From 4ae100abc461eb0f0a6eb7ad91919ec53a2cba70 Mon Sep 17 00:00:00 2001 From: shidayang <530847445@qq.com> Date: Tue, 10 Oct 2023 20:15:16 +0800 Subject: [PATCH] [AMORO-1906]Refactor code of "Table" and "Catalog" to support Paimon format integration (#1960) * Integration paimon * Integration paimon * Integration paimon * Integration paimon * tmp * Support registering for the Paimon Catalog * Support registering for the Paimon Catalog * Support registering for the Paimon Catalog * Support registering for the Paimon Catalog * Fix ut * Polish code * Fix compile error * Merge master * polish code * Merge master * Fix UT * Polish code * Polish code * polish code * polish code * Update ams/server/src/main/java/com/netease/arctic/server/dashboard/MixedAndIcebergTableDescriptor.java Co-authored-by: ZhouJinsong * Update ams/server/src/main/java/com/netease/arctic/server/dashboard/PaimonTableDescriptor.java Co-authored-by: ZhouJinsong * tmp * polish code * polish code --------- Co-authored-by: ZhouJinsong --- .../netease/arctic/ams/api/TableFormat.java | 9 + .../server/DefaultOptimizingService.java | 4 +- .../arctic/server/catalog/CatalogBuilder.java | 4 + .../server/catalog/IcebergCatalogImpl.java | 13 +- .../catalog/InternalIcebergCatalogImpl.java | 13 +- .../catalog/InternalMixedCatalogImpl.java | 12 +- .../server/catalog/MixedHiveCatalogImpl.java | 14 + .../catalog/MixedIcebergCatalogImpl.java | 9 +- .../server/catalog/PaimonServerCatalog.java | 99 ++++ .../arctic/server/catalog/ServerCatalog.java | 4 +- .../dashboard/FormatTableDescriptor.java | 69 +++ .../MixedAndIcebergTableDescriptor.java | 468 ++++++++++++++++++ .../dashboard/PaimonTableDescriptor.java | 70 +++ .../dashboard/ServerTableDescriptor.java | 295 ++--------- .../dashboard/controller/TableController.java | 178 +------ .../server/optimizing/OptimizingQueue.java | 9 +- .../maintainer/IcebergTableMaintainer.java | 423 ++++++++++++++++ .../maintainer/MixedTableMaintainer.java | 311 ++++++++++++ .../maintainer/TableMaintainer.java | 54 ++ .../persistence/mapper/TableMetaMapper.java | 8 +- .../server/table/DefaultTableService.java | 12 +- .../server/table/RuntimeHandlerChain.java | 54 +- .../arctic/server/table/TableManager.java | 4 +- .../arctic/server/table/TableRuntime.java | 16 +- .../arctic/server/table/TableRuntimeMeta.java | 10 + .../table/executor/BaseTableExecutor.java | 6 +- .../executor/HiveCommitSyncExecutor.java | 2 +- .../executor/OptimizingCommitExecutor.java | 4 +- .../executor/OrphanFilesCleaningExecutor.java | 320 +----------- .../executor/SnapshotsExpiringExecutor.java | 282 +---------- .../executor/TableRuntimeRefreshExecutor.java | 5 +- .../arctic/server/utils/IcebergTableUtil.java | 17 +- .../main/resources/derby/ams-derby-init.sql | 1 + .../main/resources/mysql/ams-mysql-init.sql | 1 + .../src/main/resources/mysql/update.sql | 1 + .../src/main/resources/postgres/update.sql | 2 + .../server/catalog/ExternalCatalogTest.java | 88 ++++ .../server/catalog/TableCatalogTestBase.java | 78 +++ .../dashboard/TestServerTableDescriptor.java | 2 +- .../optimizing/TestOptimizingQueue.java | 9 +- .../flow/CompleteOptimizingFlow.java | 2 +- .../maintainer}/TestOrphanFileClean.java | 44 +- .../maintainer}/TestOrphanFileCleanHive.java | 10 +- .../TestOrphanFileCleanIceberg.java | 9 +- .../maintainer}/TestSnapshotExpire.java | 201 ++------ .../maintainer}/TestSnapshotExpireHive.java | 12 +- .../TestSnapshotExpireIceberg.java | 4 +- .../scan/TestKeyedTableFileScanHelper.java | 23 +- .../server/table/TestTableRuntimeHandler.java | 9 +- .../server/table/TestTableRuntimeManager.java | 2 +- core/pom.xml | 5 + .../arctic/AlreadyExistsException.java | 8 + .../java/com/netease/arctic/AmoroTable.java | 2 +- .../arctic/DatabaseNotEmptyException.java | 34 ++ .../arctic/NoSuchDatabaseException.java | 12 + .../netease/arctic/NoSuchTableException.java | 8 + .../{Snapshot.java => TableSnapshot.java} | 6 +- .../iceberg/IcebergCatalogFactory.java | 12 +- .../formats/iceberg/IcebergSnapshot.java | 15 +- .../arctic/formats/iceberg/IcebergTable.java | 11 +- .../MixedHiveTable.java} | 18 +- .../formats/mixed/MixedIcebergTable.java | 76 +++ .../arctic/formats/mixed/MixedSnapshot.java | 72 +++ .../arctic/formats/paimon/PaimonCatalog.java | 98 ++++ .../formats/paimon/PaimonCatalogFactory.java | 61 +++ .../arctic/formats/paimon/PaimonSnapshot.java | 47 ++ .../arctic/formats/paimon/PaimonTable.java | 71 +++ .../com.netease.arctic.FormatCatalogFactory | 1 + .../arctic/formats/AmoroCatalogTestBase.java | 56 +++ .../formats/AmoroCatalogTestHelper.java | 95 ++++ .../IcebergHadoopCatalogTestHelper.java | 142 ++++++ .../PaimonHadoopCatalogTestHelper.java | 137 +++++ .../arctic/formats/TestAmoroCatalogBase.java | 101 ++++ .../formats/TestIcebergAmoroCatalog.java | 84 ++++ .../formats/TestPaimonAmoroCatalog.java | 81 +++ dist/src/main/arctic-bin/conf/config.yaml | 14 +- .../formats/IcebergHiveCatalogTestHelper.java | 52 ++ .../formats/PaimonHiveCatalogTestHelper.java | 52 ++ .../formats/TestIcebergHiveAmoroCatalog.java | 52 ++ .../formats/TestPaimonHiveAmoroCatalog.java | 52 ++ pom.xml | 7 + 81 files changed, 3454 insertions(+), 1324 deletions(-) create mode 100644 ams/server/src/main/java/com/netease/arctic/server/catalog/PaimonServerCatalog.java create mode 100644 ams/server/src/main/java/com/netease/arctic/server/dashboard/FormatTableDescriptor.java create mode 100644 ams/server/src/main/java/com/netease/arctic/server/dashboard/MixedAndIcebergTableDescriptor.java create mode 100644 ams/server/src/main/java/com/netease/arctic/server/dashboard/PaimonTableDescriptor.java create mode 100644 ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/IcebergTableMaintainer.java create mode 100644 ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/MixedTableMaintainer.java create mode 100644 ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/TableMaintainer.java create mode 100644 ams/server/src/main/resources/mysql/update.sql create mode 100644 ams/server/src/main/resources/postgres/update.sql create mode 100644 ams/server/src/test/java/com/netease/arctic/server/catalog/ExternalCatalogTest.java create mode 100644 ams/server/src/test/java/com/netease/arctic/server/catalog/TableCatalogTestBase.java rename ams/server/src/test/java/com/netease/arctic/server/{table/executor => optimizing/maintainer}/TestOrphanFileClean.java (90%) rename ams/server/src/test/java/com/netease/arctic/server/{table/executor => optimizing/maintainer}/TestOrphanFileCleanHive.java (89%) rename ams/server/src/test/java/com/netease/arctic/server/{table/executor => optimizing/maintainer}/TestOrphanFileCleanIceberg.java (96%) rename ams/server/src/test/java/com/netease/arctic/server/{table/executor => optimizing/maintainer}/TestSnapshotExpire.java (60%) rename ams/server/src/test/java/com/netease/arctic/server/{table/executor => optimizing/maintainer}/TestSnapshotExpireHive.java (90%) rename ams/server/src/test/java/com/netease/arctic/server/{table/executor => optimizing/maintainer}/TestSnapshotExpireIceberg.java (94%) create mode 100644 core/src/main/java/com/netease/arctic/DatabaseNotEmptyException.java rename core/src/main/java/com/netease/arctic/{Snapshot.java => TableSnapshot.java} (89%) rename core/src/main/java/com/netease/arctic/formats/{paimon/PaimonFormatCatalog.java => mixed/MixedHiveTable.java} (64%) create mode 100644 core/src/main/java/com/netease/arctic/formats/mixed/MixedIcebergTable.java create mode 100644 core/src/main/java/com/netease/arctic/formats/mixed/MixedSnapshot.java create mode 100644 core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalog.java create mode 100644 core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalogFactory.java create mode 100644 core/src/main/java/com/netease/arctic/formats/paimon/PaimonSnapshot.java create mode 100644 core/src/main/java/com/netease/arctic/formats/paimon/PaimonTable.java create mode 100644 core/src/test/java/com/netease/arctic/formats/AmoroCatalogTestBase.java create mode 100644 core/src/test/java/com/netease/arctic/formats/AmoroCatalogTestHelper.java create mode 100644 core/src/test/java/com/netease/arctic/formats/IcebergHadoopCatalogTestHelper.java create mode 100644 core/src/test/java/com/netease/arctic/formats/PaimonHadoopCatalogTestHelper.java create mode 100644 core/src/test/java/com/netease/arctic/formats/TestAmoroCatalogBase.java create mode 100644 core/src/test/java/com/netease/arctic/formats/TestIcebergAmoroCatalog.java create mode 100644 core/src/test/java/com/netease/arctic/formats/TestPaimonAmoroCatalog.java create mode 100644 hive/src/test/java/com/netease/arctic/hive/formats/IcebergHiveCatalogTestHelper.java create mode 100644 hive/src/test/java/com/netease/arctic/hive/formats/PaimonHiveCatalogTestHelper.java create mode 100644 hive/src/test/java/com/netease/arctic/hive/formats/TestIcebergHiveAmoroCatalog.java create mode 100644 hive/src/test/java/com/netease/arctic/hive/formats/TestPaimonHiveAmoroCatalog.java diff --git a/ams/api/src/main/java/com/netease/arctic/ams/api/TableFormat.java b/ams/api/src/main/java/com/netease/arctic/ams/api/TableFormat.java index 91752514ed..9d376df355 100644 --- a/ams/api/src/main/java/com/netease/arctic/ams/api/TableFormat.java +++ b/ams/api/src/main/java/com/netease/arctic/ams/api/TableFormat.java @@ -28,4 +28,13 @@ public enum TableFormat { MIXED_ICEBERG, MIXED_HIVE, PAIMON; + + public boolean in(TableFormat... tableFormats) { + for (TableFormat tableFormat : tableFormats) { + if (this == tableFormat) { + return true; + } + } + return false; + } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java index 936f1f039f..5725c54fe9 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/DefaultOptimizingService.java @@ -19,6 +19,7 @@ package com.netease.arctic.server; import com.google.common.base.Preconditions; +import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.OptimizerRegisterInfo; import com.netease.arctic.ams.api.OptimizingService; @@ -45,7 +46,6 @@ import com.netease.arctic.server.table.TableRuntime; import com.netease.arctic.server.table.TableRuntimeMeta; import com.netease.arctic.server.utils.Configurations; -import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.TableProperties; import org.apache.commons.collections.CollectionUtils; import org.slf4j.Logger; @@ -332,7 +332,7 @@ public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration or } @Override - public void handleTableAdded(ArcticTable table, TableRuntime tableRuntime) { + public void handleTableAdded(AmoroTable table, TableRuntime tableRuntime) { getOptionalQueueByGroup(tableRuntime.getOptimizerGroup()).ifPresent(q -> q.refreshTable(tableRuntime)); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/catalog/CatalogBuilder.java b/ams/server/src/main/java/com/netease/arctic/server/catalog/CatalogBuilder.java index 5359ff33bc..3075298b93 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/CatalogBuilder.java +++ b/ams/server/src/main/java/com/netease/arctic/server/catalog/CatalogBuilder.java @@ -22,6 +22,10 @@ public static ServerCatalog buildServerCatalog(CatalogMeta catalogMeta, Configur Set tableFormats = CatalogUtil.tableFormats(catalogMeta); TableFormat tableFormat = tableFormats.iterator().next(); + if (tableFormat == TableFormat.PAIMON) { + return new PaimonServerCatalog(catalogMeta); + } + switch (type) { case CATALOG_TYPE_HADOOP: if (TableFormat.ICEBERG == tableFormat) { diff --git a/ams/server/src/main/java/com/netease/arctic/server/catalog/IcebergCatalogImpl.java b/ams/server/src/main/java/com/netease/arctic/server/catalog/IcebergCatalogImpl.java index eb0847fc13..d13728424e 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/IcebergCatalogImpl.java +++ b/ams/server/src/main/java/com/netease/arctic/server/catalog/IcebergCatalogImpl.java @@ -1,10 +1,12 @@ package com.netease.arctic.server.catalog; +import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableIdentifier; import com.netease.arctic.catalog.IcebergCatalogWrapper; -import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.formats.iceberg.IcebergTable; import com.netease.arctic.utils.CatalogUtil; +import org.apache.iceberg.Table; import java.util.Collections; import java.util.List; @@ -55,8 +57,11 @@ public List toAmsIdList(List loadTable(String database, String tableName) { + com.netease.arctic.table.TableIdentifier identifier = com.netease.arctic.table.TableIdentifier.of( + catalogWrapper.name(), + database, + tableName); + return new IcebergTable(identifier, catalogWrapper.loadTable(identifier).asUnkeyedTable()); } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/catalog/InternalIcebergCatalogImpl.java b/ams/server/src/main/java/com/netease/arctic/server/catalog/InternalIcebergCatalogImpl.java index e06606c689..d417730d32 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/InternalIcebergCatalogImpl.java +++ b/ams/server/src/main/java/com/netease/arctic/server/catalog/InternalIcebergCatalogImpl.java @@ -1,7 +1,9 @@ package com.netease.arctic.server.catalog; +import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.catalog.IcebergCatalogWrapper; +import com.netease.arctic.formats.iceberg.IcebergTable; import com.netease.arctic.io.ArcticFileIO; import com.netease.arctic.io.ArcticFileIOAdapter; import com.netease.arctic.server.ArcticManagementConf; @@ -11,7 +13,6 @@ import com.netease.arctic.server.table.TableMetadata; import com.netease.arctic.server.utils.Configurations; import com.netease.arctic.server.utils.IcebergTableUtil; -import com.netease.arctic.table.ArcticTable; import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.TableOperations; @@ -54,7 +55,7 @@ public void updateMetadata(CatalogMeta metadata) { } @Override - public ArcticTable loadTable(String database, String tableName) { + public AmoroTable loadTable(String database, String tableName) { TableMetadata tableMetadata = getAs(TableMetaMapper.class, mapper -> mapper.selectTableMetaByName(getMetadata().getCatalogName(), database, tableName)); if (tableMetadata == null) { @@ -64,9 +65,11 @@ public ArcticTable loadTable(String database, String tableName) { ArcticFileIO fileIO = new ArcticFileIOAdapter(io); TableOperations ops = InternalTableOperations.buildForLoad(tableMetadata, io); BaseTable table = new BaseTable(ops, TableIdentifier.of(database, tableName).toString()); - return new IcebergCatalogWrapper.BasicIcebergTable( - com.netease.arctic.table.TableIdentifier.of(name(), database, tableName), - table, fileIO, getMetadata().getCatalogProperties() + com.netease.arctic.table.TableIdentifier tableIdentifier = + com.netease.arctic.table.TableIdentifier.of(name(), database, tableName); + return new IcebergTable(tableIdentifier, + new IcebergCatalogWrapper.BasicIcebergTable( + tableIdentifier, table, fileIO, getMetadata().getCatalogProperties()) ); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/catalog/InternalMixedCatalogImpl.java b/ams/server/src/main/java/com/netease/arctic/server/catalog/InternalMixedCatalogImpl.java index 77106a4c94..9beb8d835a 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/InternalMixedCatalogImpl.java +++ b/ams/server/src/main/java/com/netease/arctic/server/catalog/InternalMixedCatalogImpl.java @@ -1,14 +1,15 @@ package com.netease.arctic.server.catalog; +import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.catalog.MixedTables; +import com.netease.arctic.formats.mixed.MixedIcebergTable; import com.netease.arctic.server.persistence.mapper.TableMetaMapper; import com.netease.arctic.server.table.TableMetadata; -import com.netease.arctic.table.ArcticTable; public class InternalMixedCatalogImpl extends InternalCatalog { - private final MixedTables tables; + protected final MixedTables tables; protected InternalMixedCatalogImpl(CatalogMeta metadata) { super(metadata); @@ -27,10 +28,13 @@ public void updateMetadata(CatalogMeta metadata) { } @Override - public ArcticTable loadTable(String database, String tableName) { + public AmoroTable loadTable(String database, String tableName) { TableMetadata tableMetadata = getAs(TableMetaMapper.class, mapper -> mapper.selectTableMetaByName(getMetadata().getCatalogName(), database, tableName)); - return tableMetadata == null ? null : tables.loadTableByMeta(tableMetadata.buildTableMeta()); + if (tableMetadata == null) { + return null; + } + return new MixedIcebergTable(tables.loadTableByMeta(tableMetadata.buildTableMeta())); } protected MixedTables tables() { diff --git a/ams/server/src/main/java/com/netease/arctic/server/catalog/MixedHiveCatalogImpl.java b/ams/server/src/main/java/com/netease/arctic/server/catalog/MixedHiveCatalogImpl.java index 58e0b87b01..a8389b9347 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/MixedHiveCatalogImpl.java +++ b/ams/server/src/main/java/com/netease/arctic/server/catalog/MixedHiveCatalogImpl.java @@ -18,10 +18,14 @@ package com.netease.arctic.server.catalog; +import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.CatalogMeta; +import com.netease.arctic.formats.mixed.MixedHiveTable; import com.netease.arctic.hive.CachedHiveClientPool; import com.netease.arctic.hive.HMSClient; import com.netease.arctic.hive.catalog.MixedHiveTables; +import com.netease.arctic.server.persistence.mapper.TableMetaMapper; +import com.netease.arctic.server.table.TableMetadata; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.thrift.TException; @@ -42,6 +46,16 @@ public void updateMetadata(CatalogMeta metadata) { hiveClientPool = ((MixedHiveTables) tables()).getHiveClientPool(); } + @Override + public AmoroTable loadTable(String database, String tableName) { + TableMetadata tableMetadata = getAs(TableMetaMapper.class, mapper -> + mapper.selectTableMetaByName(getMetadata().getCatalogName(), database, tableName)); + if (tableMetadata == null) { + return null; + } + return new MixedHiveTable(tables.loadTableByMeta(tableMetadata.buildTableMeta())); + } + @Override public void createDatabase(String databaseName) { // do not handle database operations diff --git a/ams/server/src/main/java/com/netease/arctic/server/catalog/MixedIcebergCatalogImpl.java b/ams/server/src/main/java/com/netease/arctic/server/catalog/MixedIcebergCatalogImpl.java index 3ac26f307b..d477981bd7 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/MixedIcebergCatalogImpl.java +++ b/ams/server/src/main/java/com/netease/arctic/server/catalog/MixedIcebergCatalogImpl.java @@ -18,11 +18,11 @@ package com.netease.arctic.server.catalog; +import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableIdentifier; +import com.netease.arctic.formats.mixed.MixedIcebergTable; import com.netease.arctic.mixed.BasicMixedIcebergCatalog; -import com.netease.arctic.table.ArcticTable; - import java.util.List; import java.util.stream.Collectors; @@ -72,7 +72,8 @@ public List listTables(String database) { } @Override - public ArcticTable loadTable(String database, String tableName) { - return mixedIcebergCatalog.loadTable(com.netease.arctic.table.TableIdentifier.of(name(), database, tableName)); + public AmoroTable loadTable(String database, String tableName) { + return new MixedIcebergTable(mixedIcebergCatalog.loadTable( + com.netease.arctic.table.TableIdentifier.of(name(), database, tableName))); } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/catalog/PaimonServerCatalog.java b/ams/server/src/main/java/com/netease/arctic/server/catalog/PaimonServerCatalog.java new file mode 100644 index 0000000000..0663d57f95 --- /dev/null +++ b/ams/server/src/main/java/com/netease/arctic/server/catalog/PaimonServerCatalog.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.server.catalog; + +import com.netease.arctic.AmoroTable; +import com.netease.arctic.CommonUnifiedCatalog; +import com.netease.arctic.UnifiedCatalog; +import com.netease.arctic.ams.api.CatalogMeta; +import com.netease.arctic.ams.api.TableIdentifier; +import com.netease.arctic.table.TableMetaStore; +import com.netease.arctic.utils.CatalogUtil; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.stream.Collectors; + +public class PaimonServerCatalog extends ExternalCatalog { + + private volatile UnifiedCatalog paimonCatalog; + + private volatile TableMetaStore tableMetaStore; + + protected PaimonServerCatalog(CatalogMeta metadata) { + super(metadata); + this.tableMetaStore = CatalogUtil.buildMetaStore(metadata); + this.paimonCatalog = doAs(() -> new CommonUnifiedCatalog( + null, + metadata, + metadata.catalogProperties + )); + } + + @Override + public void updateMetadata(CatalogMeta metadata) { + super.updateMetadata(metadata); + this.tableMetaStore = CatalogUtil.buildMetaStore(metadata); + this.paimonCatalog = doAs(() -> new CommonUnifiedCatalog( + null, + metadata, + metadata.catalogProperties + )); + } + + @Override + public boolean exist(String database) { + return doAs(() -> paimonCatalog.exist(database)); + } + + @Override + public boolean exist(String database, String tableName) { + return doAs(() -> paimonCatalog.exist(database, tableName)); + } + + @Override + public List listDatabases() { + return doAs(() -> paimonCatalog.listDatabases()); + } + + @Override + public List listTables() { + return doAs(() -> paimonCatalog.listDatabases() + .stream() + .map(this::listTables) + .flatMap(List::stream) + .collect(Collectors.toList())); + } + + @Override + public List listTables(String database) { + return doAs(() -> paimonCatalog.listTableMetas(database) + .stream() + .map(t -> t.getIdentifier().buildTableIdentifier()) + .collect(Collectors.toList())); + } + + @Override + public AmoroTable loadTable(String database, String tableName) { + return doAs(() -> paimonCatalog.loadTable(database, tableName)); + } + + private T doAs(Callable callable) { + return tableMetaStore.doAs(callable); + } +} diff --git a/ams/server/src/main/java/com/netease/arctic/server/catalog/ServerCatalog.java b/ams/server/src/main/java/com/netease/arctic/server/catalog/ServerCatalog.java index 188cd980c0..f7f4989faa 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/ServerCatalog.java +++ b/ams/server/src/main/java/com/netease/arctic/server/catalog/ServerCatalog.java @@ -1,10 +1,10 @@ package com.netease.arctic.server.catalog; +import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableIdentifier; import com.netease.arctic.server.persistence.PersistentBase; import com.netease.arctic.server.persistence.mapper.CatalogMetaMapper; -import com.netease.arctic.table.ArcticTable; import java.util.List; @@ -40,5 +40,5 @@ public void updateMetadata(CatalogMeta metadata) { public abstract List listTables(String database); - public abstract ArcticTable loadTable(String database, String tableName); + public abstract AmoroTable loadTable(String database, String tableName); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/dashboard/FormatTableDescriptor.java b/ams/server/src/main/java/com/netease/arctic/server/dashboard/FormatTableDescriptor.java new file mode 100644 index 0000000000..f3f1eb7fd7 --- /dev/null +++ b/ams/server/src/main/java/com/netease/arctic/server/dashboard/FormatTableDescriptor.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.server.dashboard; + +import com.netease.arctic.AmoroTable; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.server.dashboard.model.DDLInfo; +import com.netease.arctic.server.dashboard.model.PartitionBaseInfo; +import com.netease.arctic.server.dashboard.model.PartitionFileBaseInfo; +import com.netease.arctic.server.dashboard.model.ServerTableMeta; +import com.netease.arctic.server.dashboard.model.TransactionsOfTable; +import java.util.List; + +/** + * API for obtaining metadata information of various formats. + */ +public interface FormatTableDescriptor { + + /** + * Get the format supported by this descriptor. + */ + List supportFormat(); + + /** + * Get the table metadata information of the {@link AmoroTable}. + */ + ServerTableMeta getTableDetail(AmoroTable amoroTable); + + /** + * Get the transaction information of the {@link AmoroTable}. + */ + List getTransactions(AmoroTable amoroTable); + + /** + * Get the transaction detail information of the {@link AmoroTable}. + */ + List getTransactionDetail(AmoroTable amoroTable, long transactionId); + + /** + * Get the DDL information of the {@link AmoroTable}. + */ + List getTableOperations(AmoroTable amoroTable); + + /** + * Get the partition information of the {@link AmoroTable}. + */ + List getTablePartition(AmoroTable amoroTable); + + /** + * Get the file information of the {@link AmoroTable}. + */ + List getTableFile(AmoroTable amoroTable, String partition); +} \ No newline at end of file diff --git a/ams/server/src/main/java/com/netease/arctic/server/dashboard/MixedAndIcebergTableDescriptor.java b/ams/server/src/main/java/com/netease/arctic/server/dashboard/MixedAndIcebergTableDescriptor.java new file mode 100644 index 0000000000..2c9b48753a --- /dev/null +++ b/ams/server/src/main/java/com/netease/arctic/server/dashboard/MixedAndIcebergTableDescriptor.java @@ -0,0 +1,468 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.server.dashboard; + +import com.netease.arctic.AmoroTable; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.data.DataFileType; +import com.netease.arctic.data.FileNameRules; +import com.netease.arctic.op.SnapshotSummary; +import com.netease.arctic.server.dashboard.model.AMSColumnInfo; +import com.netease.arctic.server.dashboard.model.AMSPartitionField; +import com.netease.arctic.server.dashboard.model.DDLInfo; +import com.netease.arctic.server.dashboard.model.FilesStatistics; +import com.netease.arctic.server.dashboard.model.PartitionBaseInfo; +import com.netease.arctic.server.dashboard.model.PartitionFileBaseInfo; +import com.netease.arctic.server.dashboard.model.ServerTableMeta; +import com.netease.arctic.server.dashboard.model.TableBasicInfo; +import com.netease.arctic.server.dashboard.model.TableStatistics; +import com.netease.arctic.server.dashboard.model.TransactionsOfTable; +import com.netease.arctic.server.dashboard.utils.AmsUtil; +import com.netease.arctic.server.dashboard.utils.TableStatCollector; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.KeyedTable; +import com.netease.arctic.table.PrimaryKeySpec; +import com.netease.arctic.table.TableProperties; +import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.ManifestEntryFields; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.HistoryEntry; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Descriptor for Mixed-Hive, Mixed-Iceberg, Iceberg format tables. + */ +public class MixedAndIcebergTableDescriptor implements FormatTableDescriptor { + + private static final Logger LOG = LoggerFactory.getLogger(MixedAndIcebergTableDescriptor.class); + + @Override + public List supportFormat() { + return Arrays.asList(TableFormat.ICEBERG, TableFormat.MIXED_ICEBERG, TableFormat.MIXED_HIVE); + } + + @Override + public ServerTableMeta getTableDetail(AmoroTable amoroTable) { + ArcticTable table = getTable(amoroTable); + // set basic info + TableBasicInfo tableBasicInfo = getTableBasicInfo(table); + ServerTableMeta serverTableMeta = getServerTableMeta(table); + long tableSize = 0; + long tableFileCnt = 0; + Map baseMetrics = Maps.newHashMap(); + FilesStatistics baseFilesStatistics = tableBasicInfo.getBaseStatistics().getTotalFilesStat(); + Map baseSummary = tableBasicInfo.getBaseStatistics().getSummary(); + baseMetrics.put("lastCommitTime", AmsUtil.longOrNull(baseSummary.get("visibleTime"))); + baseMetrics.put("totalSize", AmsUtil.byteToXB(baseFilesStatistics.getTotalSize())); + baseMetrics.put("fileCount", baseFilesStatistics.getFileCnt()); + baseMetrics.put("averageFileSize", AmsUtil.byteToXB(baseFilesStatistics.getAverageSize())); + if (tableBasicInfo.getChangeStatistics() == null) { + baseMetrics.put("baseWatermark", AmsUtil.longOrNull(serverTableMeta.getTableWatermark())); + } else { + baseMetrics.put("baseWatermark", AmsUtil.longOrNull(serverTableMeta.getBaseWatermark())); + } + tableSize += baseFilesStatistics.getTotalSize(); + tableFileCnt += baseFilesStatistics.getFileCnt(); + serverTableMeta.setBaseMetrics(baseMetrics); + + Map changeMetrics = Maps.newHashMap(); + if (tableBasicInfo.getChangeStatistics() != null) { + FilesStatistics changeFilesStatistics = tableBasicInfo.getChangeStatistics().getTotalFilesStat(); + Map changeSummary = tableBasicInfo.getChangeStatistics().getSummary(); + changeMetrics.put("lastCommitTime", AmsUtil.longOrNull(changeSummary.get("visibleTime"))); + changeMetrics.put("totalSize", AmsUtil.byteToXB(changeFilesStatistics.getTotalSize())); + changeMetrics.put("fileCount", changeFilesStatistics.getFileCnt()); + changeMetrics.put("averageFileSize", AmsUtil.byteToXB(changeFilesStatistics.getAverageSize())); + changeMetrics.put("tableWatermark", AmsUtil.longOrNull(serverTableMeta.getTableWatermark())); + tableSize += changeFilesStatistics.getTotalSize(); + tableFileCnt += changeFilesStatistics.getFileCnt(); + } else { + changeMetrics.put("lastCommitTime", null); + changeMetrics.put("totalSize", null); + changeMetrics.put("fileCount", null); + changeMetrics.put("averageFileSize", null); + changeMetrics.put("tableWatermark", null); + } + Map tableSummary = new HashMap<>(); + tableSummary.put("size", AmsUtil.byteToXB(tableSize)); + tableSummary.put("file", tableFileCnt); + tableSummary.put("averageFile", AmsUtil.byteToXB(tableFileCnt == 0 ? 0 : tableSize / tableFileCnt)); + tableSummary.put("tableFormat", AmsUtil.formatString(amoroTable.format().name())); + serverTableMeta.setTableSummary(tableSummary); + return serverTableMeta; + } + + public List getTransactions(AmoroTable amoroTable) { + ArcticTable arcticTable = getTable(amoroTable); + List transactionsOfTables = new ArrayList<>(); + List tables = new ArrayList<>(); + if (arcticTable.isKeyedTable()) { + tables.add(arcticTable.asKeyedTable().changeTable()); + tables.add(arcticTable.asKeyedTable().baseTable()); + } else { + tables.add(arcticTable.asUnkeyedTable()); + } + tables.forEach(table -> table.snapshots().forEach(snapshot -> { + if (snapshot.operation().equals(DataOperations.REPLACE)) { + return; + } + if (snapshot.summary().containsKey(SnapshotSummary.TRANSACTION_BEGIN_SIGNATURE)) { + return; + } + TransactionsOfTable transactionsOfTable = new TransactionsOfTable(); + transactionsOfTable.setTransactionId(snapshot.snapshotId()); + int fileCount = PropertyUtil + .propertyAsInt(snapshot.summary(), org.apache.iceberg.SnapshotSummary.ADDED_FILES_PROP, 0); + fileCount += PropertyUtil + .propertyAsInt(snapshot.summary(), org.apache.iceberg.SnapshotSummary.ADDED_DELETE_FILES_PROP, 0); + fileCount += PropertyUtil + .propertyAsInt(snapshot.summary(), org.apache.iceberg.SnapshotSummary.DELETED_FILES_PROP, 0); + fileCount += PropertyUtil + .propertyAsInt(snapshot.summary(), org.apache.iceberg.SnapshotSummary.REMOVED_DELETE_FILES_PROP, 0); + transactionsOfTable.setFileCount(fileCount); + transactionsOfTable.setFileSize(PropertyUtil + .propertyAsLong(snapshot.summary(), org.apache.iceberg.SnapshotSummary.ADDED_FILE_SIZE_PROP, 0) + + PropertyUtil + .propertyAsLong(snapshot.summary(), org.apache.iceberg.SnapshotSummary.REMOVED_FILE_SIZE_PROP, 0)); + transactionsOfTable.setCommitTime(snapshot.timestampMillis()); + transactionsOfTable.setOperation(snapshot.operation()); + transactionsOfTable.setSummary(snapshot.summary()); + transactionsOfTables.add(transactionsOfTable); + })); + transactionsOfTables.sort((o1, o2) -> Long.compare(o2.commitTime, o1.commitTime)); + return transactionsOfTables; + } + + public List getTransactionDetail(AmoroTable amoroTable, long transactionId) { + ArcticTable arcticTable = getTable(amoroTable); + List result = new ArrayList<>(); + Snapshot snapshot; + if (arcticTable.isKeyedTable()) { + snapshot = arcticTable.asKeyedTable().changeTable().snapshot(transactionId); + if (snapshot == null) { + snapshot = arcticTable.asKeyedTable().baseTable().snapshot(transactionId); + } + } else { + snapshot = arcticTable.asUnkeyedTable().snapshot(transactionId); + } + if (snapshot == null) { + throw new IllegalArgumentException("unknown snapshot " + transactionId + " of " + amoroTable.id()); + } + final long snapshotTime = snapshot.timestampMillis(); + String commitId = String.valueOf(transactionId); + snapshot.addedDataFiles(arcticTable.io()).forEach(f -> result.add(new PartitionFileBaseInfo( + commitId, + DataFileType.ofContentId(f.content().id()), + snapshotTime, + arcticTable.spec().partitionToPath(f.partition()), + f.path().toString(), + f.fileSizeInBytes(), + "add"))); + snapshot.removedDataFiles(arcticTable.io()).forEach(f -> result.add(new PartitionFileBaseInfo( + commitId, + DataFileType.ofContentId(f.content().id()), + snapshotTime, + arcticTable.spec().partitionToPath(f.partition()), + f.path().toString(), + f.fileSizeInBytes(), + "remove"))); + snapshot.addedDeleteFiles(arcticTable.io()).forEach(f -> result.add(new PartitionFileBaseInfo( + commitId, + DataFileType.ofContentId(f.content().id()), + snapshotTime, + arcticTable.spec().partitionToPath(f.partition()), + f.path().toString(), + f.fileSizeInBytes(), + "add"))); + snapshot.removedDeleteFiles(arcticTable.io()).forEach(f -> result.add(new PartitionFileBaseInfo( + commitId, + DataFileType.ofContentId(f.content().id()), + snapshotTime, + arcticTable.spec().partitionToPath(f.partition()), + f.path().toString(), + f.fileSizeInBytes(), + "remove"))); + return result; + } + + public List getTableOperations(AmoroTable amoroTable) { + ArcticTable arcticTable = getTable(amoroTable); + List result = new ArrayList<>(); + Table table; + if (arcticTable.isKeyedTable()) { + table = arcticTable.asKeyedTable().baseTable(); + } else { + table = arcticTable.asUnkeyedTable(); + } + List snapshotLog = ((HasTableOperations) table).operations().current().snapshotLog(); + List metadataLogEntries = + ((HasTableOperations) table).operations().current().previousFiles(); + Set time = new HashSet<>(); + snapshotLog.forEach(e -> time.add(e.timestampMillis())); + String lastMetadataLogEntryFile = null; + org.apache.iceberg.TableMetadata lastTableMetadata = null; + for (int i = 1; i < metadataLogEntries.size(); i++) { + org.apache.iceberg.TableMetadata.MetadataLogEntry currentEntry = metadataLogEntries.get(i); + if (!time.contains(currentEntry.timestampMillis())) { + org.apache.iceberg.TableMetadata.MetadataLogEntry previousEntry = metadataLogEntries.get(i - 1); + org.apache.iceberg.TableMetadata oldTableMetadata; + if (lastMetadataLogEntryFile == null || !lastMetadataLogEntryFile.equals(previousEntry.file())) { + oldTableMetadata = TableMetadataParser.read(table.io(), previousEntry.file()); + } else { + oldTableMetadata = lastTableMetadata; + } + + org.apache.iceberg.TableMetadata + newTableMetadata = TableMetadataParser.read(table.io(), currentEntry.file()); + lastMetadataLogEntryFile = currentEntry.file(); + lastTableMetadata = newTableMetadata; + + DDLInfo.Generator generator = new DDLInfo.Generator(); + result.addAll(generator.tableIdentify(arcticTable.id()) + .oldMeta(oldTableMetadata) + .newMeta(newTableMetadata) + .generate()); + } + } + if (metadataLogEntries.size() > 0) { + org.apache.iceberg.TableMetadata.MetadataLogEntry previousEntry = metadataLogEntries + .get(metadataLogEntries.size() - 1); + org.apache.iceberg.TableMetadata oldTableMetadata; + + if (lastMetadataLogEntryFile == null || !lastMetadataLogEntryFile.equals(previousEntry.file())) { + oldTableMetadata = TableMetadataParser.read(table.io(), previousEntry.file()); + } else { + oldTableMetadata = lastTableMetadata; + } + + org.apache.iceberg.TableMetadata newTableMetadata = ((HasTableOperations) table).operations().current(); + DDLInfo.Generator generator = new DDLInfo.Generator(); + result.addAll(generator.tableIdentify(arcticTable.id()) + .oldMeta(oldTableMetadata) + .newMeta(newTableMetadata) + .generate()); + } + return result; + } + + @Override + public List getTablePartition(AmoroTable amoroTable) { + ArcticTable arcticTable = getTable(amoroTable); + if (arcticTable.spec().isUnpartitioned()) { + return new ArrayList<>(); + } + Map partitionBaseInfoHashMap = new HashMap<>(); + getTableFile(amoroTable, null).forEach(fileInfo -> { + if (!partitionBaseInfoHashMap.containsKey(fileInfo.getPartition())) { + partitionBaseInfoHashMap.put(fileInfo.getPartition(), new PartitionBaseInfo()); + partitionBaseInfoHashMap.get(fileInfo.getPartition()).setPartition(fileInfo.getPartition()); + } + PartitionBaseInfo partitionInfo = partitionBaseInfoHashMap.get(fileInfo.getPartition()); + partitionInfo.setFileCount(partitionInfo.getFileCount() + 1); + partitionInfo.setFileSize(partitionInfo.getFileSize() + fileInfo.getFileSize()); + partitionInfo.setLastCommitTime(partitionInfo.getLastCommitTime() > fileInfo.getCommitTime() ? + partitionInfo.getLastCommitTime() : + fileInfo.getCommitTime()); + }); + + return new ArrayList<>(partitionBaseInfoHashMap.values()); + } + + public List getTableFile(AmoroTable amoroTable, String partition) { + ArcticTable arcticTable = getTable(amoroTable); + List result = new ArrayList<>(); + if (arcticTable.isKeyedTable()) { + result.addAll(collectFileInfo(arcticTable.asKeyedTable().changeTable(), true, partition)); + result.addAll(collectFileInfo(arcticTable.asKeyedTable().baseTable(), false, partition)); + } else { + result.addAll(collectFileInfo(arcticTable.asUnkeyedTable(), false, partition)); + } + return result; + } + + private List collectFileInfo(Table table, boolean isChangeTable, String partition) { + PartitionSpec spec = table.spec(); + List result = new ArrayList<>(); + Table entriesTable = MetadataTableUtils.createMetadataTableInstance(((HasTableOperations) table).operations(), + table.name(), table.name() + "#ENTRIES", + MetadataTableType.ENTRIES); + try (CloseableIterable manifests = IcebergGenerics.read(entriesTable) + .where(Expressions.notEqual(ManifestEntryFields.STATUS.name(), ManifestEntryFields.Status.DELETED.id())) + .build()) { + for (Record record : manifests) { + long snapshotId = (long) record.getField(ManifestEntryFields.SNAPSHOT_ID.name()); + GenericRecord dataFile = (GenericRecord) record.getField(ManifestEntryFields.DATA_FILE_FIELD_NAME); + Integer contentId = (Integer) dataFile.getField(DataFile.CONTENT.name()); + String filePath = (String) dataFile.getField(DataFile.FILE_PATH.name()); + String partitionPath = null; + GenericRecord parRecord = (GenericRecord) dataFile.getField(DataFile.PARTITION_NAME); + if (parRecord != null) { + InternalRecordWrapper wrapper = new InternalRecordWrapper(parRecord.struct()); + partitionPath = spec.partitionToPath(wrapper.wrap(parRecord)); + } + if (partition != null && spec.isPartitioned() && !partition.equals(partitionPath)) { + continue; + } + Long fileSize = (Long) dataFile.getField(DataFile.FILE_SIZE.name()); + DataFileType dataFileType = + isChangeTable ? FileNameRules.parseFileTypeForChange(filePath) : DataFileType.ofContentId(contentId); + long commitTime = -1; + if (table.snapshot(snapshotId) != null) { + commitTime = table.snapshot(snapshotId).timestampMillis(); + } + result.add(new PartitionFileBaseInfo(String.valueOf(snapshotId), dataFileType, commitTime, + partitionPath, filePath, fileSize)); + } + } catch (IOException exception) { + LOG.error("close manifest file error", exception); + } + return result; + } + + private TableBasicInfo getTableBasicInfo(ArcticTable table) { + try { + TableBasicInfo tableBasicInfo = new TableBasicInfo(); + tableBasicInfo.setTableIdentifier(table.id()); + TableStatistics changeInfo = null; + TableStatistics baseInfo; + + if (table.isUnkeyedTable()) { + UnkeyedTable unkeyedTable = table.asUnkeyedTable(); + baseInfo = new TableStatistics(); + TableStatCollector.fillTableStatistics(baseInfo, unkeyedTable, table); + } else if (table.isKeyedTable()) { + KeyedTable keyedTable = table.asKeyedTable(); + if (!PrimaryKeySpec.noPrimaryKey().equals(keyedTable.primaryKeySpec())) { + changeInfo = TableStatCollector.collectChangeTableInfo(keyedTable); + } + baseInfo = TableStatCollector.collectBaseTableInfo(keyedTable); + } else { + throw new IllegalStateException("unknown type of table"); + } + + tableBasicInfo.setChangeStatistics(changeInfo); + tableBasicInfo.setBaseStatistics(baseInfo); + tableBasicInfo.setTableStatistics(TableStatCollector.union(changeInfo, baseInfo)); + + long createTime + = PropertyUtil.propertyAsLong(table.properties(), TableProperties.TABLE_CREATE_TIME, + TableProperties.TABLE_CREATE_TIME_DEFAULT); + if (createTime != TableProperties.TABLE_CREATE_TIME_DEFAULT) { + if (tableBasicInfo.getTableStatistics() != null) { + if (tableBasicInfo.getTableStatistics().getSummary() == null) { + tableBasicInfo.getTableStatistics().setSummary(new HashMap<>()); + } else { + LOG.warn("{} summary is null", table.id()); + } + tableBasicInfo.getTableStatistics().getSummary() + .put("createTime", String.valueOf(createTime)); + } else { + LOG.warn("{} table statistics is null {}", table.id(), tableBasicInfo); + } + } + return tableBasicInfo; + } catch (Throwable t) { + LOG.error("{} failed to build table basic info", table.id(), t); + throw t; + } + } + + private ServerTableMeta getServerTableMeta(ArcticTable table) { + ServerTableMeta serverTableMeta = new ServerTableMeta(); + serverTableMeta.setTableType(table.format().toString()); + serverTableMeta.setTableIdentifier(table.id()); + serverTableMeta.setBaseLocation(table.location()); + fillTableProperties(serverTableMeta, table.properties()); + serverTableMeta.setPartitionColumnList(table + .spec() + .fields() + .stream() + .map(item -> AMSPartitionField.buildFromPartitionSpec(table.spec().schema(), item)) + .collect(Collectors.toList())); + serverTableMeta.setSchema(table + .schema() + .columns() + .stream() + .map(AMSColumnInfo::buildFromNestedField) + .collect(Collectors.toList())); + + serverTableMeta.setFilter(null); + LOG.debug("Table {} is keyedTable: {}", table.name(), table instanceof KeyedTable); + if (table.isKeyedTable()) { + KeyedTable kt = table.asKeyedTable(); + if (kt.primaryKeySpec() != null) { + serverTableMeta.setPkList(kt + .primaryKeySpec() + .fields() + .stream() + .map(item -> AMSColumnInfo.buildFromPartitionSpec(table.spec().schema(), item)) + .collect(Collectors.toList())); + } + } + if (serverTableMeta.getPkList() == null) { + serverTableMeta.setPkList(new ArrayList<>()); + } + return serverTableMeta; + } + + private void fillTableProperties( + ServerTableMeta serverTableMeta, + Map tableProperties) { + Map properties = com.google.common.collect.Maps.newHashMap(tableProperties); + serverTableMeta.setTableWatermark(properties.remove(TableProperties.WATERMARK_TABLE)); + serverTableMeta.setBaseWatermark(properties.remove(TableProperties.WATERMARK_BASE_STORE)); + serverTableMeta.setCreateTime(PropertyUtil.propertyAsLong(properties, TableProperties.TABLE_CREATE_TIME, + TableProperties.TABLE_CREATE_TIME_DEFAULT)); + properties.remove(TableProperties.TABLE_CREATE_TIME); + + TableProperties.READ_PROTECTED_PROPERTIES.forEach(properties::remove); + serverTableMeta.setProperties(properties); + } + + private ArcticTable getTable(AmoroTable amoroTable) { + return (ArcticTable) amoroTable.originalTable(); + } +} diff --git a/ams/server/src/main/java/com/netease/arctic/server/dashboard/PaimonTableDescriptor.java b/ams/server/src/main/java/com/netease/arctic/server/dashboard/PaimonTableDescriptor.java new file mode 100644 index 0000000000..645f2a4924 --- /dev/null +++ b/ams/server/src/main/java/com/netease/arctic/server/dashboard/PaimonTableDescriptor.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.server.dashboard; + +import com.netease.arctic.AmoroTable; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.server.dashboard.model.DDLInfo; +import com.netease.arctic.server.dashboard.model.PartitionBaseInfo; +import com.netease.arctic.server.dashboard.model.PartitionFileBaseInfo; +import com.netease.arctic.server.dashboard.model.ServerTableMeta; +import com.netease.arctic.server.dashboard.model.TransactionsOfTable; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Descriptor for Paimon format tables. + */ +public class PaimonTableDescriptor implements FormatTableDescriptor { + @Override + public List supportFormat() { + return Lists.newArrayList(TableFormat.PAIMON); + } + + @Override + public ServerTableMeta getTableDetail(AmoroTable amoroTable) { + throw new UnsupportedOperationException(); + } + + @Override + public List getTransactions(AmoroTable amoroTable) { + throw new UnsupportedOperationException(); + } + + @Override + public List getTransactionDetail(AmoroTable amoroTable, long transactionId) { + throw new UnsupportedOperationException(); + } + + @Override + public List getTableOperations(AmoroTable amoroTable) { + throw new UnsupportedOperationException(); + } + + @Override + public List getTablePartition(AmoroTable amoroTable) { + throw new UnsupportedOperationException(); + } + + @Override + public List getTableFile(AmoroTable amoroTable, String partition) { + throw new UnsupportedOperationException(); + } +} diff --git a/ams/server/src/main/java/com/netease/arctic/server/dashboard/ServerTableDescriptor.java b/ams/server/src/main/java/com/netease/arctic/server/dashboard/ServerTableDescriptor.java index 76a1b9774c..454c45989e 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/dashboard/ServerTableDescriptor.java +++ b/ams/server/src/main/java/com/netease/arctic/server/dashboard/ServerTableDescriptor.java @@ -1,225 +1,79 @@ package com.netease.arctic.server.dashboard; -import com.netease.arctic.data.DataFileType; -import com.netease.arctic.data.FileNameRules; -import com.netease.arctic.op.SnapshotSummary; +import com.netease.arctic.AmoroTable; +import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.server.dashboard.model.DDLInfo; import com.netease.arctic.server.dashboard.model.PartitionBaseInfo; import com.netease.arctic.server.dashboard.model.PartitionFileBaseInfo; +import com.netease.arctic.server.dashboard.model.ServerTableMeta; import com.netease.arctic.server.dashboard.model.TransactionsOfTable; import com.netease.arctic.server.optimizing.OptimizingProcessMeta; import com.netease.arctic.server.optimizing.OptimizingTaskMeta; import com.netease.arctic.server.persistence.PersistentBase; import com.netease.arctic.server.persistence.mapper.OptimizingMapper; -import com.netease.arctic.server.persistence.mapper.TableMetaMapper; import com.netease.arctic.server.table.ServerTableIdentifier; import com.netease.arctic.server.table.TableService; -import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.utils.ManifestEntryFields; import org.apache.commons.collections.CollectionUtils; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataOperations; -import org.apache.iceberg.HasTableOperations; -import org.apache.iceberg.HistoryEntry; -import org.apache.iceberg.MetadataTableType; -import org.apache.iceberg.MetadataTableUtils; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableMetadataParser; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.IcebergGenerics; -import org.apache.iceberg.data.InternalRecordWrapper; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.util.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.stream.Collectors; public class ServerTableDescriptor extends PersistentBase { - private static final Logger LOG = LoggerFactory.getLogger(ServerTableDescriptor.class); + private final Map formatDescriptorMap = new HashMap<>(); private final TableService tableService; public ServerTableDescriptor(TableService tableService) { this.tableService = tableService; + FormatTableDescriptor[] formatTableDescriptors = new FormatTableDescriptor[] { + new MixedAndIcebergTableDescriptor(), + new PaimonTableDescriptor() + }; + for (FormatTableDescriptor formatTableDescriptor : formatTableDescriptors) { + for (TableFormat format : formatTableDescriptor.supportFormat()) { + formatDescriptorMap.put(format, formatTableDescriptor); + } + } } - private ServerTableIdentifier getTable(String catalog, String db, String table) { - return getAs(TableMetaMapper.class, mapper -> mapper.selectTableIdentifier(catalog, db, table)); + public ServerTableMeta getTableDetail(ServerTableIdentifier tableIdentifier) { + AmoroTable amoroTable = tableService.loadTable(tableIdentifier); + FormatTableDescriptor formatTableDescriptor = formatDescriptorMap.get(amoroTable.format()); + return formatTableDescriptor.getTableDetail(amoroTable); } public List getTransactions(ServerTableIdentifier tableIdentifier) { - List transactionsOfTables = new ArrayList<>(); - ArcticTable arcticTable = tableService.loadTable(tableIdentifier); - List
tables = new ArrayList<>(); - if (arcticTable.isKeyedTable()) { - tables.add(arcticTable.asKeyedTable().changeTable()); - tables.add(arcticTable.asKeyedTable().baseTable()); - } else { - tables.add(arcticTable.asUnkeyedTable()); - } - tables.forEach(table -> table.snapshots().forEach(snapshot -> { - if (snapshot.operation().equals(DataOperations.REPLACE)) { - return; - } - if (snapshot.summary().containsKey(SnapshotSummary.TRANSACTION_BEGIN_SIGNATURE)) { - return; - } - TransactionsOfTable transactionsOfTable = new TransactionsOfTable(); - transactionsOfTable.setTransactionId(snapshot.snapshotId()); - int fileCount = PropertyUtil - .propertyAsInt(snapshot.summary(), org.apache.iceberg.SnapshotSummary.ADDED_FILES_PROP, 0); - fileCount += PropertyUtil - .propertyAsInt(snapshot.summary(), org.apache.iceberg.SnapshotSummary.ADDED_DELETE_FILES_PROP, 0); - fileCount += PropertyUtil - .propertyAsInt(snapshot.summary(), org.apache.iceberg.SnapshotSummary.DELETED_FILES_PROP, 0); - fileCount += PropertyUtil - .propertyAsInt(snapshot.summary(), org.apache.iceberg.SnapshotSummary.REMOVED_DELETE_FILES_PROP, 0); - transactionsOfTable.setFileCount(fileCount); - transactionsOfTable.setFileSize(PropertyUtil - .propertyAsLong(snapshot.summary(), org.apache.iceberg.SnapshotSummary.ADDED_FILE_SIZE_PROP, 0) + - PropertyUtil - .propertyAsLong(snapshot.summary(), org.apache.iceberg.SnapshotSummary.REMOVED_FILE_SIZE_PROP, 0)); - transactionsOfTable.setCommitTime(snapshot.timestampMillis()); - transactionsOfTable.setOperation(snapshot.operation()); - transactionsOfTable.setSummary(snapshot.summary()); - transactionsOfTables.add(transactionsOfTable); - })); - transactionsOfTables.sort((o1, o2) -> Long.compare(o2.commitTime, o1.commitTime)); - return transactionsOfTables; + AmoroTable amoroTable = tableService.loadTable(tableIdentifier); + FormatTableDescriptor formatTableDescriptor = formatDescriptorMap.get(amoroTable.format()); + return formatTableDescriptor.getTransactions(amoroTable); } public List getTransactionDetail(ServerTableIdentifier tableIdentifier, long transactionId) { - List result = new ArrayList<>(); - ArcticTable arcticTable = tableService.loadTable(tableIdentifier); - Snapshot snapshot; - if (arcticTable.isKeyedTable()) { - snapshot = arcticTable.asKeyedTable().changeTable().snapshot(transactionId); - if (snapshot == null) { - snapshot = arcticTable.asKeyedTable().baseTable().snapshot(transactionId); - } - } else { - snapshot = arcticTable.asUnkeyedTable().snapshot(transactionId); - } - if (snapshot == null) { - throw new IllegalArgumentException("unknown snapshot " + transactionId + " of " + tableIdentifier); - } - final long snapshotTime = snapshot.timestampMillis(); - String commitId = String.valueOf(transactionId); - snapshot.addedDataFiles(arcticTable.io()).forEach(f -> { - result.add(new PartitionFileBaseInfo( - commitId, - DataFileType.ofContentId(f.content().id()), - snapshotTime, - arcticTable.spec().partitionToPath(f.partition()), - f.path().toString(), - f.fileSizeInBytes(), - "add")); - }); - snapshot.removedDataFiles(arcticTable.io()).forEach(f -> { - result.add(new PartitionFileBaseInfo( - commitId, - DataFileType.ofContentId(f.content().id()), - snapshotTime, - arcticTable.spec().partitionToPath(f.partition()), - f.path().toString(), - f.fileSizeInBytes(), - "remove")); - }); - snapshot.addedDeleteFiles(arcticTable.io()).forEach(f -> { - result.add(new PartitionFileBaseInfo( - commitId, - DataFileType.ofContentId(f.content().id()), - snapshotTime, - arcticTable.spec().partitionToPath(f.partition()), - f.path().toString(), - f.fileSizeInBytes(), - "add")); - }); - snapshot.removedDeleteFiles(arcticTable.io()).forEach(f -> { - result.add(new PartitionFileBaseInfo( - commitId, - DataFileType.ofContentId(f.content().id()), - snapshotTime, - arcticTable.spec().partitionToPath(f.partition()), - f.path().toString(), - f.fileSizeInBytes(), - "remove")); - }); - return result; + AmoroTable amoroTable = tableService.loadTable(tableIdentifier); + FormatTableDescriptor formatTableDescriptor = formatDescriptorMap.get(amoroTable.format()); + return formatTableDescriptor.getTransactionDetail(amoroTable, transactionId); } public List getTableOperations(ServerTableIdentifier tableIdentifier) { - List result = new ArrayList<>(); - ArcticTable arcticTable = tableService.loadTable(tableIdentifier); - Table table; - if (arcticTable.isKeyedTable()) { - table = arcticTable.asKeyedTable().baseTable(); - } else { - table = arcticTable.asUnkeyedTable(); - } - List snapshotLog = ((HasTableOperations) table).operations().current().snapshotLog(); - List metadataLogEntries = - ((HasTableOperations) table).operations().current().previousFiles(); - Set time = new HashSet<>(); - snapshotLog.forEach(e -> time.add(e.timestampMillis())); - String lastMetadataLogEntryFile = null; - org.apache.iceberg.TableMetadata lastTableMetadata = null; - for (int i = 1; i < metadataLogEntries.size(); i++) { - org.apache.iceberg.TableMetadata.MetadataLogEntry currentEntry = metadataLogEntries.get(i); - if (!time.contains(currentEntry.timestampMillis())) { - org.apache.iceberg.TableMetadata.MetadataLogEntry previousEntry = metadataLogEntries.get(i - 1); - org.apache.iceberg.TableMetadata oldTableMetadata; - if (lastMetadataLogEntryFile == null || !lastMetadataLogEntryFile.equals(previousEntry.file())) { - oldTableMetadata = TableMetadataParser.read(table.io(), previousEntry.file()); - } else { - oldTableMetadata = lastTableMetadata; - } - - org.apache.iceberg.TableMetadata - newTableMetadata = TableMetadataParser.read(table.io(), currentEntry.file()); - lastMetadataLogEntryFile = currentEntry.file(); - lastTableMetadata = newTableMetadata; - - DDLInfo.Generator generator = new DDLInfo.Generator(); - result.addAll(generator.tableIdentify(arcticTable.id()) - .oldMeta(oldTableMetadata) - .newMeta(newTableMetadata) - .generate()); - } - } - if (metadataLogEntries.size() > 0) { - org.apache.iceberg.TableMetadata.MetadataLogEntry previousEntry = metadataLogEntries - .get(metadataLogEntries.size() - 1); - org.apache.iceberg.TableMetadata oldTableMetadata; + AmoroTable amoroTable = tableService.loadTable(tableIdentifier); + FormatTableDescriptor formatTableDescriptor = formatDescriptorMap.get(amoroTable.format()); + return formatTableDescriptor.getTableOperations(amoroTable); + } - if (lastMetadataLogEntryFile == null || !lastMetadataLogEntryFile.equals(previousEntry.file())) { - oldTableMetadata = TableMetadataParser.read(table.io(), previousEntry.file()); - } else { - oldTableMetadata = lastTableMetadata; - } + public List getTablePartition(ServerTableIdentifier tableIdentifier) { + AmoroTable amoroTable = tableService.loadTable(tableIdentifier); + FormatTableDescriptor formatTableDescriptor = formatDescriptorMap.get(amoroTable.format()); + return formatTableDescriptor.getTablePartition(amoroTable); + } - org.apache.iceberg.TableMetadata newTableMetadata = ((HasTableOperations) table).operations().current(); - DDLInfo.Generator generator = new DDLInfo.Generator(); - result.addAll(generator.tableIdentify(arcticTable.id()) - .oldMeta(oldTableMetadata) - .newMeta(newTableMetadata) - .generate()); - } - return result; + public List getTableFile(ServerTableIdentifier tableIdentifier, String partition) { + AmoroTable amoroTable = tableService.loadTable(tableIdentifier); + FormatTableDescriptor formatTableDescriptor = formatDescriptorMap.get(amoroTable.format()); + return formatTableDescriptor.getTableFile(amoroTable, partition); } public List getOptimizingProcesses(String catalog, String db, String table) { @@ -228,12 +82,6 @@ public List getOptimizingProcesses(String catalog, String mapper -> mapper.selectOptimizingProcesses(catalog, db, table)); } - public List getOptimizingTasks(long processId) { - return getAs( - OptimizingMapper.class, - mapper -> mapper.selectOptimizeTaskMetas(Collections.singletonList(processId))); - } - public List getOptimizingTasks(List processMetaList) { if (CollectionUtils.isEmpty(processMetaList)) { return Collections.emptyList(); @@ -244,75 +92,4 @@ public List getOptimizingTasks(List p OptimizingMapper.class, mapper -> mapper.selectOptimizeTaskMetas(processIds)); } - - public List getTablePartition(ArcticTable arcticTable) { - if (arcticTable.spec().isUnpartitioned()) { - return new ArrayList<>(); - } - Map partitionBaseInfoHashMap = new HashMap<>(); - getTableFile(arcticTable, null).forEach(fileInfo -> { - if (!partitionBaseInfoHashMap.containsKey(fileInfo.getPartition())) { - partitionBaseInfoHashMap.put(fileInfo.getPartition(), new PartitionBaseInfo()); - partitionBaseInfoHashMap.get(fileInfo.getPartition()).setPartition(fileInfo.getPartition()); - } - PartitionBaseInfo partitionInfo = partitionBaseInfoHashMap.get(fileInfo.getPartition()); - partitionInfo.setFileCount(partitionInfo.getFileCount() + 1); - partitionInfo.setFileSize(partitionInfo.getFileSize() + fileInfo.getFileSize()); - partitionInfo.setLastCommitTime(partitionInfo.getLastCommitTime() > fileInfo.getCommitTime() ? - partitionInfo.getLastCommitTime() : - fileInfo.getCommitTime()); - }); - - return new ArrayList<>(partitionBaseInfoHashMap.values()); - } - - public List getTableFile(ArcticTable arcticTable, String partition) { - List result = new ArrayList<>(); - if (arcticTable.isKeyedTable()) { - result.addAll(collectFileInfo(arcticTable.asKeyedTable().changeTable(), true, partition)); - result.addAll(collectFileInfo(arcticTable.asKeyedTable().baseTable(), false, partition)); - } else { - result.addAll(collectFileInfo(arcticTable.asUnkeyedTable(), false, partition)); - } - return result; - } - - private List collectFileInfo(Table table, boolean isChangeTable, String partition) { - PartitionSpec spec = table.spec(); - List result = new ArrayList<>(); - Table entriesTable = MetadataTableUtils.createMetadataTableInstance(((HasTableOperations) table).operations(), - table.name(), table.name() + "#ENTRIES", - MetadataTableType.ENTRIES); - try (CloseableIterable manifests = IcebergGenerics.read(entriesTable) - .where(Expressions.notEqual(ManifestEntryFields.STATUS.name(), ManifestEntryFields.Status.DELETED.id())) - .build()) { - for (Record record : manifests) { - long snapshotId = (long) record.getField(ManifestEntryFields.SNAPSHOT_ID.name()); - GenericRecord dataFile = (GenericRecord) record.getField(ManifestEntryFields.DATA_FILE_FIELD_NAME); - Integer contentId = (Integer) dataFile.getField(DataFile.CONTENT.name()); - String filePath = (String) dataFile.getField(DataFile.FILE_PATH.name()); - String partitionPath = null; - GenericRecord parRecord = (GenericRecord) dataFile.getField(DataFile.PARTITION_NAME); - if (parRecord != null) { - InternalRecordWrapper wrapper = new InternalRecordWrapper(parRecord.struct()); - partitionPath = spec.partitionToPath(wrapper.wrap(parRecord)); - } - if (partition != null && spec.isPartitioned() && !partition.equals(partitionPath)) { - continue; - } - Long fileSize = (Long) dataFile.getField(DataFile.FILE_SIZE.name()); - DataFileType dataFileType = - isChangeTable ? FileNameRules.parseFileTypeForChange(filePath) : DataFileType.ofContentId(contentId); - long commitTime = -1; - if (table.snapshot(snapshotId) != null) { - commitTime = table.snapshot(snapshotId).timestampMillis(); - } - result.add(new PartitionFileBaseInfo(String.valueOf(snapshotId), dataFileType, commitTime, - partitionPath, filePath, fileSize)); - } - } catch (IOException exception) { - LOG.error("close manifest file error", exception); - } - return result; - } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/dashboard/controller/TableController.java b/ams/server/src/main/java/com/netease/arctic/server/dashboard/controller/TableController.java index 141e6d10cd..a3fa432180 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/dashboard/controller/TableController.java +++ b/ams/server/src/main/java/com/netease/arctic/server/dashboard/controller/TableController.java @@ -21,7 +21,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.Constants; -import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.catalog.CatalogLoader; import com.netease.arctic.hive.HiveTableProperties; import com.netease.arctic.hive.catalog.ArcticHiveCatalog; @@ -34,19 +33,15 @@ import com.netease.arctic.server.dashboard.ServerTableDescriptor; import com.netease.arctic.server.dashboard.ServerTableProperties; import com.netease.arctic.server.dashboard.model.AMSColumnInfo; -import com.netease.arctic.server.dashboard.model.AMSPartitionField; import com.netease.arctic.server.dashboard.model.AMSTransactionsOfTable; import com.netease.arctic.server.dashboard.model.DDLInfo; -import com.netease.arctic.server.dashboard.model.FilesStatistics; import com.netease.arctic.server.dashboard.model.HiveTableInfo; import com.netease.arctic.server.dashboard.model.OptimizingProcessInfo; import com.netease.arctic.server.dashboard.model.PartitionBaseInfo; import com.netease.arctic.server.dashboard.model.PartitionFileBaseInfo; import com.netease.arctic.server.dashboard.model.ServerTableMeta; -import com.netease.arctic.server.dashboard.model.TableBasicInfo; import com.netease.arctic.server.dashboard.model.TableMeta; import com.netease.arctic.server.dashboard.model.TableOperation; -import com.netease.arctic.server.dashboard.model.TableStatistics; import com.netease.arctic.server.dashboard.model.TransactionsOfTable; import com.netease.arctic.server.dashboard.model.UpgradeHiveMeta; import com.netease.arctic.server.dashboard.model.UpgradeRunningInfo; @@ -55,25 +50,18 @@ import com.netease.arctic.server.dashboard.response.PageResult; import com.netease.arctic.server.dashboard.utils.AmsUtil; import com.netease.arctic.server.dashboard.utils.CommonUtil; -import com.netease.arctic.server.dashboard.utils.TableStatCollector; import com.netease.arctic.server.optimizing.OptimizingProcessMeta; import com.netease.arctic.server.optimizing.OptimizingTaskMeta; import com.netease.arctic.server.table.ServerTableIdentifier; import com.netease.arctic.server.table.TableService; import com.netease.arctic.server.utils.Configurations; -import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.table.KeyedTable; -import com.netease.arctic.table.PrimaryKeySpec; import com.netease.arctic.table.TableIdentifier; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.table.UnkeyedTable; import io.javalin.http.Context; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.PropertyUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -133,57 +121,9 @@ public void getTableDetail(Context ctx) { "catalog.database.tableName can not be empty in any element"); Preconditions.checkState(tableService.catalogExist(catalog), "invalid catalog!"); - ArcticTable table = tableService.loadTable(ServerTableIdentifier.of(catalog, database, tableMame)); - // set basic info - TableBasicInfo tableBasicInfo = getTableBasicInfo(table); - ServerTableMeta serverTableMeta = getServerTableMeta(table); - long tableSize = 0; - long tableFileCnt = 0; - Map baseMetrics = Maps.newHashMap(); - FilesStatistics baseFilesStatistics = tableBasicInfo.getBaseStatistics().getTotalFilesStat(); - Map baseSummary = tableBasicInfo.getBaseStatistics().getSummary(); - baseMetrics.put("lastCommitTime", AmsUtil.longOrNull(baseSummary.get("visibleTime"))); - baseMetrics.put("totalSize", AmsUtil.byteToXB(baseFilesStatistics.getTotalSize())); - baseMetrics.put("fileCount", baseFilesStatistics.getFileCnt()); - baseMetrics.put("averageFileSize", AmsUtil.byteToXB(baseFilesStatistics.getAverageSize())); - if (tableBasicInfo.getChangeStatistics() == null) { - baseMetrics.put("baseWatermark", AmsUtil.longOrNull(serverTableMeta.getTableWatermark())); - } else { - baseMetrics.put("baseWatermark", AmsUtil.longOrNull(serverTableMeta.getBaseWatermark())); - } - tableSize += baseFilesStatistics.getTotalSize(); - tableFileCnt += baseFilesStatistics.getFileCnt(); - serverTableMeta.setBaseMetrics(baseMetrics); - - Map changeMetrics = Maps.newHashMap(); - if (tableBasicInfo.getChangeStatistics() != null) { - FilesStatistics changeFilesStatistics = tableBasicInfo.getChangeStatistics().getTotalFilesStat(); - Map changeSummary = tableBasicInfo.getChangeStatistics().getSummary(); - changeMetrics.put("lastCommitTime", AmsUtil.longOrNull(changeSummary.get("visibleTime"))); - changeMetrics.put("totalSize", AmsUtil.byteToXB(changeFilesStatistics.getTotalSize())); - changeMetrics.put("fileCount", changeFilesStatistics.getFileCnt()); - changeMetrics.put("averageFileSize", AmsUtil.byteToXB(changeFilesStatistics.getAverageSize())); - changeMetrics.put("tableWatermark", AmsUtil.longOrNull(serverTableMeta.getTableWatermark())); - tableSize += changeFilesStatistics.getTotalSize(); - tableFileCnt += changeFilesStatistics.getFileCnt(); - } else { - changeMetrics.put("lastCommitTime", null); - changeMetrics.put("totalSize", null); - changeMetrics.put("fileCount", null); - changeMetrics.put("averageFileSize", null); - changeMetrics.put("tableWatermark", null); - } - serverTableMeta.setChangeMetrics(changeMetrics); - Set tableFormats = - com.netease.arctic.utils.CatalogUtil.tableFormats(tableService.getCatalogMeta(catalog)); - Preconditions.checkArgument(tableFormats.size() == 1, "Catalog support only one table format now."); - TableFormat tableFormat = tableFormats.iterator().next(); - Map tableSummary = new HashMap<>(); - tableSummary.put("size", AmsUtil.byteToXB(tableSize)); - tableSummary.put("file", tableFileCnt); - tableSummary.put("averageFile", AmsUtil.byteToXB(tableFileCnt == 0 ? 0 : tableSize / tableFileCnt)); - tableSummary.put("tableFormat", AmsUtil.formatString(tableFormat.name())); - serverTableMeta.setTableSummary(tableSummary); + ServerTableMeta serverTableMeta = + tableDescriptor.getTableDetail(ServerTableIdentifier.of(catalog, database, tableMame)); + ctx.json(OkResponse.of(serverTableMeta)); } @@ -231,7 +171,8 @@ public void upgradeHiveTable(Context ctx) { UpgradeHiveMeta upgradeHiveMeta = ctx.bodyAsClass(UpgradeHiveMeta.class); ArcticHiveCatalog arcticHiveCatalog - = (ArcticHiveCatalog) CatalogLoader.load(String.join("/", + = (ArcticHiveCatalog) CatalogLoader.load(String.join( + "/", AmsUtil.getAMSThriftAddress(serviceConfig, Constants.THRIFT_TABLE_SERVICE_NAME), catalog)); @@ -387,8 +328,8 @@ public void getTablePartitions(Context ctx) { Integer page = ctx.queryParamAsClass("page", Integer.class).getOrDefault(1); Integer pageSize = ctx.queryParamAsClass("pageSize", Integer.class).getOrDefault(20); - ArcticTable arcticTable = tableService.loadTable(ServerTableIdentifier.of(catalog, db, table)); - List partitionBaseInfos = tableDescriptor.getTablePartition(arcticTable); + List partitionBaseInfos = tableDescriptor.getTablePartition( + ServerTableIdentifier.of(catalog, db, table)); int offset = (page - 1) * pageSize; PageResult amsPageResult = PageResult.of(partitionBaseInfos, offset, pageSize); @@ -408,8 +349,9 @@ public void getPartitionFileListInfo(Context ctx) { Integer page = ctx.queryParamAsClass("page", Integer.class).getOrDefault(1); Integer pageSize = ctx.queryParamAsClass("pageSize", Integer.class).getOrDefault(20); - ArcticTable arcticTable = tableService.loadTable(ServerTableIdentifier.of(catalog, db, table)); - List partitionFileBaseInfos = tableDescriptor.getTableFile(arcticTable, partition); + + List partitionFileBaseInfos = tableDescriptor.getTableFile( + ServerTableIdentifier.of(catalog, db, table), partition); int offset = (page - 1) * pageSize; PageResult amsPageResult = PageResult.of(partitionFileBaseInfos, offset, pageSize); @@ -517,106 +459,6 @@ public void getTableDetailTabToken(Context ctx) { ctx.json(OkResponse.of(signCal)); } - private TableBasicInfo getTableBasicInfo(ArcticTable table) { - try { - TableBasicInfo tableBasicInfo = new TableBasicInfo(); - tableBasicInfo.setTableIdentifier(table.id()); - TableStatistics changeInfo = null; - TableStatistics baseInfo; - - if (table.isUnkeyedTable()) { - UnkeyedTable unkeyedTable = table.asUnkeyedTable(); - baseInfo = new TableStatistics(); - TableStatCollector.fillTableStatistics(baseInfo, unkeyedTable, table); - } else if (table.isKeyedTable()) { - KeyedTable keyedTable = table.asKeyedTable(); - if (!PrimaryKeySpec.noPrimaryKey().equals(keyedTable.primaryKeySpec())) { - changeInfo = TableStatCollector.collectChangeTableInfo(keyedTable); - } - baseInfo = TableStatCollector.collectBaseTableInfo(keyedTable); - } else { - throw new IllegalStateException("unknown type of table"); - } - - tableBasicInfo.setChangeStatistics(changeInfo); - tableBasicInfo.setBaseStatistics(baseInfo); - tableBasicInfo.setTableStatistics(TableStatCollector.union(changeInfo, baseInfo)); - - long createTime - = PropertyUtil.propertyAsLong(table.properties(), TableProperties.TABLE_CREATE_TIME, - TableProperties.TABLE_CREATE_TIME_DEFAULT); - if (createTime != TableProperties.TABLE_CREATE_TIME_DEFAULT) { - if (tableBasicInfo.getTableStatistics() != null) { - if (tableBasicInfo.getTableStatistics().getSummary() == null) { - tableBasicInfo.getTableStatistics().setSummary(new HashMap<>()); - } else { - LOG.warn("{} summary is null", table.id()); - } - tableBasicInfo.getTableStatistics().getSummary() - .put("createTime", String.valueOf(createTime)); - } else { - LOG.warn("{} table statistics is null {}", table.id(), tableBasicInfo); - } - } - return tableBasicInfo; - } catch (Throwable t) { - LOG.error("{} failed to build table basic info", table.id(), t); - throw t; - } - } - - private ServerTableMeta getServerTableMeta(ArcticTable table) { - ServerTableMeta serverTableMeta = new ServerTableMeta(); - serverTableMeta.setTableType(table.format().toString()); - serverTableMeta.setTableIdentifier(table.id()); - serverTableMeta.setBaseLocation(table.location()); - fillTableProperties(serverTableMeta, table.properties()); - serverTableMeta.setPartitionColumnList(table - .spec() - .fields() - .stream() - .map(item -> AMSPartitionField.buildFromPartitionSpec(table.spec().schema(), item)) - .collect(Collectors.toList())); - serverTableMeta.setSchema(table - .schema() - .columns() - .stream() - .map(AMSColumnInfo::buildFromNestedField) - .collect(Collectors.toList())); - - serverTableMeta.setFilter(null); - LOG.debug("Table {} is keyedTable: {}", table.name(), table instanceof KeyedTable); - if (table.isKeyedTable()) { - KeyedTable kt = table.asKeyedTable(); - if (kt.primaryKeySpec() != null) { - serverTableMeta.setPkList(kt - .primaryKeySpec() - .fields() - .stream() - .map(item -> AMSColumnInfo.buildFromPartitionSpec(table.spec().schema(), item)) - .collect(Collectors.toList())); - } - } - if (serverTableMeta.getPkList() == null) { - serverTableMeta.setPkList(new ArrayList<>()); - } - return serverTableMeta; - } - - private void fillTableProperties( - ServerTableMeta serverTableMeta, - Map tableProperties) { - Map properties = com.google.common.collect.Maps.newHashMap(tableProperties); - serverTableMeta.setTableWatermark(properties.remove(TableProperties.WATERMARK_TABLE)); - serverTableMeta.setBaseWatermark(properties.remove(TableProperties.WATERMARK_BASE_STORE)); - serverTableMeta.setCreateTime(PropertyUtil.propertyAsLong(properties, TableProperties.TABLE_CREATE_TIME, - TableProperties.TABLE_CREATE_TIME_DEFAULT)); - properties.remove(TableProperties.TABLE_CREATE_TIME); - - TableProperties.READ_PROTECTED_PROPERTIES.forEach(properties::remove); - serverTableMeta.setProperties(properties); - } - private List transformHiveSchemaToAMSColumnInfo(List fields) { return fields.stream() .map(f -> { diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java index 0f3a4eb86e..0dd8ef83ba 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/OptimizingQueue.java @@ -3,6 +3,7 @@ import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; +import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.BlockableOperation; import com.netease.arctic.ams.api.OptimizerRegisterInfo; import com.netease.arctic.ams.api.OptimizingService; @@ -307,8 +308,10 @@ private void planTasks() { for (TableRuntime tableRuntime : scheduledTables) { LOG.debug("Planning table {}", tableRuntime.getTableIdentifier()); try { - ArcticTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); - OptimizingPlanner planner = new OptimizingPlanner(tableRuntime.refresh(table), table, + AmoroTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); + OptimizingPlanner planner = new OptimizingPlanner( + tableRuntime.refresh(table), + (ArcticTable) table.originalTable(), getAvailableCore()); if (tableRuntime.isBlocked(BlockableOperation.OPTIMIZE)) { LOG.info("{} optimize is blocked, continue", tableRuntime.getTableIdentifier()); @@ -552,7 +555,7 @@ public MetricsSummary getSummary() { } private UnKeyedTableCommit buildCommit() { - ArcticTable table = tableManager.loadTable(tableRuntime.getTableIdentifier()); + ArcticTable table = (ArcticTable) tableManager.loadTable(tableRuntime.getTableIdentifier()).originalTable(); if (table.isUnkeyedTable()) { return new UnKeyedTableCommit(targetSnapshotId, table, taskMap.values()); } else { diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/IcebergTableMaintainer.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/IcebergTableMaintainer.java new file mode 100644 index 0000000000..156fcfb250 --- /dev/null +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/IcebergTableMaintainer.java @@ -0,0 +1,423 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.server.optimizing.maintainer; + +import com.google.common.base.Strings; +import com.netease.arctic.io.ArcticFileIO; +import com.netease.arctic.io.PathInfo; +import com.netease.arctic.io.SupportsFileSystemOperations; +import com.netease.arctic.server.table.TableConfiguration; +import com.netease.arctic.server.table.TableRuntime; +import com.netease.arctic.server.utils.IcebergTableUtil; +import com.netease.arctic.table.TableProperties; +import com.netease.arctic.utils.CompatiblePropertyUtil; +import com.netease.arctic.utils.TableFileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ReachableFileUtil; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.FileInfo; +import org.apache.iceberg.io.SupportsPrefixOperations; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; +import java.util.stream.Stream; + +import static org.apache.iceberg.relocated.com.google.common.primitives.Longs.min; + +/** + * Table maintainer for iceberg tables. + */ +public class IcebergTableMaintainer implements TableMaintainer { + + private static final Logger LOG = LoggerFactory.getLogger(IcebergTableMaintainer.class); + + public static final String METADATA_FOLDER_NAME = "metadata"; + public static final String DATA_FOLDER_NAME = "data"; + public static final String FLINK_JOB_ID = "flink.job-id"; + + public static final String FLINK_MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + + protected Table table; + + public IcebergTableMaintainer(Table table) { + this.table = table; + } + + @Override + public void cleanOrphanFiles(TableRuntime tableRuntime) { + TableConfiguration tableConfiguration = tableRuntime.getTableConfiguration(); + + if (!tableConfiguration.isCleanOrphanEnabled()) { + return; + } + + long keepTime = tableConfiguration.getOrphanExistingMinutes() * 60 * 1000; + + cleanContentFiles(System.currentTimeMillis() - keepTime); + + //refresh + table.refresh(); + + // clear metadata files + cleanMetadata(System.currentTimeMillis() - keepTime); + + if (!tableConfiguration.isDeleteDanglingDeleteFilesEnabled()) { + return; + } + + //refresh + table.refresh(); + + // clear dangling delete files + cleanDanglingDeleteFiles(); + } + + public void expireSnapshots(TableRuntime tableRuntime) { + TableConfiguration tableConfiguration = tableRuntime.getTableConfiguration(); + if (!tableConfiguration.isExpireSnapshotEnabled()) { + return; + } + expireSnapshots(olderThanSnapshotNeedToExpire(tableRuntime), expireSnapshotNeedToExcludeFiles()); + } + + public void expireSnapshots(long mustOlderThan) { + expireSnapshots(olderThanSnapshotNeedToExpire(mustOlderThan), expireSnapshotNeedToExcludeFiles()); + } + + @VisibleForTesting + public void expireSnapshots(long olderThan, Set exclude) { + LOG.debug("start expire snapshots older than {}, the exclude is {}", olderThan, exclude); + final AtomicInteger toDeleteFiles = new AtomicInteger(0); + final AtomicInteger deleteFiles = new AtomicInteger(0); + Set parentDirectory = new HashSet<>(); + table.expireSnapshots() + .retainLast(1) + .expireOlderThan(olderThan) + .deleteWith(file -> { + try { + String filePath = TableFileUtil.getUriPath(file); + if (!exclude.contains(filePath) && !exclude.contains(new Path(filePath).getParent().toString())) { + arcticFileIO().deleteFile(file); + } + parentDirectory.add(new Path(file).getParent().toString()); + deleteFiles.incrementAndGet(); + } catch (Throwable t) { + LOG.warn("failed to delete file " + file, t); + } finally { + toDeleteFiles.incrementAndGet(); + } + }) + .cleanExpiredFiles(true) + .commit(); + if (arcticFileIO().supportFileSystemOperations()) { + parentDirectory.forEach(parent -> TableFileUtil.deleteEmptyDirectory(arcticFileIO(), parent, exclude)); + } + LOG.info("to delete {} files, success delete {} files", toDeleteFiles.get(), deleteFiles.get()); + } + + protected void cleanContentFiles(long lastTime) { + // For clean data files, should getRuntime valid files in the base store and the change store, so acquire in advance + // to prevent repeated acquisition + Set validFiles = orphanFileCleanNeedToExcludeFiles(); + LOG.info("{} start clean content files of change store", table.name()); + int deleteFilesCnt = clearInternalTableContentsFiles(lastTime, validFiles); + LOG.info("{} total delete {} files from change store", table.name(), deleteFilesCnt); + } + + protected void cleanMetadata(long lastTime) { + LOG.info("{} start clean metadata files", table.name()); + int deleteFilesCnt = clearInternalTableMetadata(lastTime); + LOG.info("{} total delete {} metadata files", table.name(), deleteFilesCnt); + } + + protected void cleanDanglingDeleteFiles() { + LOG.info("{} start delete dangling delete files", table.name()); + int danglingDeleteFilesCnt = clearInternalTableDanglingDeleteFiles(); + LOG.info("{} total delete {} dangling delete files", table.name(), danglingDeleteFilesCnt); + } + + protected long olderThanSnapshotNeedToExpire(TableRuntime tableRuntime) { + long optimizingSnapshotTime = fetchOptimizingSnapshotTime(table, tableRuntime); + return olderThanSnapshotNeedToExpire(optimizingSnapshotTime); + } + + protected long olderThanSnapshotNeedToExpire(long mustOlderThan) { + long baseSnapshotsKeepTime = CompatiblePropertyUtil.propertyAsLong( + table.properties(), + TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, + TableProperties.BASE_SNAPSHOT_KEEP_MINUTES_DEFAULT) * 60 * 1000; + // Latest checkpoint of flink need retain. If Flink does not continuously commit new snapshots, + // it can lead to issues with table partitions not expiring. + long latestFlinkCommitTime = fetchLatestFlinkCommittedSnapshotTime(table); + long olderThan = System.currentTimeMillis() - baseSnapshotsKeepTime; + return min(latestFlinkCommitTime, mustOlderThan, olderThan); + } + + protected Set expireSnapshotNeedToExcludeFiles() { + return Collections.emptySet(); + } + + protected Set orphanFileCleanNeedToExcludeFiles() { + return IcebergTableUtil.getAllContentFilePath(table); + } + + protected ArcticFileIO arcticFileIO() { + return (ArcticFileIO) table.io(); + } + + private int clearInternalTableContentsFiles(long lastTime, Set exclude) { + String dataLocation = table.location() + File.separator + DATA_FOLDER_NAME; + + try (ArcticFileIO io = arcticFileIO()) { + // listPrefix will not return the directory and the orphan file clean should clean the empty dir. + if (io.supportFileSystemOperations()) { + SupportsFileSystemOperations fio = io.asFileSystemIO(); + return deleteInvalidFilesInFs(fio, dataLocation, lastTime, exclude); + } else if (io.supportPrefixOperations()) { + SupportsPrefixOperations pio = io.asPrefixFileIO(); + return deleteInvalidFilesByPrefix(pio, dataLocation, lastTime, exclude); + } else { + LOG.warn(String.format( + "Table %s doesn't support a fileIo with listDirectory or listPrefix, so skip clear files.", + table.name() + )); + } + } + + return 0; + } + + private int clearInternalTableMetadata(long lastTime) { + Set validFiles = getValidMetadataFiles(table); + LOG.info("{} table getRuntime {} valid files", table.name(), validFiles.size()); + Pattern excludeFileNameRegex = getExcludeFileNameRegex(table); + LOG.info("{} table getRuntime exclude file name pattern {}", table.name(), excludeFileNameRegex); + String metadataLocation = table.location() + File.separator + METADATA_FOLDER_NAME; + LOG.info("start orphan files clean in {}", metadataLocation); + + try (ArcticFileIO io = arcticFileIO()) { + if (io.supportPrefixOperations()) { + SupportsPrefixOperations pio = io.asPrefixFileIO(); + return deleteInvalidMetadataFile(pio, metadataLocation, lastTime, validFiles, excludeFileNameRegex); + } else { + LOG.warn(String.format( + "Table %s doesn't support a fileIo with listDirectory or listPrefix, so skip clear files.", + table.name() + )); + } + } + return 0; + } + + private int clearInternalTableDanglingDeleteFiles() { + Set danglingDeleteFiles = IcebergTableUtil.getDanglingDeleteFiles(table); + if (danglingDeleteFiles.isEmpty()) { + return 0; + } + RewriteFiles rewriteFiles = table.newRewrite(); + rewriteFiles.rewriteFiles( + Collections.emptySet(), danglingDeleteFiles, + Collections.emptySet(), Collections.emptySet()); + try { + rewriteFiles.commit(); + } catch (ValidationException e) { + LOG.warn("Iceberg RewriteFiles commit failed on clear danglingDeleteFiles, but ignore", e); + return 0; + } + return danglingDeleteFiles.size(); + } + + /** + * When committing a snapshot, Flink will write a checkpoint id into the snapshot summary. + * The latest snapshot with checkpoint id should not be expired or the flink job can't recover from state. + * + * @param table - + * @return commit time of snapshot with the latest flink checkpointId in summary + */ + public static long fetchLatestFlinkCommittedSnapshotTime(Table table) { + long latestCommitTime = Long.MAX_VALUE; + for (Snapshot snapshot : table.snapshots()) { + if (snapshot.summary().containsKey(FLINK_MAX_COMMITTED_CHECKPOINT_ID)) { + latestCommitTime = snapshot.timestampMillis(); + } + } + return latestCommitTime; + } + + /** + * When optimizing tasks are not committed, the snapshot with which it planned should not be expired, since + * it will use the snapshot to check conflict when committing. + * + * @param table - table + * @return commit time of snapshot for optimizing + */ + public static long fetchOptimizingSnapshotTime(Table table, TableRuntime tableRuntime) { + if (tableRuntime.getOptimizingStatus().isProcessing()) { + long fromSnapshotId = tableRuntime.getOptimizingProcess().getTargetSnapshotId(); + + for (Snapshot snapshot : table.snapshots()) { + if (snapshot.snapshotId() == fromSnapshotId) { + return snapshot.timestampMillis(); + } + } + } + return Long.MAX_VALUE; + } + + private static int deleteInvalidFilesInFs( + SupportsFileSystemOperations fio, String location, long lastTime, Set excludes + ) { + if (!fio.exists(location)) { + return 0; + } + + int deleteCount = 0; + for (PathInfo p : fio.listDirectory(location)) { + String uriPath = TableFileUtil.getUriPath(p.location()); + if (p.isDirectory()) { + int deleted = deleteInvalidFilesInFs(fio, p.location(), lastTime, excludes); + deleteCount += deleted; + if (!p.location().endsWith(METADATA_FOLDER_NAME) && + !p.location().endsWith(DATA_FOLDER_NAME) && + p.createdAtMillis() < lastTime && + fio.isEmptyDirectory(p.location())) { + TableFileUtil.deleteEmptyDirectory(fio, p.location(), excludes); + } + } else { + String parentLocation = TableFileUtil.getParent(p.location()); + String parentUriPath = TableFileUtil.getUriPath(parentLocation); + if (!excludes.contains(uriPath) && + !excludes.contains(parentUriPath) && + p.createdAtMillis() < lastTime) { + fio.deleteFile(p.location()); + deleteCount += 1; + } + } + } + return deleteCount; + } + + private static int deleteInvalidFilesByPrefix( + SupportsPrefixOperations pio, String prefix, long lastTime, Set excludes + ) { + int deleteCount = 0; + for (FileInfo fileInfo : pio.listPrefix(prefix)) { + String uriPath = TableFileUtil.getUriPath(fileInfo.location()); + if (!excludes.contains(uriPath) && fileInfo.createdAtMillis() < lastTime) { + pio.deleteFile(fileInfo.location()); + deleteCount += 1; + } + } + return deleteCount; + } + + private static Set getValidMetadataFiles(Table internalTable) { + String tableName = internalTable.name(); + Set validFiles = new HashSet<>(); + Iterable snapshots = internalTable.snapshots(); + int size = Iterables.size(snapshots); + LOG.info("{} getRuntime {} snapshots to scan", tableName, size); + int cnt = 0; + for (Snapshot snapshot : snapshots) { + cnt++; + int before = validFiles.size(); + String manifestListLocation = snapshot.manifestListLocation(); + + validFiles.add(TableFileUtil.getUriPath(manifestListLocation)); + + // valid data files + List manifestFiles = snapshot.allManifests(internalTable.io()); + for (ManifestFile manifestFile : manifestFiles) { + validFiles.add(TableFileUtil.getUriPath(manifestFile.path())); + } + + LOG.info( + "{} scan snapshot {}: {} and getRuntime {} files, complete {}/{}", + tableName, + snapshot.snapshotId(), + formatTime(snapshot.timestampMillis()), + validFiles.size() - before, + cnt, + size); + } + Stream.of( + ReachableFileUtil.metadataFileLocations(internalTable, false).stream(), + ReachableFileUtil.statisticsFilesLocations(internalTable).stream(), + Stream.of(ReachableFileUtil.versionHintLocation(internalTable))) + .reduce(Stream::concat) + .orElse(Stream.empty()) + .map(TableFileUtil::getUriPath) + .forEach(validFiles::add); + + return validFiles; + } + + private static Pattern getExcludeFileNameRegex(Table table) { + String latestFlinkJobId = null; + for (Snapshot snapshot : table.snapshots()) { + String flinkJobId = snapshot.summary().get(FLINK_JOB_ID); + if (!Strings.isNullOrEmpty(flinkJobId)) { + latestFlinkJobId = flinkJobId; + } + } + if (latestFlinkJobId != null) { + // file name starting with flink.job-id should not be deleted + return Pattern.compile(latestFlinkJobId + ".*"); + } + return null; + } + + private static int deleteInvalidMetadataFile( + SupportsPrefixOperations pio, String location, long lastTime, Set exclude, Pattern excludeRegex + ) { + int count = 0; + for (FileInfo fileInfo : pio.listPrefix(location)) { + String uriPath = TableFileUtil.getUriPath(fileInfo.location()); + if (!exclude.contains(uriPath) && + fileInfo.createdAtMillis() < lastTime && + (excludeRegex == null || !excludeRegex.matcher( + TableFileUtil.getFileName(fileInfo.location())).matches())) { + pio.deleteFile(fileInfo.location()); + count += 1; + } + } + return count; + } + + private static String formatTime(long timestamp) { + return LocalDateTime.ofInstant(Instant.ofEpochMilli(timestamp), ZoneId.systemDefault()).toString(); + } +} diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/MixedTableMaintainer.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/MixedTableMaintainer.java new file mode 100644 index 0000000000..52c7e72e17 --- /dev/null +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/MixedTableMaintainer.java @@ -0,0 +1,311 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.server.optimizing.maintainer; + +import com.netease.arctic.IcebergFileEntry; +import com.netease.arctic.data.FileNameRules; +import com.netease.arctic.hive.utils.TableTypeUtil; +import com.netease.arctic.scan.TableEntriesScan; +import com.netease.arctic.server.table.TableRuntime; +import com.netease.arctic.server.utils.HiveLocationUtil; +import com.netease.arctic.server.utils.IcebergTableUtil; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.KeyedTable; +import com.netease.arctic.table.TableProperties; +import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.CompatiblePropertyUtil; +import com.netease.arctic.utils.TablePropertyUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.collections.MapUtils; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFiles; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.primitives.Longs; +import org.apache.iceberg.util.StructLikeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Table maintainer for mixed-iceberg and mixed-hive tables. + */ +public class MixedTableMaintainer implements TableMaintainer { + + private static final Logger LOG = LoggerFactory.getLogger(MixedTableMaintainer.class); + + private final ArcticTable arcticTable; + + private ChangeTableMaintainer changeMaintainer; + + private final BaseTableMaintainer baseMaintainer; + + private final Set changeFiles; + + private final Set baseFiles; + + private final Set hiveFiles; + + public MixedTableMaintainer(ArcticTable arcticTable) { + this.arcticTable = arcticTable; + if (arcticTable.isKeyedTable()) { + changeMaintainer = new ChangeTableMaintainer(arcticTable.asKeyedTable().changeTable()); + baseMaintainer = new BaseTableMaintainer(arcticTable.asKeyedTable().baseTable()); + changeFiles = IcebergTableUtil.getAllContentFilePath(arcticTable.asKeyedTable().changeTable()); + baseFiles = IcebergTableUtil.getAllContentFilePath(arcticTable.asKeyedTable().baseTable()); + } else { + baseMaintainer = new BaseTableMaintainer(arcticTable.asUnkeyedTable()); + changeFiles = new HashSet<>(); + baseFiles = IcebergTableUtil.getAllContentFilePath(arcticTable.asUnkeyedTable()); + } + + if (TableTypeUtil.isHive(arcticTable)) { + hiveFiles = HiveLocationUtil.getHiveLocation(arcticTable); + } else { + hiveFiles = new HashSet<>(); + } + } + + @Override + public void cleanOrphanFiles(TableRuntime tableRuntime) { + if (changeMaintainer != null) { + changeMaintainer.cleanOrphanFiles(tableRuntime); + } + baseMaintainer.cleanOrphanFiles(tableRuntime); + } + + @Override + public void expireSnapshots(TableRuntime tableRuntime) { + if (changeMaintainer != null) { + changeMaintainer.expireSnapshots(tableRuntime); + } + baseMaintainer.expireSnapshots(tableRuntime); + } + + protected void expireSnapshots(long mustOlderThan) { + if (changeMaintainer != null) { + changeMaintainer.expireSnapshots(mustOlderThan); + } + baseMaintainer.expireSnapshots(mustOlderThan); + } + + protected void cleanContentFiles(long lastTime) { + if (changeMaintainer != null) { + changeMaintainer.cleanContentFiles(lastTime); + } + baseMaintainer.cleanContentFiles(lastTime); + } + + protected void cleanMetadata(long lastTime) { + if (changeMaintainer != null) { + changeMaintainer.cleanMetadata(lastTime); + } + baseMaintainer.cleanMetadata(lastTime); + } + + protected void cleanDanglingDeleteFiles() { + if (changeMaintainer != null) { + changeMaintainer.cleanDanglingDeleteFiles(); + } + baseMaintainer.cleanDanglingDeleteFiles(); + } + + public ChangeTableMaintainer getChangeMaintainer() { + return changeMaintainer; + } + + public BaseTableMaintainer getBaseMaintainer() { + return baseMaintainer; + } + + @SafeVarargs + private final Set mergeSets(Set... sets) { + Set result = new HashSet<>(); + for (Set set : sets) { + result.addAll(set); + } + return result; + } + + public class ChangeTableMaintainer extends IcebergTableMaintainer { + + private static final int DATA_FILE_LIST_SPLIT = 3000; + + private final UnkeyedTable unkeyedTable; + + public ChangeTableMaintainer(UnkeyedTable unkeyedTable) { + super(unkeyedTable); + this.unkeyedTable = unkeyedTable; + } + + @Override + public Set orphanFileCleanNeedToExcludeFiles() { + return mergeSets(changeFiles, baseFiles, hiveFiles); + } + + @Override + public void expireSnapshots(TableRuntime tableRuntime) { + expireSnapshots(Long.MAX_VALUE); + } + + @Override + public void expireSnapshots(long mustOlderThan) { + long changeTTLPoint = getChangeTTLPoint(); + expireFiles(Longs.min(getChangeTTLPoint(), mustOlderThan)); + super.expireSnapshots(Longs.min(changeTTLPoint, mustOlderThan)); + } + + @Override + protected long olderThanSnapshotNeedToExpire(long mustOlderThan) { + long latestChangeFlinkCommitTime = fetchLatestFlinkCommittedSnapshotTime(unkeyedTable); + return Longs.min(latestChangeFlinkCommitTime, mustOlderThan); + } + + @Override + protected Set expireSnapshotNeedToExcludeFiles() { + return mergeSets(baseFiles, hiveFiles); + } + + public void expireFiles(long ttlPoint) { + List expiredDataFileEntries = getExpiredDataFileEntries(ttlPoint); + deleteChangeFile(expiredDataFileEntries); + } + + private long getChangeTTLPoint() { + return System.currentTimeMillis() - CompatiblePropertyUtil.propertyAsLong( + unkeyedTable.properties(), + TableProperties.CHANGE_DATA_TTL, + TableProperties.CHANGE_DATA_TTL_DEFAULT) * 60 * 1000; + } + + private List getExpiredDataFileEntries(long ttlPoint) { + TableEntriesScan entriesScan = TableEntriesScan.builder(unkeyedTable) + .includeFileContent(FileContent.DATA) + .build(); + List changeTTLFileEntries = new ArrayList<>(); + + try (CloseableIterable entries = entriesScan.entries()) { + entries.forEach(entry -> { + Snapshot snapshot = unkeyedTable.snapshot(entry.getSnapshotId()); + if (snapshot == null || snapshot.timestampMillis() < ttlPoint) { + changeTTLFileEntries.add(entry); + } + }); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close manifest entry scan of " + table.name(), e); + } + return changeTTLFileEntries; + } + + private void deleteChangeFile(List expiredDataFileEntries) { + KeyedTable keyedTable = arcticTable.asKeyedTable(); + if (CollectionUtils.isEmpty(expiredDataFileEntries)) { + return; + } + + StructLikeMap partitionMaxTransactionId = TablePropertyUtil.getPartitionOptimizedSequence(keyedTable); + if (MapUtils.isEmpty(partitionMaxTransactionId)) { + LOG.info("table {} not contains max transaction id", keyedTable.id()); + return; + } + + Map> partitionDataFileMap = expiredDataFileEntries.stream() + .collect(Collectors.groupingBy(entry -> + keyedTable.spec().partitionToPath(entry.getFile().partition()), Collectors.toList())); + + List changeDeleteFiles = new ArrayList<>(); + if (keyedTable.spec().isUnpartitioned()) { + List partitionDataFiles = + partitionDataFileMap.get(keyedTable.spec().partitionToPath( + expiredDataFileEntries.get(0).getFile().partition())); + + Long optimizedSequence = partitionMaxTransactionId.get(TablePropertyUtil.EMPTY_STRUCT); + if (optimizedSequence != null && CollectionUtils.isNotEmpty(partitionDataFiles)) { + changeDeleteFiles.addAll(partitionDataFiles.stream() + .filter(entry -> FileNameRules.parseChangeTransactionId( + entry.getFile().path().toString(), entry.getSequenceNumber()) <= optimizedSequence) + .map(entry -> (DataFile) entry.getFile()) + .collect(Collectors.toList())); + } + } else { + partitionMaxTransactionId.forEach((key, value) -> { + List partitionDataFiles = + partitionDataFileMap.get(keyedTable.spec().partitionToPath(key)); + + if (CollectionUtils.isNotEmpty(partitionDataFiles)) { + changeDeleteFiles.addAll(partitionDataFiles.stream() + .filter(entry -> FileNameRules.parseChangeTransactionId( + entry.getFile().path().toString(), entry.getSequenceNumber()) <= value) + .map(entry -> (DataFile) entry.getFile()) + .collect(Collectors.toList())); + } + }); + } + tryClearChangeFiles(changeDeleteFiles); + } + + private void tryClearChangeFiles(List changeFiles) { + if (CollectionUtils.isEmpty(changeFiles)) { + return; + } + try { + for (int startIndex = 0; startIndex < changeFiles.size(); startIndex += DATA_FILE_LIST_SPLIT) { + int end = Math.min(startIndex + DATA_FILE_LIST_SPLIT, changeFiles.size()); + List tableFiles = changeFiles.subList(startIndex, end); + LOG.info("{} delete {} change files", unkeyedTable.name(), tableFiles.size()); + if (!tableFiles.isEmpty()) { + DeleteFiles changeDelete = unkeyedTable.newDelete(); + changeFiles.forEach(changeDelete::deleteFile); + changeDelete.commit(); + } + LOG.info("{} change committed, delete {} files, complete {}/{}", unkeyedTable.name(), + tableFiles.size(), end, changeFiles.size()); + } + } catch (Throwable t) { + LOG.error(unkeyedTable.name() + " failed to delete change files, ignore", t); + } + } + } + + public class BaseTableMaintainer extends IcebergTableMaintainer { + + public BaseTableMaintainer(UnkeyedTable unkeyedTable) { + super(unkeyedTable); + } + + @Override + public Set orphanFileCleanNeedToExcludeFiles() { + return mergeSets(changeFiles, baseFiles, hiveFiles); + } + + @Override + protected Set expireSnapshotNeedToExcludeFiles() { + return mergeSets(changeFiles, hiveFiles); + } + } +} diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/TableMaintainer.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/TableMaintainer.java new file mode 100644 index 0000000000..abe9921610 --- /dev/null +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/maintainer/TableMaintainer.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.server.optimizing.maintainer; + +import com.netease.arctic.AmoroTable; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.server.table.TableRuntime; +import com.netease.arctic.table.ArcticTable; +import org.apache.iceberg.Table; + +/** + * API for maintaining table. + *

Includes: clean content files, clean metadata, clean dangling delete files, expire snapshots. + */ +public interface TableMaintainer { + + /** + * Clean table orphan files. Includes: data files, metadata files, dangling delete files. + */ + void cleanOrphanFiles(TableRuntime tableRuntime); + + /** + * Expire snapshots,The optimizing based on the snapshot that the current table relies + * on will not expire according to TableRuntime. + */ + void expireSnapshots(TableRuntime tableRuntime); + + static TableMaintainer ofTable(AmoroTable amoroTable) { + TableFormat format = amoroTable.format(); + if (format == TableFormat.MIXED_HIVE || format == TableFormat.MIXED_ICEBERG) { + return new MixedTableMaintainer((ArcticTable) amoroTable.originalTable()); + } else if (format == TableFormat.ICEBERG) { + return new IcebergTableMaintainer((Table) amoroTable.originalTable()); + } else { + throw new RuntimeException("Unsupported table type" + amoroTable.originalTable().getClass()); + } + } +} diff --git a/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/TableMetaMapper.java b/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/TableMetaMapper.java index eb74ac0401..2709f79637 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/TableMetaMapper.java +++ b/ams/server/src/main/java/com/netease/arctic/server/persistence/mapper/TableMetaMapper.java @@ -282,13 +282,14 @@ List selectTableIdentifiersByDb( @Delete("DELETE FROM table_runtime WHERE table_id = #{tableId}") void deleteOptimizingRuntime(@Param("tableId") long tableId); - @Insert("INSERT INTO table_runtime (table_id, catalog_name, db_name, table_name, current_snapshot_id," + + @Insert("INSERT INTO table_runtime (table_id, catalog_name, db_name, table_name, format, current_snapshot_id," + " current_change_snapshotId, last_optimized_snapshotId, last_optimized_change_snapshotId," + " last_major_optimizing_time, last_minor_optimizing_time," + " last_full_optimizing_time, optimizing_status, optimizing_status_start_time, optimizing_process_id," + " optimizer_group, table_config, pending_input) VALUES" + " (#{runtime.tableIdentifier.id}, #{runtime.tableIdentifier.catalog}," + - " #{runtime.tableIdentifier.database}, #{runtime.tableIdentifier.tableName},#{runtime.currentSnapshotId}," + + " #{runtime.tableIdentifier.database}, #{runtime.tableIdentifier.tableName}, #{runtime.format}, #{runtime" + + ".currentSnapshotId}," + " #{runtime.currentChangeSnapshotId}, #{runtime.lastOptimizedSnapshotId}," + " #{runtime.lastOptimizedChangeSnapshotId}, #{runtime.lastMajorOptimizingTime," + " typeHandler=com.netease.arctic.server.persistence.converter.Long2TsConverter}," + @@ -306,7 +307,7 @@ List selectTableIdentifiersByDb( " typeHandler=com.netease.arctic.server.persistence.converter.JsonObjectConverter})") void insertTableRuntime(@Param("runtime") TableRuntime runtime); - @Select("SELECT a.table_id, a.catalog_name, a.db_name, a.table_name, a.current_snapshot_id, a" + + @Select("SELECT a.table_id, a.catalog_name, a.db_name, a.table_name, a.format, a.current_snapshot_id, a" + ".current_change_snapshotId, a.last_optimized_snapshotId, a.last_optimized_change_snapshotId," + " a.last_major_optimizing_time, a.last_minor_optimizing_time, a.last_full_optimizing_time, a.optimizing_status," + " a.optimizing_status_start_time, a.optimizing_process_id," + @@ -318,6 +319,7 @@ List selectTableIdentifiersByDb( @Result(property = "catalogName", column = "catalog_name"), @Result(property = "dbName", column = "db_name"), @Result(property = "tableName", column = "table_name"), + @Result(property = "format", column = "format"), @Result(property = "currentSnapshotId", column = "current_snapshot_id"), @Result(property = "currentChangeSnapshotId", column = "current_change_snapshotId"), @Result(property = "lastOptimizedSnapshotId", column = "last_optimized_snapshotId"), diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/DefaultTableService.java b/ams/server/src/main/java/com/netease/arctic/server/table/DefaultTableService.java index 9945bb0ecc..c66710e903 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/DefaultTableService.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/DefaultTableService.java @@ -2,6 +2,7 @@ import com.google.common.base.Objects; import com.google.common.collect.Sets; +import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.BlockableOperation; import com.netease.arctic.ams.api.Blocker; import com.netease.arctic.ams.api.CatalogMeta; @@ -20,7 +21,6 @@ import com.netease.arctic.server.persistence.mapper.TableMetaMapper; import com.netease.arctic.server.table.blocker.TableBlocker; import com.netease.arctic.server.utils.Configurations; -import com.netease.arctic.table.ArcticTable; import org.apache.commons.lang.StringUtils; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -160,8 +160,8 @@ public void createTable(String catalogName, TableMetadata tableMetadata) { InternalCatalog catalog = getInternalCatalog(catalogName); ServerTableIdentifier tableIdentifier = catalog.createTable(tableMetadata); - ArcticTable table = catalog.loadTable(tableIdentifier.getDatabase(), tableIdentifier.getTableName()); - TableRuntime tableRuntime = new TableRuntime(tableIdentifier, this, table.properties()); + AmoroTable table = catalog.loadTable(tableIdentifier.getDatabase(), tableIdentifier.getTableName()); + TableRuntime tableRuntime = new TableRuntime(tableIdentifier, table.format(), this, table.properties()); tableRuntimeMap.put(tableIdentifier, tableRuntime); if (headHandler != null) { headHandler.fireTableAdded(table, tableRuntime); @@ -205,7 +205,7 @@ public void dropDatabase(String catalogName, String dbName) { } @Override - public ArcticTable loadTable(ServerTableIdentifier tableIdentifier) { + public AmoroTable loadTable(ServerTableIdentifier tableIdentifier) { checkStarted(); return getServerCatalog(tableIdentifier.getCatalog()) .loadTable(tableIdentifier.getDatabase(), tableIdentifier.getTableName()); @@ -454,10 +454,10 @@ private void syncTable(ExternalCatalog externalCatalog, TableIdentity tableIdent private void handleTableRuntimeAdded(ExternalCatalog externalCatalog, TableIdentity tableIdentity) { ServerTableIdentifier tableIdentifier = externalCatalog.getServerTableIdentifier(tableIdentity.getDatabase(), tableIdentity.getTableName()); - ArcticTable table = externalCatalog.loadTable( + AmoroTable table = externalCatalog.loadTable( tableIdentifier.getDatabase(), tableIdentifier.getTableName()); - TableRuntime tableRuntime = new TableRuntime(tableIdentifier, this, table.properties()); + TableRuntime tableRuntime = new TableRuntime(tableIdentifier, table.format(), this, table.properties()); tableRuntimeMap.put(tableIdentifier, tableRuntime); if (headHandler != null) { headHandler.fireTableAdded(table, tableRuntime); diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/RuntimeHandlerChain.java b/ams/server/src/main/java/com/netease/arctic/server/table/RuntimeHandlerChain.java index dda095a609..db4d440b95 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/RuntimeHandlerChain.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/RuntimeHandlerChain.java @@ -1,16 +1,18 @@ package com.netease.arctic.server.table; +import com.netease.arctic.AmoroTable; +import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.server.optimizing.OptimizingStatus; -import com.netease.arctic.table.ArcticTable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; public abstract class RuntimeHandlerChain { - + private static final Logger LOG = LoggerFactory.getLogger(RuntimeHandlerChain.class); private RuntimeHandlerChain next; @@ -29,7 +31,10 @@ protected void appendNext(RuntimeHandlerChain handler) { } public final void initialize(List tableRuntimeMetaList) { - initHandler(tableRuntimeMetaList); + List supportedtableRuntimeMetaList = tableRuntimeMetaList.stream() + .filter(tableRuntimeMeta -> formatSupported(tableRuntimeMeta.getTableRuntime().getFormat())) + .collect(Collectors.toList()); + initHandler(supportedtableRuntimeMetaList); initialized = true; if (next != null) { next.initialize(tableRuntimeMetaList); @@ -37,39 +42,55 @@ public final void initialize(List tableRuntimeMetaList) { } public final void fireStatusChanged(TableRuntime tableRuntime, OptimizingStatus originalStatus) { - if (!initialized) return; - - doSilently(() -> handleStatusChanged(tableRuntime, originalStatus)); + if (!initialized) { + return; + } + if (formatSupported(tableRuntime.getFormat())) { + doSilently(() -> handleStatusChanged(tableRuntime, originalStatus)); + } if (next != null) { next.fireStatusChanged(tableRuntime, originalStatus); } } public final void fireConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { - if (!initialized) return; + if (!initialized) { + return; + } - doSilently(() -> handleConfigChanged(tableRuntime, originalConfig)); + if (formatSupported(tableRuntime.getFormat())) { + doSilently(() -> handleConfigChanged(tableRuntime, originalConfig)); + } if (next != null) { next.fireConfigChanged(tableRuntime, originalConfig); } } - public final void fireTableAdded(ArcticTable table, TableRuntime tableRuntime) { - if (!initialized) return; + public final void fireTableAdded(AmoroTable table, TableRuntime tableRuntime) { + if (!initialized) { + return; + } - doSilently(() -> handleTableAdded(table, tableRuntime)); + if (formatSupported(tableRuntime.getFormat())) { + doSilently(() -> handleTableAdded(table, tableRuntime)); + } if (next != null) { next.fireTableAdded(table, tableRuntime); } } public final void fireTableRemoved(TableRuntime tableRuntime) { - if (!initialized) return; + if (!initialized) { + return; + } if (next != null) { next.fireTableRemoved(tableRuntime); } - doSilently(() -> handleTableRemoved(tableRuntime)); + + if (formatSupported(tableRuntime.getFormat())) { + doSilently(() -> handleTableRemoved(tableRuntime)); + } } public final void dispose() { @@ -87,11 +108,16 @@ private void doSilently(Runnable runnable) { } } + //Currently, paimon is unsupported + protected boolean formatSupported(TableFormat format) { + return format.in(TableFormat.ICEBERG, TableFormat.MIXED_ICEBERG, TableFormat.MIXED_HIVE); + } + protected abstract void handleStatusChanged(TableRuntime tableRuntime, OptimizingStatus originalStatus); protected abstract void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig); - protected abstract void handleTableAdded(ArcticTable table, TableRuntime tableRuntime); + protected abstract void handleTableAdded(AmoroTable table, TableRuntime tableRuntime); protected abstract void handleTableRemoved(TableRuntime tableRuntime); diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/TableManager.java b/ams/server/src/main/java/com/netease/arctic/server/table/TableManager.java index b97955d907..e114d93312 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/TableManager.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/TableManager.java @@ -1,10 +1,10 @@ package com.netease.arctic.server.table; -import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.AmoroTable; public interface TableManager extends TableRuntimeHandler { - ArcticTable loadTable(ServerTableIdentifier tableIdentifier); + AmoroTable loadTable(ServerTableIdentifier tableIdentifier); TableRuntime getRuntime(ServerTableIdentifier tableIdentifier); diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntime.java b/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntime.java index d54bea5946..cde39a33ab 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntime.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntime.java @@ -18,7 +18,9 @@ package com.netease.arctic.server.table; +import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.BlockableOperation; +import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.server.ArcticServiceConstants; import com.netease.arctic.server.exception.BlockerConflictException; import com.netease.arctic.server.exception.ObjectNotExistsException; @@ -61,6 +63,7 @@ public class TableRuntime extends StatedPersistentBase { private final TableRuntimeHandler tableHandler; private final ServerTableIdentifier tableIdentifier; + private final TableFormat format; private final List taskQuotas = Collections.synchronizedList(new ArrayList<>()); // for unKeyedTable or base table @@ -97,11 +100,12 @@ public class TableRuntime extends StatedPersistentBase { private final ReentrantLock blockerLock = new ReentrantLock(); protected TableRuntime( - ServerTableIdentifier tableIdentifier, TableRuntimeHandler tableHandler, + ServerTableIdentifier tableIdentifier, TableFormat format, TableRuntimeHandler tableHandler, Map properties) { Preconditions.checkNotNull(tableIdentifier, tableHandler); this.tableHandler = tableHandler; this.tableIdentifier = tableIdentifier; + this.format = format; this.tableConfiguration = TableConfiguration.parseConfig(properties); this.optimizerGroup = tableConfiguration.getOptimizingConfig().getOptimizerGroup(); persistTableRuntime(); @@ -125,6 +129,7 @@ protected TableRuntime(TableRuntimeMeta tableRuntimeMeta, TableRuntimeHandler ta this.processId = tableRuntimeMeta.getOptimizingProcessId(); this.optimizingStatus = tableRuntimeMeta.getTableStatus(); this.pendingInput = tableRuntimeMeta.getPendingInput(); + this.format = tableRuntimeMeta.getFormat(); } public void recover(OptimizingProcess optimizingProcess) { @@ -178,7 +183,7 @@ public void setPendingInput(OptimizingEvaluator.PendingInput pendingInput) { }); } - public TableRuntime refresh(ArcticTable table) { + public TableRuntime refresh(AmoroTable table) { return invokeConsisitency(() -> { TableConfiguration configuration = tableConfiguration; boolean configChanged = updateConfigInternal(table.properties()); @@ -242,7 +247,8 @@ private void updateOptimizingStatus(OptimizingStatus status) { this.currentStatusStartTime = System.currentTimeMillis(); } - private boolean refreshSnapshots(ArcticTable table) { + private boolean refreshSnapshots(AmoroTable amoroTable) { + ArcticTable table = (ArcticTable) amoroTable.originalTable(); if (table.isKeyedTable()) { long lastSnapshotId = currentSnapshotId; long changeSnapshotId = currentChangeSnapshotId; @@ -315,6 +321,10 @@ public ServerTableIdentifier getTableIdentifier() { return tableIdentifier; } + public TableFormat getFormat() { + return format; + } + public OptimizingStatus getOptimizingStatus() { return optimizingStatus; } diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntimeMeta.java b/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntimeMeta.java index e6025048f0..3969fb1320 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntimeMeta.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/TableRuntimeMeta.java @@ -1,5 +1,6 @@ package com.netease.arctic.server.table; +import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.server.optimizing.OptimizingProcess; import com.netease.arctic.server.optimizing.OptimizingStatus; import com.netease.arctic.server.optimizing.OptimizingType; @@ -12,6 +13,7 @@ public class TableRuntimeMeta { private String catalogName; private String dbName; private String tableName; + private TableFormat format; private long currentSnapshotId; private long lastOptimizedSnapshotId; private long lastOptimizedChangeSnapshotId; @@ -91,6 +93,10 @@ public String getTableName() { return tableName; } + public TableFormat getFormat() { + return format; + } + public long getCurrentSnapshotId() { return currentSnapshotId; } @@ -167,6 +173,10 @@ public void setTableName(String tableName) { this.tableName = tableName; } + public void setFormat(TableFormat format) { + this.format = format; + } + public void setCurrentSnapshotId(long currentSnapshotId) { this.currentSnapshotId = currentSnapshotId; } diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/executor/BaseTableExecutor.java b/ams/server/src/main/java/com/netease/arctic/server/table/executor/BaseTableExecutor.java index 1050521473..c5f7aa531a 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/executor/BaseTableExecutor.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/executor/BaseTableExecutor.java @@ -1,13 +1,13 @@ package com.netease.arctic.server.table.executor; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.netease.arctic.AmoroTable; import com.netease.arctic.server.optimizing.OptimizingStatus; import com.netease.arctic.server.table.RuntimeHandlerChain; import com.netease.arctic.server.table.TableConfiguration; import com.netease.arctic.server.table.TableManager; import com.netease.arctic.server.table.TableRuntime; import com.netease.arctic.server.table.TableRuntimeMeta; -import com.netease.arctic.table.ArcticTable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -97,7 +97,7 @@ public void handleStatusChanged(TableRuntime tableRuntime, OptimizingStatus orig } @Override - public void handleTableAdded(ArcticTable table, TableRuntime tableRuntime) { + public void handleTableAdded(AmoroTable table, TableRuntime tableRuntime) { scheduleIfNecessary(tableRuntime, getStartDelay()); } @@ -111,7 +111,7 @@ protected long getStartDelay() { return START_DELAY; } - protected ArcticTable loadTable(TableRuntime tableRuntime) { + protected AmoroTable loadTable(TableRuntime tableRuntime) { return tableManager.loadTable(tableRuntime.getTableIdentifier()); } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/executor/HiveCommitSyncExecutor.java b/ams/server/src/main/java/com/netease/arctic/server/table/executor/HiveCommitSyncExecutor.java index f03ef149f3..a8bdd15ca4 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/executor/HiveCommitSyncExecutor.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/executor/HiveCommitSyncExecutor.java @@ -35,7 +35,7 @@ protected void execute(TableRuntime tableRuntime) { long startTime = System.currentTimeMillis(); ServerTableIdentifier tableIdentifier = tableRuntime.getTableIdentifier(); try { - ArcticTable arcticTable = loadTable(tableRuntime); + ArcticTable arcticTable = (ArcticTable) loadTable(tableRuntime).originalTable(); if (!TableTypeUtil.isHive(arcticTable)) { LOG.debug("{} is not a support hive table", tableIdentifier); return; diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/executor/OptimizingCommitExecutor.java b/ams/server/src/main/java/com/netease/arctic/server/table/executor/OptimizingCommitExecutor.java index 317489ea21..dc8f684bdf 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/executor/OptimizingCommitExecutor.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/executor/OptimizingCommitExecutor.java @@ -1,9 +1,9 @@ package com.netease.arctic.server.table.executor; +import com.netease.arctic.AmoroTable; import com.netease.arctic.server.optimizing.OptimizingStatus; import com.netease.arctic.server.table.TableManager; import com.netease.arctic.server.table.TableRuntime; -import com.netease.arctic.table.ArcticTable; import java.util.Optional; @@ -38,7 +38,7 @@ public void handleStatusChanged(TableRuntime tableRuntime, OptimizingStatus orig } @Override - public void handleTableAdded(ArcticTable table, TableRuntime tableRuntime) { + public void handleTableAdded(AmoroTable table, TableRuntime tableRuntime) { } protected long getStartDelay() { diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/executor/OrphanFilesCleaningExecutor.java b/ams/server/src/main/java/com/netease/arctic/server/table/executor/OrphanFilesCleaningExecutor.java index aa524786e9..0622ab0c1d 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/executor/OrphanFilesCleaningExecutor.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/executor/OrphanFilesCleaningExecutor.java @@ -18,50 +18,20 @@ package com.netease.arctic.server.table.executor; -import com.google.common.base.Strings; -import com.netease.arctic.io.ArcticFileIO; -import com.netease.arctic.io.PathInfo; -import com.netease.arctic.io.SupportsFileSystemOperations; +import com.netease.arctic.AmoroTable; +import com.netease.arctic.server.optimizing.maintainer.TableMaintainer; import com.netease.arctic.server.table.TableConfiguration; import com.netease.arctic.server.table.TableManager; import com.netease.arctic.server.table.TableRuntime; -import com.netease.arctic.server.utils.HiveLocationUtil; -import com.netease.arctic.server.utils.IcebergTableUtil; -import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.table.KeyedTable; -import com.netease.arctic.table.TableIdentifier; -import com.netease.arctic.table.UnkeyedTable; -import com.netease.arctic.utils.TableFileUtil; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ReachableFileUtil; -import org.apache.iceberg.RewriteFiles; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.io.FileInfo; -import org.apache.iceberg.io.SupportsPrefixOperations; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.regex.Pattern; -import java.util.stream.Stream; +import static com.netease.arctic.server.optimizing.maintainer.TableMaintainer.ofTable; public class OrphanFilesCleaningExecutor extends BaseTableExecutor { private static final Logger LOG = LoggerFactory.getLogger(OrphanFilesCleaningExecutor.class); // same as org.apache.iceberg.flink.sink.IcebergFilesCommitter#FLINK_JOB_ID - public static final String FLINK_JOB_ID = "flink.job-id"; - public static final String METADATA_FOLDER_NAME = "metadata"; - public static final String DATA_FOLDER_NAME = "data"; - // 1 days + private static final long INTERVAL = 24 * 60 * 60 * 1000L; public OrphanFilesCleaningExecutor(TableManager tableRuntimes, int poolSize) { @@ -87,289 +57,13 @@ public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration or public void execute(TableRuntime tableRuntime) { try { LOG.info("{} start cleaning orphan files", tableRuntime.getTableIdentifier()); - TableConfiguration tableConfiguration = tableRuntime.getTableConfiguration(); - - if (!tableConfiguration.isCleanOrphanEnabled()) { - return; - } - - long keepTime = tableConfiguration.getOrphanExistingMinutes() * 60 * 1000; - // clear data files - ArcticTable arcticTable = loadTable(tableRuntime); - cleanContentFiles(arcticTable, System.currentTimeMillis() - keepTime); - - // it may cost a long time to clean content files, so refresh the table to the current snapshot before cleaning - // the metadata files - arcticTable.refresh(); - // clear metadata files - cleanMetadata(arcticTable, System.currentTimeMillis() - keepTime); - - if (!tableConfiguration.isDeleteDanglingDeleteFilesEnabled()) { - return; - } - // refresh to the current snapshot before clean dangling delete files - arcticTable.refresh(); - // clear dangling delete files - cleanDanglingDeleteFiles(arcticTable); + AmoroTable amoroTable = loadTable(tableRuntime); + TableMaintainer tableMaintainer = ofTable(amoroTable); + tableMaintainer.cleanOrphanFiles(tableRuntime); } catch (Throwable t) { LOG.error("{} failed to clean orphan file", tableRuntime.getTableIdentifier(), t); } } - - public static void cleanContentFiles(ArcticTable arcticTable, long lastTime) { - // For clean data files, should get valid files in the base store and the change store, so acquire in advance - // to prevent repeated acquisition - Set validFiles = getValidContentFiles(arcticTable); - if (arcticTable.isKeyedTable()) { - KeyedTable keyedArcticTable = arcticTable.asKeyedTable(); - LOG.info("{} start cleaning content files of base store", arcticTable.id()); - int deleteFilesCnt = clearInternalTableContentsFiles(keyedArcticTable.baseTable(), lastTime, validFiles); - LOG.info("{} deleted {} content files from base store", arcticTable.id(), deleteFilesCnt); - - LOG.info("{} start cleaning content files of change store", arcticTable.id()); - deleteFilesCnt = clearInternalTableContentsFiles(keyedArcticTable.changeTable(), lastTime, validFiles); - LOG.info("{} deleted {} content files from change store", arcticTable.id(), deleteFilesCnt); - } else { - LOG.info("{} start cleaning content files", arcticTable.id()); - int deleteFilesCnt = clearInternalTableContentsFiles(arcticTable.asUnkeyedTable(), lastTime, validFiles); - LOG.info("{} deleted {} content files", arcticTable.id(), deleteFilesCnt); - } - } - - public static void cleanMetadata(ArcticTable arcticTable, long lastTime) { - if (arcticTable.isKeyedTable()) { - KeyedTable keyedArcticTable = arcticTable.asKeyedTable(); - LOG.info("{} start cleaning metadata files of base store", arcticTable.id()); - int deleteFilesCnt = clearInternalTableMetadata(keyedArcticTable.baseTable(), lastTime); - LOG.info("{} deleted {} metadata files from base store", arcticTable.id(), deleteFilesCnt); - - LOG.info("{} start cleaning metadata files of change store", arcticTable.id()); - deleteFilesCnt = clearInternalTableMetadata(keyedArcticTable.changeTable(), lastTime); - LOG.info("{} deleted {} metadata files from change store", arcticTable.id(), deleteFilesCnt); - } else { - LOG.info("{} start cleaning metadata files", arcticTable.id()); - int deleteFilesCnt = clearInternalTableMetadata(arcticTable.asUnkeyedTable(), lastTime); - LOG.info("{} deleted {} metadata files", arcticTable.id(), deleteFilesCnt); - } - } - - public static void cleanDanglingDeleteFiles(ArcticTable arcticTable) { - if (!arcticTable.isKeyedTable()) { - LOG.info("{} start deleting dangling delete files", arcticTable.id()); - int danglingDeleteFilesCnt = clearInternalTableDanglingDeleteFiles(arcticTable.asUnkeyedTable()); - LOG.info("{} deleted {} dangling delete files", arcticTable.id(), danglingDeleteFilesCnt); - } - } - - private static Set getValidContentFiles(ArcticTable arcticTable) { - Set validFiles = new HashSet<>(); - if (arcticTable.isKeyedTable()) { - Set baseValidFiles = IcebergTableUtil.getAllContentFilePath(arcticTable.asKeyedTable().baseTable()); - LOG.info("{} found {} valid files in the base store", arcticTable.id(), baseValidFiles.size()); - Set changeValidFiles = IcebergTableUtil.getAllContentFilePath(arcticTable.asKeyedTable().changeTable()); - LOG.info("{} found {} valid files in the change store", arcticTable.id(), baseValidFiles.size()); - validFiles.addAll(baseValidFiles); - validFiles.addAll(changeValidFiles); - } else { - Set baseValidFiles = IcebergTableUtil.getAllContentFilePath(arcticTable.asUnkeyedTable()); - validFiles.addAll(baseValidFiles); - LOG.info("{} found {} valid files", arcticTable.id(), validFiles.size()); - } - - // add hive location to exclude - Set hiveValidLocations = HiveLocationUtil.getHiveLocation(arcticTable); - if (hiveValidLocations.size() > 0) { - validFiles.addAll(hiveValidLocations); - LOG.info("{} found {} valid locations in the Hive location", arcticTable.id(), hiveValidLocations.size()); - } - - return validFiles; - } - - private static int clearInternalTableContentsFiles( - UnkeyedTable internalTable, long lastTime, Set exclude) { - String dataLocation = internalTable.location() + File.separator + DATA_FOLDER_NAME; - - try (ArcticFileIO io = internalTable.io()) { - // listPrefix will not return the directory and the orphan file clean should clean the empty dir. - if (io.supportFileSystemOperations()) { - SupportsFileSystemOperations fio = io.asFileSystemIO(); - return deleteInvalidFilesInFs(fio, dataLocation, lastTime, exclude); - } else if (io.supportPrefixOperations()) { - SupportsPrefixOperations pio = io.asPrefixFileIO(); - return deleteInvalidFilesByPrefix(pio, dataLocation, lastTime, exclude); - } else { - LOG.warn(String.format( - "Table %s doesn't support a fileIo with listDirectory or listPrefix, so skip clear files.", - internalTable.name() - )); - } - } - - return 0; - } - - private static int deleteInvalidFilesInFs( - SupportsFileSystemOperations fio, String location, long lastTime, Set excludes - ) { - if (!fio.exists(location)) { - return 0; - } - - int deleteCount = 0; - for (PathInfo p : fio.listDirectory(location)) { - String uriPath = TableFileUtil.getUriPath(p.location()); - if (p.isDirectory()) { - int deleted = deleteInvalidFilesInFs(fio, p.location(), lastTime, excludes); - deleteCount += deleted; - if (!p.location().endsWith(METADATA_FOLDER_NAME) && - !p.location().endsWith(DATA_FOLDER_NAME) && - p.createdAtMillis() < lastTime && - fio.isEmptyDirectory(p.location())) { - TableFileUtil.deleteEmptyDirectory(fio, p.location(), excludes); - } - } else { - String parentLocation = TableFileUtil.getParent(p.location()); - String parentUriPath = TableFileUtil.getUriPath(parentLocation); - if (!excludes.contains(uriPath) && - !excludes.contains(parentUriPath) && - p.createdAtMillis() < lastTime) { - fio.deleteFile(p.location()); - deleteCount += 1; - } - } - } - return deleteCount; - } - - private static int deleteInvalidFilesByPrefix( - SupportsPrefixOperations pio, String prefix, long lastTime, Set excludes - ) { - int deleteCount = 0; - for (FileInfo fileInfo : pio.listPrefix(prefix)) { - String uriPath = TableFileUtil.getUriPath(fileInfo.location()); - if (!excludes.contains(uriPath) && fileInfo.createdAtMillis() < lastTime) { - pio.deleteFile(fileInfo.location()); - deleteCount += 1; - } - } - return deleteCount; - } - - private static int clearInternalTableMetadata(UnkeyedTable internalTable, long lastTime) { - Set validFiles = getValidMetadataFiles(internalTable); - LOG.info("{} found {} valid metadata files", internalTable.id(), validFiles.size()); - Pattern excludeFileNameRegex = getExcludeFileNameRegex(internalTable); - LOG.info("{} generated exclude file name pattern {}", internalTable.id(), excludeFileNameRegex); - String metadataLocation = internalTable.location() + File.separator + METADATA_FOLDER_NAME; - - try (ArcticFileIO io = internalTable.io()) { - if (io.supportPrefixOperations()) { - SupportsPrefixOperations pio = io.asPrefixFileIO(); - return deleteInvalidMetadataFile(pio, metadataLocation, lastTime, validFiles, excludeFileNameRegex); - } else { - LOG.warn(String.format( - "Table %s doesn't support a fileIo with listDirectory or listPrefix, so skip clear files.", - internalTable.name() - )); - } - } - return 0; - } - - private static int clearInternalTableDanglingDeleteFiles(UnkeyedTable internalTable) { - Set danglingDeleteFiles = IcebergTableUtil.getDanglingDeleteFiles(internalTable); - if (danglingDeleteFiles.isEmpty()) { - return 0; - } - RewriteFiles rewriteFiles = internalTable.newRewrite(); - rewriteFiles.rewriteFiles(Collections.emptySet(), danglingDeleteFiles, - Collections.emptySet(), Collections.emptySet()); - try { - rewriteFiles.commit(); - } catch (ValidationException e) { - LOG.warn("Iceberg RewriteFiles commit failed on clear danglingDeleteFiles, but ignore", e); - return 0; - } - return danglingDeleteFiles.size(); - } - - private static Set getValidMetadataFiles(UnkeyedTable internalTable) { - TableIdentifier tableIdentifier = internalTable.id(); - Set validFiles = new HashSet<>(); - Iterable snapshots = internalTable.snapshots(); - int size = Iterables.size(snapshots); - LOG.info("{} needs to scan {} snapshots to find all validate metadata files", tableIdentifier, size); - int cnt = 0; - for (Snapshot snapshot : snapshots) { - cnt++; - int before = validFiles.size(); - String manifestListLocation = snapshot.manifestListLocation(); - - validFiles.add(TableFileUtil.getUriPath(manifestListLocation)); - - // valid data files - List manifestFiles = snapshot.allManifests(internalTable.io()); - for (ManifestFile manifestFile : manifestFiles) { - validFiles.add(TableFileUtil.getUriPath(manifestFile.path())); - } - - LOG.info( - "{} scan snapshot {}: {} and get {} files, complete {}/{}", - tableIdentifier, - snapshot.snapshotId(), - formatTime(snapshot.timestampMillis()), - validFiles.size() - before, - cnt, - size); - } - Stream.of( - ReachableFileUtil.metadataFileLocations(internalTable, false).stream(), - ReachableFileUtil.statisticsFilesLocations(internalTable).stream(), - Stream.of(ReachableFileUtil.versionHintLocation(internalTable))) - .reduce(Stream::concat) - .orElse(Stream.empty()) - .map(TableFileUtil::getUriPath) - .forEach(validFiles::add); - - return validFiles; - } - - private static Pattern getExcludeFileNameRegex(UnkeyedTable table) { - String latestFlinkJobId = null; - for (Snapshot snapshot : table.snapshots()) { - String flinkJobId = snapshot.summary().get(FLINK_JOB_ID); - if (!Strings.isNullOrEmpty(flinkJobId)) { - latestFlinkJobId = flinkJobId; - } - } - if (latestFlinkJobId != null) { - // file name starting with flink.job-id should not be deleted - return Pattern.compile(latestFlinkJobId + ".*"); - } - return null; - } - - private static int deleteInvalidMetadataFile( - SupportsPrefixOperations pio, String location, long lastTime, Set exclude, Pattern excludeRegex - ) { - int count = 0; - for (FileInfo fileInfo : pio.listPrefix(location)) { - String uriPath = TableFileUtil.getUriPath(fileInfo.location()); - if (!exclude.contains(uriPath) && - fileInfo.createdAtMillis() < lastTime && - (excludeRegex == null || !excludeRegex.matcher( - TableFileUtil.getFileName(fileInfo.location())).matches())) { - pio.deleteFile(fileInfo.location()); - count += 1; - } - } - return count; - } - - private static String formatTime(long timestamp) { - return LocalDateTime.ofInstant(Instant.ofEpochMilli(timestamp), ZoneId.systemDefault()).toString(); - } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/executor/SnapshotsExpiringExecutor.java b/ams/server/src/main/java/com/netease/arctic/server/table/executor/SnapshotsExpiringExecutor.java index 92d2d91011..216b9aa5f1 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/executor/SnapshotsExpiringExecutor.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/executor/SnapshotsExpiringExecutor.java @@ -18,55 +18,21 @@ package com.netease.arctic.server.table.executor; -import com.netease.arctic.IcebergFileEntry; -import com.netease.arctic.data.FileNameRules; -import com.netease.arctic.hive.utils.TableTypeUtil; -import com.netease.arctic.scan.TableEntriesScan; +import com.netease.arctic.AmoroTable; +import com.netease.arctic.server.optimizing.maintainer.TableMaintainer; import com.netease.arctic.server.table.TableConfiguration; import com.netease.arctic.server.table.TableManager; import com.netease.arctic.server.table.TableRuntime; -import com.netease.arctic.server.utils.HiveLocationUtil; -import com.netease.arctic.server.utils.IcebergTableUtil; -import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.table.KeyedTable; -import com.netease.arctic.table.UnkeyedTable; -import com.netease.arctic.utils.TableFileUtil; -import com.netease.arctic.utils.TablePropertyUtil; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.collections.MapUtils; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFiles; -import org.apache.iceberg.FileContent; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.util.StructLikeMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; - -import static org.apache.iceberg.relocated.com.google.common.primitives.Longs.min; - /** * Service for expiring tables periodically. */ public class SnapshotsExpiringExecutor extends BaseTableExecutor { private static final Logger LOG = LoggerFactory.getLogger(SnapshotsExpiringExecutor.class); - public static final String FLINK_MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; - - private static final int DATA_FILE_LIST_SPLIT = 3000; - - // 1 hour + // 1 days private static final long INTERVAL = 60 * 60 * 1000L; public SnapshotsExpiringExecutor(TableManager tableRuntimes, int poolSize) { @@ -91,247 +57,11 @@ public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration or @Override public void execute(TableRuntime tableRuntime) { try { - TableConfiguration tableConfiguration = tableRuntime.getTableConfiguration(); - if (!tableConfiguration.isExpireSnapshotEnabled()) { - return; - } - - ArcticTable arcticTable = loadTable(tableRuntime); - expireArcticTable(arcticTable, tableRuntime); + AmoroTable amoroTable = loadTable(tableRuntime); + TableMaintainer tableMaintainer = TableMaintainer.ofTable(amoroTable); + tableMaintainer.expireSnapshots(tableRuntime); } catch (Throwable t) { LOG.error("unexpected expire error of table {} ", tableRuntime.getTableIdentifier(), t); } } - - public static void expireArcticTable(ArcticTable arcticTable, TableRuntime tableRuntime) { - long startTime = System.currentTimeMillis(); - TableConfiguration tableConfiguration = tableRuntime.getTableConfiguration(); - LOG.info("{} start expire", tableRuntime.getTableIdentifier()); - - long changeDataTTL = tableConfiguration.getChangeDataTTLMinutes() * 60 * 1000; - long baseSnapshotsKeepTime = tableConfiguration.getSnapshotTTLMinutes() * 60 * 1000; - - Set hiveLocations = new HashSet<>(); - if (TableTypeUtil.isHive(arcticTable)) { - hiveLocations = HiveLocationUtil.getHiveLocation(arcticTable); - } - - if (arcticTable.isKeyedTable()) { - KeyedTable keyedArcticTable = arcticTable.asKeyedTable(); - Set finalHiveLocations = hiveLocations; - keyedArcticTable.io().doAs(() -> { - UnkeyedTable baseTable = keyedArcticTable.baseTable(); - UnkeyedTable changeTable = keyedArcticTable.changeTable(); - - // get valid files in the change store which shouldn't physically delete when expire the snapshot - // in the base store - Set baseExcludePaths = IcebergTableUtil.getAllContentFilePath(changeTable); - baseExcludePaths.addAll(finalHiveLocations); - long latestBaseFlinkCommitTime = fetchLatestFlinkCommittedSnapshotTime(baseTable); - long optimizingSnapshotTime = fetchOptimizingSnapshotTime(baseTable, tableRuntime); - long baseOlderThan = startTime - baseSnapshotsKeepTime; - LOG.info("{} base table expire with latestFlinkCommitTime={}, optimizingSnapshotTime={}, olderThan={}", - arcticTable.id(), latestBaseFlinkCommitTime, optimizingSnapshotTime, baseOlderThan); - expireSnapshots( - baseTable, - min(latestBaseFlinkCommitTime, optimizingSnapshotTime, baseOlderThan), - baseExcludePaths); - long baseCleanedTime = System.currentTimeMillis(); - LOG.info("{} base expire cost {} ms", arcticTable.id(), baseCleanedTime - startTime); - // delete ttl files - long changeTTLPoint = startTime - changeDataTTL; - List expiredDataFileEntries = getExpiredDataFileEntries( - changeTable, System.currentTimeMillis() - changeDataTTL); - deleteChangeFile(keyedArcticTable, expiredDataFileEntries); - - // get valid files in the base store which shouldn't physically delete when expire the snapshot - // in the change store - Set changeExclude = IcebergTableUtil.getAllContentFilePath(baseTable); - changeExclude.addAll(finalHiveLocations); - - long latestChangeFlinkCommitTime = fetchLatestFlinkCommittedSnapshotTime(changeTable); - long changeOlderThan = changeTTLPoint; - LOG.info("{} change table expire with latestFlinkCommitTime={}, olderThan={}", arcticTable.id(), - latestChangeFlinkCommitTime, changeOlderThan); - expireSnapshots( - changeTable, - Math.min(latestChangeFlinkCommitTime, changeOlderThan), - changeExclude); - return null; - }); - LOG.info("{} expire cost total {} ms", arcticTable.id(), System.currentTimeMillis() - startTime); - } else { - UnkeyedTable unKeyedArcticTable = arcticTable.asUnkeyedTable(); - long latestFlinkCommitTime = fetchLatestFlinkCommittedSnapshotTime(unKeyedArcticTable); - long optimizingSnapshotTime = fetchOptimizingSnapshotTime(unKeyedArcticTable, tableRuntime); - long olderThan = startTime - baseSnapshotsKeepTime; - LOG.info("{} unKeyedTable expire with latestFlinkCommitTime={}, optimizingSnapshotTime={}, olderThan={}", - arcticTable.id(), latestFlinkCommitTime, optimizingSnapshotTime, olderThan); - expireSnapshots( - unKeyedArcticTable, - min(latestFlinkCommitTime, optimizingSnapshotTime, olderThan), - hiveLocations); - long baseCleanedTime = System.currentTimeMillis(); - LOG.info("{} unKeyedTable expire cost {} ms", arcticTable.id(), baseCleanedTime - startTime); - } - } - - /** - * When committing a snapshot, Flink will write a checkpoint id into the snapshot summary. - * The latest snapshot with checkpoint id should not be expired or the flink job can't recover from state. - * - * @param table - - * @return commit time of snapshot with the latest flink checkpointId in summary - */ - public static long fetchLatestFlinkCommittedSnapshotTime(UnkeyedTable table) { - long latestCommitTime = Long.MAX_VALUE; - for (Snapshot snapshot : table.snapshots()) { - if (snapshot.summary().containsKey(FLINK_MAX_COMMITTED_CHECKPOINT_ID)) { - latestCommitTime = snapshot.timestampMillis(); - } - } - return latestCommitTime; - } - - /** - * When optimizing tasks are not committed, the snapshot with which it planned should not be expired, since - * it will use the snapshot to check conflict when committing. - * - * @param table - table - * @return commit time of snapshot for optimizing - */ - public static long fetchOptimizingSnapshotTime(UnkeyedTable table, TableRuntime tableRuntime) { - if (tableRuntime.getOptimizingStatus().isProcessing()) { - long targetSnapshotId = tableRuntime.getOptimizingProcess().getTargetSnapshotId(); - for (Snapshot snapshot : table.snapshots()) { - if (snapshot.snapshotId() == targetSnapshotId) { - return snapshot.timestampMillis(); - } - } - } - return Long.MAX_VALUE; - } - - public static void expireSnapshots( - UnkeyedTable arcticInternalTable, - long olderThan, - Set exclude) { - LOG.debug("start expire snapshots older than {}, the exclude is {}", olderThan, exclude); - final AtomicInteger toDeleteFiles = new AtomicInteger(0); - final AtomicInteger deleteFiles = new AtomicInteger(0); - Set parentDirectory = new HashSet<>(); - arcticInternalTable.expireSnapshots() - .retainLast(1) - .expireOlderThan(olderThan) - .deleteWith(file -> { - try { - String filePath = TableFileUtil.getUriPath(file); - if (!exclude.contains(filePath) && !exclude.contains(new Path(filePath).getParent().toString())) { - arcticInternalTable.io().deleteFile(file); - } - parentDirectory.add(new Path(file).getParent().toString()); - deleteFiles.incrementAndGet(); - } catch (Throwable t) { - LOG.warn("failed to delete file " + file, t); - } finally { - toDeleteFiles.incrementAndGet(); - } - }) - .cleanExpiredFiles(true) - .commit(); - if (arcticInternalTable.io().supportFileSystemOperations()) { - parentDirectory.forEach(parent -> TableFileUtil.deleteEmptyDirectory(arcticInternalTable.io(), parent, exclude)); - } - LOG.info("to delete {} files, success delete {} files", toDeleteFiles.get(), deleteFiles.get()); - } - - - public static List getExpiredDataFileEntries(UnkeyedTable changeTable, long ttlPoint) { - TableEntriesScan entriesScan = TableEntriesScan.builder(changeTable) - .includeFileContent(FileContent.DATA) - .build(); - List changeTTLFileEntries = new ArrayList<>(); - - try (CloseableIterable entries = entriesScan.entries()) { - entries.forEach(entry -> { - Snapshot snapshot = changeTable.snapshot(entry.getSnapshotId()); - if (snapshot == null || snapshot.timestampMillis() < ttlPoint) { - changeTTLFileEntries.add(entry); - } - }); - } catch (IOException e) { - throw new UncheckedIOException("Failed to close manifest entry scan of " + changeTable.name(), e); - } - return changeTTLFileEntries; - } - - public static void deleteChangeFile(KeyedTable keyedTable, List expiredDataFileEntries) { - if (CollectionUtils.isEmpty(expiredDataFileEntries)) { - return; - } - - StructLikeMap partitionMaxTransactionId = TablePropertyUtil.getPartitionOptimizedSequence(keyedTable); - if (MapUtils.isEmpty(partitionMaxTransactionId)) { - LOG.info("table {} not contains max transaction id", keyedTable.id()); - return; - } - - Map> partitionDataFileMap = expiredDataFileEntries.stream() - .collect(Collectors.groupingBy(entry -> - keyedTable.spec().partitionToPath(entry.getFile().partition()), Collectors.toList())); - - List changeDeleteFiles = new ArrayList<>(); - if (keyedTable.spec().isUnpartitioned()) { - List partitionDataFiles = - partitionDataFileMap.get(keyedTable.spec().partitionToPath( - expiredDataFileEntries.get(0).getFile().partition())); - - Long optimizedSequence = partitionMaxTransactionId.get(TablePropertyUtil.EMPTY_STRUCT); - if (optimizedSequence != null && CollectionUtils.isNotEmpty(partitionDataFiles)) { - changeDeleteFiles.addAll(partitionDataFiles.stream() - .filter(entry -> FileNameRules.parseChangeTransactionId( - entry.getFile().path().toString(), entry.getSequenceNumber()) <= optimizedSequence) - .map(entry -> (DataFile) entry.getFile()) - .collect(Collectors.toList())); - } - } else { - partitionMaxTransactionId.forEach((key, value) -> { - List partitionDataFiles = - partitionDataFileMap.get(keyedTable.spec().partitionToPath(key)); - - if (CollectionUtils.isNotEmpty(partitionDataFiles)) { - changeDeleteFiles.addAll(partitionDataFiles.stream() - .filter(entry -> FileNameRules.parseChangeTransactionId( - entry.getFile().path().toString(), entry.getSequenceNumber()) <= value) - .map(entry -> (DataFile) entry.getFile()) - .collect(Collectors.toList())); - } - }); - } - tryClearChangeFiles(keyedTable, changeDeleteFiles); - } - - public static void tryClearChangeFiles(KeyedTable keyedTable, List changeFiles) { - if (CollectionUtils.isEmpty(changeFiles)) { - return; - } - try { - if (keyedTable.primaryKeySpec().primaryKeyExisted()) { - for (int startIndex = 0; startIndex < changeFiles.size(); startIndex += DATA_FILE_LIST_SPLIT) { - int end = Math.min(startIndex + DATA_FILE_LIST_SPLIT, changeFiles.size()); - List tableFiles = changeFiles.subList(startIndex, end); - LOG.info("{} delete {} change files", keyedTable.id(), tableFiles.size()); - if (!tableFiles.isEmpty()) { - DeleteFiles changeDelete = keyedTable.changeTable().newDelete(); - changeFiles.forEach(changeDelete::deleteFile); - changeDelete.commit(); - } - LOG.info("{} change committed, delete {} files, complete {}/{}", keyedTable.id(), - tableFiles.size(), end, changeFiles.size()); - } - } - } catch (Throwable t) { - LOG.error(keyedTable.id() + " failed to delete change files, ignore", t); - } - } } diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/executor/TableRuntimeRefreshExecutor.java b/ams/server/src/main/java/com/netease/arctic/server/table/executor/TableRuntimeRefreshExecutor.java index 0c58e0cb92..c6358d8008 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/executor/TableRuntimeRefreshExecutor.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/executor/TableRuntimeRefreshExecutor.java @@ -18,6 +18,7 @@ package com.netease.arctic.server.table.executor; +import com.netease.arctic.AmoroTable; import com.netease.arctic.server.optimizing.plan.OptimizingEvaluator; import com.netease.arctic.server.table.TableManager; import com.netease.arctic.server.table.TableRuntime; @@ -62,11 +63,11 @@ public void execute(TableRuntime tableRuntime) { try { long lastOptimizedSnapshotId = tableRuntime.getLastOptimizedSnapshotId(); long lastOptimizedChangeSnapshotId = tableRuntime.getLastOptimizedChangeSnapshotId(); - ArcticTable table = loadTable(tableRuntime); + AmoroTable table = loadTable(tableRuntime); tableRuntime.refresh(table); if (lastOptimizedSnapshotId != tableRuntime.getCurrentSnapshotId() || lastOptimizedChangeSnapshotId != tableRuntime.getCurrentChangeSnapshotId()) { - tryEvaluatingPendingInput(tableRuntime, table); + tryEvaluatingPendingInput(tableRuntime, (ArcticTable) table.originalTable()); } } catch (Throwable throwable) { logger.error("Refreshing table {} failed.", tableRuntime.getTableIdentifier(), throwable); diff --git a/ams/server/src/main/java/com/netease/arctic/server/utils/IcebergTableUtil.java b/ams/server/src/main/java/com/netease/arctic/server/utils/IcebergTableUtil.java index 2c60f76905..fc3fdcdab0 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/utils/IcebergTableUtil.java +++ b/ams/server/src/main/java/com/netease/arctic/server/utils/IcebergTableUtil.java @@ -33,7 +33,6 @@ import com.netease.arctic.server.table.TableSnapshot; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.TableMetaStore; -import com.netease.arctic.table.UnkeyedTable; import com.netease.arctic.utils.CatalogUtil; import com.netease.arctic.utils.TableFileUtil; import com.netease.arctic.utils.TablePropertyUtil; @@ -45,6 +44,7 @@ import org.apache.iceberg.FileContent; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.TableProperties; @@ -76,8 +76,8 @@ public class IcebergTableUtil { public static final String PROPERTIES_METADATA_LOCATION = "iceberg.metadata.location"; public static final String PROPERTIES_PREV_METADATA_LOCATION = "iceberg.metadata.prev-location"; - public static long getSnapshotId(UnkeyedTable internalTable, boolean refresh) { - Snapshot currentSnapshot = getSnapshot(internalTable, refresh); + public static long getSnapshotId(Table table, boolean refresh) { + Snapshot currentSnapshot = getSnapshot(table, refresh); if (currentSnapshot == null) { return ArcticServiceConstants.INVALID_SNAPSHOT_ID; } else { @@ -86,7 +86,6 @@ public static long getSnapshotId(UnkeyedTable internalTable, boolean refresh) { } public static TableSnapshot getSnapshot(ArcticTable arcticTable, TableRuntime tableRuntime) { - tableRuntime.refresh(arcticTable); if (arcticTable.isUnkeyedTable()) { return new BasicTableSnapshot(tableRuntime.getCurrentSnapshotId()); } else { @@ -101,14 +100,14 @@ public static TableSnapshot getSnapshot(ArcticTable arcticTable, TableRuntime ta } } - public static Snapshot getSnapshot(UnkeyedTable internalTable, boolean refresh) { + public static Snapshot getSnapshot(Table table, boolean refresh) { if (refresh) { - internalTable.refresh(); + table.refresh(); } - return internalTable.currentSnapshot(); + return table.currentSnapshot(); } - public static Set getAllContentFilePath(UnkeyedTable internalTable) { + public static Set getAllContentFilePath(Table internalTable) { Set validFilesPath = new HashSet<>(); TableEntriesScan entriesScan = TableEntriesScan.builder(internalTable) @@ -125,7 +124,7 @@ public static Set getAllContentFilePath(UnkeyedTable internalTable) { return validFilesPath; } - public static Set getDanglingDeleteFiles(UnkeyedTable internalTable) { + public static Set getDanglingDeleteFiles(Table internalTable) { if (internalTable.currentSnapshot() == null) { return Collections.emptySet(); } diff --git a/ams/server/src/main/resources/derby/ams-derby-init.sql b/ams/server/src/main/resources/derby/ams-derby-init.sql index a8af75b9de..3ef6c2acc8 100644 --- a/ams/server/src/main/resources/derby/ams-derby-init.sql +++ b/ams/server/src/main/resources/derby/ams-derby-init.sql @@ -89,6 +89,7 @@ CREATE TABLE table_runtime ( catalog_name VARCHAR(64) NOT NULL, db_name VARCHAR(128) NOT NULL, table_name VARCHAR(128) NOT NULL, + format VARCHAR(32) NOT NULL, current_snapshot_id BIGINT NOT NULL DEFAULT -1, current_change_snapshotId BIGINT, last_optimized_snapshotId BIGINT NOT NULL DEFAULT -1, diff --git a/ams/server/src/main/resources/mysql/ams-mysql-init.sql b/ams/server/src/main/resources/mysql/ams-mysql-init.sql index 76fd59b6b4..b63f6f206b 100644 --- a/ams/server/src/main/resources/mysql/ams-mysql-init.sql +++ b/ams/server/src/main/resources/mysql/ams-mysql-init.sql @@ -100,6 +100,7 @@ CREATE TABLE `table_runtime` `catalog_name` varchar(64) NOT NULL COMMENT 'Catalog name', `db_name` varchar(128) NOT NULL COMMENT 'Database name', `table_name` varchar(128) NOT NULL COMMENT 'Table name', + `format` varchar(32) NOT NULL COMMENT "format", `current_snapshot_id` bigint(20) NOT NULL DEFAULT '-1' COMMENT 'Base table current snapshot id', `current_change_snapshotId` bigint(20) DEFAULT NULL COMMENT 'Change table current snapshot id', `last_optimized_snapshotId` bigint(20) NOT NULL DEFAULT '-1' COMMENT 'last optimized snapshot id', diff --git a/ams/server/src/main/resources/mysql/update.sql b/ams/server/src/main/resources/mysql/update.sql new file mode 100644 index 0000000000..ef541f397c --- /dev/null +++ b/ams/server/src/main/resources/mysql/update.sql @@ -0,0 +1 @@ +ALTER TABLE `table_runtime` ADD `format` varchar(32) NOT NULL COMMENT "format"; diff --git a/ams/server/src/main/resources/postgres/update.sql b/ams/server/src/main/resources/postgres/update.sql new file mode 100644 index 0000000000..ce79089ad6 --- /dev/null +++ b/ams/server/src/main/resources/postgres/update.sql @@ -0,0 +1,2 @@ +ALTER TABLE table_runtime ADD format VARCHAR(32) NOT NULL; +COMMENT ON COLUMN table_runtime.format IS 'Format'; \ No newline at end of file diff --git a/ams/server/src/test/java/com/netease/arctic/server/catalog/ExternalCatalogTest.java b/ams/server/src/test/java/com/netease/arctic/server/catalog/ExternalCatalogTest.java new file mode 100644 index 0000000000..cdb9f6bad5 --- /dev/null +++ b/ams/server/src/test/java/com/netease/arctic/server/catalog/ExternalCatalogTest.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.server.catalog; + +import com.netease.arctic.formats.AmoroCatalogTestHelper; +import com.netease.arctic.formats.IcebergHadoopCatalogTestHelper; +import com.netease.arctic.formats.PaimonHadoopCatalogTestHelper; +import com.netease.arctic.hive.formats.IcebergHiveCatalogTestHelper; +import com.netease.arctic.hive.formats.PaimonHiveCatalogTestHelper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class ExternalCatalogTest extends TableCatalogTestBase { + + private final String testDatabaseName = "test_database"; + + private final String testTableName = "test_table"; + + public ExternalCatalogTest(AmoroCatalogTestHelper amoroCatalogTestHelper) { + super(amoroCatalogTestHelper); + } + + @Parameterized.Parameters(name = "{0}") + public static Object[] parameters() { + return new Object[] { + PaimonHadoopCatalogTestHelper.defaultHelper(), + PaimonHiveCatalogTestHelper.defaultHelper(), + IcebergHadoopCatalogTestHelper.defaultHelper(), + IcebergHiveCatalogTestHelper.defaultHelper() + }; + } + + @Before + public void setUp() throws Exception { + getAmoroCatalog().createDatabase(testDatabaseName); + getAmoroCatalogTestHelper().createTable(testDatabaseName, testTableName); + } + + @Test + public void listDatabases() { + Assert.assertTrue(getExternalCatalog().listDatabases().contains(testDatabaseName)); + } + + @Test + public void dataBaseExists() { + Assert.assertTrue(getExternalCatalog().exist(testDatabaseName)); + } + + @Test + public void tableExists() { + Assert.assertTrue(getExternalCatalog().exist(testDatabaseName, testTableName)); + } + + @Test + public void listTables() { + Assert.assertEquals(1, getExternalCatalog().listTables(testDatabaseName).size()); + Assert.assertEquals(testTableName, getExternalCatalog().listTables(testDatabaseName).get(0).getTableName()); + } + + @Test + public void loadTable() { + Assert.assertNotNull(getExternalCatalog().loadTable(testDatabaseName, testTableName)); + } + + private ServerCatalog getExternalCatalog() { + return tableService().getServerCatalog(getAmoroCatalogTestHelper().catalogName()); + } +} diff --git a/ams/server/src/test/java/com/netease/arctic/server/catalog/TableCatalogTestBase.java b/ams/server/src/test/java/com/netease/arctic/server/catalog/TableCatalogTestBase.java new file mode 100644 index 0000000000..37814ffd5a --- /dev/null +++ b/ams/server/src/test/java/com/netease/arctic/server/catalog/TableCatalogTestBase.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.server.catalog; + +import com.netease.arctic.AmoroCatalog; +import com.netease.arctic.formats.AmoroCatalogTestHelper; +import com.netease.arctic.hive.TestHMS; +import com.netease.arctic.server.table.TableServiceTestBase; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; + +public class TableCatalogTestBase extends TableServiceTestBase { + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + @ClassRule + public static TestHMS TEST_HMS = new TestHMS(); + + private final AmoroCatalogTestHelper amoroCatalogTestHelper; + + private AmoroCatalog amoroCatalog; + + private Object originalTableCatalog; + + public TableCatalogTestBase(AmoroCatalogTestHelper amoroCatalogTestHelper) { + this.amoroCatalogTestHelper = amoroCatalogTestHelper; + } + + @Before + public void init() throws IOException { + String path = temp.newFolder().getPath(); + amoroCatalogTestHelper.initWarehouse(path); + amoroCatalogTestHelper.initHiveConf(TEST_HMS.getHiveConf()); + this.amoroCatalog = amoroCatalogTestHelper.amoroCatalog(); + tableService().createCatalog(amoroCatalogTestHelper.getCatalogMeta()); + this.originalTableCatalog = amoroCatalogTestHelper.originalCatalog(); + } + + @After + public void clean() { + tableService().dropCatalog(amoroCatalogTestHelper.catalogName()); + amoroCatalogTestHelper.clean(); + } + + public AmoroCatalog getAmoroCatalog() { + return amoroCatalog; + } + + public Object getOriginalTableCatalog() { + return originalTableCatalog; + } + + public AmoroCatalogTestHelper getAmoroCatalogTestHelper() { + return amoroCatalogTestHelper; + } +} diff --git a/ams/server/src/test/java/com/netease/arctic/server/dashboard/TestServerTableDescriptor.java b/ams/server/src/test/java/com/netease/arctic/server/dashboard/TestServerTableDescriptor.java index db3bce8d0e..5eb4833fca 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/dashboard/TestServerTableDescriptor.java +++ b/ams/server/src/test/java/com/netease/arctic/server/dashboard/TestServerTableDescriptor.java @@ -54,7 +54,7 @@ public TestServerTableDescriptor(CatalogTestHelper catalogTestHelper, @Test public void getTableOperations() { ServerTableDescriptor serverTableDescriptor = new ServerTableDescriptor(tableService()); - ArcticTable arcticTable = tableService().loadTable(serverTableIdentifier()); + ArcticTable arcticTable = (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); arcticTable.updateProperties().set("key", "value1").commit(); List tableOperations = serverTableDescriptor.getTableOperations(serverTableIdentifier()); Assert.assertEquals(1, tableOperations.size()); diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java index 530aad0d48..dafcedbff1 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestOptimizingQueue.java @@ -13,7 +13,6 @@ import com.netease.arctic.io.MixedDataTestHelpers; import com.netease.arctic.optimizing.RewriteFilesOutput; import com.netease.arctic.optimizing.TableOptimizing; -import com.netease.arctic.server.ArcticServiceConstants; import com.netease.arctic.server.persistence.PersistentBase; import com.netease.arctic.server.persistence.mapper.TableMetaMapper; import com.netease.arctic.server.resource.OptimizerInstance; @@ -296,7 +295,7 @@ public void testReloadFailTask() { } private TableRuntimeMeta initTableWithFiles() { - ArcticTable arcticTable = tableService().loadTable(serverTableIdentifier()); + ArcticTable arcticTable = (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); appendData(arcticTable.asUnkeyedTable(), 1); appendData(arcticTable.asUnkeyedTable(), 2); TableRuntimeMeta tableRuntimeMeta = buildTableRuntimeMeta(OptimizingStatus.PENDING, defaultResourceGroup()); @@ -390,7 +389,7 @@ private ResourceGroup defaultResourceGroup() { } private TableRuntimeMeta buildTableRuntimeMeta(OptimizingStatus status, ResourceGroup resourceGroup) { - ArcticTable arcticTable = tableService().loadTable(serverTableIdentifier()); + ArcticTable arcticTable = (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); TableRuntimeMeta tableRuntimeMeta = new TableRuntimeMeta(); tableRuntimeMeta.setCatalogName(serverTableIdentifier().getCatalog()); tableRuntimeMeta.setDbName(serverTableIdentifier().getDatabase()); @@ -398,10 +397,6 @@ private TableRuntimeMeta buildTableRuntimeMeta(OptimizingStatus status, Resource tableRuntimeMeta.setTableId(serverTableIdentifier().getId()); tableRuntimeMeta.setTableStatus(status); tableRuntimeMeta.setTableConfig(TableConfiguration.parseConfig(arcticTable.properties())); - tableRuntimeMeta.setCurrentChangeSnapshotId(ArcticServiceConstants.INVALID_SNAPSHOT_ID); - tableRuntimeMeta.setCurrentSnapshotId(ArcticServiceConstants.INVALID_SNAPSHOT_ID); - tableRuntimeMeta.setLastOptimizedChangeSnapshotId(ArcticServiceConstants.INVALID_SNAPSHOT_ID); - tableRuntimeMeta.setLastOptimizedSnapshotId(ArcticServiceConstants.INVALID_SNAPSHOT_ID); tableRuntimeMeta.setOptimizerGroup(resourceGroup.getName()); tableRuntimeMeta.constructTableRuntime(tableService()); return tableRuntimeMeta; diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/flow/CompleteOptimizingFlow.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/flow/CompleteOptimizingFlow.java index 8cbd9c057c..ca6adb97e3 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/flow/CompleteOptimizingFlow.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/flow/CompleteOptimizingFlow.java @@ -164,13 +164,13 @@ private OptimizingPlanner planner() { table.refresh(); TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); Mockito.when(tableRuntime.getCurrentSnapshotId()).thenAnswer(f -> getCurrentSnapshotId()); + Mockito.when(tableRuntime.getCurrentChangeSnapshotId()).thenAnswer(f -> getCurrentChangeSnapshotId()); Mockito.when(tableRuntime.getNewestProcessId()).thenReturn(1L); Mockito.when(tableRuntime.getPendingInput()).thenReturn(null); Mockito.doCallRealMethod().when(tableRuntime).getLastMinorOptimizingTime(); Mockito.doCallRealMethod().when(tableRuntime).getLastMajorOptimizingTime(); Mockito.doCallRealMethod().when(tableRuntime).getLastFullOptimizingTime(); Mockito.when(tableRuntime.getOptimizingConfig()).thenAnswer(f -> optimizingConfig()); - Mockito.when(tableRuntime.getCurrentChangeSnapshotId()).thenAnswer(f -> getCurrentChangeSnapshotId()); Mockito.when(tableRuntime.getTableIdentifier()).thenReturn(ServerTableIdentifier.of(1L, "a", "b", "c")); return new OptimizingPlanner(tableRuntime, table, availableCore); } diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestOrphanFileClean.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestOrphanFileClean.java similarity index 90% rename from ams/server/src/test/java/com/netease/arctic/server/table/executor/TestOrphanFileClean.java rename to ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestOrphanFileClean.java index 7cc3dcf571..442cc30478 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestOrphanFileClean.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestOrphanFileClean.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.netease.arctic.server.table.executor; +package com.netease.arctic.server.optimizing.maintainer; import com.netease.arctic.BasicTableTestHelper; import com.netease.arctic.TableTestHelper; @@ -24,6 +24,7 @@ import com.netease.arctic.catalog.BasicCatalogTestHelper; import com.netease.arctic.catalog.CatalogTestHelper; import com.netease.arctic.data.ChangeAction; +import com.netease.arctic.server.table.executor.ExecutorTestBase; import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.UnkeyedTable; @@ -51,23 +52,23 @@ import java.util.Set; import java.util.stream.Collectors; -import static com.netease.arctic.server.table.executor.OrphanFilesCleaningExecutor.DATA_FOLDER_NAME; -import static com.netease.arctic.server.table.executor.OrphanFilesCleaningExecutor.FLINK_JOB_ID; +import static com.netease.arctic.server.optimizing.maintainer.IcebergTableMaintainer.DATA_FOLDER_NAME; +import static com.netease.arctic.server.optimizing.maintainer.IcebergTableMaintainer.FLINK_JOB_ID; @RunWith(Parameterized.class) public class TestOrphanFileClean extends ExecutorTestBase { @Parameterized.Parameters(name = "{0}, {1}") public static Object[] parameters() { - return new Object[][]{ + return new Object[][] { {new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(true, true)}, + new BasicTableTestHelper(true, true)}, {new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(true, false)}, + new BasicTableTestHelper(true, false)}, {new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(false, true)}, + new BasicTableTestHelper(false, true)}, {new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(false, false)}}; + new BasicTableTestHelper(false, false)}}; } public TestOrphanFileClean(CatalogTestHelper catalogTestHelper, TableTestHelper tableTestHelper) { @@ -100,12 +101,12 @@ public void orphanDataFileClean() throws IOException { Assert.assertTrue(getArcticTable().io().exists(changeOrphanFilePath)); } - OrphanFilesCleaningExecutor.cleanContentFiles( - getArcticTable(), + MixedTableMaintainer maintainer = new MixedTableMaintainer(getArcticTable()); + maintainer.cleanContentFiles( System.currentTimeMillis() - TableProperties.MIN_ORPHAN_FILE_EXISTING_TIME_DEFAULT * 60 * 1000); - OrphanFilesCleaningExecutor.cleanMetadata( - getArcticTable(), + maintainer.cleanMetadata( System.currentTimeMillis() - TableProperties.MIN_ORPHAN_FILE_EXISTING_TIME_DEFAULT * 60 * 1000); + Assert.assertTrue(getArcticTable().io().exists(baseOrphanFileDir)); Assert.assertTrue(getArcticTable().io().exists(baseOrphanFilePath)); @@ -113,8 +114,8 @@ public void orphanDataFileClean() throws IOException { Assert.assertTrue(getArcticTable().io().exists(changeOrphanFilePath)); } - OrphanFilesCleaningExecutor.cleanContentFiles(getArcticTable(), System.currentTimeMillis()); - OrphanFilesCleaningExecutor.cleanMetadata(getArcticTable(), System.currentTimeMillis()); + maintainer.cleanContentFiles(System.currentTimeMillis()); + maintainer.cleanMetadata(System.currentTimeMillis()); Assert.assertFalse(getArcticTable().io().exists(baseOrphanFileDir)); Assert.assertFalse(getArcticTable().io().exists(baseOrphanFilePath)); @@ -157,7 +158,8 @@ public void orphanMetadataFileClean() throws IOException { Assert.assertTrue(getArcticTable().io().exists(changeInvalidMetadataJson)); } - OrphanFilesCleaningExecutor.cleanMetadata(getArcticTable(), System.currentTimeMillis()); + MixedTableMaintainer maintainer = new MixedTableMaintainer(getArcticTable()); + maintainer.cleanMetadata(System.currentTimeMillis()); Assert.assertFalse(getArcticTable().io().exists(baseOrphanFilePath)); if (isKeyedTable()) { @@ -191,7 +193,9 @@ public void orphanChangeDataFileInBaseClean() { } pathAll.forEach(path -> Assert.assertTrue(testKeyedTable.io().exists(path))); - OrphanFilesCleaningExecutor.cleanContentFiles(testKeyedTable, System.currentTimeMillis()); + MixedTableMaintainer maintainer = new MixedTableMaintainer(getArcticTable()); + maintainer.cleanContentFiles(System.currentTimeMillis()); + fileInBaseStore.forEach(path -> Assert.assertTrue(testKeyedTable.io().exists(path))); fileOnlyInChangeLocation.forEach(path -> Assert.assertFalse(testKeyedTable.io().exists(path))); } @@ -242,7 +246,8 @@ public void notDeleteFlinkTemporaryFile() throws IOException { Assert.assertTrue(getArcticTable().io().exists(changeInvalidMetadataJson)); } - OrphanFilesCleaningExecutor.cleanMetadata(getArcticTable(), System.currentTimeMillis()); + MixedTableMaintainer tableMaintainer = new MixedTableMaintainer(getArcticTable()); + tableMaintainer.cleanMetadata(System.currentTimeMillis()); Assert.assertFalse(getArcticTable().io().exists(baseOrphanFilePath)); if (isKeyedTable()) { // files whose file name starts with flink.job-id should not be deleted @@ -268,7 +273,7 @@ public void notDeleteStatisticsFile() { unkeyedTable.updateStatistics().setStatistics(snapshot.snapshotId(), file).commit(); Assert.assertTrue(unkeyedTable.io().exists(file.path())); - OrphanFilesCleaningExecutor.cleanMetadata(unkeyedTable, System.currentTimeMillis() + 1); + new MixedTableMaintainer(getArcticTable()).cleanMetadata(System.currentTimeMillis() + 1); Assert.assertTrue(unkeyedTable.io().exists(file.path())); } @@ -295,5 +300,4 @@ private StatisticsFile writeStatisticsFile(UnkeyedTable table, Snapshot snapshot footerSize, collect); } - } diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestOrphanFileCleanHive.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestOrphanFileCleanHive.java similarity index 89% rename from ams/server/src/test/java/com/netease/arctic/server/table/executor/TestOrphanFileCleanHive.java rename to ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestOrphanFileCleanHive.java index 212de8ed11..0947ca2513 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestOrphanFileCleanHive.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestOrphanFileCleanHive.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.netease.arctic.server.table.executor; +package com.netease.arctic.server.optimizing.maintainer; import com.netease.arctic.TableTestHelper; import com.netease.arctic.ams.api.TableFormat; @@ -35,7 +35,7 @@ import java.io.File; import java.io.IOException; -import static com.netease.arctic.server.table.executor.OrphanFilesCleaningExecutor.DATA_FOLDER_NAME; +import static com.netease.arctic.server.optimizing.maintainer.IcebergTableMaintainer.DATA_FOLDER_NAME; @RunWith(Parameterized.class) public class TestOrphanFileCleanHive extends TestOrphanFileClean { @@ -67,7 +67,9 @@ public void hiveLocationOrphanDataFileClean() throws IOException { OutputFile changeOrphanDataFile = getArcticTable().io().newOutputFile(hiveOrphanFilePath); changeOrphanDataFile.createOrOverwrite().close(); Assert.assertTrue(getArcticTable().io().exists(hiveOrphanFilePath)); - OrphanFilesCleaningExecutor.cleanContentFiles(getArcticTable(), System.currentTimeMillis()); + + MixedTableMaintainer maintainer = new MixedTableMaintainer(getArcticTable()); + maintainer.cleanContentFiles(System.currentTimeMillis()); Assert.assertTrue(getArcticTable().io().exists(hiveOrphanFilePath)); } diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestOrphanFileCleanIceberg.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestOrphanFileCleanIceberg.java similarity index 96% rename from ams/server/src/test/java/com/netease/arctic/server/table/executor/TestOrphanFileCleanIceberg.java rename to ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestOrphanFileCleanIceberg.java index 85bfce772b..f25c66f2e5 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestOrphanFileCleanIceberg.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestOrphanFileCleanIceberg.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.netease.arctic.server.table.executor; +package com.netease.arctic.server.optimizing.maintainer; import com.google.common.collect.Lists; import com.netease.arctic.BasicTableTestHelper; @@ -95,7 +95,10 @@ public void cleanDanglingDeleteFiles() throws IOException { Collections.singleton(dataFiles2.get(0))) .validateFromSnapshot(testTable.currentSnapshot().snapshotId()).commit(); assertDanglingDeleteFiles(testTable, 1); - OrphanFilesCleaningExecutor.cleanDanglingDeleteFiles(testTable); + + MixedTableMaintainer tableMaintainer = new MixedTableMaintainer(testTable); + tableMaintainer.cleanDanglingDeleteFiles(); + assertDanglingDeleteFiles(testTable, 0); } diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestSnapshotExpire.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpire.java similarity index 60% rename from ams/server/src/test/java/com/netease/arctic/server/table/executor/TestSnapshotExpire.java rename to ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpire.java index ba19601527..0b16b93414 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestSnapshotExpire.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpire.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,10 +16,9 @@ * limitations under the License. */ -package com.netease.arctic.server.table.executor; +package com.netease.arctic.server.optimizing.maintainer; import com.netease.arctic.BasicTableTestHelper; -import com.netease.arctic.IcebergFileEntry; import com.netease.arctic.TableTestHelper; import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.catalog.BasicCatalogTestHelper; @@ -29,10 +28,11 @@ import com.netease.arctic.server.dashboard.utils.AmsUtil; import com.netease.arctic.server.optimizing.OptimizingProcess; import com.netease.arctic.server.optimizing.OptimizingStatus; +import com.netease.arctic.server.optimizing.maintainer.MixedTableMaintainer; import com.netease.arctic.server.table.ServerTableIdentifier; import com.netease.arctic.server.table.TableConfiguration; import com.netease.arctic.server.table.TableRuntime; -import com.netease.arctic.server.utils.IcebergTableUtil; +import com.netease.arctic.server.table.executor.ExecutorTestBase; import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.UnkeyedTable; @@ -40,12 +40,11 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFiles; -import org.apache.iceberg.FileScanTask; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StructLike; -import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.jetbrains.annotations.NotNull; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -59,6 +58,7 @@ import java.util.Set; import java.util.stream.Collectors; +import static com.netease.arctic.server.optimizing.maintainer.IcebergTableMaintainer.FLINK_MAX_COMMITTED_CHECKPOINT_ID; @RunWith(Parameterized.class) public class TestSnapshotExpire extends ExecutorTestBase { @@ -82,47 +82,12 @@ public TestSnapshotExpire(CatalogTestHelper catalogTestHelper, TableTestHelper t super(catalogTestHelper, tableTestHelper); } - @Test - public void testDeleteChangeFiles() throws Exception { - Assume.assumeTrue(isKeyedTable()); - Assume.assumeTrue(isPartitionedTable()); - KeyedTable testKeyedTable = getArcticTable().asKeyedTable(); - List s1Files = insertChangeDataFiles(testKeyedTable, 1); - long l = testKeyedTable.changeTable().currentSnapshot().timestampMillis(); - List partitions = - new ArrayList<>(s1Files.stream().collect(Collectors.groupingBy(ContentFile::partition)).keySet()); - Assert.assertEquals(2, partitions.size()); - - UpdatePartitionProperties updateProperties = testKeyedTable.baseTable().updatePartitionProperties(null); - updateProperties.set(partitions.get(0), TableProperties.PARTITION_OPTIMIZED_SEQUENCE, "3"); - updateProperties.set(partitions.get(1), TableProperties.PARTITION_OPTIMIZED_SEQUENCE, "0"); - updateProperties.commit(); - List existedDataFiles = new ArrayList<>(); - try (CloseableIterable fileScanTasks = testKeyedTable.changeTable().newScan().planFiles()) { - fileScanTasks.forEach(fileScanTask -> existedDataFiles.add(fileScanTask.file())); - } - Assert.assertEquals(4, existedDataFiles.size()); - - List expiredDataFileEntries = - SnapshotsExpiringExecutor.getExpiredDataFileEntries(testKeyedTable.changeTable(), l + 1); - SnapshotsExpiringExecutor.deleteChangeFile( - testKeyedTable, expiredDataFileEntries); - Assert.assertEquals(2, Iterables.size(testKeyedTable.changeTable().snapshots())); - Set currentDataFiles = new HashSet<>(); - try (CloseableIterable fileScanTasks = testKeyedTable.changeTable().newScan().planFiles()) { - fileScanTasks.forEach(fileScanTask -> currentDataFiles.add(fileScanTask.file().path().toString())); - } - Set expectedDataFiles = existedDataFiles.stream().filter( - file -> file.partition().equals(partitions.get(1))).map(f -> f.path().toString()).collect(Collectors.toSet()); - Assert.assertEquals(expectedDataFiles, currentDataFiles); - changeTableFiles.forEach( - file -> Assert.assertTrue(testKeyedTable.changeTable().io().exists(file.path().toString()))); - } - @Test public void testExpireChangeTableFiles() throws Exception { Assume.assumeTrue(isKeyedTable()); KeyedTable testKeyedTable = getArcticTable().asKeyedTable(); + testKeyedTable.updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); + testKeyedTable.updateProperties().set(TableProperties.CHANGE_DATA_TTL, "0").commit(); List s1Files = insertChangeDataFiles(testKeyedTable, 1); long l = testKeyedTable.changeTable().currentSnapshot().timestampMillis(); List partitions = @@ -140,23 +105,16 @@ public void testExpireChangeTableFiles() throws Exception { } updateProperties.commit(); s1Files.forEach(file -> Assert.assertTrue(testKeyedTable.changeTable().io().exists(file.path().toString()))); - List expiredDataFileEntries = - SnapshotsExpiringExecutor.getExpiredDataFileEntries(testKeyedTable.changeTable(), l + 1); - SnapshotsExpiringExecutor.deleteChangeFile( - testKeyedTable, expiredDataFileEntries); - Assert.assertEquals(2, Iterables.size(testKeyedTable.changeTable().snapshots())); - List existedDataFiles = new ArrayList<>(); - try (CloseableIterable fileScanTasks = testKeyedTable.changeTable().newScan().planFiles()) { - fileScanTasks.forEach(fileScanTask -> existedDataFiles.add(fileScanTask.file())); - } - Assert.assertEquals(0, existedDataFiles.size()); + MixedTableMaintainer tableMaintainer = new MixedTableMaintainer(testKeyedTable); + tableMaintainer.getChangeMaintainer().expireFiles(l + 1); + + //In order to advance the snapshot insertChangeDataFiles(testKeyedTable, 2); - Snapshot expectedSnapshot = testKeyedTable.changeTable().currentSnapshot(); - SnapshotsExpiringExecutor.expireSnapshots( - testKeyedTable.changeTable(), System.currentTimeMillis(), new HashSet<>()); + + tableMaintainer.getChangeMaintainer().expireSnapshots(System.currentTimeMillis()); + Assert.assertEquals(1, Iterables.size(testKeyedTable.changeTable().snapshots())); - Assert.assertEquals(expectedSnapshot, testKeyedTable.changeTable().snapshots().iterator().next()); s1Files.forEach(file -> Assert.assertFalse(testKeyedTable.changeTable().io().exists(file.path().toString()))); } @@ -165,6 +123,8 @@ public void testExpiredChangeTableFilesInBase() { Assume.assumeTrue(isKeyedTable()); Assume.assumeTrue(isPartitionedTable()); KeyedTable testKeyedTable = getArcticTable().asKeyedTable(); + testKeyedTable.updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); + testKeyedTable.updateProperties().set(TableProperties.CHANGE_DATA_TTL, "0").commit(); List s1Files = insertChangeDataFiles(testKeyedTable, 1); long l = testKeyedTable.changeTable().currentSnapshot().timestampMillis(); testKeyedTable.baseTable().newAppend().appendFile(s1Files.get(0)).commit(); @@ -177,82 +137,36 @@ public void testExpiredChangeTableFilesInBase() { updateProperties.set(partitions.get(1), TableProperties.PARTITION_OPTIMIZED_SEQUENCE, "1"); updateProperties.commit(); s1Files.forEach(file -> Assert.assertTrue(testKeyedTable.io().exists(file.path().toString()))); - List expiredDataFileEntries = - SnapshotsExpiringExecutor.getExpiredDataFileEntries(testKeyedTable.changeTable(), l + 1); - SnapshotsExpiringExecutor.deleteChangeFile( - testKeyedTable, expiredDataFileEntries); - Assert.assertEquals(2, Iterables.size(testKeyedTable.changeTable().snapshots())); - Set exclude = IcebergTableUtil.getAllContentFilePath(testKeyedTable.baseTable()); + MixedTableMaintainer tableMaintainer = new MixedTableMaintainer(testKeyedTable); + tableMaintainer.getChangeMaintainer().expireFiles(l + 1); + //In order to advance the snapshot insertChangeDataFiles(testKeyedTable, 2); - Snapshot expectedSnapshot = testKeyedTable.changeTable().currentSnapshot(); - SnapshotsExpiringExecutor.expireSnapshots(testKeyedTable.changeTable(), System.currentTimeMillis(), exclude); + tableMaintainer.getChangeMaintainer().expireSnapshots(System.currentTimeMillis()); Assert.assertEquals(1, Iterables.size(testKeyedTable.changeTable().snapshots())); - Assert.assertEquals(expectedSnapshot, testKeyedTable.changeTable().currentSnapshot()); Assert.assertTrue(testKeyedTable.io().exists(s1Files.get(0).path().toString())); Assert.assertFalse(testKeyedTable.io().exists(s1Files.get(1).path().toString())); } - @Test - public void testGetClosestExpiredFiles() { - Assume.assumeTrue(isKeyedTable()); - KeyedTable testKeyedTable = getArcticTable().asKeyedTable(); - insertChangeDataFiles(testKeyedTable, 1); - Snapshot firstSnapshot = testKeyedTable.changeTable().currentSnapshot(); - insertChangeDataFiles(testKeyedTable, 2); - long secondCommitTime = testKeyedTable.changeTable().currentSnapshot().timestampMillis(); - Snapshot secondSnapshot = testKeyedTable.changeTable().currentSnapshot(); - - testKeyedTable.changeTable().newAppend().commit(); - long thirdCommitTime = testKeyedTable.changeTable().currentSnapshot().timestampMillis(); - Snapshot thirdSnapshot = testKeyedTable.changeTable().currentSnapshot(); - - Set top8Files = new HashSet<>(); - testKeyedTable.changeTable().newScan().planFiles().forEach(task -> top8Files.add(task.file())); - Assert.assertEquals(8, top8Files.size()); - Assert.assertEquals(3, Iterables.size(testKeyedTable.changeTable().snapshots())); - - insertChangeDataFiles(testKeyedTable, 3); - Assert.assertEquals(12, Iterables.size(testKeyedTable.changeTable().newScan().planFiles())); - Assert.assertEquals(4, Iterables.size(testKeyedTable.changeTable().snapshots())); - - List expiredDataFileEntries = SnapshotsExpiringExecutor.getExpiredDataFileEntries( - testKeyedTable.changeTable(), thirdCommitTime + 1); - Set closestFilesPath = expiredDataFileEntries.stream().map( - entry -> entry.getFile().path()).collect(Collectors.toSet()); - Set top8FilesPath = top8Files.stream().map(ContentFile::path).collect(Collectors.toSet()); - - Assert.assertTrue(top8FilesPath.equals(closestFilesPath)); - } - @Test public void testNotExpireFlinkLatestCommit4ChangeTable() { Assume.assumeTrue(isKeyedTable()); KeyedTable testKeyedTable = getArcticTable().asKeyedTable(); insertChangeDataFiles(testKeyedTable, 1); insertChangeDataFiles(testKeyedTable, 2); - Assert.assertEquals(Long.MAX_VALUE, - SnapshotsExpiringExecutor.fetchLatestFlinkCommittedSnapshotTime(testKeyedTable.changeTable())); AppendFiles appendFiles = testKeyedTable.changeTable().newAppend(); - appendFiles.set(SnapshotsExpiringExecutor.FLINK_MAX_COMMITTED_CHECKPOINT_ID, "100"); + appendFiles.set(FLINK_MAX_COMMITTED_CHECKPOINT_ID, "100"); appendFiles.commit(); - long checkpointTime = testKeyedTable.changeTable().currentSnapshot().timestampMillis(); - Assert.assertEquals(checkpointTime, - SnapshotsExpiringExecutor.fetchLatestFlinkCommittedSnapshotTime(testKeyedTable.changeTable())); AppendFiles appendFiles2 = testKeyedTable.changeTable().newAppend(); - appendFiles2.set(SnapshotsExpiringExecutor.FLINK_MAX_COMMITTED_CHECKPOINT_ID, "101"); + appendFiles2.set(FLINK_MAX_COMMITTED_CHECKPOINT_ID, "101"); appendFiles2.commit(); Snapshot checkpointTime2Snapshot = testKeyedTable.changeTable().currentSnapshot(); - Assert.assertEquals(checkpointTime2Snapshot.timestampMillis(), - SnapshotsExpiringExecutor.fetchLatestFlinkCommittedSnapshotTime(testKeyedTable.changeTable())); insertChangeDataFiles(testKeyedTable, 2); Snapshot lastSnapshot = testKeyedTable.changeTable().currentSnapshot(); - Assert.assertEquals(checkpointTime2Snapshot.timestampMillis(), - SnapshotsExpiringExecutor.fetchLatestFlinkCommittedSnapshotTime(testKeyedTable.changeTable())); testKeyedTable.updateProperties().set(TableProperties.CHANGE_DATA_TTL, "0").commit(); TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); @@ -263,8 +177,9 @@ public void testNotExpireFlinkLatestCommit4ChangeTable() { TableConfiguration.parseConfig(testKeyedTable.properties())); Assert.assertEquals(5, Iterables.size(testKeyedTable.changeTable().snapshots())); - SnapshotsExpiringExecutor.expireArcticTable( - testKeyedTable, tableRuntime); + + MixedTableMaintainer tableMaintainer = new MixedTableMaintainer(testKeyedTable); + tableMaintainer.expireSnapshots(tableRuntime); Assert.assertEquals(2, Iterables.size(testKeyedTable.changeTable().snapshots())); HashSet expectedSnapshots = new HashSet<>(); @@ -278,27 +193,19 @@ public void testNotExpireFlinkLatestCommit4All() { UnkeyedTable table = isKeyedTable() ? getArcticTable().asKeyedTable().baseTable() : getArcticTable().asUnkeyedTable(); writeAndCommitBaseStore(table); - Assert.assertEquals(Long.MAX_VALUE, - SnapshotsExpiringExecutor.fetchLatestFlinkCommittedSnapshotTime(table)); AppendFiles appendFiles = table.newAppend(); - appendFiles.set(SnapshotsExpiringExecutor.FLINK_MAX_COMMITTED_CHECKPOINT_ID, "100"); + appendFiles.set(FLINK_MAX_COMMITTED_CHECKPOINT_ID, "100"); appendFiles.commit(); long checkpointTime = table.currentSnapshot().timestampMillis(); - Assert.assertEquals(checkpointTime, - SnapshotsExpiringExecutor.fetchLatestFlinkCommittedSnapshotTime(table)); AppendFiles appendFiles2 = table.newAppend(); - appendFiles2.set(SnapshotsExpiringExecutor.FLINK_MAX_COMMITTED_CHECKPOINT_ID, "101"); + appendFiles2.set(FLINK_MAX_COMMITTED_CHECKPOINT_ID, "101"); appendFiles2.commit(); Snapshot checkpointTime2Snapshot = table.currentSnapshot(); - Assert.assertEquals(checkpointTime2Snapshot.timestampMillis(), - SnapshotsExpiringExecutor.fetchLatestFlinkCommittedSnapshotTime(table)); writeAndCommitBaseStore(table); Snapshot lastSnapshot = table.currentSnapshot(); - Assert.assertEquals(checkpointTime2Snapshot.timestampMillis(), - SnapshotsExpiringExecutor.fetchLatestFlinkCommittedSnapshotTime(table)); table.updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); TableRuntime tableRuntime = Mockito.mock(TableRuntime.class); @@ -309,8 +216,9 @@ public void testNotExpireFlinkLatestCommit4All() { TableConfiguration.parseConfig(table.properties())); Assert.assertEquals(4, Iterables.size(table.snapshots())); - SnapshotsExpiringExecutor.expireArcticTable( - table, tableRuntime); + + MixedTableMaintainer tableMaintainer = new MixedTableMaintainer(table); + tableMaintainer.expireSnapshots(tableRuntime); Assert.assertEquals(2, Iterables.size(table.snapshots())); HashSet expectedSnapshots = new HashSet<>(); @@ -333,7 +241,8 @@ public void testNotExpireOptimizeCommit4All() { Mockito.when(tableRuntime.getOptimizingStatus()).thenReturn(OptimizingStatus.IDLE); Mockito.when(tableRuntime.getTableConfiguration()).thenReturn( TableConfiguration.parseConfig(table.properties())); - SnapshotsExpiringExecutor.expireArcticTable(table, tableRuntime); + + new MixedTableMaintainer(table).expireSnapshots(tableRuntime); Assert.assertEquals(1, Iterables.size(table.snapshots())); table.newAppend().commit(); @@ -352,7 +261,7 @@ public void testNotExpireOptimizeCommit4All() { table.newAppend().commit(); expectedSnapshots.add(table.currentSnapshot()); - SnapshotsExpiringExecutor.expireArcticTable(table, tableRuntime); + new MixedTableMaintainer(table).expireSnapshots(tableRuntime); Assert.assertEquals(3, Iterables.size(table.snapshots())); Iterators.elementsEqual(expectedSnapshots.iterator(), table.snapshots().iterator()); } @@ -361,6 +270,7 @@ public void testNotExpireOptimizeCommit4All() { public void testExpireTableFiles4All() { UnkeyedTable table = isKeyedTable() ? getArcticTable().asKeyedTable().baseTable() : getArcticTable().asUnkeyedTable(); + table.updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); List dataFiles = writeAndCommitBaseStore(table); DeleteFiles deleteFiles = table.newDelete(); @@ -372,8 +282,7 @@ public void testExpireTableFiles4All() { List newDataFiles = writeAndCommitBaseStore(table); Assert.assertEquals(3, Iterables.size(table.snapshots())); - SnapshotsExpiringExecutor.expireSnapshots(table, System.currentTimeMillis(), - new HashSet<>()); + new MixedTableMaintainer(table).expireSnapshots(System.currentTimeMillis()); Assert.assertEquals(1, Iterables.size(table.snapshots())); dataFiles.forEach(file -> Assert.assertFalse(table.io().exists(file.path().toString()))); @@ -384,6 +293,10 @@ public void testExpireTableFiles4All() { public void testExpireTableFilesRepeatedly() { Assume.assumeTrue(isKeyedTable()); KeyedTable testKeyedTable = getArcticTable().asKeyedTable(); + + testKeyedTable.updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); + testKeyedTable.updateProperties().set(TableProperties.CHANGE_DATA_TTL, "0").commit(); + List s1Files = insertChangeDataFiles(testKeyedTable, 1); insertChangeDataFiles(testKeyedTable, 2); long secondCommitTime = testKeyedTable.changeTable().currentSnapshot().timestampMillis(); @@ -413,33 +326,23 @@ public void testExpireTableFilesRepeatedly() { Assert.assertEquals(12, Iterables.size(testKeyedTable.changeTable().newScan().planFiles())); Assert.assertEquals(3, Iterables.size(testKeyedTable.changeTable().snapshots())); - List expiredDataFileEntries = SnapshotsExpiringExecutor.getExpiredDataFileEntries( - testKeyedTable.changeTable(), secondCommitTime + 1); - Set top8FilesPath = top8Files.stream().map(ContentFile::path).collect(Collectors.toSet()); - Set expiredFilesPath = expiredDataFileEntries.stream() - .map(entry -> entry.getFile().path()).collect(Collectors.toSet()); - Assert.assertTrue(top8FilesPath.equals(expiredFilesPath)); - SnapshotsExpiringExecutor.deleteChangeFile(testKeyedTable, expiredDataFileEntries); - SnapshotsExpiringExecutor.expireSnapshots( - testKeyedTable.changeTable(), secondCommitTime + 1, new HashSet<>()); + MixedTableMaintainer tableMaintainer = new MixedTableMaintainer(testKeyedTable); + tableMaintainer.getChangeMaintainer().expireFiles(secondCommitTime + 1); + tableMaintainer.getChangeMaintainer().expireSnapshots(secondCommitTime + 1); + + Set dataFiles = getDataFiles(testKeyedTable); + Assert.assertEquals(last4File, dataFiles); + Assert.assertEquals(2, Iterables.size(testKeyedTable.changeTable().snapshots())); + } + + @NotNull + private static Set getDataFiles(KeyedTable testKeyedTable) { Set dataFiles = new HashSet<>(); testKeyedTable.changeTable().newScan().planFiles().forEach( task -> dataFiles.add(task.file().path()) ); - Assert.assertEquals(last4File, dataFiles); - - List expiredDataFileEntries2 = SnapshotsExpiringExecutor.getExpiredDataFileEntries( - testKeyedTable.changeTable(), secondCommitTime + 1); - Assert.assertTrue(expiredDataFileEntries2.isEmpty()); - - List expiredDataFileEntries3 = SnapshotsExpiringExecutor.getExpiredDataFileEntries( - testKeyedTable.changeTable(), thirdCommitTime + 1); - Set fileEntry3 = expiredDataFileEntries3.stream() - .map(entry -> entry.getFile().path()).collect(Collectors.toSet()); - Assert.assertTrue(last4File.equals(fileEntry3)); - - Assert.assertEquals(2, Iterables.size(testKeyedTable.changeTable().snapshots())); + return dataFiles; } private List insertChangeDataFiles(KeyedTable testKeyedTable, long transactionId) { diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestSnapshotExpireHive.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpireHive.java similarity index 90% rename from ams/server/src/test/java/com/netease/arctic/server/table/executor/TestSnapshotExpireHive.java rename to ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpireHive.java index c052f7e896..db34164a91 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestSnapshotExpireHive.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpireHive.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.netease.arctic.server.table.executor; +package com.netease.arctic.server.optimizing.maintainer; import com.netease.arctic.TableTestHelper; import com.netease.arctic.ams.api.TableFormat; @@ -24,6 +24,7 @@ import com.netease.arctic.hive.TestHMS; import com.netease.arctic.hive.catalog.HiveCatalogTestHelper; import com.netease.arctic.hive.catalog.HiveTableTestHelper; +import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.UnkeyedTable; import com.netease.arctic.utils.TableFileUtil; import org.apache.iceberg.DataFile; @@ -69,6 +70,10 @@ public void testExpireTableFiles() { DeleteFiles deleteHiveFiles = isKeyedTable() ? getArcticTable().asKeyedTable().baseTable().newDelete() : getArcticTable().asUnkeyedTable().newDelete(); + + getArcticTable().updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); + getArcticTable().updateProperties().set(TableProperties.CHANGE_DATA_TTL, "0").commit(); + for (DataFile hiveFile : hiveFiles) { Assert.assertTrue(getArcticTable().io().exists(hiveFile.path().toString())); deleteHiveFiles.deleteFile(hiveFile); @@ -97,7 +102,8 @@ public void testExpireTableFiles() { } UnkeyedTable unkeyedTable = isKeyedTable() ? getArcticTable().asKeyedTable().baseTable() : getArcticTable().asUnkeyedTable(); - SnapshotsExpiringExecutor.expireSnapshots(unkeyedTable, System.currentTimeMillis(), hiveLocation); + new MixedTableMaintainer(unkeyedTable) + .getBaseMaintainer().expireSnapshots(System.currentTimeMillis(), hiveLocation); Assert.assertEquals(1, Iterables.size(unkeyedTable.snapshots())); hiveFiles.forEach(file -> Assert.assertTrue(getArcticTable().io().exists(file.path().toString()))); diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestSnapshotExpireIceberg.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpireIceberg.java similarity index 94% rename from ams/server/src/test/java/com/netease/arctic/server/table/executor/TestSnapshotExpireIceberg.java rename to ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpireIceberg.java index 44ed09b1bb..715cb59c16 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestSnapshotExpireIceberg.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpireIceberg.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.netease.arctic.server.table.executor; +package com.netease.arctic.server.optimizing.maintainer; import com.netease.arctic.BasicTableTestHelper; import com.netease.arctic.TableTestHelper; diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/scan/TestKeyedTableFileScanHelper.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/scan/TestKeyedTableFileScanHelper.java index 2757a98166..8572e4e356 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/scan/TestKeyedTableFileScanHelper.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/scan/TestKeyedTableFileScanHelper.java @@ -48,8 +48,9 @@ @RunWith(Parameterized.class) public class TestKeyedTableFileScanHelper extends TableFileScanHelperTestBase { - public TestKeyedTableFileScanHelper(CatalogTestHelper catalogTestHelper, - TableTestHelper tableTestHelper) { + public TestKeyedTableFileScanHelper( + CatalogTestHelper catalogTestHelper, + TableTestHelper tableTestHelper) { super(catalogTestHelper, tableTestHelper); } @@ -57,9 +58,9 @@ public TestKeyedTableFileScanHelper(CatalogTestHelper catalogTestHelper, public static Object[][] parameters() { return new Object[][] { {new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(true, true)}, + new BasicTableTestHelper(true, true)}, {new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), - new BasicTableTestHelper(true, false)}}; + new BasicTableTestHelper(true, false)}}; } @Test @@ -71,7 +72,8 @@ public void testScanEmpty() { @Test public void testScanEmptySnapshot() { long transactionId = getArcticTable().beginTransaction(""); - OptimizingTestHelpers.appendBase(getArcticTable(), + OptimizingTestHelpers.appendBase( + getArcticTable(), tableTestHelper().writeBaseStore(getArcticTable(), transactionId, Collections.emptyList(), false)); List scan = scanFiles(); @@ -87,7 +89,8 @@ public void testScanOnlyBase() { tableTestHelper().generateTestRecord(4, "444", 0, "2022-01-01T12:00:00") ); long transactionId = getArcticTable().beginTransaction(""); - OptimizingTestHelpers.appendBase(getArcticTable(), + OptimizingTestHelpers.appendBase( + getArcticTable(), tableTestHelper().writeBaseStore(getArcticTable(), transactionId, newRecords, false)); List scan = scanFiles(); @@ -123,7 +126,8 @@ public void testScanChangeAndBase() { ); long transactionId = getArcticTable().beginTransaction(""); - OptimizingTestHelpers.appendBase(getArcticTable(), + OptimizingTestHelpers.appendBase( + getArcticTable(), tableTestHelper().writeBaseStore(getArcticTable(), transactionId, newRecords, false)); transactionId = getArcticTable().beginTransaction(""); @@ -164,7 +168,8 @@ public void testScanWithPosDelete() { tableTestHelper().generateTestRecord(4, "444", 0, "2022-01-01T12:00:00") ); long transactionId = getArcticTable().beginTransaction(""); - List dataFiles = OptimizingTestHelpers.appendBase(getArcticTable(), + List dataFiles = OptimizingTestHelpers.appendBase( + getArcticTable(), tableTestHelper().writeBaseStore(getArcticTable(), transactionId, newRecords, false)); List posDeleteFiles = Lists.newArrayList(); for (DataFile dataFile : dataFiles) { @@ -267,9 +272,7 @@ public void testGetMaxSequenceLimit() { Assert.assertEquals(Long.MAX_VALUE, KeyedTableFileScanHelper.getMaxSequenceKeepingTxIdInOrder(sequenceGroups, 9)); } - private static KeyedTableFileScanHelper.SnapshotFileGroup buildSequenceGroup(long sequence, long txId, int cnt) { return new KeyedTableFileScanHelper.SnapshotFileGroup(sequence, txId, cnt); } - } diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/TestTableRuntimeHandler.java b/ams/server/src/test/java/com/netease/arctic/server/table/TestTableRuntimeHandler.java index a2aad5b725..41fa73015d 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/TestTableRuntimeHandler.java +++ b/ams/server/src/test/java/com/netease/arctic/server/table/TestTableRuntimeHandler.java @@ -18,6 +18,7 @@ package com.netease.arctic.server.table; +import com.netease.arctic.AmoroTable; import com.netease.arctic.BasicTableTestHelper; import com.netease.arctic.TableTestHelper; import com.netease.arctic.ams.api.TableFormat; @@ -83,7 +84,9 @@ public void testInitialize() { Assert.assertEquals(createTableId.getId().longValue(), handler.getInitTables().get(0).getTableId()); // test change properties - tableService().loadTable(createTableId).updateProperties() + ArcticTable arcticTable = (ArcticTable)tableService().loadTable(createTableId).originalTable(); + + arcticTable.updateProperties() .set(TableProperties.ENABLE_ORPHAN_CLEAN, "true").commit(); tableService().getRuntime(createTableId).refresh(tableService.loadTable(serverTableIdentifier())); Assert.assertEquals(1, handler.getConfigChangedTables().size()); @@ -131,8 +134,8 @@ protected void handleConfigChanged( @Override protected void handleTableAdded( - ArcticTable table, TableRuntime tableRuntime) { - addedTables.add(Pair.of(table, tableRuntime)); + AmoroTable table, TableRuntime tableRuntime) { + addedTables.add(Pair.of((ArcticTable) table.originalTable(), tableRuntime)); } @Override diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/TestTableRuntimeManager.java b/ams/server/src/test/java/com/netease/arctic/server/table/TestTableRuntimeManager.java index 8c5009e89e..6e5206a0d8 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/TestTableRuntimeManager.java +++ b/ams/server/src/test/java/com/netease/arctic/server/table/TestTableRuntimeManager.java @@ -52,7 +52,7 @@ public TestTableRuntimeManager(CatalogTestHelper catalogTestHelper, @Test public void testLoadTable() { - ArcticTable arcticTable = tableService().loadTable(serverTableIdentifier()); + ArcticTable arcticTable = (ArcticTable) tableService().loadTable(serverTableIdentifier()).originalTable(); validateArcticTable(arcticTable); // test load not existed table diff --git a/core/pom.xml b/core/pom.xml index 39d089d2b6..6d03ec4941 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -109,6 +109,11 @@ commons-lang3 + + org.apache.paimon + paimon-bundle + + diff --git a/core/src/main/java/com/netease/arctic/AlreadyExistsException.java b/core/src/main/java/com/netease/arctic/AlreadyExistsException.java index e2b7c220a6..16b09c5bdb 100644 --- a/core/src/main/java/com/netease/arctic/AlreadyExistsException.java +++ b/core/src/main/java/com/netease/arctic/AlreadyExistsException.java @@ -22,4 +22,12 @@ public class AlreadyExistsException extends RuntimeException { public AlreadyExistsException(String message) { super(message); } + + public AlreadyExistsException(String message, Throwable cause) { + super(message, cause); + } + + public AlreadyExistsException(Throwable cause) { + super(cause); + } } diff --git a/core/src/main/java/com/netease/arctic/AmoroTable.java b/core/src/main/java/com/netease/arctic/AmoroTable.java index 420c16c3f5..75e48bba03 100644 --- a/core/src/main/java/com/netease/arctic/AmoroTable.java +++ b/core/src/main/java/com/netease/arctic/AmoroTable.java @@ -55,5 +55,5 @@ default String name() { /** * Returns the current snapshot of this table */ - Snapshot currentSnapshot(); + TableSnapshot currentSnapshot(); } diff --git a/core/src/main/java/com/netease/arctic/DatabaseNotEmptyException.java b/core/src/main/java/com/netease/arctic/DatabaseNotEmptyException.java new file mode 100644 index 0000000000..5ae6305191 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/DatabaseNotEmptyException.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic; + +public class DatabaseNotEmptyException extends RuntimeException { + + public DatabaseNotEmptyException(String message) { + super(message); + } + + public DatabaseNotEmptyException(String message, Throwable cause) { + super(message, cause); + } + + public DatabaseNotEmptyException(Throwable cause) { + super(cause); + } +} diff --git a/core/src/main/java/com/netease/arctic/NoSuchDatabaseException.java b/core/src/main/java/com/netease/arctic/NoSuchDatabaseException.java index f7f176c61f..a55bf4a362 100644 --- a/core/src/main/java/com/netease/arctic/NoSuchDatabaseException.java +++ b/core/src/main/java/com/netease/arctic/NoSuchDatabaseException.java @@ -30,4 +30,16 @@ public NoSuchDatabaseException(String message, Object... args) { public NoSuchDatabaseException(Throwable e, String message, Object... args) { super(String.format(message, args), e); } + + public NoSuchDatabaseException(String message) { + super(message); + } + + public NoSuchDatabaseException(String message, Throwable cause) { + super(message, cause); + } + + public NoSuchDatabaseException(Throwable cause) { + super(cause); + } } diff --git a/core/src/main/java/com/netease/arctic/NoSuchTableException.java b/core/src/main/java/com/netease/arctic/NoSuchTableException.java index 6ac1da90c1..d588722e0a 100644 --- a/core/src/main/java/com/netease/arctic/NoSuchTableException.java +++ b/core/src/main/java/com/netease/arctic/NoSuchTableException.java @@ -22,4 +22,12 @@ public class NoSuchTableException extends RuntimeException { public NoSuchTableException(String message) { super(message); } + + public NoSuchTableException(String message, Throwable cause) { + super(message, cause); + } + + public NoSuchTableException(Throwable cause) { + super(cause); + } } diff --git a/core/src/main/java/com/netease/arctic/Snapshot.java b/core/src/main/java/com/netease/arctic/TableSnapshot.java similarity index 89% rename from core/src/main/java/com/netease/arctic/Snapshot.java rename to core/src/main/java/com/netease/arctic/TableSnapshot.java index e15fc99c0b..4da2a7c8e9 100644 --- a/core/src/main/java/com/netease/arctic/Snapshot.java +++ b/core/src/main/java/com/netease/arctic/TableSnapshot.java @@ -18,10 +18,10 @@ package com.netease.arctic; -public interface Snapshot { +public interface TableSnapshot { /** - * Returns the watermark of this snapshot. + * Returns the watermark of this snapshot. Return -1 if the watermark does not exist. */ long watermark(); @@ -34,4 +34,4 @@ public interface Snapshot { * Returns the id of this snapshot. */ String id(); -} +} \ No newline at end of file diff --git a/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergCatalogFactory.java b/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergCatalogFactory.java index df24288f66..84e31776c4 100644 --- a/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergCatalogFactory.java +++ b/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergCatalogFactory.java @@ -38,6 +38,15 @@ public class IcebergCatalogFactory implements FormatCatalogFactory { @Override public FormatCatalog create( String name, String metastoreType, Map properties, Configuration configuration) { + Catalog icebergCatalog = icebergCatalog(name, metastoreType, properties, configuration); + return new IcebergCatalog(icebergCatalog); + } + + public static Catalog icebergCatalog( + String name, + String metastoreType, + Map properties, + Configuration configuration) { Preconditions.checkArgument(StringUtils.isNotBlank(metastoreType), "metastore type is blank"); Map icebergProperties = Maps.newHashMap(properties); if (CatalogMetaProperties.CATALOG_TYPE_HADOOP.equalsIgnoreCase(metastoreType) || @@ -56,8 +65,7 @@ public FormatCatalog create( icebergProperties.put(CatalogProperties.CATALOG_IMPL, icebergCatalogImpl); } - Catalog icebergCatalog = CatalogUtil.buildIcebergCatalog(name, icebergProperties, configuration); - return new IcebergCatalog(icebergCatalog); + return CatalogUtil.buildIcebergCatalog(name, icebergProperties, configuration); } @Override diff --git a/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergSnapshot.java b/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergSnapshot.java index d8d06a97b6..684a31a369 100644 --- a/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergSnapshot.java +++ b/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergSnapshot.java @@ -18,18 +18,19 @@ package com.netease.arctic.formats.iceberg; -import com.netease.arctic.Snapshot; +import com.netease.arctic.TableSnapshot; +import org.apache.iceberg.Snapshot; -public class IcebergSnapshot implements Snapshot { - org.apache.iceberg.Snapshot snapshot; +public class IcebergSnapshot implements TableSnapshot { + private final Snapshot snapshot; - public IcebergSnapshot(org.apache.iceberg.Snapshot snapshot) { + public IcebergSnapshot(Snapshot snapshot) { this.snapshot = snapshot; } @Override public long watermark() { - return commitTime(); + return -1L; } @Override @@ -41,4 +42,8 @@ public long commitTime() { public String id() { return String.valueOf(snapshot.snapshotId()); } + + public Snapshot icebergSnapshot() { + return snapshot; + } } diff --git a/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergTable.java b/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergTable.java index 8f050eeb33..c6b792fc41 100644 --- a/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergTable.java +++ b/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergTable.java @@ -19,17 +19,18 @@ package com.netease.arctic.formats.iceberg; import com.netease.arctic.AmoroTable; -import com.netease.arctic.Snapshot; +import com.netease.arctic.TableSnapshot; import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.table.TableIdentifier; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import java.util.Map; public class IcebergTable implements AmoroTable

{ - TableIdentifier identifier; - Table table; + private final TableIdentifier identifier; + private final Table table; public IcebergTable(TableIdentifier identifier, Table table) { this.table = table; @@ -57,8 +58,8 @@ public Table originalTable() { } @Override - public Snapshot currentSnapshot() { - org.apache.iceberg.Snapshot snapshot = table.currentSnapshot(); + public TableSnapshot currentSnapshot() { + Snapshot snapshot = table.currentSnapshot(); return new IcebergSnapshot(snapshot); } } diff --git a/core/src/main/java/com/netease/arctic/formats/paimon/PaimonFormatCatalog.java b/core/src/main/java/com/netease/arctic/formats/mixed/MixedHiveTable.java similarity index 64% rename from core/src/main/java/com/netease/arctic/formats/paimon/PaimonFormatCatalog.java rename to core/src/main/java/com/netease/arctic/formats/mixed/MixedHiveTable.java index f873593305..2bd0da1ef3 100644 --- a/core/src/main/java/com/netease/arctic/formats/paimon/PaimonFormatCatalog.java +++ b/core/src/main/java/com/netease/arctic/formats/mixed/MixedHiveTable.java @@ -16,25 +16,19 @@ * limitations under the License. */ -package com.netease.arctic.formats.paimon; +package com.netease.arctic.formats.mixed; -import com.netease.arctic.FormatCatalog; -import com.netease.arctic.FormatCatalogFactory; import com.netease.arctic.ams.api.TableFormat; -import org.apache.hadoop.conf.Configuration; +import com.netease.arctic.table.ArcticTable; -import java.util.Map; +public class MixedHiveTable extends MixedIcebergTable { -public class PaimonFormatCatalog implements FormatCatalogFactory { - @Override - public FormatCatalog create( - String name, String metastoreType, Map properties, Configuration configuration) { - // TODO: implement this method - return null; + public MixedHiveTable(ArcticTable arcticTable) { + super(arcticTable); } @Override public TableFormat format() { - return TableFormat.PAIMON; + return TableFormat.MIXED_HIVE; } } diff --git a/core/src/main/java/com/netease/arctic/formats/mixed/MixedIcebergTable.java b/core/src/main/java/com/netease/arctic/formats/mixed/MixedIcebergTable.java new file mode 100644 index 0000000000..fbb52c5cb3 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/formats/mixed/MixedIcebergTable.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats.mixed; + +import com.netease.arctic.AmoroTable; +import com.netease.arctic.TableSnapshot; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.TableIdentifier; +import org.apache.iceberg.Snapshot; + +import java.util.Map; + +public class MixedIcebergTable implements AmoroTable { + + private final ArcticTable arcticTable; + + public MixedIcebergTable(ArcticTable arcticTable) { + this.arcticTable = arcticTable; + } + + @Override + public TableIdentifier id() { + return arcticTable.id(); + } + + @Override + public TableFormat format() { + return TableFormat.MIXED_ICEBERG; + } + + @Override + public Map properties() { + return arcticTable.properties(); + } + + @Override + public ArcticTable originalTable() { + return arcticTable; + } + + @Override + public TableSnapshot currentSnapshot() { + Snapshot changeSnapshot; + Snapshot baseSnapshot; + if (arcticTable.isKeyedTable()) { + changeSnapshot = arcticTable.asKeyedTable().changeTable().currentSnapshot(); + baseSnapshot = arcticTable.asKeyedTable().baseTable().currentSnapshot(); + } else { + changeSnapshot = null; + baseSnapshot = arcticTable.asUnkeyedTable().currentSnapshot(); + } + + if (changeSnapshot == null && baseSnapshot == null) { + return null; + } + + return new MixedSnapshot(changeSnapshot, baseSnapshot); + } +} diff --git a/core/src/main/java/com/netease/arctic/formats/mixed/MixedSnapshot.java b/core/src/main/java/com/netease/arctic/formats/mixed/MixedSnapshot.java new file mode 100644 index 0000000000..ea50de0f82 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/formats/mixed/MixedSnapshot.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats.mixed; + +import com.netease.arctic.TableSnapshot; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.relocated.com.google.common.primitives.Longs; + +import java.util.Optional; + +public class MixedSnapshot implements TableSnapshot { + + private final Snapshot changeSnapshot; + + private final Snapshot baseSnapshot; + + public MixedSnapshot( + Snapshot changeSnapshot, + Snapshot baseSnapshot) { + this.changeSnapshot = changeSnapshot; + this.baseSnapshot = baseSnapshot; + } + + @Override + public long watermark() { + return -1; + } + + @Override + public long commitTime() { + Long changCommit = Optional.ofNullable(changeSnapshot).map(Snapshot::timestampMillis).orElse(-1L); + Long baseCommit = Optional.ofNullable(baseSnapshot).map(Snapshot::timestampMillis).orElse(-1L); + return Longs.max(changCommit, baseCommit); + } + + public Snapshot getChangeSnapshot() { + return changeSnapshot; + } + + public Snapshot getBaseSnapshot() { + return baseSnapshot; + } + + public long getChangeSnapshotId() { + return Optional.ofNullable(changeSnapshot).map(Snapshot::snapshotId).orElse(-1L); + } + + public long getBaseSnapshotId() { + return Optional.ofNullable(baseSnapshot).map(Snapshot::snapshotId).orElse(-1L); + } + + @Override + public String id() { + return changeSnapshot + "_" + baseSnapshot; + } +} diff --git a/core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalog.java b/core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalog.java new file mode 100644 index 0000000000..3d54f5cd86 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalog.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats.paimon; + +import com.netease.arctic.AlreadyExistsException; +import com.netease.arctic.AmoroTable; +import com.netease.arctic.DatabaseNotEmptyException; +import com.netease.arctic.FormatCatalog; +import com.netease.arctic.NoSuchDatabaseException; +import com.netease.arctic.NoSuchTableException; +import com.netease.arctic.table.TableIdentifier; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; + +import java.util.List; + +public class PaimonCatalog implements FormatCatalog { + + private final Catalog catalog; + + private final String name; + + public PaimonCatalog(Catalog catalog, String name) { + this.catalog = catalog; + this.name = name; + } + + @Override + public List listDatabases() { + return catalog.listDatabases(); + } + + @Override + public boolean exist(String database) { + return catalog.databaseExists(database); + } + + @Override + public boolean exist(String database, String table) { + return catalog.tableExists(Identifier.create(database, table)); + } + + @Override + public void createDatabase(String database) { + try { + catalog.createDatabase(database, false); + } catch (Catalog.DatabaseAlreadyExistException e) { + throw new AlreadyExistsException(e); + } + } + + @Override + public void dropDatabase(String database) { + try { + catalog.dropDatabase(database, false, false); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchDatabaseException(e); + } catch (Catalog.DatabaseNotEmptyException e) { + throw new DatabaseNotEmptyException(e); + } + } + + @Override + public AmoroTable loadTable(String database, String table) { + try { + return new PaimonTable( + TableIdentifier.of(name, database, table), + catalog.getTable(Identifier.create(database, table))); + } catch (Catalog.TableNotExistException e) { + throw new NoSuchTableException(e); + } + } + + @Override + public List listTables(String database) { + try { + return catalog.listTables(database); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchDatabaseException(e); + } + } +} diff --git a/core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalogFactory.java b/core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalogFactory.java new file mode 100644 index 0000000000..0284d1723b --- /dev/null +++ b/core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalogFactory.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats.paimon; + +import com.netease.arctic.FormatCatalogFactory; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.ams.api.properties.CatalogMetaProperties; +import org.apache.hadoop.conf.Configuration; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.FileSystemCatalogFactory; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; + +import java.util.Map; + +public class PaimonCatalogFactory implements FormatCatalogFactory { + @Override + public PaimonCatalog create( + String name, String metastoreType, Map properties, Configuration configuration) { + return new PaimonCatalog(paimonCatalog(metastoreType, properties, configuration), name); + } + + public static Catalog paimonCatalog( + String metastoreType, Map properties, Configuration configuration) { + Options options = Options.fromMap(properties); + + String type; + if (CatalogMetaProperties.CATALOG_TYPE_HADOOP.equalsIgnoreCase(metastoreType)) { + type = FileSystemCatalogFactory.IDENTIFIER; + } else { + type = metastoreType; + } + options.set(CatalogOptions.METASTORE, type); + + CatalogContext catalogContext = CatalogContext.create(options, configuration); + return CatalogFactory.createCatalog(catalogContext); + } + + @Override + public TableFormat format() { + return TableFormat.PAIMON; + } +} diff --git a/core/src/main/java/com/netease/arctic/formats/paimon/PaimonSnapshot.java b/core/src/main/java/com/netease/arctic/formats/paimon/PaimonSnapshot.java new file mode 100644 index 0000000000..388cc8cbab --- /dev/null +++ b/core/src/main/java/com/netease/arctic/formats/paimon/PaimonSnapshot.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats.paimon; + +import com.netease.arctic.TableSnapshot; +import org.apache.paimon.Snapshot; + +public class PaimonSnapshot implements TableSnapshot { + + private final Snapshot snapshot; + + public PaimonSnapshot(Snapshot snapshot) { + this.snapshot = snapshot; + } + + @Override + public long watermark() { + Long watermark = snapshot.watermark(); + return watermark == null ? -1 : watermark; + } + + @Override + public long commitTime() { + return snapshot.timeMillis(); + } + + @Override + public String id() { + return String.valueOf(snapshot.id()); + } +} diff --git a/core/src/main/java/com/netease/arctic/formats/paimon/PaimonTable.java b/core/src/main/java/com/netease/arctic/formats/paimon/PaimonTable.java new file mode 100644 index 0000000000..0bd29cd8e4 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/formats/paimon/PaimonTable.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats.paimon; + +import com.netease.arctic.AmoroTable; +import com.netease.arctic.TableSnapshot; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.table.TableIdentifier; +import org.apache.paimon.Snapshot; +import org.apache.paimon.table.DataTable; +import org.apache.paimon.table.Table; + +import java.util.Map; + +public class PaimonTable implements AmoroTable
{ + + private final TableIdentifier tableIdentifier; + + private final Table table; + + public PaimonTable(TableIdentifier tableIdentifier, Table table) { + this.tableIdentifier = tableIdentifier; + this.table = table; + } + + @Override + public TableIdentifier id() { + return tableIdentifier; + } + + @Override + public TableFormat format() { + return TableFormat.PAIMON; + } + + @Override + public Map properties() { + return table.options(); + } + + @Override + public Table originalTable() { + return table; + } + + @Override + public TableSnapshot currentSnapshot() { + if (!(table instanceof DataTable)) { + return null; + } + + Snapshot snapshot = ((DataTable) table).snapshotManager().latestSnapshot(); + return snapshot == null ? null : new PaimonSnapshot(snapshot); + } +} diff --git a/core/src/main/resources/META-INF/services/com.netease.arctic.FormatCatalogFactory b/core/src/main/resources/META-INF/services/com.netease.arctic.FormatCatalogFactory index fba21f2079..b97c258b2f 100644 --- a/core/src/main/resources/META-INF/services/com.netease.arctic.FormatCatalogFactory +++ b/core/src/main/resources/META-INF/services/com.netease.arctic.FormatCatalogFactory @@ -17,3 +17,4 @@ # com.netease.arctic.formats.iceberg.IcebergCatalogFactory +com.netease.arctic.formats.paimon.PaimonCatalogFactory \ No newline at end of file diff --git a/core/src/test/java/com/netease/arctic/formats/AmoroCatalogTestBase.java b/core/src/test/java/com/netease/arctic/formats/AmoroCatalogTestBase.java new file mode 100644 index 0000000000..27229a5e3a --- /dev/null +++ b/core/src/test/java/com/netease/arctic/formats/AmoroCatalogTestBase.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats; + +import com.netease.arctic.AmoroCatalog; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; + +public abstract class AmoroCatalogTestBase { + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + protected AmoroCatalogTestHelper catalogTestHelper; + + protected AmoroCatalog amoroCatalog; + + protected Object originalCatalog; + + public AmoroCatalogTestBase(AmoroCatalogTestHelper catalogTestHelper) { + this.catalogTestHelper = catalogTestHelper; + } + + @Before + public void setupCatalog() throws IOException { + String path = temp.newFolder().getPath(); + catalogTestHelper.initWarehouse(path); + this.amoroCatalog = catalogTestHelper.amoroCatalog(); + this.originalCatalog = catalogTestHelper.originalCatalog(); + } + + @After + public void cleanCatalog() { + catalogTestHelper.clean(); + } +} diff --git a/core/src/test/java/com/netease/arctic/formats/AmoroCatalogTestHelper.java b/core/src/test/java/com/netease/arctic/formats/AmoroCatalogTestHelper.java new file mode 100644 index 0000000000..a387564448 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/formats/AmoroCatalogTestHelper.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats; + +import com.netease.arctic.AmoroCatalog; +import com.netease.arctic.ams.api.CatalogMeta; +import org.apache.hadoop.conf.Configuration; + +/** + * Helper Interface for testing AmoroCatalog. + */ +public interface AmoroCatalogTestHelper { + + /** + * Table option key when using {@link #createTable(String, String)} to create table. + */ + String DEFAULT_TABLE_OPTION_KEY = "amoro.test.key"; + + /** + * Table option value when using {@link #createTable(String, String)} to create table. + */ + String DEFAULT_TABLE_OPTION_VALUE = "amoro.test.value"; + + /** + * Table column named 'id' when using {@link #createTable(String, String)} to create table. + */ + String DEFAULT_SCHEMA_ID_NAME = "id"; + + /** + * Table column named 'name' when using {@link #createTable(String, String)} to create table. + */ + String DEFAULT_SCHEMA_NAME_NAME = "name"; + + /** + * Table column named 'age' when using {@link #createTable(String, String)} to create table. + */ + String DEFAULT_SCHEMA_AGE_NAME = "age"; + + /** + * Will be called first to inject the warehouse location. + */ + void initWarehouse(String warehouseLocation); + + /** + * Will be called first to inject the hive configuration. + */ + void initHiveConf(Configuration hiveConf); + + /** + * Get the {@link CatalogMeta} for the catalog. + */ + CatalogMeta getCatalogMeta(); + + /** + * Get the {@link AmoroCatalog} for the catalog. + */ + AmoroCatalog amoroCatalog(); + + /** + * Get the original catalog for the catalog. It will be paimon catalog or iceberg catalog or other. + */ + T originalCatalog(); + + /** + * Get the catalog name. + */ + String catalogName(); + + /** + * Clean the catalog. drop databases and tables. + */ + void clean(); + + /** + * Create a table. The schema, properties, etc. of a table depend on its implementation. + */ + void createTable(String db, String tableName) + throws Exception; +} diff --git a/core/src/test/java/com/netease/arctic/formats/IcebergHadoopCatalogTestHelper.java b/core/src/test/java/com/netease/arctic/formats/IcebergHadoopCatalogTestHelper.java new file mode 100644 index 0000000000..7ec997ed31 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/formats/IcebergHadoopCatalogTestHelper.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.netease.arctic.AmoroCatalog; +import com.netease.arctic.ams.api.CatalogMeta; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.ams.api.properties.CatalogMetaProperties; +import com.netease.arctic.catalog.CatalogTestHelpers; +import com.netease.arctic.formats.iceberg.IcebergCatalogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; + +import java.util.HashMap; +import java.util.Map; + +public class IcebergHadoopCatalogTestHelper implements AmoroCatalogTestHelper { + + public static final Schema schema = new Schema( + Lists.newArrayList( + Types.NestedField.required(1, DEFAULT_SCHEMA_ID_NAME, Types.LongType.get()), + Types.NestedField.required(2, DEFAULT_SCHEMA_NAME_NAME, Types.StringType.get()), + Types.NestedField.optional(3, DEFAULT_SCHEMA_AGE_NAME, Types.IntegerType.get())), + Sets.newHashSet(1) + ); + + public static final PartitionSpec spec = PartitionSpec.builderFor(schema) + .identity(DEFAULT_SCHEMA_AGE_NAME) + .build(); + + public static final Map properties = new HashMap<>(); + + static { + properties.put(DEFAULT_TABLE_OPTION_KEY, DEFAULT_TABLE_OPTION_VALUE); + } + + protected final String catalogName; + + protected final Map catalogProperties; + + public IcebergHadoopCatalogTestHelper(String catalogName, Map catalogProperties) { + this.catalogName = catalogName; + this.catalogProperties = catalogProperties == null ? new HashMap<>() : catalogProperties; + } + + @Override + public void initWarehouse(String warehouseLocation) { + catalogProperties.put(CatalogMetaProperties.KEY_WAREHOUSE, warehouseLocation); + } + + @Override + public void initHiveConf(Configuration hiveConf) { + //Do nothing + } + + @Override + public CatalogMeta getCatalogMeta() { + return CatalogTestHelpers.buildCatalogMeta( + catalogName, + getMetastoreType(), + catalogProperties, + TableFormat.ICEBERG); + } + + @Override + public AmoroCatalog amoroCatalog() { + IcebergCatalogFactory icebergCatalogFactory = new IcebergCatalogFactory(); + return icebergCatalogFactory.create( + catalogName, + getMetastoreType(), + catalogProperties, + new Configuration() + ); + } + + @Override + public Catalog originalCatalog() { + return IcebergCatalogFactory.icebergCatalog( + catalogName, + getMetastoreType(), + catalogProperties, + new Configuration()); + } + + @Override + public String catalogName() { + return catalogName; + } + + @Override + public void clean() { + Catalog catalog = originalCatalog(); + if (catalog instanceof SupportsNamespaces) { + for (Namespace ns : ((SupportsNamespaces) catalog).listNamespaces()) { + catalog.listTables(ns).forEach(tableIdentifier -> catalog.dropTable(tableIdentifier, true)); + try { + ((SupportsNamespaces) catalog).dropNamespace(ns); + } catch (Exception e) { + //'default' database can not be dropped in hive catalog. + } + } + } + } + + @Override + public void createTable(String db, String tableName) throws Exception { + Catalog catalog = originalCatalog(); + catalog.createTable(TableIdentifier.of(db, tableName), schema, spec, properties); + } + + protected String getMetastoreType() { + return CatalogMetaProperties.CATALOG_TYPE_HADOOP; + } + + public static IcebergHadoopCatalogTestHelper defaultHelper() { + return new IcebergHadoopCatalogTestHelper("test_iceberg_catalog", new HashMap<>()); + } +} diff --git a/core/src/test/java/com/netease/arctic/formats/PaimonHadoopCatalogTestHelper.java b/core/src/test/java/com/netease/arctic/formats/PaimonHadoopCatalogTestHelper.java new file mode 100644 index 0000000000..31da2ae9c8 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/formats/PaimonHadoopCatalogTestHelper.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats; + +import com.netease.arctic.AmoroCatalog; +import com.netease.arctic.ams.api.CatalogMeta; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.ams.api.properties.CatalogMetaProperties; +import com.netease.arctic.catalog.CatalogTestHelpers; +import com.netease.arctic.formats.paimon.PaimonCatalogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.types.DataTypes; + +import java.util.HashMap; +import java.util.Map; + +public class PaimonHadoopCatalogTestHelper implements AmoroCatalogTestHelper { + + public static final Schema schema = Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .column("age", DataTypes.INT()) + .primaryKey("id", "age") + .partitionKeys("age") + .option("amoro.test.key", "amoro.test.value") + .build(); + + protected final String catalogName; + + protected final Map catalogProperties; + + public PaimonHadoopCatalogTestHelper( + String catalogName, + Map catalogProperties) { + this.catalogName = catalogName; + this.catalogProperties = catalogProperties == null ? new HashMap<>() : catalogProperties; + } + + public void initWarehouse(String warehouseLocation) { + catalogProperties.put(CatalogOptions.WAREHOUSE.key(), warehouseLocation); + } + + @Override + public void initHiveConf(Configuration hiveConf) { + //Do nothing + } + + @Override + public CatalogMeta getCatalogMeta() { + return CatalogTestHelpers.buildCatalogMeta( + catalogName, + getMetastoreType(), + catalogProperties, + TableFormat.PAIMON); + } + + @Override + public AmoroCatalog amoroCatalog() { + PaimonCatalogFactory paimonCatalogFactory = new PaimonCatalogFactory(); + return paimonCatalogFactory.create( + catalogName, + getMetastoreType(), + catalogProperties, + new Configuration() + ); + } + + @Override + public Catalog originalCatalog() { + return PaimonCatalogFactory.paimonCatalog(getMetastoreType(), catalogProperties, null); + } + + @Override + public String catalogName() { + return catalogName; + } + + @Override + public void clean() { + try (Catalog catalog = originalCatalog()) { + for (String dbName : catalog.listDatabases()) { + try { + catalog.dropDatabase(dbName, true, true); + continue; + } catch (Exception e) { + // If drop database failed, drop all tables in this database. Because 'default' database can not be + // dropped in hive catalog. + } + for (String tableName : catalog.listTables(dbName)) { + catalog.dropTable(Identifier.create(dbName, tableName), true); + } + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void createTable(String db, String tableName) + throws Exception { + try (Catalog catalog = originalCatalog()) { + catalog.createTable(Identifier.create(db, tableName), schema, false); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected String getMetastoreType() { + return CatalogMetaProperties.CATALOG_TYPE_HADOOP; + } + + public static PaimonHadoopCatalogTestHelper defaultHelper() { + return new PaimonHadoopCatalogTestHelper( + "test_paimon_catalog", + new HashMap<>()); + } +} diff --git a/core/src/test/java/com/netease/arctic/formats/TestAmoroCatalogBase.java b/core/src/test/java/com/netease/arctic/formats/TestAmoroCatalogBase.java new file mode 100644 index 0000000000..4f3bc11ba4 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/formats/TestAmoroCatalogBase.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats; + +import com.netease.arctic.AmoroTable; +import com.netease.arctic.table.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +public abstract class TestAmoroCatalogBase extends AmoroCatalogTestBase { + + private static final String DB1 = "db1"; + private static final String DB2 = "db2"; + private static final String DB3 = "db3"; + + private static final String TABLE = "table"; + + public TestAmoroCatalogBase(AmoroCatalogTestHelper catalogTestHelper) { + super(catalogTestHelper); + } + + protected abstract void createDatabase(String dbName); + + protected abstract void createTable(String dbName, String tableName, Map properties); + + protected abstract List listDatabases(); + + @Test + public void testListDatabases() { + createDatabase(DB1); + createDatabase(DB2); + createDatabase(DB3); + HashSet databases = Sets.newHashSet(amoroCatalog.listDatabases()); + Assert.assertTrue(databases.contains(DB1)); + Assert.assertTrue(databases.contains(DB2)); + Assert.assertTrue(databases.contains(DB3)); + } + + @Test + public void testDropDatabases() { + createDatabase(DB1); + amoroCatalog.dropDatabase(DB1); + + Assert.assertFalse(amoroCatalog.listDatabases().contains(DB1)); + } + + @Test + public void testCreateDatabases() { + amoroCatalog.createDatabase(DB1); + Assert.assertTrue(listDatabases().contains(DB1)); + } + + @Test + public void testExistsDatabase() { + createDatabase(DB1); + Assert.assertTrue(amoroCatalog.exist(DB1)); + } + + @Test + public void testExistsTable() { + createDatabase(DB1); + createTable(DB1, TABLE, new HashMap<>()); + Assert.assertTrue(amoroCatalog.exist(DB1, TABLE)); + } + + @Test + public void testLoadTable() { + createDatabase(DB1); + Map properties = new HashMap<>(); + properties.put("key1", "value1"); + createTable(DB1, TABLE, properties); + AmoroTable amoroTable = amoroCatalog.loadTable(DB1, TABLE); + Assert.assertEquals(amoroTable.properties().get("key1"), "value1"); + Assert.assertEquals(amoroTable.name(), catalogTestHelper.catalogName() + "." + DB1 + "." + TABLE); + Assert.assertEquals( + amoroTable.id(), + TableIdentifier.of(catalogTestHelper.catalogName(), DB1, TABLE)); + } +} diff --git a/core/src/test/java/com/netease/arctic/formats/TestIcebergAmoroCatalog.java b/core/src/test/java/com/netease/arctic/formats/TestIcebergAmoroCatalog.java new file mode 100644 index 0000000000..563dea02a6 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/formats/TestIcebergAmoroCatalog.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +@RunWith(Parameterized.class) +public class TestIcebergAmoroCatalog extends TestAmoroCatalogBase { + + public TestIcebergAmoroCatalog(AmoroCatalogTestHelper amoroCatalogTestHelper) { + super(amoroCatalogTestHelper); + } + + @Parameterized.Parameters(name = "{0}") + public static Object[] parameters() { + return new Object[] { + IcebergHadoopCatalogTestHelper.defaultHelper() + }; + } + + @Override + protected void createDatabase(String dbName) { + if (catalog() instanceof SupportsNamespaces) { + Namespace ns = Namespace.of(dbName); + ((SupportsNamespaces) catalog()).createNamespace(ns); + } + } + + @Override + protected void createTable(String dbName, String tableName, Map properties) { + TableIdentifier identifier = TableIdentifier.of(dbName, tableName); + + Schema schema = new Schema( + Types.NestedField.of(1, false, "id", Types.IntegerType.get()), + Types.NestedField.of(2, false, "data", Types.StringType.get()) + ); + + catalog().newCreateTableTransaction(identifier, schema, PartitionSpec.unpartitioned(), properties) + .commitTransaction(); + } + + @Override + protected List listDatabases() { + if (catalog() instanceof SupportsNamespaces) { + return ((SupportsNamespaces) catalog()).listNamespaces() + .stream().map(ns -> ns.level(0)) + .collect(Collectors.toList()); + } + return Lists.newArrayList(); + } + + private Catalog catalog() { + return (Catalog) originalCatalog; + } +} diff --git a/core/src/test/java/com/netease/arctic/formats/TestPaimonAmoroCatalog.java b/core/src/test/java/com/netease/arctic/formats/TestPaimonAmoroCatalog.java new file mode 100644 index 0000000000..6e52f8a3c2 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/formats/TestPaimonAmoroCatalog.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.formats; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.types.DataTypes; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.List; +import java.util.Map; + +@RunWith(Parameterized.class) +public class TestPaimonAmoroCatalog extends TestAmoroCatalogBase { + + public TestPaimonAmoroCatalog(AmoroCatalogTestHelper amoroCatalogTestHelper) { + super(amoroCatalogTestHelper); + } + + @Parameterized.Parameters(name = "{0}") + public static Object[] parameters() { + return new Object[] { + PaimonHadoopCatalogTestHelper.defaultHelper() + }; + } + + @Override + protected void createDatabase(String dbName) { + try (Catalog catalog = catalog()) { + catalog.createDatabase(dbName, false); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + protected void createTable(String dbName, String tableName, Map properties) { + Schema schema = Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id", "name") + .options(properties) + .build(); + try (Catalog catalog = catalog()) { + catalog.createTable(Identifier.create(dbName, tableName), schema, false); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + protected List listDatabases() { + try (Catalog catalog = catalog()) { + return catalog.listDatabases(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private Catalog catalog() { + return (Catalog) originalCatalog; + } +} diff --git a/dist/src/main/arctic-bin/conf/config.yaml b/dist/src/main/arctic-bin/conf/config.yaml index f1bc9d7020..addf41aab4 100644 --- a/dist/src/main/arctic-bin/conf/config.yaml +++ b/dist/src/main/arctic-bin/conf/config.yaml @@ -52,13 +52,13 @@ ams: jdbc-driver-class: org.apache.derby.jdbc.EmbeddedDriver url: jdbc:derby:/tmp/amoro/derby;create=true - # MySQL database configuration. - # database: - # type: mysql - # jdbc-driver-class: com.mysql.cj.jdbc.Driver - # url: jdbc:mysql://127.0.0.1:3306/db?useUnicode=true&characterEncoding=UTF8&autoReconnect=true&useAffectedRows=true&useSSL=false - # username: root - # password: root +# MySQL database configuration. +# database: +# type: mysql +# jdbc-driver-class: com.mysql.cj.jdbc.Driver +# url: jdbc:mysql://127.0.0.1:3306/db?useUnicode=true&characterEncoding=UTF8&autoReconnect=true&useAffectedRows=true&useSSL=false +# username: root +# password: root # Postgres database configuration. # database: diff --git a/hive/src/test/java/com/netease/arctic/hive/formats/IcebergHiveCatalogTestHelper.java b/hive/src/test/java/com/netease/arctic/hive/formats/IcebergHiveCatalogTestHelper.java new file mode 100644 index 0000000000..d7bf0fda87 --- /dev/null +++ b/hive/src/test/java/com/netease/arctic/hive/formats/IcebergHiveCatalogTestHelper.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.hive.formats; + +import com.netease.arctic.ams.api.properties.CatalogMetaProperties; +import com.netease.arctic.formats.IcebergHadoopCatalogTestHelper; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; + +import java.util.HashMap; +import java.util.Map; + +public class IcebergHiveCatalogTestHelper extends IcebergHadoopCatalogTestHelper { + + public IcebergHiveCatalogTestHelper( + String catalogName, + Map catalogProperties) { + super(catalogName, catalogProperties); + } + + @Override + public void initHiveConf(Configuration hiveConf) { + properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); + } + + protected String getMetastoreType() { + return CatalogMetaProperties.CATALOG_TYPE_HIVE; + } + + public static IcebergHiveCatalogTestHelper defaultHelper() { + return new IcebergHiveCatalogTestHelper( + "test_iceberg_catalog", + new HashMap<>()); + } +} diff --git a/hive/src/test/java/com/netease/arctic/hive/formats/PaimonHiveCatalogTestHelper.java b/hive/src/test/java/com/netease/arctic/hive/formats/PaimonHiveCatalogTestHelper.java new file mode 100644 index 0000000000..83d3d16240 --- /dev/null +++ b/hive/src/test/java/com/netease/arctic/hive/formats/PaimonHiveCatalogTestHelper.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.hive.formats; + +import com.netease.arctic.ams.api.properties.CatalogMetaProperties; +import com.netease.arctic.formats.PaimonHadoopCatalogTestHelper; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.paimon.options.CatalogOptions; + +import java.util.HashMap; +import java.util.Map; + +public class PaimonHiveCatalogTestHelper extends PaimonHadoopCatalogTestHelper { + + public PaimonHiveCatalogTestHelper( + String catalogName, + Map catalogProperties) { + super(catalogName, catalogProperties); + } + + @Override + public void initHiveConf(Configuration hiveConf) { + catalogProperties.put(CatalogOptions.URI.key(), hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); + } + + protected String getMetastoreType() { + return CatalogMetaProperties.CATALOG_TYPE_HIVE; + } + + public static PaimonHiveCatalogTestHelper defaultHelper() { + return new PaimonHiveCatalogTestHelper( + "test_paimon_catalog", + new HashMap<>()); + } +} diff --git a/hive/src/test/java/com/netease/arctic/hive/formats/TestIcebergHiveAmoroCatalog.java b/hive/src/test/java/com/netease/arctic/hive/formats/TestIcebergHiveAmoroCatalog.java new file mode 100644 index 0000000000..6fcff55745 --- /dev/null +++ b/hive/src/test/java/com/netease/arctic/hive/formats/TestIcebergHiveAmoroCatalog.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.hive.formats; + +import com.netease.arctic.formats.AmoroCatalogTestHelper; +import com.netease.arctic.formats.TestIcebergAmoroCatalog; +import com.netease.arctic.hive.TestHMS; +import org.junit.ClassRule; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; + +@RunWith(Parameterized.class) +public class TestIcebergHiveAmoroCatalog extends TestIcebergAmoroCatalog { + + @ClassRule + public static TestHMS TEST_HMS = new TestHMS(); + + public TestIcebergHiveAmoroCatalog(AmoroCatalogTestHelper amoroCatalogTestHelper) { + super(amoroCatalogTestHelper); + } + + @Parameterized.Parameters(name = "{0}") + public static Object[] parameters() { + return new Object[] { + IcebergHiveCatalogTestHelper.defaultHelper() + }; + } + + @Override + public void setupCatalog() throws IOException { + catalogTestHelper.initHiveConf(TEST_HMS.getHiveConf()); + super.setupCatalog(); + } +} diff --git a/hive/src/test/java/com/netease/arctic/hive/formats/TestPaimonHiveAmoroCatalog.java b/hive/src/test/java/com/netease/arctic/hive/formats/TestPaimonHiveAmoroCatalog.java new file mode 100644 index 0000000000..edf71bd30f --- /dev/null +++ b/hive/src/test/java/com/netease/arctic/hive/formats/TestPaimonHiveAmoroCatalog.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netease.arctic.hive.formats; + +import com.netease.arctic.formats.AmoroCatalogTestHelper; +import com.netease.arctic.formats.TestPaimonAmoroCatalog; +import com.netease.arctic.hive.TestHMS; +import org.junit.ClassRule; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; + +@RunWith(Parameterized.class) +public class TestPaimonHiveAmoroCatalog extends TestPaimonAmoroCatalog { + + @ClassRule + public static TestHMS TEST_HMS = new TestHMS(); + + public TestPaimonHiveAmoroCatalog(AmoroCatalogTestHelper amoroCatalogTestHelper) { + super(amoroCatalogTestHelper); + } + + @Parameterized.Parameters(name = "{0}") + public static Object[] parameters() { + return new Object[] { + PaimonHiveCatalogTestHelper.defaultHelper() + }; + } + + @Override + public void setupCatalog() throws IOException { + catalogTestHelper.initHiveConf(TEST_HMS.getHiveConf()); + super.setupCatalog(); + } +} diff --git a/pom.xml b/pom.xml index 1df1ff75f8..964786b6a4 100644 --- a/pom.xml +++ b/pom.xml @@ -88,6 +88,7 @@ 3.2.0 1.3.0 + 0.5.0-incubating 3.1.1 3.2.2 2.12 @@ -384,6 +385,12 @@ ${dropwizard.metrics.version} + + org.apache.paimon + paimon-bundle + ${paimon.version} + + org.apache.thrift libthrift