diff --git a/ams/api/src/main/java/com/netease/arctic/ams/api/properties/CatalogMetaProperties.java b/ams/api/src/main/java/com/netease/arctic/ams/api/properties/CatalogMetaProperties.java index b86c154ccd..98782f7d26 100644 --- a/ams/api/src/main/java/com/netease/arctic/ams/api/properties/CatalogMetaProperties.java +++ b/ams/api/src/main/java/com/netease/arctic/ams/api/properties/CatalogMetaProperties.java @@ -74,6 +74,9 @@ public class CatalogMetaProperties { public static final long CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS_DEFAULT = TimeUnit.MINUTES.toMillis(5); + // only used for unified catalog + public static final String AMS_URI = "ams.uri"; + // only used for engine properties public static final String LOAD_AUTH_FROM_AMS = "auth.load-from-ams"; public static final boolean LOAD_AUTH_FROM_AMS_DEFAULT = true; 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 10e2698ffb..88e8fe8dbd 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 @@ -11,45 +11,60 @@ import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.server.utils.Configurations; import com.netease.arctic.utils.CatalogUtil; -import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import java.util.Map; import java.util.Set; public class CatalogBuilder { - // TODO: use internal or external concepts + /** matrix of catalog type and supported table formats */ + private static final Map> formatSupportedMatrix = + ImmutableMap.of( + CATALOG_TYPE_HADOOP, + Sets.newHashSet(TableFormat.ICEBERG, TableFormat.MIXED_ICEBERG, TableFormat.PAIMON), + CATALOG_TYPE_GLUE, Sets.newHashSet(TableFormat.ICEBERG, TableFormat.MIXED_ICEBERG), + CATALOG_TYPE_CUSTOM, Sets.newHashSet(TableFormat.ICEBERG, TableFormat.MIXED_ICEBERG), + CATALOG_TYPE_HIVE, + Sets.newHashSet( + TableFormat.ICEBERG, + TableFormat.MIXED_ICEBERG, + TableFormat.MIXED_HIVE, + TableFormat.PAIMON), + CATALOG_TYPE_AMS, Sets.newHashSet(TableFormat.ICEBERG, TableFormat.MIXED_ICEBERG)); + public static ServerCatalog buildServerCatalog( CatalogMeta catalogMeta, Configurations serverConfiguration) { String type = catalogMeta.getCatalogType(); Set tableFormats = CatalogUtil.tableFormats(catalogMeta); - TableFormat tableFormat = tableFormats.iterator().next(); - if (tableFormat == TableFormat.PAIMON) { - return new PaimonServerCatalog(catalogMeta); - } + Preconditions.checkState( + formatSupportedMatrix.containsKey(type), "unsupported catalog type: %s", type); + Preconditions.checkState( + tableFormats.size() == 1, + "only 1 types format is supported: %s", + Joiner.on(",").join(tableFormats)); + + Set supportedFormats = formatSupportedMatrix.get(type); + TableFormat tableFormat = tableFormats.iterator().next(); + Preconditions.checkState( + supportedFormats.contains(tableFormat), + "Table format %s is not supported for metastore type: %s", + tableFormat, + type); switch (type) { case CATALOG_TYPE_HADOOP: - if (TableFormat.ICEBERG == tableFormat) { - return new IcebergCatalogImpl(catalogMeta); - } else if (TableFormat.MIXED_ICEBERG == tableFormat) { - return new MixedIcebergCatalogImpl(catalogMeta); - } else { - throw new IllegalStateException( - "Hadoop catalog support iceberg/mixed-iceberg table only."); - } - + case CATALOG_TYPE_GLUE: + case CATALOG_TYPE_CUSTOM: + return new ExternalCatalog(catalogMeta); case CATALOG_TYPE_HIVE: - if (tableFormat.equals(TableFormat.ICEBERG)) { - return new IcebergCatalogImpl(catalogMeta); - } else if (tableFormat.equals(TableFormat.MIXED_HIVE)) { + if (tableFormat.equals(TableFormat.MIXED_HIVE)) { return new MixedHiveCatalogImpl(catalogMeta); - } else if (TableFormat.MIXED_ICEBERG == tableFormat) { - return new MixedIcebergCatalogImpl(catalogMeta); - } else { - throw new IllegalArgumentException( - "Hive Catalog support iceberg/mixed-hive/mixed-iceberg table only"); } + return new ExternalCatalog(catalogMeta); case CATALOG_TYPE_AMS: if (tableFormat.equals(TableFormat.MIXED_ICEBERG)) { return new InternalMixedCatalogImpl(catalogMeta); @@ -58,26 +73,6 @@ public static ServerCatalog buildServerCatalog( } else { throw new IllegalStateException("AMS catalog support iceberg/mixed-iceberg table only."); } - case CATALOG_TYPE_GLUE: - if (TableFormat.ICEBERG == tableFormat) { - return new IcebergCatalogImpl(catalogMeta); - } else if (TableFormat.MIXED_ICEBERG == tableFormat) { - return new MixedIcebergCatalogImpl(catalogMeta); - } else { - throw new IllegalStateException("Glue catalog support iceberg table only."); - } - case CATALOG_TYPE_CUSTOM: - Preconditions.checkArgument( - catalogMeta.getCatalogProperties().containsKey(CatalogProperties.CATALOG_IMPL), - "Custom catalog properties must contains " + CatalogProperties.CATALOG_IMPL); - if (TableFormat.ICEBERG == tableFormat) { - return new IcebergCatalogImpl(catalogMeta); - } else if (TableFormat.MIXED_ICEBERG == tableFormat) { - return new MixedIcebergCatalogImpl(catalogMeta); - } else { - throw new IllegalStateException( - "Custom catalog support iceberg/mixed-iceberg table only."); - } default: throw new IllegalStateException("unsupported catalog type:" + type); } diff --git a/ams/server/src/main/java/com/netease/arctic/server/catalog/ExternalCatalog.java b/ams/server/src/main/java/com/netease/arctic/server/catalog/ExternalCatalog.java index c7d946e7a0..afaf2308ce 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/ExternalCatalog.java +++ b/ams/server/src/main/java/com/netease/arctic/server/catalog/ExternalCatalog.java @@ -1,14 +1,33 @@ package com.netease.arctic.server.catalog; +import com.netease.arctic.AmoroTable; +import com.netease.arctic.CommonUnifiedCatalog; +import com.netease.arctic.TableIDWithFormat; +import com.netease.arctic.UnifiedCatalog; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.server.persistence.mapper.TableMetaMapper; import com.netease.arctic.server.table.ServerTableIdentifier; +import com.netease.arctic.table.TableMetaStore; +import com.netease.arctic.utils.CatalogUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; -public abstract class ExternalCatalog extends ServerCatalog { +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.stream.Collectors; + +public class ExternalCatalog extends ServerCatalog { + + UnifiedCatalog unifiedCatalog; + TableMetaStore tableMetaStore; protected ExternalCatalog(CatalogMeta metadata) { super(metadata); + this.tableMetaStore = CatalogUtil.buildMetaStore(metadata); + this.unifiedCatalog = + this.tableMetaStore.doAs( + () -> new CommonUnifiedCatalog(this::getMetadata, Maps.newHashMap())); } public void syncTable(String database, String tableName, TableFormat format) { @@ -29,4 +48,50 @@ public void disposeTable(String database, String tableName) { TableMetaMapper.class, mapper -> mapper.deleteTableIdByName(getMetadata().getCatalogName(), database, tableName)); } + + @Override + public void updateMetadata(CatalogMeta metadata) { + super.updateMetadata(metadata); + this.tableMetaStore = CatalogUtil.buildMetaStore(metadata); + this.unifiedCatalog.refresh(); + } + + @Override + public boolean exist(String database) { + return doAs(() -> unifiedCatalog.exist(database)); + } + + @Override + public boolean exist(String database, String tableName) { + return doAs(() -> unifiedCatalog.exist(database, tableName)); + } + + @Override + public List listDatabases() { + return doAs(() -> unifiedCatalog.listDatabases()); + } + + @Override + public List listTables() { + return doAs( + () -> + unifiedCatalog.listDatabases().stream() + .map(this::listTables) + .flatMap(List::stream) + .collect(Collectors.toList())); + } + + @Override + public List listTables(String database) { + return doAs(() -> new ArrayList<>(unifiedCatalog.listTables(database))); + } + + @Override + public AmoroTable loadTable(String database, String tableName) { + return doAs(() -> unifiedCatalog.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/IcebergCatalogImpl.java b/ams/server/src/main/java/com/netease/arctic/server/catalog/IcebergCatalogImpl.java deleted file mode 100644 index f7661c18d9..0000000000 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/IcebergCatalogImpl.java +++ /dev/null @@ -1,65 +0,0 @@ -package com.netease.arctic.server.catalog; - -import com.netease.arctic.AmoroTable; -import com.netease.arctic.TableIDWithFormat; -import com.netease.arctic.ams.api.CatalogMeta; -import com.netease.arctic.ams.api.TableFormat; -import com.netease.arctic.catalog.IcebergCatalogWrapper; -import com.netease.arctic.formats.iceberg.IcebergTable; -import org.apache.iceberg.Table; - -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; - -public class IcebergCatalogImpl extends ExternalCatalog { - - private final IcebergCatalogWrapper catalogWrapper; - - protected IcebergCatalogImpl(CatalogMeta metadata) { - super(metadata); - this.catalogWrapper = new IcebergCatalogWrapper(getMetadata(), Collections.emptyMap()); - } - - @Override - public void updateMetadata(CatalogMeta metadata) { - super.updateMetadata(metadata); - this.catalogWrapper.refreshCatalogMeta(getMetadata()); - } - - @Override - public boolean exist(String database) { - return catalogWrapper.listDatabases().contains(database); - } - - @Override - public boolean exist(String database, String tableName) { - return loadTable(database, tableName) != null; - } - - @Override - public List listDatabases() { - return catalogWrapper.listDatabases(); - } - - @Override - public List listTables() { - return catalogWrapper.listTables().stream() - .map(id -> TableIDWithFormat.of(id, TableFormat.ICEBERG)) - .collect(Collectors.toList()); - } - - @Override - public List listTables(String database) { - return catalogWrapper.listTables(database).stream() - .map(id -> TableIDWithFormat.of(id, TableFormat.ICEBERG)) - .collect(Collectors.toList()); - } - - @Override - public AmoroTable 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 a713cc8902..9f35022612 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 @@ -2,10 +2,7 @@ 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; import com.netease.arctic.server.IcebergRestCatalogService; import com.netease.arctic.server.iceberg.InternalTableOperations; @@ -13,6 +10,7 @@ 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.utils.CatalogUtil; import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.TableOperations; @@ -64,15 +62,16 @@ public AmoroTable loadTable(String database, String tableName) { return null; } FileIO io = IcebergTableUtil.newIcebergFileIo(getMetadata()); - ArcticFileIO fileIO = new ArcticFileIOAdapter(io); TableOperations ops = InternalTableOperations.buildForLoad(tableMetadata, io); BaseTable table = new BaseTable(ops, TableIdentifier.of(database, tableName).toString()); com.netease.arctic.table.TableIdentifier tableIdentifier = com.netease.arctic.table.TableIdentifier.of(name(), database, tableName); - return new IcebergTable( + + return IcebergTable.newIcebergTable( tableIdentifier, - new IcebergCatalogWrapper.BasicIcebergTable( - tableIdentifier, table, fileIO, getMetadata().getCatalogProperties())); + table, + CatalogUtil.buildMetaStore(getMetadata()), + getMetadata().getCatalogProperties()); } private String defaultRestURI() { 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 1e64457bce..d4cc353598 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 @@ -2,29 +2,33 @@ import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.CatalogMeta; +import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.catalog.MixedTables; -import com.netease.arctic.formats.mixed.MixedIcebergTable; +import com.netease.arctic.formats.mixed.MixedTable; import com.netease.arctic.server.persistence.mapper.TableMetaMapper; import com.netease.arctic.server.table.TableMetadata; +import com.netease.arctic.table.TableMetaStore; +import com.netease.arctic.utils.CatalogUtil; + +import java.util.Map; public class InternalMixedCatalogImpl extends InternalCatalog { - protected final MixedTables tables; + protected MixedTables tables; protected InternalMixedCatalogImpl(CatalogMeta metadata) { super(metadata); - this.tables = new MixedTables(metadata); + this.tables = newTables(metadata.getCatalogProperties(), CatalogUtil.buildMetaStore(metadata)); } - protected InternalMixedCatalogImpl(CatalogMeta metadata, MixedTables tables) { - super(metadata); - this.tables = tables; + protected MixedTables newTables(Map catalogProperties, TableMetaStore metaStore) { + return new MixedTables(catalogProperties, metaStore); } @Override public void updateMetadata(CatalogMeta metadata) { super.updateMetadata(metadata); - this.tables.refreshCatalogMeta(getMetadata()); + this.tables = newTables(metadata.getCatalogProperties(), CatalogUtil.buildMetaStore(metadata)); } @Override @@ -37,7 +41,8 @@ public AmoroTable loadTable(String database, String tableName) { if (tableMetadata == null) { return null; } - return new MixedIcebergTable(tables.loadTableByMeta(tableMetadata.buildTableMeta())); + return new MixedTable( + tables.loadTableByMeta(tableMetadata.buildTableMeta()), TableFormat.MIXED_ICEBERG); } 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 c300404a5d..a218117ae1 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 @@ -20,26 +20,35 @@ import com.netease.arctic.AmoroTable; import com.netease.arctic.ams.api.CatalogMeta; -import com.netease.arctic.formats.mixed.MixedHiveTable; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.catalog.MixedTables; +import com.netease.arctic.formats.mixed.MixedTable; 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 com.netease.arctic.table.TableMetaStore; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.thrift.TException; import java.util.List; +import java.util.Map; public class MixedHiveCatalogImpl extends InternalMixedCatalogImpl { private volatile CachedHiveClientPool hiveClientPool; protected MixedHiveCatalogImpl(CatalogMeta catalogMeta) { - super(catalogMeta, new MixedHiveTables(catalogMeta)); + super(catalogMeta); hiveClientPool = ((MixedHiveTables) tables()).getHiveClientPool(); } + @Override + protected MixedTables newTables(Map catalogProperties, TableMetaStore metaStore) { + return new MixedHiveTables(catalogProperties, metaStore); + } + @Override public void updateMetadata(CatalogMeta metadata) { super.updateMetadata(metadata); @@ -56,7 +65,8 @@ public AmoroTable loadTable(String database, String tableName) { if (tableMetadata == null) { return null; } - return new MixedHiveTable(tables.loadTableByMeta(tableMetadata.buildTableMeta())); + return new MixedTable( + tables.loadTableByMeta(tableMetadata.buildTableMeta()), TableFormat.MIXED_HIVE); } @Override 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 deleted file mode 100644 index 402ed53fda..0000000000 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/MixedIcebergCatalogImpl.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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.TableIDWithFormat; -import com.netease.arctic.ams.api.CatalogMeta; -import com.netease.arctic.ams.api.TableFormat; -import com.netease.arctic.formats.mixed.MixedIcebergTable; -import com.netease.arctic.mixed.BasicMixedIcebergCatalog; - -import java.util.List; -import java.util.stream.Collectors; - -public class MixedIcebergCatalogImpl extends ExternalCatalog { - private BasicMixedIcebergCatalog mixedIcebergCatalog; - - protected MixedIcebergCatalogImpl(CatalogMeta metadata) { - super(metadata); - this.mixedIcebergCatalog = new BasicMixedIcebergCatalog(metadata); - } - - @Override - public void updateMetadata(CatalogMeta metadata) { - super.updateMetadata(metadata); - this.mixedIcebergCatalog = new BasicMixedIcebergCatalog(metadata); - } - - @Override - public boolean exist(String database) { - return mixedIcebergCatalog.listDatabases().contains(database); - } - - @Override - public boolean exist(String database, String tableName) { - return mixedIcebergCatalog.tableExists( - com.netease.arctic.table.TableIdentifier.of(name(), database, tableName)); - } - - @Override - public List listDatabases() { - return mixedIcebergCatalog.listDatabases(); - } - - @Override - public List listTables() { - return listDatabases().stream() - .map(this::listTables) - .flatMap(List::stream) - .collect(Collectors.toList()); - } - - @Override - public List listTables(String database) { - return mixedIcebergCatalog.listTables(database).stream() - .map(id -> TableIDWithFormat.of(id, TableFormat.MIXED_ICEBERG)) - .collect(Collectors.toList()); - } - - @Override - 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 deleted file mode 100644 index 49d5673158..0000000000 --- a/ams/server/src/main/java/com/netease/arctic/server/catalog/PaimonServerCatalog.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.TableIDWithFormat; -import com.netease.arctic.UnifiedCatalog; -import com.netease.arctic.ams.api.CatalogMeta; -import com.netease.arctic.table.TableMetaStore; -import com.netease.arctic.utils.CatalogUtil; -import org.apache.paimon.hive.HiveCatalogOptions; - -import java.io.File; -import java.net.URL; -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -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); - Optional hiveSiteLocation = tableMetaStore.getHiveSiteLocation(); - hiveSiteLocation.ifPresent( - url -> - metadata.catalogProperties.put( - HiveCatalogOptions.HIVE_CONF_DIR.key(), new File(url.getPath()).getParent())); - 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(() -> new ArrayList<>(paimonCatalog.listTables(database))); - } - - @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/dashboard/DashboardServer.java b/ams/server/src/main/java/com/netease/arctic/server/dashboard/DashboardServer.java index c38be6cfd0..53fc2f5b6b 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/dashboard/DashboardServer.java +++ b/ams/server/src/main/java/com/netease/arctic/server/dashboard/DashboardServer.java @@ -186,6 +186,9 @@ public EndpointGroup endpoints() { get( "/tables/catalogs/{catalog}/dbs/{db}/tables/{table}/optimizing-processes", tableController::getOptimizingProcesses); + get( + "/tables/catalogs/{catalog}/dbs/{db}/tables/{table}/optimizing-processes/{processId}/tasks", + tableController::getOptimizingProcessTasks); get( "/tables/catalogs/{catalog}/dbs/{db}/tables/{table}/transactions", tableController::getTableTransactions); @@ -297,6 +300,9 @@ public EndpointGroup endpoints() { get( "/tables/catalogs/{catalog}/dbs/{db}/tables/{table}/optimizing-processes", tableController::getOptimizingProcesses); + get( + "/tables/catalogs/{catalog}/dbs/{db}/tables/{table}/optimizing-processes/{processId}/tasks", + tableController::getOptimizingProcessTasks); get( "/tables/catalogs/{catalog}/dbs/{db}/tables/{table}/transactions", tableController::getTableTransactions); 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 index 27c308b7db..5628d9cc2b 100644 --- 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 @@ -23,6 +23,7 @@ import com.netease.arctic.server.dashboard.model.AMSTransactionsOfTable; import com.netease.arctic.server.dashboard.model.DDLInfo; import com.netease.arctic.server.dashboard.model.OptimizingProcessInfo; +import com.netease.arctic.server.dashboard.model.OptimizingTaskInfo; import com.netease.arctic.server.dashboard.model.PartitionBaseInfo; import com.netease.arctic.server.dashboard.model.PartitionFileBaseInfo; import com.netease.arctic.server.dashboard.model.ServerTableMeta; @@ -59,6 +60,9 @@ public interface FormatTableDescriptor { Pair, Integer> getOptimizingProcessesInfo( AmoroTable amoroTable, int limit, int offset); + /** Get the paged optimizing process tasks information of the {@link AmoroTable}. */ + List getOptimizingTaskInfos(AmoroTable amoroTable, long processId); + /** Get the tag information of the {@link AmoroTable}. */ List getTableTags(AmoroTable amoroTable); 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 index 4635638448..15140cc132 100644 --- 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 @@ -33,6 +33,7 @@ import com.netease.arctic.server.dashboard.model.DDLInfo; import com.netease.arctic.server.dashboard.model.FilesStatistics; import com.netease.arctic.server.dashboard.model.OptimizingProcessInfo; +import com.netease.arctic.server.dashboard.model.OptimizingTaskInfo; import com.netease.arctic.server.dashboard.model.PartitionBaseInfo; import com.netease.arctic.server.dashboard.model.PartitionFileBaseInfo; import com.netease.arctic.server.dashboard.model.ServerTableMeta; @@ -432,6 +433,35 @@ public Pair, Integer> getOptimizingProcessesInfo( total); } + @Override + public List getOptimizingTaskInfos(AmoroTable amoroTable, long processId) { + List optimizingTaskMetaList = + getAs( + OptimizingMapper.class, + mapper -> mapper.selectOptimizeTaskMetas(Collections.singletonList(processId))); + if (CollectionUtils.isEmpty(optimizingTaskMetaList)) { + return Collections.emptyList(); + } + return optimizingTaskMetaList.stream() + .map( + taskMeta -> + new OptimizingTaskInfo( + taskMeta.getTableId(), + taskMeta.getProcessId(), + taskMeta.getTaskId(), + taskMeta.getPartitionData(), + taskMeta.getStatus(), + taskMeta.getRetryNum(), + taskMeta.getThreadId(), + taskMeta.getStartTime(), + taskMeta.getEndTime(), + taskMeta.getCostTime(), + taskMeta.getFailReason(), + taskMeta.getMetricsSummary(), + taskMeta.getProperties())) + .collect(Collectors.toList()); + } + private List collectFileInfo( Table table, boolean isChangeTable, String partition) { PartitionSpec spec = table.spec(); 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 index 691a0f00c1..a99e4b8c5c 100644 --- 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 @@ -31,6 +31,7 @@ import com.netease.arctic.server.dashboard.model.AMSTransactionsOfTable; import com.netease.arctic.server.dashboard.model.DDLInfo; import com.netease.arctic.server.dashboard.model.OptimizingProcessInfo; +import com.netease.arctic.server.dashboard.model.OptimizingTaskInfo; import com.netease.arctic.server.dashboard.model.PartitionBaseInfo; import com.netease.arctic.server.dashboard.model.PartitionFileBaseInfo; import com.netease.arctic.server.dashboard.model.ServerTableMeta; @@ -387,6 +388,11 @@ public Pair, Integer> getOptimizingProcessesInfo( return Pair.of(processInfoList, total); } + @Override + public List getOptimizingTaskInfos(AmoroTable amoroTable, long processId) { + throw new UnsupportedOperationException(); + } + @NotNull private AMSTransactionsOfTable getTransactionsOfTable( AbstractFileStore store, Snapshot snapshot) { 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 86b51deb30..f65bbbf702 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 @@ -26,6 +26,7 @@ import com.netease.arctic.server.dashboard.model.AMSTransactionsOfTable; import com.netease.arctic.server.dashboard.model.DDLInfo; import com.netease.arctic.server.dashboard.model.OptimizingProcessInfo; +import com.netease.arctic.server.dashboard.model.OptimizingTaskInfo; import com.netease.arctic.server.dashboard.model.PartitionBaseInfo; import com.netease.arctic.server.dashboard.model.PartitionFileBaseInfo; import com.netease.arctic.server.dashboard.model.ServerTableMeta; @@ -128,6 +129,13 @@ public Pair, Integer> getOptimizingProcessesInfo( return formatTableDescriptor.getOptimizingProcessesInfo(amoroTable, limit, offset); } + public List getOptimizingProcessTaskInfos( + TableIdentifier tableIdentifier, long tableId) { + AmoroTable amoroTable = loadTable(tableIdentifier); + FormatTableDescriptor formatTableDescriptor = formatDescriptorMap.get(amoroTable.format()); + return formatTableDescriptor.getOptimizingTaskInfos(amoroTable, tableId); + } + private AmoroTable loadTable(TableIdentifier identifier) { ServerCatalog catalog = tableService.getServerCatalog(identifier.getCatalog()); return catalog.loadTable(identifier.getDatabase(), identifier.getTableName()); 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 76bfd7ff44..6ec00f299e 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 @@ -35,6 +35,7 @@ import com.netease.arctic.server.dashboard.model.DDLInfo; import com.netease.arctic.server.dashboard.model.HiveTableInfo; import com.netease.arctic.server.dashboard.model.OptimizingProcessInfo; +import com.netease.arctic.server.dashboard.model.OptimizingTaskInfo; import com.netease.arctic.server.dashboard.model.PartitionBaseInfo; import com.netease.arctic.server.dashboard.model.PartitionFileBaseInfo; import com.netease.arctic.server.dashboard.model.ServerTableMeta; @@ -286,6 +287,35 @@ public void getOptimizingProcesses(Context ctx) { ctx.json(OkResponse.of(PageResult.of(result, total))); } + /** + * Get tasks of optimizing process. + * + * @param ctx - context for handling the request and response + */ + public void getOptimizingProcessTasks(Context ctx) { + String catalog = ctx.pathParam("catalog"); + String db = ctx.pathParam("db"); + String table = ctx.pathParam("table"); + String processId = ctx.pathParam("processId"); + Integer page = ctx.queryParamAsClass("page", Integer.class).getOrDefault(1); + Integer pageSize = ctx.queryParamAsClass("pageSize", Integer.class).getOrDefault(20); + + int offset = (page - 1) * pageSize; + int limit = pageSize; + ServerCatalog serverCatalog = tableService.getServerCatalog(catalog); + Preconditions.checkArgument(offset >= 0, "offset[%s] must >= 0", offset); + Preconditions.checkArgument(limit >= 0, "limit[%s] must >= 0", limit); + Preconditions.checkState(serverCatalog.exist(db, table), "no such table"); + + TableIdentifier tableIdentifier = TableIdentifier.of(catalog, db, table); + List optimizingTaskInfos = + tableDescriptor.getOptimizingProcessTaskInfos( + tableIdentifier.buildTableIdentifier(), Long.parseLong(processId)); + + PageResult pageResult = PageResult.of(optimizingTaskInfos, offset, limit); + ctx.json(OkResponse.of(pageResult)); + } + /** * get list of transactions. * diff --git a/ams/server/src/main/java/com/netease/arctic/server/dashboard/model/OptimizingTaskInfo.java b/ams/server/src/main/java/com/netease/arctic/server/dashboard/model/OptimizingTaskInfo.java new file mode 100644 index 0000000000..e2f31e109f --- /dev/null +++ b/ams/server/src/main/java/com/netease/arctic/server/dashboard/model/OptimizingTaskInfo.java @@ -0,0 +1,173 @@ +/* + * 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.model; + +import com.netease.arctic.server.optimizing.MetricsSummary; +import com.netease.arctic.server.optimizing.TaskRuntime; + +import java.util.Map; + +public class OptimizingTaskInfo { + private Long tableId; + private Long processId; + private int taskId; + private String partitionData; + private TaskRuntime.Status status; + private int retryNum; + private int threadId; + private long startTime; + private long endTime; + private long costTime; + private String failReason; + private MetricsSummary summary; + private Map properties; + + public OptimizingTaskInfo( + Long tableId, + Long processId, + int taskId, + String partitionData, + TaskRuntime.Status status, + int retryNum, + int threadId, + long startTime, + long endTime, + long costTime, + String failReason, + MetricsSummary summary, + Map properties) { + this.tableId = tableId; + this.processId = processId; + this.taskId = taskId; + this.partitionData = partitionData; + this.status = status; + this.retryNum = retryNum; + this.threadId = threadId; + this.startTime = startTime; + this.endTime = endTime; + this.costTime = costTime; + this.failReason = failReason; + this.summary = summary; + this.properties = properties; + } + + public Long getTableId() { + return tableId; + } + + public void setTableId(Long tableId) { + this.tableId = tableId; + } + + public Long getProcessId() { + return processId; + } + + public void setProcessId(Long processId) { + this.processId = processId; + } + + public int getTaskId() { + return taskId; + } + + public void setTaskId(int taskId) { + this.taskId = taskId; + } + + public String getPartitionData() { + return partitionData; + } + + public void setPartitionData(String partitionData) { + this.partitionData = partitionData; + } + + public TaskRuntime.Status getStatus() { + return status; + } + + public void setStatus(TaskRuntime.Status status) { + this.status = status; + } + + public int getRetryNum() { + return retryNum; + } + + public void setRetryNum(int retryNum) { + this.retryNum = retryNum; + } + + public int getThreadId() { + return threadId; + } + + public void setThreadId(int threadId) { + this.threadId = threadId; + } + + public long getStartTime() { + return startTime; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + public long getEndTime() { + return endTime; + } + + public void setEndTime(long endTime) { + this.endTime = endTime; + } + + public long getCostTime() { + return costTime; + } + + public void setCostTime(long costTime) { + this.costTime = costTime; + } + + public String getFailReason() { + return failReason; + } + + public void setFailReason(String failReason) { + this.failReason = failReason; + } + + public MetricsSummary getSummary() { + return summary; + } + + public void setSummary(MetricsSummary summary) { + this.summary = summary; + } + + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } +} diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/KeyedTableCommit.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/KeyedTableCommit.java index c106418b3f..969452dcc6 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/KeyedTableCommit.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/KeyedTableCommit.java @@ -14,8 +14,8 @@ import com.netease.arctic.optimizing.RewriteFilesOutput; import com.netease.arctic.server.exception.OptimizingCommitException; import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.utils.ArcticTableUtil; import com.netease.arctic.utils.ContentFiles; -import com.netease.arctic.utils.TablePropertyUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -78,7 +78,7 @@ public void commit() throws OptimizingCommitException { Set removedDeleteFiles = Sets.newHashSet(); StructLikeMap partitionOptimizedSequence = - TablePropertyUtil.getPartitionOptimizedSequence(table.asKeyedTable()); + ArcticTableUtil.readOptimizedSequence(table.asKeyedTable()); for (TaskRuntime taskRuntime : tasks) { RewriteFilesInput input = taskRuntime.getInput(); 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 index 54208f05e6..18c3ad40b7 100644 --- 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 @@ -29,6 +29,7 @@ import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.ArcticTableUtil; import com.netease.arctic.utils.CompatiblePropertyUtil; import com.netease.arctic.utils.TablePropertyUtil; import org.apache.commons.collections.CollectionUtils; @@ -230,9 +231,8 @@ private void deleteChangeFile(List expiredDataFileEntries) { return; } - StructLikeMap partitionMaxTransactionId = - TablePropertyUtil.getPartitionOptimizedSequence(keyedTable); - if (MapUtils.isEmpty(partitionMaxTransactionId)) { + StructLikeMap optimizedSequences = ArcticTableUtil.readOptimizedSequence(keyedTable); + if (MapUtils.isEmpty(optimizedSequences)) { LOG.info("table {} not contains max transaction id", keyedTable.id()); return; } @@ -252,7 +252,7 @@ private void deleteChangeFile(List expiredDataFileEntries) { .spec() .partitionToPath(expiredDataFileEntries.get(0).getFile().partition())); - Long optimizedSequence = partitionMaxTransactionId.get(TablePropertyUtil.EMPTY_STRUCT); + Long optimizedSequence = optimizedSequences.get(TablePropertyUtil.EMPTY_STRUCT); if (optimizedSequence != null && CollectionUtils.isNotEmpty(partitionDataFiles)) { changeDeleteFiles.addAll( partitionDataFiles.stream() @@ -265,7 +265,7 @@ private void deleteChangeFile(List expiredDataFileEntries) { .collect(Collectors.toList())); } } else { - partitionMaxTransactionId.forEach( + optimizedSequences.forEach( (key, value) -> { List partitionDataFiles = partitionDataFileMap.get(keyedTable.spec().partitionToPath(key)); diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/AbstractPartitionPlan.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/AbstractPartitionPlan.java index 0fa9979d36..f24bd38a02 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/AbstractPartitionPlan.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/AbstractPartitionPlan.java @@ -38,7 +38,6 @@ import java.util.stream.Collectors; public abstract class AbstractPartitionPlan implements PartitionEvaluator { - public static final int INVALID_SEQUENCE = -1; protected final String partition; protected final OptimizingConfig config; @@ -47,8 +46,8 @@ public abstract class AbstractPartitionPlan implements PartitionEvaluator { private TaskSplitter taskSplitter; protected ArcticTable tableObject; - private long fromSequence = INVALID_SEQUENCE; - private long toSequence = INVALID_SEQUENCE; + private Long fromSequence = null; + private Long toSequence = null; protected final long planTime; protected final Map partitionProperties; @@ -135,19 +134,19 @@ protected void beforeSplit() {} protected abstract OptimizingInputProperties buildTaskProperties(); protected void markSequence(long sequence) { - if (fromSequence == INVALID_SEQUENCE || fromSequence > sequence) { + if (fromSequence == null || fromSequence > sequence) { fromSequence = sequence; } - if (toSequence == INVALID_SEQUENCE || toSequence < sequence) { + if (toSequence == null || toSequence < sequence) { toSequence = sequence; } } - public long getFromSequence() { + public Long getFromSequence() { return fromSequence; } - public long getToSequence() { + public Long getToSequence() { return toSequence; } diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/OptimizingEvaluator.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/OptimizingEvaluator.java index abfc351f13..00e6db2db8 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/OptimizingEvaluator.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/OptimizingEvaluator.java @@ -18,6 +18,7 @@ package com.netease.arctic.server.optimizing.plan; +import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.hive.table.SupportHive; import com.netease.arctic.server.optimizing.scan.IcebergTableFileScanHelper; import com.netease.arctic.server.optimizing.scan.KeyedTableFileScanHelper; @@ -29,7 +30,6 @@ import com.netease.arctic.server.utils.IcebergTableUtil; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.utils.TablePropertyUtil; -import com.netease.arctic.utils.TableTypeUtil; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; import org.apache.iceberg.io.CloseableIterable; @@ -73,7 +73,7 @@ public TableRuntime getTableRuntime() { protected void initEvaluator() { long startTime = System.currentTimeMillis(); TableFileScanHelper tableFileScanHelper; - if (TableTypeUtil.isIcebergTableFormat(arcticTable)) { + if (TableFormat.ICEBERG == arcticTable.format()) { tableFileScanHelper = new IcebergTableFileScanHelper( arcticTable.asUnkeyedTable(), currentSnapshot.snapshotId()); @@ -133,7 +133,7 @@ private Map partitionProperties(String partitionPath) { protected PartitionEvaluator buildEvaluator(String partitionPath) { Map partitionProperties = partitionProperties(partitionPath); - if (TableTypeUtil.isIcebergTableFormat(arcticTable)) { + if (TableFormat.ICEBERG == arcticTable.format()) { return new CommonPartitionEvaluator( tableRuntime, partitionPath, partitionProperties, System.currentTimeMillis()); } else { diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/OptimizingPlanner.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/OptimizingPlanner.java index 2fbad3effb..8f2c9ce389 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/OptimizingPlanner.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/plan/OptimizingPlanner.java @@ -18,6 +18,7 @@ package com.netease.arctic.server.optimizing.plan; +import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.hive.table.SupportHive; import com.netease.arctic.server.ArcticServiceConstants; import com.netease.arctic.server.optimizing.OptimizingType; @@ -25,7 +26,6 @@ import com.netease.arctic.server.table.KeyedTableSnapshot; import com.netease.arctic.server.table.TableRuntime; import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.utils.TableTypeUtil; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,6 +51,8 @@ public class OptimizingPlanner extends OptimizingEvaluator { private final PartitionPlannerFactory partitionPlannerFactory; private List tasks; + private List actualPartitionPlans; + public OptimizingPlanner(TableRuntime tableRuntime, ArcticTable table, double availableCore) { super(tableRuntime, table); this.partitionFilter = @@ -69,17 +71,19 @@ protected PartitionEvaluator buildEvaluator(String partitionPath) { } public Map getFromSequence() { - return partitionPlanMap.entrySet().stream() + return actualPartitionPlans.stream() + .filter(p -> p.getFromSequence() != null) .collect( Collectors.toMap( - Map.Entry::getKey, e -> ((AbstractPartitionPlan) e.getValue()).getFromSequence())); + AbstractPartitionPlan::getPartition, AbstractPartitionPlan::getFromSequence)); } public Map getToSequence() { - return partitionPlanMap.entrySet().stream() + return actualPartitionPlans.stream() + .filter(p -> p.getToSequence() != null) .collect( Collectors.toMap( - Map.Entry::getKey, e -> ((AbstractPartitionPlan) e.getValue()).getToSequence())); + AbstractPartitionPlan::getPartition, AbstractPartitionPlan::getToSequence)); } @Override @@ -126,10 +130,10 @@ public List planTasks() { evaluators.sort(Comparator.comparing(PartitionEvaluator::getWeight, Comparator.reverseOrder())); double maxInputSize = MAX_INPUT_FILE_SIZE_PER_THREAD * availableCore; - List inputPartitions = Lists.newArrayList(); + actualPartitionPlans = Lists.newArrayList(); long actualInputSize = 0; for (PartitionEvaluator evaluator : evaluators) { - inputPartitions.add(evaluator); + actualPartitionPlans.add((AbstractPartitionPlan) evaluator); actualInputSize += evaluator.getCost(); if (actualInputSize > maxInputSize) { break; @@ -138,9 +142,8 @@ public List planTasks() { double avgThreadCost = actualInputSize / availableCore; List tasks = Lists.newArrayList(); - for (PartitionEvaluator evaluator : inputPartitions) { - tasks.addAll( - ((AbstractPartitionPlan) evaluator).splitTasks((int) (actualInputSize / avgThreadCost))); + for (AbstractPartitionPlan partitionPlan : actualPartitionPlans) { + tasks.addAll(partitionPlan.splitTasks((int) (actualInputSize / avgThreadCost))); } if (!tasks.isEmpty()) { if (evaluators.stream() @@ -200,7 +203,7 @@ public PartitionPlannerFactory( } public PartitionEvaluator buildPartitionPlanner(String partitionPath) { - if (TableTypeUtil.isIcebergTableFormat(arcticTable)) { + if (TableFormat.ICEBERG == arcticTable.format()) { return new IcebergPartitionPlan(tableRuntime, arcticTable, partitionPath, planTime); } else { if (com.netease.arctic.hive.utils.TableTypeUtil.isHive(arcticTable)) { diff --git a/ams/server/src/main/java/com/netease/arctic/server/optimizing/scan/KeyedTableFileScanHelper.java b/ams/server/src/main/java/com/netease/arctic/server/optimizing/scan/KeyedTableFileScanHelper.java index 72dbae9af8..a33521d466 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/optimizing/scan/KeyedTableFileScanHelper.java +++ b/ams/server/src/main/java/com/netease/arctic/server/optimizing/scan/KeyedTableFileScanHelper.java @@ -29,6 +29,7 @@ import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.ArcticTableUtil; import com.netease.arctic.utils.CompatiblePropertyUtil; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -64,16 +65,12 @@ public class KeyedTableFileScanHelper implements TableFileScanHelper { private final KeyedTable arcticTable; private final long changeSnapshotId; private final long baseSnapshotId; - private final StructLikeMap partitionOptimizedSequence; - private final StructLikeMap legacyPartitionMaxTransactionId; private PartitionFilter partitionFilter; public KeyedTableFileScanHelper(KeyedTable arcticTable, KeyedTableSnapshot snapshot) { this.arcticTable = arcticTable; this.baseSnapshotId = snapshot.baseSnapshotId(); this.changeSnapshotId = snapshot.changeSnapshotId(); - this.partitionOptimizedSequence = snapshot.partitionOptimizedSequence(); - this.legacyPartitionMaxTransactionId = snapshot.legacyPartitionMaxTransactionId(); } /** @@ -165,18 +162,16 @@ public CloseableIterable scan() { UnkeyedTable baseTable = arcticTable.baseTable(); ChangeTable changeTable = arcticTable.changeTable(); if (changeSnapshotId != ArcticServiceConstants.INVALID_SNAPSHOT_ID) { - long maxSequence = - getMaxSequenceLimit( - arcticTable, - changeSnapshotId, - partitionOptimizedSequence, - legacyPartitionMaxTransactionId); + StructLikeMap optimizedSequence = + baseSnapshotId == ArcticServiceConstants.INVALID_SNAPSHOT_ID + ? StructLikeMap.create(arcticTable.spec().partitionType()) + : ArcticTableUtil.readOptimizedSequence(arcticTable, baseSnapshotId); + long maxSequence = getMaxSequenceLimit(arcticTable, changeSnapshotId, optimizedSequence); if (maxSequence != Long.MIN_VALUE) { ChangeTableIncrementalScan changeTableIncrementalScan = changeTable .newScan() - .fromSequence(partitionOptimizedSequence) - .fromLegacyTransaction(legacyPartitionMaxTransactionId) + .fromSequence(optimizedSequence) .toSequence(maxSequence) .useSnapshot(changeSnapshotId); try (CloseableIterable fileScanTasks = @@ -251,8 +246,7 @@ private boolean filterFilePartition(PartitionSpec partitionSpec, ContentFile private long getMaxSequenceLimit( KeyedTable arcticTable, long changeSnapshotId, - StructLikeMap partitionOptimizedSequence, - StructLikeMap legacyPartitionMaxTransactionId) { + StructLikeMap partitionOptimizedSequence) { ChangeTable changeTable = arcticTable.changeTable(); Snapshot changeSnapshot = changeTable.snapshot(changeSnapshotId); int totalFilesInSummary = @@ -272,7 +266,6 @@ private long getMaxSequenceLimit( changeTable .newScan() .fromSequence(partitionOptimizedSequence) - .fromLegacyTransaction(legacyPartitionMaxTransactionId) .useSnapshot(changeSnapshot.snapshotId()); Map changeFilesGroupBySequence = new HashMap<>(); try (CloseableIterable tasks = changeTableIncrementalScan.planFiles()) { diff --git a/ams/server/src/main/java/com/netease/arctic/server/table/KeyedTableSnapshot.java b/ams/server/src/main/java/com/netease/arctic/server/table/KeyedTableSnapshot.java index 46eaed94f5..c99bda3f23 100644 --- a/ams/server/src/main/java/com/netease/arctic/server/table/KeyedTableSnapshot.java +++ b/ams/server/src/main/java/com/netease/arctic/server/table/KeyedTableSnapshot.java @@ -18,23 +18,13 @@ package com.netease.arctic.server.table; -import org.apache.iceberg.util.StructLikeMap; - public class KeyedTableSnapshot implements TableSnapshot { private final long baseSnapshotId; private final long changeSnapshotId; - private final StructLikeMap partitionOptimizedSequence; - private final StructLikeMap legacyPartitionMaxTransactionId; - public KeyedTableSnapshot( - long baseSnapshotId, - long changeSnapshotId, - StructLikeMap partitionOptimizedSequence, - StructLikeMap legacyPartitionMaxTransactionId) { + public KeyedTableSnapshot(long baseSnapshotId, long changeSnapshotId) { this.baseSnapshotId = baseSnapshotId; this.changeSnapshotId = changeSnapshotId; - this.partitionOptimizedSequence = partitionOptimizedSequence; - this.legacyPartitionMaxTransactionId = legacyPartitionMaxTransactionId; } public long baseSnapshotId() { @@ -45,14 +35,6 @@ public long changeSnapshotId() { return changeSnapshotId; } - public StructLikeMap partitionOptimizedSequence() { - return partitionOptimizedSequence; - } - - public StructLikeMap legacyPartitionMaxTransactionId() { - return legacyPartitionMaxTransactionId; - } - @Override public long snapshotId() { return baseSnapshotId; 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 6de6ef41f9..c72f1fe359 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 @@ -19,7 +19,9 @@ package com.netease.arctic.server.table.executor; import com.netease.arctic.AmoroTable; +import com.netease.arctic.server.optimizing.OptimizingProcess; import com.netease.arctic.server.optimizing.plan.OptimizingEvaluator; +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.table.ArcticTable; @@ -58,6 +60,18 @@ private void tryEvaluatingPendingInput(TableRuntime tableRuntime, ArcticTable ta } } + @Override + public void handleConfigChanged(TableRuntime tableRuntime, TableConfiguration originalConfig) { + // After disabling self-optimizing, close the currently running optimizing process. + if (originalConfig.getOptimizingConfig().isEnabled() + && !tableRuntime.getTableConfiguration().getOptimizingConfig().isEnabled()) { + OptimizingProcess optimizingProcess = tableRuntime.getOptimizingProcess(); + if (optimizingProcess.getStatus() == OptimizingProcess.Status.RUNNING) { + optimizingProcess.close(); + } + } + } + @Override public void execute(TableRuntime tableRuntime) { try { 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 8f9b2f7e4d..98ca6e2640 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 @@ -35,7 +35,6 @@ import com.netease.arctic.table.TableMetaStore; import com.netease.arctic.utils.CatalogUtil; import com.netease.arctic.utils.TableFileUtil; -import com.netease.arctic.utils.TablePropertyUtil; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; @@ -54,7 +53,6 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.util.LocationUtil; -import org.apache.iceberg.util.StructLikeMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -89,15 +87,8 @@ public static TableSnapshot getSnapshot(ArcticTable arcticTable, TableRuntime ta if (arcticTable.isUnkeyedTable()) { return new BasicTableSnapshot(tableRuntime.getCurrentSnapshotId()); } else { - StructLikeMap partitionOptimizedSequence = - TablePropertyUtil.getPartitionOptimizedSequence(arcticTable.asKeyedTable()); - StructLikeMap legacyPartitionMaxTransactionId = - TablePropertyUtil.getLegacyPartitionMaxTransactionId(arcticTable.asKeyedTable()); return new KeyedTableSnapshot( - tableRuntime.getCurrentSnapshotId(), - tableRuntime.getCurrentChangeSnapshotId(), - partitionOptimizedSequence, - legacyPartitionMaxTransactionId); + tableRuntime.getCurrentSnapshotId(), tableRuntime.getCurrentChangeSnapshotId()); } } diff --git a/ams/server/src/test/java/com/netease/arctic/server/AmsEnvironment.java b/ams/server/src/test/java/com/netease/arctic/server/AmsEnvironment.java index 9ae32ef2a1..8e746edb29 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/AmsEnvironment.java +++ b/ams/server/src/test/java/com/netease/arctic/server/AmsEnvironment.java @@ -203,9 +203,6 @@ private void createInternalIceberg() { TableFormat.ICEBERG); tableService.createCatalog(catalogMeta); - catalogs.put( - INTERNAL_ICEBERG_CATALOG, - CatalogLoader.load(getTableServiceUrl() + "/" + INTERNAL_ICEBERG_CATALOG)); } private void createIcebergCatalog() { @@ -220,7 +217,6 @@ private void createIcebergCatalog() { properties, TableFormat.ICEBERG); tableService.createCatalog(catalogMeta); - catalogs.put(ICEBERG_CATALOG, CatalogLoader.load(getTableServiceUrl() + "/" + ICEBERG_CATALOG)); } private void createMixIcebergCatalog() { diff --git a/ams/server/src/test/java/com/netease/arctic/server/TestIcebergRestCatalogService.java b/ams/server/src/test/java/com/netease/arctic/server/TestIcebergRestCatalogService.java index fce3e0e833..394d5c836b 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/TestIcebergRestCatalogService.java +++ b/ams/server/src/test/java/com/netease/arctic/server/TestIcebergRestCatalogService.java @@ -231,7 +231,7 @@ public void testTableTransaction() throws IOException { } @Test - public void testArcticCatalogLoader() throws IOException { + public void testServerCatalogLoadTable() throws IOException { Table tbl = nsCatalog.createTable(identifier, schema, spec); DataFile[] files = IcebergDataTestHelpers.insert(tbl, newRecords).dataFiles(); AppendFiles appendFiles = tbl.newAppend(); @@ -240,9 +240,7 @@ public void testArcticCatalogLoader() throws IOException { ArcticCatalog catalog = ams.catalog(AmsEnvironment.INTERNAL_ICEBERG_CATALOG); ArcticTable arcticTable = - catalog.loadTable( - com.netease.arctic.table.TableIdentifier.of( - AmsEnvironment.INTERNAL_ICEBERG_CATALOG, database, table)); + (ArcticTable) serverCatalog.loadTable(database, table).originalTable(); Assertions.assertEquals(TableFormat.ICEBERG, arcticTable.format()); GenericUnkeyedDataReader reader = diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/BaseOptimizingChecker.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/BaseOptimizingChecker.java index 9fc24da190..fd8ff145f0 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/BaseOptimizingChecker.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/BaseOptimizingChecker.java @@ -75,7 +75,8 @@ protected void assertOptimizingProcess( Assert.assertEquals(optimizeType, optimizingProcess.getOptimizingType()); Assert.assertEquals( fileCntBefore, - optimizingProcess.getSummary().getRewriteDataFileCnt() + optimizingProcess.getSummary().getReRowDeletedDataFileCnt() + + optimizingProcess.getSummary().getRewriteDataFileCnt() + optimizingProcess.getSummary().getEqDeleteFileCnt() + optimizingProcess.getSummary().getPosDeleteFileCnt()); Assert.assertEquals(fileCntAfter, optimizingProcess.getSummary().getNewFileCnt()); diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/OptimizingTestHelpers.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/OptimizingTestHelpers.java index e1261ee299..dedfada90e 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/OptimizingTestHelpers.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/OptimizingTestHelpers.java @@ -25,14 +25,12 @@ import com.netease.arctic.server.utils.IcebergTableUtil; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.KeyedTable; -import com.netease.arctic.utils.TablePropertyUtil; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.RowDelta; import org.apache.iceberg.data.Record; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.StructLikeMap; import java.util.List; @@ -49,16 +47,8 @@ public static TableSnapshot getCurrentTableSnapshot(ArcticTable table) { public static KeyedTableSnapshot getCurrentKeyedTableSnapshot(KeyedTable keyedTable) { long baseSnapshotId = IcebergTableUtil.getSnapshotId(keyedTable.baseTable(), true); long changeSnapshotId = IcebergTableUtil.getSnapshotId(keyedTable.changeTable(), true); - StructLikeMap partitionOptimizedSequence = - TablePropertyUtil.getPartitionOptimizedSequence(keyedTable); - StructLikeMap legacyPartitionMaxTransactionId = - TablePropertyUtil.getLegacyPartitionMaxTransactionId(keyedTable); - return new KeyedTableSnapshot( - baseSnapshotId, - changeSnapshotId, - partitionOptimizedSequence, - legacyPartitionMaxTransactionId); + return new KeyedTableSnapshot(baseSnapshotId, changeSnapshotId); } public static List generateRecord( diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestMixedHiveOptimizing.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestMixedHiveOptimizing.java index 993f15bbff..7ee578d027 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestMixedHiveOptimizing.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestMixedHiveOptimizing.java @@ -61,27 +61,30 @@ public void testHiveKeyedTableMajorOptimizeNotMove() throws TException, IOExcept checker.assertOptimizingProcess(optimizeHistory, OptimizingType.FULL, 1, 1); assertIdRange(readRecords(table), 1, 100); // assert file are in hive location - // assertIdRange(readHiveTableData(), 1, 100); + assertIdRange(readHiveTableData(), 1, 100); // Step2: write 1 change delete record - writeChange(table, null, Lists.newArrayList(newRecord(1, "aaa", quickDateWithZone(3)))); - // wait Minor Optimize result, generate 1 pos-delete file + writeChange( + table, + Lists.newArrayList(newRecord(101, "aaa", quickDateWithZone(3))), + Lists.newArrayList(newRecord(1, "aaa", quickDateWithZone(3)))); + // wait Minor Optimize result, generate 1 pos-delete file and 1 data file, but not move to hive + // location optimizeHistory = checker.waitOptimizeResult(); - checker.assertOptimizingProcess(optimizeHistory, OptimizingType.MINOR, 2, 1); - assertIdRange(readRecords(table), 2, 100); - // assertIdRange(readHiveTableData(), 1, 100); + checker.assertOptimizingProcess(optimizeHistory, OptimizingType.MINOR, 3, 2); + assertIdRange(readRecords(table), 2, 101); + assertIdRange(readHiveTableData(), 1, 100); // Step3: write 2 small files to base - writeBase(table, rangeFromTo(101, 102, "aaa", quickDateWithZone(3))); + writeBase(table, rangeFromTo(102, 103, "aaa", quickDateWithZone(3))); // should not optimize with 1 small file - optimizeHistory = checker.waitOptimizeResult(); - checker.assertOptimizingProcess(optimizeHistory, OptimizingType.MINOR, 2, 1); - writeBase(table, rangeFromTo(103, 104, "aaa", quickDateWithZone(3))); - // wait Major Optimize result, generate 1 data file from 2 small files, but not move to hive - // location + checker.assertOptimizeHangUp(); + writeBase(table, rangeFromTo(104, 105, "aaa", quickDateWithZone(3))); + // wait Minor Optimize result, generate 1 data file, but not move to hive location optimizeHistory = checker.waitOptimizeResult(); checker.assertOptimizingProcess(optimizeHistory, OptimizingType.MINOR, 3, 1); - assertIdRange(readRecords(table), 2, 104); + assertIdRange(readRecords(table), 2, 105); + assertIdRange(readHiveTableData(), 1, 100); checker.assertOptimizeHangUp(); } @@ -90,22 +93,24 @@ public void testHiveKeyedTableMajorOptimizeAndMove() throws TException, IOExcept KeyedTable table = arcticTable.asKeyedTable(); // Step1: write 1 data file into base node(0,0) updateProperties(table, TableProperties.BASE_FILE_INDEX_HASH_BUCKET, 1 + ""); + updateProperties(table, TableProperties.SELF_OPTIMIZING_FULL_TRIGGER_INTERVAL, 1000 + ""); + updateProperties(table, TableProperties.SELF_OPTIMIZING_FULL_REWRITE_ALL_FILES, false + ""); writeBase(table, rangeFromTo(1, 100, "aaa", quickDateWithZone(3))); // wait Full Optimize result OptimizingProcessMeta optimizeHistory = checker.waitOptimizeResult(); checker.assertOptimizingProcess(optimizeHistory, OptimizingType.FULL, 1, 1); assertIdRange(readRecords(table), 1, 100); // assert file are in hive location - // assertIdRange(readHiveTableData(), 1, 100); + assertIdRange(readHiveTableData(), 1, 100); // Step2: write 1 small file to base writeBase(table, rangeFromTo(101, 102, "aaa", quickDateWithZone(3))); // wait Major Optimize result, generate 1 data file from 2 small files, but not move to hive // location optimizeHistory = checker.waitOptimizeResult(); - checker.assertOptimizingProcess(optimizeHistory, OptimizingType.MINOR, 2, 1); + checker.assertOptimizingProcess(optimizeHistory, OptimizingType.FULL, 1, 1); assertIdRange(readRecords(table), 1, 102); - // assertIdRange(readHiveTableData(), 1, 102); + assertIdRange(readHiveTableData(), 1, 102); checker.assertOptimizeHangUp(); } diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestMixedIcebergOptimizing.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestMixedIcebergOptimizing.java index 73082d8263..bface49d86 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestMixedIcebergOptimizing.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/TestMixedIcebergOptimizing.java @@ -211,7 +211,7 @@ public void testPkTableMajorOptimizeLeftPosDelete() { // wait Minor Optimize result, no major optimize because there is only 1 base file for each node optimizeHistory = checker.waitOptimizeResult(); - checker.assertOptimizingProcess(optimizeHistory, OptimizingType.MAJOR, 10, 4); + checker.assertOptimizingProcess(optimizeHistory, OptimizingType.MINOR, 6, 1); assertIds( readRecords(table), 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 21, 25, 29); diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpire.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpire.java index 4e381df002..2f4f0fb792 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpire.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/maintainer/TestSnapshotExpire.java @@ -26,7 +26,6 @@ import com.netease.arctic.catalog.BasicCatalogTestHelper; import com.netease.arctic.catalog.CatalogTestHelper; import com.netease.arctic.data.ChangeAction; -import com.netease.arctic.op.UpdatePartitionProperties; import com.netease.arctic.server.dashboard.utils.AmsUtil; import com.netease.arctic.server.optimizing.OptimizingProcess; import com.netease.arctic.server.optimizing.OptimizingStatus; @@ -34,17 +33,22 @@ import com.netease.arctic.server.table.TableConfiguration; import com.netease.arctic.server.table.TableRuntime; import com.netease.arctic.server.table.executor.ExecutorTestBase; +import com.netease.arctic.table.BaseTable; import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.ArcticTableUtil; +import com.netease.arctic.utils.StatisticsFileUtil; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFiles; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.apache.iceberg.util.StructLikeMap; import org.jetbrains.annotations.NotNull; import org.junit.Assert; import org.junit.Assume; @@ -62,8 +66,6 @@ @RunWith(Parameterized.class) public class TestSnapshotExpire extends ExecutorTestBase { - private final List changeTableFiles = new ArrayList<>(); - @Parameterized.Parameters(name = "{0}, {1}") public static Object[] parameters() { return new Object[][] { @@ -86,7 +88,7 @@ public TestSnapshotExpire(CatalogTestHelper catalogTestHelper, TableTestHelper t } @Test - public void testExpireChangeTableFiles() throws Exception { + public void testExpireChangeTableFiles() { Assume.assumeTrue(isKeyedTable()); KeyedTable testKeyedTable = getArcticTable().asKeyedTable(); testKeyedTable.updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); @@ -102,13 +104,13 @@ public void testExpireChangeTableFiles() throws Exception { Assert.assertEquals(1, partitions.size()); } - UpdatePartitionProperties updateProperties = - testKeyedTable.baseTable().updatePartitionProperties(null); - updateProperties.set(partitions.get(0), TableProperties.PARTITION_OPTIMIZED_SEQUENCE, "3"); + StructLikeMap optimizedSequence = + StructLikeMap.create(testKeyedTable.spec().partitionType()); + optimizedSequence.put(partitions.get(0), 3L); if (isPartitionedTable()) { - updateProperties.set(partitions.get(1), TableProperties.PARTITION_OPTIMIZED_SEQUENCE, "1"); + optimizedSequence.put(partitions.get(1), 1L); } - updateProperties.commit(); + writeOptimizedSequence(testKeyedTable, optimizedSequence); s1Files.forEach( file -> Assert.assertTrue(testKeyedTable.changeTable().io().exists(file.path().toString()))); @@ -127,6 +129,23 @@ public void testExpireChangeTableFiles() throws Exception { Assert.assertFalse(testKeyedTable.changeTable().io().exists(file.path().toString()))); } + private void writeOptimizedSequence( + KeyedTable testKeyedTable, StructLikeMap optimizedSequence) { + BaseTable baseTable = testKeyedTable.baseTable(); + baseTable.newAppend().set(ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE_EXIST, "true").commit(); + Snapshot snapshot = baseTable.currentSnapshot(); + StatisticsFile statisticsFile = + StatisticsFileUtil.writerBuilder(baseTable) + .withSnapshotId(snapshot.snapshotId()) + .build() + .add( + ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE, + optimizedSequence, + StatisticsFileUtil.createPartitionDataSerializer(baseTable.spec(), Long.class)) + .complete(); + baseTable.updateStatistics().setStatistics(snapshot.snapshotId(), statisticsFile).commit(); + } + @Test public void testExpiredChangeTableFilesInBase() { Assume.assumeTrue(isKeyedTable()); @@ -142,11 +161,11 @@ public void testExpiredChangeTableFilesInBase() { 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, "1"); - updateProperties.commit(); + StructLikeMap optimizedSequence = + StructLikeMap.create(testKeyedTable.spec().partitionType()); + optimizedSequence.put(partitions.get(0), 3L); + optimizedSequence.put(partitions.get(1), 1L); + writeOptimizedSequence(testKeyedTable, optimizedSequence); s1Files.forEach(file -> Assert.assertTrue(testKeyedTable.io().exists(file.path().toString()))); MixedTableMaintainer tableMaintainer = new MixedTableMaintainer(testKeyedTable); @@ -195,11 +214,12 @@ public void testNotExpireFlinkLatestCommit4ChangeTable() { tableMaintainer.expireSnapshots(tableRuntime); Assert.assertEquals(2, Iterables.size(testKeyedTable.changeTable().snapshots())); - HashSet expectedSnapshots = new HashSet<>(); + List expectedSnapshots = new ArrayList<>(); expectedSnapshots.add(checkpointTime2Snapshot); expectedSnapshots.add(lastSnapshot); - Iterators.elementsEqual( - expectedSnapshots.iterator(), testKeyedTable.changeTable().snapshots().iterator()); + Assert.assertTrue( + Iterators.elementsEqual( + expectedSnapshots.iterator(), testKeyedTable.changeTable().snapshots().iterator())); } @Test @@ -213,7 +233,6 @@ public void testNotExpireFlinkLatestCommit4All() { AppendFiles appendFiles = table.newAppend(); appendFiles.set(FLINK_MAX_COMMITTED_CHECKPOINT_ID, "100"); appendFiles.commit(); - long checkpointTime = table.currentSnapshot().timestampMillis(); AppendFiles appendFiles2 = table.newAppend(); appendFiles2.set(FLINK_MAX_COMMITTED_CHECKPOINT_ID, "101"); @@ -238,10 +257,11 @@ public void testNotExpireFlinkLatestCommit4All() { tableMaintainer.expireSnapshots(tableRuntime); Assert.assertEquals(2, Iterables.size(table.snapshots())); - HashSet expectedSnapshots = new HashSet<>(); + List expectedSnapshots = new ArrayList<>(); expectedSnapshots.add(checkpointTime2Snapshot); expectedSnapshots.add(lastSnapshot); - Iterators.elementsEqual(expectedSnapshots.iterator(), table.snapshots().iterator()); + Assert.assertTrue( + Iterators.elementsEqual(expectedSnapshots.iterator(), table.snapshots().iterator())); } @Test @@ -273,7 +293,7 @@ public void testNotExpireOptimizeCommit4All() { Mockito.when(optimizingProcess.getTargetSnapshotId()).thenReturn(optimizeSnapshotId); Mockito.when(tableRuntime.getOptimizingStatus()).thenReturn(OptimizingStatus.COMMITTING); Mockito.when(tableRuntime.getOptimizingProcess()).thenReturn(optimizingProcess); - HashSet expectedSnapshots = new HashSet<>(); + List expectedSnapshots = new ArrayList<>(); expectedSnapshots.add(table.currentSnapshot()); table.newAppend().commit(); @@ -283,7 +303,8 @@ public void testNotExpireOptimizeCommit4All() { new MixedTableMaintainer(table).expireSnapshots(tableRuntime); Assert.assertEquals(3, Iterables.size(table.snapshots())); - Iterators.elementsEqual(expectedSnapshots.iterator(), table.snapshots().iterator()); + Assert.assertTrue( + Iterators.elementsEqual(expectedSnapshots.iterator(), table.snapshots().iterator())); } @Test @@ -331,13 +352,14 @@ public void testExpireTableFilesRepeatedly() { } else { Assert.assertEquals(1, partitions.size()); } - UpdatePartitionProperties updateProperties = - testKeyedTable.baseTable().updatePartitionProperties(null); - updateProperties.set(partitions.get(0), TableProperties.PARTITION_OPTIMIZED_SEQUENCE, "3"); + + StructLikeMap optimizedSequence = + StructLikeMap.create(testKeyedTable.spec().partitionType()); + optimizedSequence.put(partitions.get(0), 3L); if (isPartitionedTable()) { - updateProperties.set(partitions.get(1), TableProperties.PARTITION_OPTIMIZED_SEQUENCE, "3"); + optimizedSequence.put(partitions.get(1), 3L); } - updateProperties.commit(); + writeOptimizedSequence(testKeyedTable, optimizedSequence); Set top8Files = new HashSet<>(); testKeyedTable.changeTable().newScan().planFiles().forEach(task -> top8Files.add(task.file())); @@ -348,7 +370,6 @@ public void testExpireTableFilesRepeatedly() { insertChangeDataFiles(testKeyedTable, 3).stream() .map(DataFile::path) .collect(Collectors.toSet()); - long thirdCommitTime = testKeyedTable.changeTable().currentSnapshot().timestampMillis(); Assert.assertEquals(12, Iterables.size(testKeyedTable.changeTable().newScan().planFiles())); Assert.assertEquals(3, Iterables.size(testKeyedTable.changeTable().snapshots())); @@ -373,11 +394,72 @@ private static Set getDataFiles(KeyedTable testKeyedTable) { return dataFiles; } + @Test + public void testExpireStatisticsFiles() { + Assume.assumeTrue(isKeyedTable()); + KeyedTable testKeyedTable = getArcticTable().asKeyedTable(); + BaseTable baseTable = testKeyedTable.baseTable(); + testKeyedTable.updateProperties().set(TableProperties.BASE_SNAPSHOT_KEEP_MINUTES, "0").commit(); + // commit an empty snapshot and its statistic file + baseTable.newAppend().commit(); + Snapshot s1 = baseTable.currentSnapshot(); + StatisticsFile file1 = + StatisticsFileUtil.writerBuilder(baseTable) + .withSnapshotId(s1.snapshotId()) + .build() + .add( + ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE, + StructLikeMap.create(baseTable.spec().partitionType()), + StatisticsFileUtil.createPartitionDataSerializer(baseTable.spec(), Long.class)) + .complete(); + baseTable.updateStatistics().setStatistics(s1.snapshotId(), file1).commit(); + + // commit an empty snapshot and its statistic file + baseTable.newAppend().commit(); + Snapshot s2 = baseTable.currentSnapshot(); + StatisticsFile file2 = + StatisticsFileUtil.writerBuilder(baseTable) + .withSnapshotId(s2.snapshotId()) + .build() + .add( + ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE, + StructLikeMap.create(baseTable.spec().partitionType()), + StatisticsFileUtil.createPartitionDataSerializer(baseTable.spec(), Long.class)) + .complete(); + baseTable.updateStatistics().setStatistics(s2.snapshotId(), file2).commit(); + + long expireTime = waitUntilAfter(s2.timestampMillis()); + + // commit an empty snapshot and its statistic file + baseTable.newAppend().commit(); + Snapshot s3 = baseTable.currentSnapshot(); + // note: s2 ans s3 use the same statistics file + StatisticsFile file3 = StatisticsFileUtil.copyToSnapshot(file2, s3.snapshotId()); + baseTable.updateStatistics().setStatistics(s3.snapshotId(), file3).commit(); + + Assert.assertEquals(3, Iterables.size(baseTable.snapshots())); + Assert.assertTrue(baseTable.io().exists(file1.path())); + Assert.assertTrue(baseTable.io().exists(file2.path())); + Assert.assertTrue(baseTable.io().exists(file3.path())); + new MixedTableMaintainer(testKeyedTable).expireSnapshots(expireTime); + + Assert.assertEquals(1, Iterables.size(baseTable.snapshots())); + Assert.assertFalse(baseTable.io().exists(file1.path())); + // file2 should not be removed, since it is used by s3 + Assert.assertTrue(baseTable.io().exists(file2.path())); + Assert.assertTrue(baseTable.io().exists(file3.path())); + } + + private long waitUntilAfter(long timestampMillis) { + long current = System.currentTimeMillis(); + while (current <= timestampMillis) { + current = System.currentTimeMillis(); + } + return current; + } + private List insertChangeDataFiles(KeyedTable testKeyedTable, long transactionId) { - List changeInsertFiles = - writeAndCommitChangeStore( - testKeyedTable, transactionId, ChangeAction.INSERT, createRecords(1, 100)); - changeTableFiles.addAll(changeInsertFiles); - return changeInsertFiles; + return writeAndCommitChangeStore( + testKeyedTable, transactionId, ChangeAction.INSERT, createRecords(1, 100)); } } diff --git a/ams/server/src/test/java/com/netease/arctic/server/optimizing/plan/TestKeyedPartitionPlan.java b/ams/server/src/test/java/com/netease/arctic/server/optimizing/plan/TestKeyedPartitionPlan.java index c468ede9d8..7a1e6cd4c6 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/optimizing/plan/TestKeyedPartitionPlan.java +++ b/ams/server/src/test/java/com/netease/arctic/server/optimizing/plan/TestKeyedPartitionPlan.java @@ -148,8 +148,8 @@ public void testChangeFilesWithDelete() { Snapshot toSnapshot = getArcticTable().changeTable().currentSnapshot(); AbstractPartitionPlan plan = buildPlanWithCurrentFiles(); - Assert.assertEquals(fromSnapshot.sequenceNumber(), plan.getFromSequence()); - Assert.assertEquals(toSnapshot.sequenceNumber(), plan.getToSequence()); + Assert.assertEquals(fromSnapshot.sequenceNumber(), (long) plan.getFromSequence()); + Assert.assertEquals(toSnapshot.sequenceNumber(), (long) plan.getToSequence()); List taskDescriptors = plan.splitTasks(0); 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 148c50f66c..4daa7a629a 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 @@ -30,13 +30,11 @@ import com.netease.arctic.server.utils.IcebergTableUtil; import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.utils.TablePropertyUtil; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.data.Record; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.StructLikeMap; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -243,17 +241,8 @@ public void testScanPartialChange() { protected KeyedTableFileScanHelper buildFileScanHelper() { long baseSnapshotId = IcebergTableUtil.getSnapshotId(getArcticTable().baseTable(), true); long changeSnapshotId = IcebergTableUtil.getSnapshotId(getArcticTable().changeTable(), true); - StructLikeMap partitionOptimizedSequence = - TablePropertyUtil.getPartitionOptimizedSequence(getArcticTable()); - StructLikeMap legacyPartitionMaxTransactionId = - TablePropertyUtil.getLegacyPartitionMaxTransactionId(getArcticTable()); return new KeyedTableFileScanHelper( - getArcticTable(), - new KeyedTableSnapshot( - baseSnapshotId, - changeSnapshotId, - partitionOptimizedSequence, - legacyPartitionMaxTransactionId)); + getArcticTable(), new KeyedTableSnapshot(baseSnapshotId, changeSnapshotId)); } private void appendChange(List dataFiles) { diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/AMSTableTestBase.java b/ams/server/src/test/java/com/netease/arctic/server/table/AMSTableTestBase.java index 6f4c7a9342..bd9e1391b1 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/AMSTableTestBase.java +++ b/ams/server/src/test/java/com/netease/arctic/server/table/AMSTableTestBase.java @@ -18,20 +18,25 @@ package com.netease.arctic.server.table; +import com.netease.arctic.CommonUnifiedCatalog; import com.netease.arctic.TableTestHelper; +import com.netease.arctic.UnifiedCatalog; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.ams.api.TableMeta; import com.netease.arctic.catalog.ArcticCatalog; +import com.netease.arctic.catalog.CatalogLoader; import com.netease.arctic.catalog.CatalogTestHelper; -import com.netease.arctic.catalog.IcebergCatalogWrapper; import com.netease.arctic.catalog.MixedTables; import com.netease.arctic.hive.TestHMS; -import com.netease.arctic.mixed.BasicMixedIcebergCatalog; import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.utils.CatalogUtil; import com.netease.arctic.utils.ConvertStructUtil; import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.thrift.TException; import org.junit.After; import org.junit.Assert; @@ -50,7 +55,7 @@ public class AMSTableTestBase extends TableServiceTestBase { private final TableTestHelper tableTestHelper; private String catalogWarehouse; private MixedTables mixedTables; - private ArcticCatalog externalCatalog; + private UnifiedCatalog externalCatalog; private CatalogMeta catalogMeta; private TableMeta tableMeta; @@ -73,20 +78,15 @@ public AMSTableTestBase( public void init() throws IOException, TException { catalogWarehouse = temp.newFolder().getPath(); catalogMeta = catalogTestHelper.buildCatalogMeta(catalogWarehouse); - if (catalogTestHelper.isInternalCatalog()) { - if (TableFormat.MIXED_ICEBERG.equals(catalogTestHelper.tableFormat())) { + if (catalogTestHelper.isInternalCatalog() + || TableFormat.MIXED_HIVE.equals(catalogTestHelper.tableFormat())) { + if (TableFormat.MIXED_ICEBERG.equals(catalogTestHelper.tableFormat()) + || TableFormat.MIXED_HIVE.equals(catalogTestHelper.tableFormat())) { mixedTables = catalogTestHelper.buildMixedTables(catalogMeta); tableMeta = buildTableMeta(); } } else { - if (TableFormat.ICEBERG.equals(catalogTestHelper.tableFormat())) { - externalCatalog = new IcebergCatalogWrapper(catalogMeta); - } else if (TableFormat.MIXED_ICEBERG.equals(catalogTestHelper.tableFormat())) { - externalCatalog = new BasicMixedIcebergCatalog(catalogMeta); - } else if (TableFormat.MIXED_HIVE.equals(catalogTestHelper.tableFormat())) { - mixedTables = catalogTestHelper.buildMixedTables(catalogMeta); - tableMeta = buildTableMeta(); - } + externalCatalog = new CommonUnifiedCatalog(() -> catalogMeta, Maps.newHashMap()); } tableService().createCatalog(catalogMeta); @@ -164,24 +164,54 @@ protected void createTable() { TableMetadata tableMetadata = tableMetadata(); tableService().createTable(catalogMeta.getCatalogName(), tableMetadata); } else { - externalCatalog - .newTableBuilder(tableTestHelper.id(), tableTestHelper.tableSchema()) - .withPartitionSpec(tableTestHelper.partitionSpec()) - .withProperties(tableTestHelper.tableProperties()) - .withPrimaryKeySpec(tableTestHelper.primaryKeySpec()) - .create(); + switch (catalogTestHelper.tableFormat()) { + case ICEBERG: + createIcebergTable(); + break; + case MIXED_ICEBERG: + createMixedIcebergTable(); + break; + default: + throw new IllegalStateException("un-support format"); + } tableService().exploreExternalCatalog(); } serverTableIdentifier = tableService().listManagedTables().get(0); } + private void createMixedIcebergTable() { + ArcticCatalog catalog = + CatalogLoader.createCatalog( + catalogMeta.getCatalogName(), + catalogMeta.getCatalogType(), + catalogMeta.getCatalogProperties(), + CatalogUtil.buildMetaStore(catalogMeta)); + catalog + .newTableBuilder(tableTestHelper.id(), tableTestHelper.tableSchema()) + .withPartitionSpec(tableTestHelper.partitionSpec()) + .withProperties(tableTestHelper.tableProperties()) + .withPrimaryKeySpec(tableTestHelper.primaryKeySpec()) + .create(); + } + + private void createIcebergTable() { + Catalog catalog = catalogTestHelper.buildIcebergCatalog(catalogMeta); + catalog.createTable( + TableIdentifier.of(tableTestHelper.id().getDatabase(), tableTestHelper.id().getTableName()), + tableTestHelper.tableSchema(), + tableTestHelper.partitionSpec(), + tableTestHelper.tableProperties()); + } + protected void dropTable() { if (externalCatalog == null) { mixedTables.dropTableByMeta(tableMeta, true); tableService().dropTableMetadata(tableMeta.getTableIdentifier(), true); } else { - externalCatalog.dropTable(tableTestHelper.id(), true); + String database = tableTestHelper.id().getDatabase(); + String table = tableTestHelper.id().getTableName(); + externalCatalog.dropTable(database, table, true); tableService().exploreExternalCatalog(); } } diff --git a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestDataExpire.java b/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestDataExpire.java index dd5932e8b5..8cdc26b05f 100644 --- a/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestDataExpire.java +++ b/ams/server/src/test/java/com/netease/arctic/server/table/executor/TestDataExpire.java @@ -44,7 +44,6 @@ import com.netease.arctic.table.TableProperties; import com.netease.arctic.utils.CompatiblePropertyUtil; import com.netease.arctic.utils.ContentFiles; -import com.netease.arctic.utils.TablePropertyUtil; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.PartitionSpec; @@ -54,7 +53,6 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.StructLikeMap; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -478,17 +476,8 @@ protected KeyedTableFileScanHelper buildKeyedFileScanHelper() { IcebergTableUtil.getSnapshotId(getArcticTable().asKeyedTable().baseTable(), true); long changeSnapshotId = IcebergTableUtil.getSnapshotId(getArcticTable().asKeyedTable().changeTable(), true); - StructLikeMap partitionOptimizedSequence = - TablePropertyUtil.getPartitionOptimizedSequence(getArcticTable().asKeyedTable()); - StructLikeMap legacyPartitionMaxTransactionId = - TablePropertyUtil.getLegacyPartitionMaxTransactionId(getArcticTable().asKeyedTable()); return new KeyedTableFileScanHelper( - getArcticTable().asKeyedTable(), - new KeyedTableSnapshot( - baseSnapshotId, - changeSnapshotId, - partitionOptimizedSequence, - legacyPartitionMaxTransactionId)); + getArcticTable().asKeyedTable(), new KeyedTableSnapshot(baseSnapshotId, changeSnapshotId)); } protected TableFileScanHelper getTableFileScanHelper() { diff --git a/core/src/main/java/com/netease/arctic/AmoroCatalog.java b/core/src/main/java/com/netease/arctic/AmoroCatalog.java index 2b9e958c3e..0dedd12069 100644 --- a/core/src/main/java/com/netease/arctic/AmoroCatalog.java +++ b/core/src/main/java/com/netease/arctic/AmoroCatalog.java @@ -73,4 +73,13 @@ public interface AmoroCatalog { * @throws NoSuchTableException when table not exists. */ AmoroTable loadTable(String database, String table); + + /** + * drop table from catalog. + * + * @param database database name + * @param table table name + * @param purge purge table data. + */ + boolean dropTable(String database, String table, boolean purge); } diff --git a/core/src/main/java/com/netease/arctic/CommonUnifiedCatalog.java b/core/src/main/java/com/netease/arctic/CommonUnifiedCatalog.java index e1973b2034..96dae7bcdf 100644 --- a/core/src/main/java/com/netease/arctic/CommonUnifiedCatalog.java +++ b/core/src/main/java/com/netease/arctic/CommonUnifiedCatalog.java @@ -42,8 +42,10 @@ public class CommonUnifiedCatalog implements UnifiedCatalog { private final Map properties = Maps.newHashMap(); public CommonUnifiedCatalog( - Supplier catalogMetaSupplier, CatalogMeta meta, Map properties) { - this.meta = meta; + Supplier catalogMetaSupplier, Map properties) { + CatalogMeta catalogMeta = catalogMetaSupplier.get(); + CatalogUtil.mergeCatalogProperties(catalogMeta, properties); + this.meta = catalogMeta; this.properties.putAll(properties); this.metaSupplier = catalogMetaSupplier; initializeFormatCatalogs(); @@ -79,7 +81,7 @@ public void dropDatabase(String database) { if (!exist(database)) { throw new NoSuchDatabaseException("Database: " + database + " does not exist."); } - if (listTables(database).size() > 0) { + if (!listTables(database).isEmpty()) { throw new IllegalStateException("Database: " + database + " is not empty."); } findFirstFormatCatalog(TableFormat.values()).dropDatabase(database); @@ -91,7 +93,11 @@ public AmoroTable loadTable(String database, String table) { throw new NoSuchDatabaseException("Database: " + database + " does not exist."); } - return formatCatalogAsOrder(TableFormat.MIXED_ICEBERG, TableFormat.ICEBERG, TableFormat.PAIMON) + return formatCatalogAsOrder( + TableFormat.MIXED_HIVE, + TableFormat.MIXED_ICEBERG, + TableFormat.ICEBERG, + TableFormat.PAIMON) .map( formatCatalog -> { try { @@ -116,7 +122,9 @@ public List listTables(String database) { throw new NoSuchDatabaseException("Database: " + database + " does not exist."); } TableFormat[] formats = - new TableFormat[] {TableFormat.MIXED_ICEBERG, TableFormat.ICEBERG, TableFormat.PAIMON}; + new TableFormat[] { + TableFormat.MIXED_HIVE, TableFormat.MIXED_ICEBERG, TableFormat.ICEBERG, TableFormat.PAIMON + }; Map tableNameToFormat = Maps.newHashMap(); for (TableFormat format : formats) { @@ -138,10 +146,25 @@ public List listTables(String database) { .collect(Collectors.toList()); } + @Override + public boolean dropTable(String database, String table, boolean purge) { + try { + AmoroTable t = loadTable(database, table); + return findFirstFormatCatalog(t.format()).dropTable(database, table, purge); + } catch (NoSuchTableException e) { + return false; + } + } + @Override public synchronized void refresh() { - this.meta = metaSupplier.get(); + CatalogMeta newMeta = metaSupplier.get(); CatalogUtil.mergeCatalogProperties(meta, properties); + if (newMeta.equals(this.meta)) { + return; + } + this.meta = newMeta; + this.initializeFormatCatalogs(); } protected void initializeFormatCatalogs() { @@ -152,11 +175,7 @@ protected void initializeFormatCatalogs() { for (FormatCatalogFactory factory : loader) { if (formats.contains(factory.format())) { FormatCatalog catalog = - factory.create( - name(), - meta.getCatalogType(), - meta.getCatalogProperties(), - store.getConfiguration()); + factory.create(name(), meta.getCatalogType(), meta.getCatalogProperties(), store); formatCatalogs.put(factory.format(), catalog); } } diff --git a/core/src/main/java/com/netease/arctic/FormatCatalogFactory.java b/core/src/main/java/com/netease/arctic/FormatCatalogFactory.java index 029f807349..20c05801fc 100644 --- a/core/src/main/java/com/netease/arctic/FormatCatalogFactory.java +++ b/core/src/main/java/com/netease/arctic/FormatCatalogFactory.java @@ -19,7 +19,7 @@ package com.netease.arctic; import com.netease.arctic.ams.api.TableFormat; -import org.apache.hadoop.conf.Configuration; +import com.netease.arctic.table.TableMetaStore; import java.util.Map; @@ -32,14 +32,14 @@ public interface FormatCatalogFactory { * @param catalogName catalog name * @param metastoreType metastore type * @param properties catalog properties - * @param configuration hadoop configuration + * @param metaStore authentication context * @return a new {@link FormatCatalog} */ FormatCatalog create( String catalogName, String metastoreType, Map properties, - Configuration configuration); + TableMetaStore metaStore); /** format of this catalog factory */ TableFormat format(); diff --git a/core/src/main/java/com/netease/arctic/UnifiedCatalogLoader.java b/core/src/main/java/com/netease/arctic/UnifiedCatalogLoader.java index d4738d5fc1..7ac2621650 100644 --- a/core/src/main/java/com/netease/arctic/UnifiedCatalogLoader.java +++ b/core/src/main/java/com/netease/arctic/UnifiedCatalogLoader.java @@ -20,7 +20,7 @@ import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.NoSuchObjectException; -import com.netease.arctic.utils.CatalogUtil; +import com.netease.arctic.ams.api.properties.CatalogMetaProperties; import java.util.Map; import java.util.function.Supplier; @@ -28,17 +28,14 @@ public class UnifiedCatalogLoader { public static UnifiedCatalog loadUnifiedCatalog( - String amsUri, String catalogName, Map clientSideProperties) { + String amsUri, String catalogName, Map props) { AmsClient client = new PooledAmsClient(amsUri); - return loadUnifiedCatalog(client, catalogName, clientSideProperties); - } - - private static UnifiedCatalog loadUnifiedCatalog( - AmsClient client, String catalogName, Map props) { Supplier metaSupplier = () -> { try { - return client.getCatalog(catalogName); + CatalogMeta meta = client.getCatalog(catalogName); + meta.putToCatalogProperties(CatalogMetaProperties.AMS_URI, amsUri); + return meta; } catch (NoSuchObjectException e) { throw new IllegalStateException( "catalog not found, please check catalog name:" + catalogName, e); @@ -47,8 +44,6 @@ private static UnifiedCatalog loadUnifiedCatalog( } }; - CatalogMeta catalogMeta = metaSupplier.get(); - CatalogUtil.mergeCatalogProperties(catalogMeta, props); - return new CommonUnifiedCatalog(metaSupplier, catalogMeta, props); + return new CommonUnifiedCatalog(metaSupplier, props); } } diff --git a/core/src/main/java/com/netease/arctic/catalog/ArcticCatalog.java b/core/src/main/java/com/netease/arctic/catalog/ArcticCatalog.java index ac1d8e0d46..919d7b5cac 100644 --- a/core/src/main/java/com/netease/arctic/catalog/ArcticCatalog.java +++ b/core/src/main/java/com/netease/arctic/catalog/ArcticCatalog.java @@ -18,11 +18,10 @@ package com.netease.arctic.catalog; -import com.netease.arctic.AmsClient; -import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.TableBuilder; import com.netease.arctic.table.TableIdentifier; +import com.netease.arctic.table.TableMetaStore; import com.netease.arctic.table.blocker.TableBlockerManager; import org.apache.iceberg.Schema; import org.apache.iceberg.exceptions.AlreadyExistsException; @@ -46,11 +45,11 @@ public interface ArcticCatalog { * Initialize a catalog given a custom name and a map of catalog properties. all catalog implement * must be no-args construct. Catalogs will call this method after implement object created. * - * @param client client of arctic metastore - * @param meta catalog init struct + * @param name name of catalog * @param properties client side catalog properties + * @param metaStore auth context. */ - void initialize(AmsClient client, CatalogMeta meta, Map properties); + void initialize(String name, Map properties, TableMetaStore metaStore); /** * Show database list of catalog. @@ -135,9 +134,6 @@ default boolean tableExists(TableIdentifier tableIdentifier) { */ TableBuilder newTableBuilder(TableIdentifier identifier, Schema schema); - /** Refresh catalog meta */ - void refresh(); - /** * Return a table blocker manager. * diff --git a/core/src/main/java/com/netease/arctic/catalog/BasicArcticCatalog.java b/core/src/main/java/com/netease/arctic/catalog/BasicArcticCatalog.java index 818c20747e..2f79deafa6 100644 --- a/core/src/main/java/com/netease/arctic/catalog/BasicArcticCatalog.java +++ b/core/src/main/java/com/netease/arctic/catalog/BasicArcticCatalog.java @@ -24,8 +24,8 @@ import com.netease.arctic.AmsClient; import com.netease.arctic.NoSuchDatabaseException; +import com.netease.arctic.PooledAmsClient; import com.netease.arctic.ams.api.AlreadyExistsException; -import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.NoSuchObjectException; import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.ams.api.TableMeta; @@ -72,34 +72,43 @@ public class BasicArcticCatalog implements ArcticCatalog { private static final Logger LOG = LoggerFactory.getLogger(BasicArcticCatalog.class); protected AmsClient client; - protected CatalogMeta catalogMeta; - protected Map customProperties; + protected String name; + protected Map catalogProperties; protected MixedTables tables; protected transient TableMetaStore tableMetaStore; @Override public String name() { - return catalogMeta.getCatalogName(); + return name; } @Override - public void initialize(AmsClient client, CatalogMeta meta, Map properties) { - this.client = client; - this.catalogMeta = meta; - this.customProperties = properties; - CatalogUtil.mergeCatalogProperties(catalogMeta, properties); - tables = newMixedTables(catalogMeta); - tableMetaStore = CatalogUtil.buildMetaStore(meta); + public void initialize(String name, Map properties, TableMetaStore metaStore) { + if (properties.containsKey(CatalogMetaProperties.AMS_URI)) { + this.client = new PooledAmsClient(properties.get(CatalogMetaProperties.AMS_URI)); + } + this.name = name; + this.catalogProperties = properties; + this.tableMetaStore = metaStore; + this.tables = newMixedTables(properties, metaStore); } - protected MixedTables newMixedTables(CatalogMeta catalogMeta) { - return new MixedTables(catalogMeta); + protected MixedTables newMixedTables( + Map catalogProperties, TableMetaStore metaStore) { + return new MixedTables(catalogProperties, metaStore); + } + + protected AmsClient getClient() { + if (client == null) { + throw new IllegalStateException("AMSClient is not initialized"); + } + return client; } @Override public List listDatabases() { try { - return client.getDatabases(this.catalogMeta.getCatalogName()); + return getClient().getDatabases(name()); } catch (TException e) { throw new IllegalStateException("failed load database", e); } @@ -108,7 +117,7 @@ public List listDatabases() { @Override public void createDatabase(String databaseName) { try { - client.createDatabase(this.catalogMeta.getCatalogName(), databaseName); + getClient().createDatabase(name(), databaseName); } catch (AlreadyExistsException e) { throw new org.apache.iceberg.exceptions.AlreadyExistsException( "Database already exists, %s", databaseName); @@ -120,7 +129,7 @@ public void createDatabase(String databaseName) { @Override public void dropDatabase(String databaseName) { try { - client.dropDatabase(this.catalogMeta.getCatalogName(), databaseName); + getClient().dropDatabase(name(), databaseName); } catch (NoSuchObjectException e0) { throw new NoSuchDatabaseException(e0, databaseName); } catch (TException e) { @@ -131,13 +140,8 @@ public void dropDatabase(String databaseName) { @Override public List listTables(String database) { try { - return client.listTables(this.catalogMeta.getCatalogName(), database).stream() - .map( - t -> - TableIdentifier.of( - this.catalogMeta.getCatalogName(), - database, - t.getTableIdentifier().getTableName())) + return getClient().listTables(name(), database).stream() + .map(t -> TableIdentifier.of(name(), database, t.getTableIdentifier().getTableName())) .collect(Collectors.toList()); } catch (TException e) { throw new IllegalStateException("failed load tables", e); @@ -175,7 +179,7 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) { protected void doDropTable(TableMeta meta, boolean purge) { try { - client.removeTable(meta.getTableIdentifier(), purge); + getClient().removeTable(meta.getTableIdentifier(), purge); } catch (TException e) { throw new IllegalStateException("error when delete table metadata from metastore"); } @@ -189,17 +193,6 @@ public TableBuilder newTableBuilder(TableIdentifier identifier, Schema schema) { return new ArcticTableBuilder(identifier, schema); } - @Override - public void refresh() { - try { - this.catalogMeta = client.getCatalog(catalogMeta.getCatalogName()); - tables.refreshCatalogMeta(catalogMeta); - } catch (TException e) { - throw new IllegalStateException( - String.format("failed load catalog %s.", catalogMeta.getCatalogName()), e); - } - } - @Override public TableBlockerManager getTableBlockerManager(TableIdentifier tableIdentifier) { validate(tableIdentifier); @@ -208,13 +201,13 @@ public TableBlockerManager getTableBlockerManager(TableIdentifier tableIdentifie @Override public Map properties() { - return catalogMeta.getCatalogProperties(); + return catalogProperties; } protected TableMeta getArcticTableMeta(TableIdentifier identifier) { TableMeta tableMeta; try { - tableMeta = client.getTable(CatalogUtil.amsTaleId(identifier)); + tableMeta = getClient().getTable(CatalogUtil.amsTaleId(identifier)); return tableMeta; } catch (NoSuchObjectException e) { throw new NoSuchTableException(e, "load table failed %s.", identifier); @@ -242,10 +235,10 @@ protected class ArcticTableBuilder implements TableBuilder { public ArcticTableBuilder(TableIdentifier identifier, Schema schema) { Preconditions.checkArgument( - identifier.getCatalog().equals(catalogMeta.getCatalogName()), + identifier.getCatalog().equals(name()), "Illegal table id:%s for catalog:%s", identifier.toString(), - catalogMeta.getCatalogName()); + name()); this.identifier = identifier; this.schema = schema; this.partitionSpec = PartitionSpec.unpartitioned(); @@ -358,7 +351,7 @@ protected void doCreateCheck() { protected void checkProperties() { Map mergedProperties = - CatalogUtil.mergeCatalogPropertiesToTable(properties, catalogMeta.getCatalogProperties()); + CatalogUtil.mergeCatalogPropertiesToTable(properties, catalogProperties); boolean enableStream = CompatiblePropertyUtil.propertyAsBoolean( mergedProperties, @@ -465,16 +458,12 @@ protected void fillTableProperties(TableMeta meta) { } protected String getDatabaseLocation() { - if (catalogMeta.getCatalogProperties() != null) { + if (catalogProperties != null) { String catalogWarehouse = - catalogMeta - .getCatalogProperties() - .getOrDefault(CatalogMetaProperties.KEY_WAREHOUSE, null); + catalogProperties.getOrDefault(CatalogMetaProperties.KEY_WAREHOUSE, null); if (catalogWarehouse == null) { catalogWarehouse = - catalogMeta - .getCatalogProperties() - .getOrDefault(CatalogMetaProperties.KEY_WAREHOUSE_DIR, null); + catalogProperties.getOrDefault(CatalogMetaProperties.KEY_WAREHOUSE_DIR, null); } if (catalogWarehouse == null) { throw new NullPointerException("Catalog warehouse is null."); diff --git a/core/src/main/java/com/netease/arctic/catalog/BasicIcebergCatalog.java b/core/src/main/java/com/netease/arctic/catalog/BasicIcebergCatalog.java deleted file mode 100644 index 5009cf21e0..0000000000 --- a/core/src/main/java/com/netease/arctic/catalog/BasicIcebergCatalog.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.catalog; - -import com.netease.arctic.AmsClient; -import com.netease.arctic.ams.api.CatalogMeta; -import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.table.TableBuilder; -import com.netease.arctic.table.TableIdentifier; -import com.netease.arctic.table.blocker.BasicTableBlockerManager; -import com.netease.arctic.table.blocker.TableBlockerManager; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.Catalog; -import org.apache.thrift.TException; - -import java.util.List; -import java.util.Map; - -/** A wrapper class around {@link Catalog} and implement {@link ArcticCatalog}. */ -public class BasicIcebergCatalog implements ArcticCatalog { - - private AmsClient client; - private IcebergCatalogWrapper catalogWrapper; - - @Override - public String name() { - return catalogWrapper.name(); - } - - @Override - public void initialize(AmsClient client, CatalogMeta meta, Map properties) { - this.client = client; - this.catalogWrapper = new IcebergCatalogWrapper(meta, properties); - } - - @Override - public List listDatabases() { - return catalogWrapper.listDatabases(); - } - - @Override - public void createDatabase(String databaseName) { - catalogWrapper.createDatabase(databaseName); - } - - @Override - public void dropDatabase(String databaseName) { - catalogWrapper.dropDatabase(databaseName); - } - - @Override - public List listTables(String database) { - return catalogWrapper.listTables(database); - } - - @Override - public ArcticTable loadTable(TableIdentifier tableIdentifier) { - return catalogWrapper.loadTable(tableIdentifier); - } - - @Override - public void renameTable(TableIdentifier from, String newTableName) { - catalogWrapper.renameTable(from, newTableName); - } - - @Override - public boolean dropTable(TableIdentifier tableIdentifier, boolean purge) { - return catalogWrapper.dropTable(tableIdentifier, purge); - } - - @Override - public TableBuilder newTableBuilder(TableIdentifier identifier, Schema schema) { - return catalogWrapper.newTableBuilder(identifier, schema); - } - - @Override - public void refresh() { - try { - catalogWrapper.refreshCatalogMeta(client.getCatalog(catalogWrapper.name())); - } catch (TException e) { - throw new IllegalStateException( - String.format("failed load catalog %s.", catalogWrapper.name()), e); - } - } - - @Override - public TableBlockerManager getTableBlockerManager(TableIdentifier tableIdentifier) { - return BasicTableBlockerManager.build(tableIdentifier, client); - } - - @Override - public Map properties() { - return catalogWrapper.properties(); - } -} diff --git a/core/src/main/java/com/netease/arctic/catalog/CatalogLoader.java b/core/src/main/java/com/netease/arctic/catalog/CatalogLoader.java index 51aea1e7d9..6914ab2e0b 100644 --- a/core/src/main/java/com/netease/arctic/catalog/CatalogLoader.java +++ b/core/src/main/java/com/netease/arctic/catalog/CatalogLoader.java @@ -33,13 +33,14 @@ import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.ams.api.client.AmsClientPools; import com.netease.arctic.ams.api.client.ArcticThriftUrl; +import com.netease.arctic.ams.api.properties.CatalogMetaProperties; import com.netease.arctic.mixed.BasicMixedIcebergCatalog; +import com.netease.arctic.table.TableMetaStore; import com.netease.arctic.utils.CatalogUtil; -import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.aws.glue.GlueCatalog; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.rest.RESTCatalog; import org.apache.thrift.TException; import java.util.List; @@ -51,14 +52,11 @@ public class CatalogLoader { public static final String AMS_CATALOG_IMPL = BasicArcticCatalog.class.getName(); - public static final String ICEBERG_CATALOG_IMPL = BasicIcebergCatalog.class.getName(); public static final String HIVE_CATALOG_IMPL = "com.netease.arctic.hive.catalog.ArcticHiveCatalog"; - public static final String GLUE_CATALOG_IMPL = "org.apache.iceberg.aws.glue.GlueCatalog"; + public static final String GLUE_CATALOG_IMPL = GlueCatalog.class.getName(); public static final String MIXED_ICEBERG_CATALOG_IMP = BasicMixedIcebergCatalog.class.getName(); - public static final String ICEBERG_REST_CATALOG = RESTCatalog.class.getName(); - /** * Entrypoint for loading Catalog. * @@ -86,95 +84,51 @@ public static ArcticCatalog load(String catalogUrl) { } /** - * Entrypoint for loading catalog + * get mixed-format catalog implement class name * - * @param client arctic metastore client - * @param catalogName arctic catalog name - * @param props client side catalog configs - * @return arctic catalog object + * @param metastoreType - metastore type + * @param catalogProperties - catalog properties + * @return class name for catalog */ - public static ArcticCatalog load( - AmsClient client, String catalogName, Map props) { - try { - CatalogMeta catalogMeta = client.getCatalog(catalogName); - String type = catalogMeta.getCatalogType(); - CatalogUtil.mergeCatalogProperties(catalogMeta, props); - String catalogImpl; - Set tableFormats = CatalogUtil.tableFormats(catalogMeta); - Preconditions.checkArgument( - tableFormats.size() == 1, "Catalog support only one table format now."); - TableFormat tableFormat = tableFormats.iterator().next(); - switch (type) { - case CATALOG_TYPE_HADOOP: - Preconditions.checkArgument( - TableFormat.ICEBERG == tableFormat || TableFormat.MIXED_ICEBERG == tableFormat, - "Hadoop catalog support iceberg/mixed-iceberg table only."); - if (TableFormat.ICEBERG == tableFormat) { - catalogImpl = ICEBERG_CATALOG_IMPL; - } else { - catalogImpl = MIXED_ICEBERG_CATALOG_IMP; - } - break; - case CATALOG_TYPE_HIVE: - if (TableFormat.ICEBERG == tableFormat) { - catalogImpl = ICEBERG_CATALOG_IMPL; - } else if (TableFormat.MIXED_HIVE == tableFormat) { - catalogImpl = HIVE_CATALOG_IMPL; - } else if (TableFormat.MIXED_ICEBERG == tableFormat) { - catalogImpl = MIXED_ICEBERG_CATALOG_IMP; - } else { - throw new IllegalArgumentException( - "Hive Catalog support iceberg/mixed-iceberg/mixed-hive table only"); - } - break; - case CATALOG_TYPE_AMS: - if (TableFormat.MIXED_ICEBERG == tableFormat) { - catalogImpl = AMS_CATALOG_IMPL; - } else if (TableFormat.ICEBERG == tableFormat) { - catalogMeta.putToCatalogProperties(CatalogProperties.WAREHOUSE_LOCATION, catalogName); - catalogMeta.putToCatalogProperties( - CatalogProperties.CATALOG_IMPL, ICEBERG_REST_CATALOG); - catalogImpl = ICEBERG_CATALOG_IMPL; - } else { - throw new IllegalArgumentException( - "Internal Catalog support iceberg or mixed-iceberg table only"); - } - - break; - case CATALOG_TYPE_GLUE: - if (TableFormat.ICEBERG == tableFormat) { - catalogImpl = GLUE_CATALOG_IMPL; - } else if (TableFormat.MIXED_ICEBERG == tableFormat) { - catalogImpl = MIXED_ICEBERG_CATALOG_IMP; - } else { - throw new IllegalArgumentException( - "Glue Catalog support iceberg/mixed-iceberg table only"); - } - break; - case CATALOG_TYPE_CUSTOM: - Preconditions.checkArgument( - TableFormat.ICEBERG == tableFormat || TableFormat.MIXED_ICEBERG == tableFormat, - "Custom catalog support iceberg/mixed-iceberg table only."); - Preconditions.checkArgument( - catalogMeta.getCatalogProperties().containsKey(CatalogProperties.CATALOG_IMPL), - "Custom catalog properties must contains " + CatalogProperties.CATALOG_IMPL); - if (TableFormat.ICEBERG == tableFormat) { - catalogImpl = ICEBERG_CATALOG_IMPL; - } else { - catalogImpl = MIXED_ICEBERG_CATALOG_IMP; - } - break; - default: - throw new IllegalStateException("unsupported catalog type:" + type); - } - ArcticCatalog catalog = buildCatalog(catalogImpl); - catalog.initialize(client, catalogMeta, props); - return catalog; - } catch (NoSuchObjectException e1) { - throw new IllegalArgumentException("catalog not found, please check catalog name", e1); - } catch (Exception e) { - throw new IllegalStateException("failed when load catalog " + catalogName, e); + private static String catalogImpl(String metastoreType, Map catalogProperties) { + Set tableFormats = CatalogUtil.tableFormats(metastoreType, catalogProperties); + Preconditions.checkArgument( + tableFormats.size() == 1, "Catalog support only one table format now."); + TableFormat tableFormat = tableFormats.iterator().next(); + Preconditions.checkArgument( + TableFormat.MIXED_HIVE == tableFormat || TableFormat.MIXED_ICEBERG == tableFormat, + "MixedCatalogLoader only support mixed-format, format: %s", + tableFormat.name()); + + String catalogImpl; + switch (metastoreType) { + case CATALOG_TYPE_HADOOP: + case CATALOG_TYPE_GLUE: + case CATALOG_TYPE_CUSTOM: + Preconditions.checkArgument( + TableFormat.MIXED_ICEBERG == tableFormat, + "%s catalog support mixed-iceberg table only.", + metastoreType); + catalogImpl = MIXED_ICEBERG_CATALOG_IMP; + break; + case CATALOG_TYPE_HIVE: + if (TableFormat.MIXED_HIVE == tableFormat) { + catalogImpl = HIVE_CATALOG_IMPL; + } else { + catalogImpl = MIXED_ICEBERG_CATALOG_IMP; + } + break; + case CATALOG_TYPE_AMS: + if (TableFormat.MIXED_ICEBERG == tableFormat) { + catalogImpl = AMS_CATALOG_IMPL; + } else { + throw new IllegalArgumentException("Internal Catalog mixed-iceberg table only"); + } + break; + default: + throw new IllegalStateException("unsupported metastore type:" + metastoreType); } + return catalogImpl; } /** @@ -211,10 +165,54 @@ public static List catalogs(String metastoreUrl) { } } + /** + * Entrypoint for loading catalog + * + * @param metaStoreUrl arctic metastore url + * @param catalogName arctic catalog name + * @param properties client side catalog configs + * @return arctic catalog object + */ private static ArcticCatalog loadCatalog( String metaStoreUrl, String catalogName, Map properties) { AmsClient client = new PooledAmsClient(metaStoreUrl); - return load(client, catalogName, properties); + try { + CatalogMeta catalogMeta = client.getCatalog(catalogName); + String type = catalogMeta.getCatalogType(); + catalogMeta.putToCatalogProperties(CatalogMetaProperties.AMS_URI, metaStoreUrl); + CatalogUtil.mergeCatalogProperties(catalogMeta, properties); + return createCatalog( + catalogName, + type, + catalogMeta.getCatalogProperties(), + CatalogUtil.buildMetaStore(catalogMeta)); + } catch (NoSuchObjectException e1) { + throw new IllegalArgumentException("catalog not found, please check catalog name", e1); + } catch (Exception e) { + throw new IllegalStateException("failed when load catalog " + catalogName, e); + } + } + + /** + * build and initialize a mixed-format catalog. + * + * @param catalogName catalog-name + * @param metastoreType metastore type + * @param properties catalog properties + * @param metaStore auth context + * @return initialized catalog. + */ + public static ArcticCatalog createCatalog( + String catalogName, + String metastoreType, + Map properties, + TableMetaStore metaStore) { + properties = + CatalogUtil.withIcebergCatalogInitializeProperties(catalogName, metastoreType, properties); + String catalogImpl = catalogImpl(metastoreType, properties); + ArcticCatalog catalog = buildCatalog(catalogImpl); + catalog.initialize(catalogName, properties, metaStore); + return catalog; } private static ArcticCatalog buildCatalog(String impl) { diff --git a/core/src/main/java/com/netease/arctic/catalog/IcebergCatalogWrapper.java b/core/src/main/java/com/netease/arctic/catalog/IcebergCatalogWrapper.java deleted file mode 100644 index 138f4557c4..0000000000 --- a/core/src/main/java/com/netease/arctic/catalog/IcebergCatalogWrapper.java +++ /dev/null @@ -1,338 +0,0 @@ -/* - * 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.catalog; - -import com.netease.arctic.AmsClient; -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.io.ArcticFileIO; -import com.netease.arctic.io.ArcticFileIOAdapter; -import com.netease.arctic.io.ArcticFileIOs; -import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.table.BasicTableBuilder; -import com.netease.arctic.table.BasicUnkeyedTable; -import com.netease.arctic.table.PrimaryKeySpec; -import com.netease.arctic.table.TableBuilder; -import com.netease.arctic.table.TableIdentifier; -import com.netease.arctic.table.TableMetaStore; -import com.netease.arctic.table.blocker.TableBlockerManager; -import com.netease.arctic.utils.CatalogUtil; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.Transaction; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.SupportsNamespaces; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -/** A wrapper class around {@link Catalog} and implement {@link ArcticCatalog}. */ -public class IcebergCatalogWrapper implements ArcticCatalog { - - private CatalogMeta meta; - private Map customProperties; - private Pattern databaseFilterPattern; - private Pattern tableFilterPattern; - private transient TableMetaStore tableMetaStore; - private transient Catalog icebergCatalog; - - public IcebergCatalogWrapper() {} - - public IcebergCatalogWrapper(CatalogMeta meta) { - initialize(meta, Maps.newHashMap()); - } - - public IcebergCatalogWrapper(CatalogMeta meta, Map properties) { - initialize(meta, properties); - } - - @Override - public String name() { - return meta.getCatalogName(); - } - - @Override - public void initialize(AmsClient client, CatalogMeta meta, Map properties) {} - - private void initialize(CatalogMeta meta, Map properties) { - this.meta = meta; - this.customProperties = properties; - CatalogUtil.mergeCatalogProperties(meta, properties); - meta.putToCatalogProperties( - org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE, meta.getCatalogType()); - this.tableMetaStore = CatalogUtil.buildMetaStore(meta); - if (CatalogMetaProperties.CATALOG_TYPE_GLUE.equals(meta.getCatalogType())) { - meta.getCatalogProperties() - .put(CatalogProperties.CATALOG_IMPL, CatalogLoader.GLUE_CATALOG_IMPL); - } - if (meta.getCatalogProperties().containsKey(CatalogProperties.CATALOG_IMPL)) { - meta.getCatalogProperties().remove(org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE); - } - - icebergCatalog = - tableMetaStore.doAs( - () -> - org.apache.iceberg.CatalogUtil.buildIcebergCatalog( - name(), meta.getCatalogProperties(), tableMetaStore.getConfiguration())); - if (meta.getCatalogProperties() - .containsKey(CatalogMetaProperties.KEY_DATABASE_FILTER_REGULAR_EXPRESSION)) { - String databaseFilter = - meta.getCatalogProperties() - .get(CatalogMetaProperties.KEY_DATABASE_FILTER_REGULAR_EXPRESSION); - databaseFilterPattern = Pattern.compile(databaseFilter); - } else { - databaseFilterPattern = null; - } - - if (meta.getCatalogProperties().containsKey(CatalogMetaProperties.KEY_TABLE_FILTER)) { - String tableFilter = meta.getCatalogProperties().get(CatalogMetaProperties.KEY_TABLE_FILTER); - tableFilterPattern = Pattern.compile(tableFilter); - } else { - tableFilterPattern = null; - } - } - - @Override - public List listDatabases() { - if (!(icebergCatalog instanceof SupportsNamespaces)) { - throw new UnsupportedOperationException( - String.format( - "Iceberg catalog: %s doesn't implement SupportsNamespaces", - icebergCatalog.getClass().getName())); - } - - List databases = - tableMetaStore.doAs( - () -> - ((SupportsNamespaces) icebergCatalog) - .listNamespaces(Namespace.empty()).stream() - .map(namespace -> namespace.level(0)) - .distinct() - .collect(Collectors.toList())); - return databases.stream() - .filter( - database -> - databaseFilterPattern == null || databaseFilterPattern.matcher(database).matches()) - .collect(Collectors.toList()); - } - - @Override - public void createDatabase(String databaseName) { - if (icebergCatalog instanceof SupportsNamespaces) { - tableMetaStore.doAs( - () -> { - ((SupportsNamespaces) icebergCatalog).createNamespace(Namespace.of(databaseName)); - return null; - }); - } else { - throw new UnsupportedOperationException( - String.format( - "Iceberg catalog: %s doesn't implement SupportsNamespaces", - icebergCatalog.getClass().getName())); - } - } - - @Override - public void dropDatabase(String databaseName) { - if (icebergCatalog instanceof SupportsNamespaces) { - tableMetaStore.doAs( - () -> { - ((SupportsNamespaces) icebergCatalog).dropNamespace(Namespace.of(databaseName)); - return null; - }); - } else { - throw new UnsupportedOperationException( - String.format( - "Iceberg catalog: %s doesn't implement SupportsNamespaces", - icebergCatalog.getClass().getName())); - } - } - - @Override - public List listTables(String database) { - return tableMetaStore.doAs( - () -> - icebergCatalog.listTables(Namespace.of(database)).stream() - .filter( - tableIdentifier -> - tableIdentifier.namespace().levels().length == 1 - && (tableFilterPattern == null - || tableFilterPattern - .matcher((database + "." + tableIdentifier.name())) - .matches())) - .map( - tableIdentifier -> TableIdentifier.of(name(), database, tableIdentifier.name())) - .collect(Collectors.toList())); - } - - public List listTables() { - List tables = new ArrayList<>(); - List dbs = listDatabases(); - for (String db : dbs) { - try { - tables.addAll(listTables(db)); - } catch (Exception ignored) { - continue; - } - } - return tables; - } - - @Override - public ArcticTable loadTable(TableIdentifier tableIdentifier) { - Table icebergTable = - tableMetaStore.doAs( - () -> icebergCatalog.loadTable(toIcebergTableIdentifier(tableIdentifier))); - FileIO io = icebergTable.io(); - ArcticFileIO arcticFileIO = createArcticFileIO(io); - return new BasicIcebergTable( - tableIdentifier, - CatalogUtil.useArcticTableOperations( - icebergTable, icebergTable.location(), arcticFileIO, tableMetaStore.getConfiguration()), - arcticFileIO, - meta.getCatalogProperties()); - } - - @Override - public boolean tableExists(TableIdentifier tableIdentifier) { - return ArcticCatalog.super.tableExists(tableIdentifier); - } - - @Override - public void renameTable(TableIdentifier from, String newTableName) { - tableMetaStore.doAs( - () -> { - icebergCatalog.renameTable( - toIcebergTableIdentifier(from), - org.apache.iceberg.catalog.TableIdentifier.of( - Namespace.of(from.getDatabase()), newTableName)); - return null; - }); - } - - @Override - public boolean dropTable(TableIdentifier tableIdentifier, boolean purge) { - return tableMetaStore.doAs( - () -> icebergCatalog.dropTable(toIcebergTableIdentifier(tableIdentifier), purge)); - } - - @Override - public TableBuilder newTableBuilder(TableIdentifier identifier, Schema schema) { - return new IcebergTableBuilder(schema, identifier); - } - - @Override - public void refresh() {} - - @Override - public TableBlockerManager getTableBlockerManager(TableIdentifier tableIdentifier) { - return null; - } - - @Override - public Map properties() { - return meta.getCatalogProperties(); - } - - public void refreshCatalogMeta(CatalogMeta meta) { - initialize(meta, customProperties); - } - - private org.apache.iceberg.catalog.TableIdentifier toIcebergTableIdentifier( - TableIdentifier tableIdentifier) { - return org.apache.iceberg.catalog.TableIdentifier.of( - Namespace.of(tableIdentifier.getDatabase()), tableIdentifier.getTableName()); - } - - private ArcticFileIO createArcticFileIO(FileIO io) { - if (io instanceof HadoopFileIO) { - return ArcticFileIOs.buildHadoopFileIO(tableMetaStore); - } else { - return new ArcticFileIOAdapter(io); - } - } - - protected class IcebergTableBuilder extends BasicTableBuilder { - - public IcebergTableBuilder(Schema schema, TableIdentifier identifier) { - super(schema, TableFormat.ICEBERG, identifier); - } - - @Override - public ArcticTable create() { - - Table table = - icebergCatalog - .buildTable(toIcebergTableIdentifier(identifier), schema) - .withPartitionSpec(spec) - .withProperties(properties) - .withSortOrder(sortOrder) - .create(); - - FileIO io = table.io(); - ArcticFileIO arcticFileIO = createArcticFileIO(io); - return new BasicIcebergTable(identifier, table, arcticFileIO, meta.getCatalogProperties()); - } - - @Override - public Transaction createTransaction() { - return icebergCatalog.newCreateTableTransaction( - toIcebergTableIdentifier(identifier), schema, spec, properties); - } - - @Override - public TableBuilder withPrimaryKeySpec(PrimaryKeySpec primaryKeySpec) { - Preconditions.checkArgument( - primaryKeySpec == null || !primaryKeySpec.primaryKeyExisted(), - "can't create an iceberg table with primary key"); - return this; - } - - @Override - protected IcebergTableBuilder self() { - return this; - } - } - - public static class BasicIcebergTable extends BasicUnkeyedTable { - - public BasicIcebergTable( - TableIdentifier tableIdentifier, - Table icebergTable, - ArcticFileIO arcticFileIO, - Map catalogProperties) { - super(tableIdentifier, icebergTable, arcticFileIO, catalogProperties); - } - - @Override - public TableFormat format() { - return TableFormat.ICEBERG; - } - } -} diff --git a/core/src/main/java/com/netease/arctic/catalog/MixedTables.java b/core/src/main/java/com/netease/arctic/catalog/MixedTables.java index 843b55c3da..29a7543988 100644 --- a/core/src/main/java/com/netease/arctic/catalog/MixedTables.java +++ b/core/src/main/java/com/netease/arctic/catalog/MixedTables.java @@ -1,8 +1,6 @@ package com.netease.arctic.catalog; -import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableMeta; -import com.netease.arctic.ams.api.properties.CatalogMetaProperties; import com.netease.arctic.ams.api.properties.MetaTableProperties; import com.netease.arctic.io.ArcticFileIO; import com.netease.arctic.io.ArcticFileIOs; @@ -40,29 +38,13 @@ public class MixedTables { private static final Logger LOG = LoggerFactory.getLogger(MixedTables.class); - protected CatalogMeta catalogMeta; protected Tables tables; protected TableMetaStore tableMetaStore; + protected Map catalogProperties; - public MixedTables(CatalogMeta catalogMeta) { - initialize(catalogMeta); - } - - private void initialize(CatalogMeta meta) { - this.catalogMeta = meta; - if (meta.getStorageConfigs() != null - && CatalogMetaProperties.STORAGE_CONFIGS_VALUE_TYPE_HADOOP.equalsIgnoreCase( - CatalogUtil.getCompatibleStorageType(meta.getStorageConfigs()))) { - if (!meta.getStorageConfigs() - .containsKey(CatalogMetaProperties.STORAGE_CONFIGS_KEY_HDFS_SITE)) { - throw new IllegalStateException("lack hdfs.site config"); - } - if (!meta.getStorageConfigs() - .containsKey(CatalogMetaProperties.STORAGE_CONFIGS_KEY_CORE_SITE)) { - throw new IllegalStateException("lack core.site config"); - } - } - this.tableMetaStore = CatalogUtil.buildMetaStore(meta); + public MixedTables(Map catalogProperties, TableMetaStore metaStore) { + this.tableMetaStore = metaStore; + this.catalogProperties = catalogProperties; this.tables = new HadoopTables(tableMetaStore.getConfiguration()); } @@ -88,7 +70,7 @@ protected KeyedTable loadKeyedTable(TableMeta tableMeta) { tableLocation, tableMeta.getProperties(), tableMetaStore, - catalogMeta.getCatalogProperties()); + catalogProperties); Table baseIcebergTable = tableMetaStore.doAs(() -> tables.load(baseLocation)); BaseTable baseTable = new BasicKeyedTable.BaseInternalTable( @@ -96,7 +78,7 @@ protected KeyedTable loadKeyedTable(TableMeta tableMeta) { CatalogUtil.useArcticTableOperations( baseIcebergTable, baseLocation, fileIO, tableMetaStore.getConfiguration()), fileIO, - catalogMeta.getCatalogProperties()); + catalogProperties); Table changeIcebergTable = tableMetaStore.doAs(() -> tables.load(changeLocation)); ChangeTable changeTable = @@ -105,7 +87,7 @@ protected KeyedTable loadKeyedTable(TableMeta tableMeta) { CatalogUtil.useArcticTableOperations( changeIcebergTable, changeLocation, fileIO, tableMetaStore.getConfiguration()), fileIO, - catalogMeta.getCatalogProperties()); + catalogProperties); PrimaryKeySpec keySpec = buildPrimaryKeySpec(baseTable.schema(), tableMeta); return new BasicKeyedTable(tableLocation, keySpec, baseTable, changeTable); } @@ -120,7 +102,7 @@ protected PrimaryKeySpec buildPrimaryKeySpec(Schema schema, TableMeta tableMeta) PrimaryKeySpec.Builder builder = PrimaryKeySpec.builderFor(schema); if (tableMeta.getKeySpec() != null && tableMeta.getKeySpec().getFields() != null - && tableMeta.getKeySpec().getFields().size() > 0) { + && !tableMeta.getKeySpec().getFields().isEmpty()) { for (String field : tableMeta.getKeySpec().getFields()) { builder.addColumn(field); } @@ -140,13 +122,13 @@ protected UnkeyedTable loadUnKeyedTable(TableMeta tableMeta) { tableLocation, tableMeta.getProperties(), tableMetaStore, - catalogMeta.getCatalogProperties()); + catalogProperties); return new BasicUnkeyedTable( tableIdentifier, CatalogUtil.useArcticTableOperations( table, baseLocation, fileIO, tableMetaStore.getConfiguration()), fileIO, - catalogMeta.getCatalogProperties()); + catalogProperties); } public ArcticTable createTableByMeta( @@ -178,7 +160,7 @@ protected KeyedTable createKeyedTable( tableLocation, tableMeta.getProperties(), tableMetaStore, - catalogMeta.getCatalogProperties()); + catalogProperties); Table baseIcebergTable = tableMetaStore.doAs( () -> { @@ -195,7 +177,7 @@ protected KeyedTable createKeyedTable( CatalogUtil.useArcticTableOperations( baseIcebergTable, baseLocation, fileIO, tableMetaStore.getConfiguration()), fileIO, - catalogMeta.getCatalogProperties()); + catalogProperties); Table changeIcebergTable = tableMetaStore.doAs( @@ -213,7 +195,7 @@ protected KeyedTable createKeyedTable( CatalogUtil.useArcticTableOperations( changeIcebergTable, changeLocation, fileIO, tableMetaStore.getConfiguration()), fileIO, - catalogMeta.getCatalogProperties()); + catalogProperties); return new BasicKeyedTable(tableLocation, primaryKeySpec, baseTable, changeTable); } @@ -253,13 +235,13 @@ protected UnkeyedTable createUnKeyedTable( tableLocation, tableMeta.getProperties(), tableMetaStore, - catalogMeta.getCatalogProperties()); + catalogProperties); return new BasicUnkeyedTable( tableIdentifier, CatalogUtil.useArcticTableOperations( table, baseLocation, fileIO, tableMetaStore.getConfiguration()), fileIO, - catalogMeta.getCatalogProperties()); + catalogProperties); } public void dropTableByMeta(TableMeta tableMeta, boolean purge) { @@ -322,10 +304,6 @@ private void dropInternalTable( }); } - public void refreshCatalogMeta(CatalogMeta meta) { - initialize(meta); - } - protected TableMetaStore getTableMetaStore() { return tableMetaStore; } diff --git a/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergCatalog.java b/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergCatalog.java index 19158ac557..0b61a4fe11 100644 --- a/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergCatalog.java +++ b/core/src/main/java/com/netease/arctic/formats/iceberg/IcebergCatalog.java @@ -20,22 +20,30 @@ import com.netease.arctic.AmoroTable; import com.netease.arctic.FormatCatalog; +import com.netease.arctic.table.TableMetaStore; import org.apache.iceberg.Table; 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.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; public class IcebergCatalog implements FormatCatalog { private final Catalog icebergCatalog; + private final TableMetaStore metaStore; + private final Map properties; - public IcebergCatalog(Catalog icebergCatalog) { + public IcebergCatalog( + Catalog icebergCatalog, Map properties, TableMetaStore metaStore) { this.icebergCatalog = icebergCatalog; + this.metaStore = metaStore; + this.properties = properties; } @Override @@ -83,9 +91,20 @@ public List listTables(String database) { @Override public AmoroTable loadTable(String database, String table) { - Table icebergTable = icebergCatalog.loadTable(TableIdentifier.of(database, table)); - return new IcebergTable( - com.netease.arctic.table.TableIdentifier.of(icebergCatalog.name(), database, table), - icebergTable); + try { + Table icebergTable = icebergCatalog.loadTable(TableIdentifier.of(database, table)); + return IcebergTable.newIcebergTable( + com.netease.arctic.table.TableIdentifier.of(icebergCatalog.name(), database, table), + icebergTable, + metaStore, + properties); + } catch (NoSuchTableException e) { + throw new com.netease.arctic.NoSuchTableException(e); + } + } + + @Override + public boolean dropTable(String database, String table, boolean purge) { + return icebergCatalog.dropTable(TableIdentifier.of(database, table), purge); } } 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 faad07bd9c..58636a5a1c 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 @@ -21,15 +21,11 @@ import com.netease.arctic.FormatCatalog; import com.netease.arctic.FormatCatalogFactory; import com.netease.arctic.ams.api.TableFormat; -import com.netease.arctic.ams.api.properties.CatalogMetaProperties; -import com.netease.arctic.catalog.CatalogLoader; +import com.netease.arctic.table.TableMetaStore; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import java.util.Map; @@ -37,40 +33,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) { + String name, String metastoreType, Map properties, TableMetaStore metaStore) { Preconditions.checkArgument(StringUtils.isNotBlank(metastoreType), "metastore type is blank"); - Map icebergProperties = Maps.newHashMap(properties); - if (CatalogMetaProperties.CATALOG_TYPE_HADOOP.equalsIgnoreCase(metastoreType) - || CatalogMetaProperties.CATALOG_TYPE_HIVE.equalsIgnoreCase(metastoreType)) { - icebergProperties.put(CatalogUtil.ICEBERG_CATALOG_TYPE, metastoreType); - icebergProperties.remove(CatalogProperties.CATALOG_IMPL); - } else if (CatalogMetaProperties.CATALOG_TYPE_AMS.equalsIgnoreCase(metastoreType)) { - icebergProperties.remove(CatalogUtil.ICEBERG_CATALOG_TYPE); - icebergProperties.put(CatalogProperties.CATALOG_IMPL, CatalogLoader.ICEBERG_REST_CATALOG); - } else if (CatalogMetaProperties.CATALOG_TYPE_GLUE.equals(metastoreType)) { - icebergProperties.remove(CatalogUtil.ICEBERG_CATALOG_TYPE); - icebergProperties.put(CatalogProperties.CATALOG_IMPL, CatalogLoader.GLUE_CATALOG_IMPL); - } else { - String icebergCatalogImpl = icebergProperties.get(CatalogProperties.CATALOG_IMPL); - Preconditions.checkArgument( - StringUtils.isNotBlank(icebergCatalogImpl), "iceberg catalog impl is blank"); - icebergProperties.remove(CatalogUtil.ICEBERG_CATALOG_TYPE); - icebergProperties.put(CatalogProperties.CATALOG_IMPL, icebergCatalogImpl); - } + properties = + com.netease.arctic.utils.CatalogUtil.withIcebergCatalogInitializeProperties( + name, metastoreType, properties); - return CatalogUtil.buildIcebergCatalog(name, icebergProperties, configuration); + Catalog icebergCatalog = + CatalogUtil.buildIcebergCatalog(name, properties, metaStore.getConfiguration()); + return new IcebergCatalog(icebergCatalog, properties, metaStore); } @Override 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 c6b792fc41..55fb6ea9e7 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 @@ -21,18 +21,55 @@ import com.netease.arctic.AmoroTable; import com.netease.arctic.TableSnapshot; import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.io.ArcticFileIO; +import com.netease.arctic.io.ArcticFileIOs; +import com.netease.arctic.table.BasicUnkeyedTable; import com.netease.arctic.table.TableIdentifier; +import com.netease.arctic.table.TableMetaStore; +import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.CatalogUtil; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import java.util.Map; -public class IcebergTable implements AmoroTable
{ +public class IcebergTable implements AmoroTable { + + /** + * build an amoro table implement which table format is iceberg. + * + * @param identifier table identifier. + * @param icebergTable iceberg table + * @param metaStore auth context + * @param catalogProperties catalog properties. + * @return amoro table. + */ + public static IcebergTable newIcebergTable( + TableIdentifier identifier, + Table icebergTable, + TableMetaStore metaStore, + Map catalogProperties) { + ArcticFileIO io = ArcticFileIOs.buildAdaptIcebergFileIO(metaStore, icebergTable.io()); + + UnkeyedTable wrapped = + new BasicUnkeyedTable( + identifier, + CatalogUtil.useArcticTableOperations( + icebergTable, icebergTable.location(), io, metaStore.getConfiguration()), + io, + catalogProperties) { + @Override + public TableFormat format() { + return TableFormat.ICEBERG; + } + }; + return new IcebergTable(identifier, wrapped); + } private final TableIdentifier identifier; - private final Table table; + private final UnkeyedTable table; - public IcebergTable(TableIdentifier identifier, Table table) { + protected IcebergTable(TableIdentifier identifier, UnkeyedTable table) { this.table = table; this.identifier = identifier; } @@ -53,7 +90,7 @@ public Map properties() { } @Override - public Table originalTable() { + public UnkeyedTable originalTable() { return table; } diff --git a/core/src/main/java/com/netease/arctic/formats/mixed/MixedCatalog.java b/core/src/main/java/com/netease/arctic/formats/mixed/MixedCatalog.java new file mode 100644 index 0000000000..3592504850 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/formats/mixed/MixedCatalog.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.formats.mixed; + +import com.netease.arctic.AmoroTable; +import com.netease.arctic.FormatCatalog; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.catalog.ArcticCatalog; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; + +import java.util.List; +import java.util.stream.Collectors; + +public class MixedCatalog implements FormatCatalog { + final ArcticCatalog catalog; + final TableFormat format; + + public MixedCatalog(ArcticCatalog catalog, TableFormat format) { + this.catalog = catalog; + this.format = format; + } + + @Override + public List listDatabases() { + return catalog.listDatabases(); + } + + @Override + public boolean exist(String database) { + return catalog.listDatabases().contains(database); + } + + @Override + public boolean exist(String database, String table) { + return catalog.tableExists(TableIdentifier.of(catalog.name(), database, table)); + } + + @Override + public void createDatabase(String database) { + catalog.createDatabase(database); + } + + @Override + public void dropDatabase(String database) { + catalog.dropDatabase(database); + } + + @Override + public AmoroTable loadTable(String database, String table) { + try { + ArcticTable mixedTable = + catalog.loadTable(TableIdentifier.of(catalog.name(), database, table)); + return new MixedTable(mixedTable, format); + } catch (NoSuchTableException e) { + throw new com.netease.arctic.NoSuchTableException(e); + } + } + + @Override + public List listTables(String database) { + return catalog.listTables(database).stream() + .map(TableIdentifier::getTableName) + .collect(Collectors.toList()); + } + + @Override + public boolean dropTable(String database, String table, boolean purge) { + return catalog.dropTable(TableIdentifier.of(catalog.name(), database, table), purge); + } +} diff --git a/core/src/main/java/com/netease/arctic/formats/mixed/MixedHiveTable.java b/core/src/main/java/com/netease/arctic/formats/mixed/MixedHiveCatalogFactory.java similarity index 84% rename from core/src/main/java/com/netease/arctic/formats/mixed/MixedHiveTable.java rename to core/src/main/java/com/netease/arctic/formats/mixed/MixedHiveCatalogFactory.java index 2bd0da1ef3..4bfcbd5717 100644 --- a/core/src/main/java/com/netease/arctic/formats/mixed/MixedHiveTable.java +++ b/core/src/main/java/com/netease/arctic/formats/mixed/MixedHiveCatalogFactory.java @@ -19,14 +19,8 @@ package com.netease.arctic.formats.mixed; import com.netease.arctic.ams.api.TableFormat; -import com.netease.arctic.table.ArcticTable; - -public class MixedHiveTable extends MixedIcebergTable { - - public MixedHiveTable(ArcticTable arcticTable) { - super(arcticTable); - } +public class MixedHiveCatalogFactory extends MixedIcebergCatalogFactory { @Override public TableFormat format() { return TableFormat.MIXED_HIVE; diff --git a/core/src/main/java/com/netease/arctic/formats/mixed/MixedIcebergCatalogFactory.java b/core/src/main/java/com/netease/arctic/formats/mixed/MixedIcebergCatalogFactory.java new file mode 100644 index 0000000000..6d5ad0a3b8 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/formats/mixed/MixedIcebergCatalogFactory.java @@ -0,0 +1,46 @@ +/* + * 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.FormatCatalog; +import com.netease.arctic.FormatCatalogFactory; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.catalog.ArcticCatalog; +import com.netease.arctic.catalog.CatalogLoader; +import com.netease.arctic.table.TableMetaStore; + +import java.util.Map; + +public class MixedIcebergCatalogFactory implements FormatCatalogFactory { + @Override + public FormatCatalog create( + String catalogName, + String metastoreType, + Map properties, + TableMetaStore metaStore) { + ArcticCatalog catalog = + CatalogLoader.createCatalog(catalogName, metastoreType, properties, metaStore); + return new MixedCatalog(catalog, format()); + } + + @Override + public TableFormat format() { + return TableFormat.MIXED_ICEBERG; + } +} diff --git a/core/src/main/java/com/netease/arctic/formats/mixed/MixedIcebergTable.java b/core/src/main/java/com/netease/arctic/formats/mixed/MixedTable.java similarity index 90% rename from core/src/main/java/com/netease/arctic/formats/mixed/MixedIcebergTable.java rename to core/src/main/java/com/netease/arctic/formats/mixed/MixedTable.java index fbb52c5cb3..7a35e5f4d0 100644 --- a/core/src/main/java/com/netease/arctic/formats/mixed/MixedIcebergTable.java +++ b/core/src/main/java/com/netease/arctic/formats/mixed/MixedTable.java @@ -27,12 +27,14 @@ import java.util.Map; -public class MixedIcebergTable implements AmoroTable { +public class MixedTable implements AmoroTable { private final ArcticTable arcticTable; + private final TableFormat format; - public MixedIcebergTable(ArcticTable arcticTable) { + public MixedTable(ArcticTable arcticTable, TableFormat format) { this.arcticTable = arcticTable; + this.format = format; } @Override @@ -42,7 +44,7 @@ public TableIdentifier id() { @Override public TableFormat format() { - return TableFormat.MIXED_ICEBERG; + return format; } @Override 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 index 3d54f5cd86..da93d3c438 100644 --- a/core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalog.java +++ b/core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalog.java @@ -95,4 +95,14 @@ public List listTables(String database) { throw new NoSuchDatabaseException(e); } } + + @Override + public boolean dropTable(String database, String table, boolean purge) { + try { + catalog.dropTable(Identifier.create(database, table), purge); + return true; + } catch (Catalog.TableNotExistException e) { + return false; + } + } } 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 index 347470fd2c..e54c33e30c 100644 --- a/core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalogFactory.java +++ b/core/src/main/java/com/netease/arctic/formats/paimon/PaimonCatalogFactory.java @@ -21,24 +21,36 @@ import com.netease.arctic.FormatCatalogFactory; import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.ams.api.properties.CatalogMetaProperties; +import com.netease.arctic.table.TableMetaStore; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; 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.hive.HiveCatalogOptions; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; +import java.io.File; +import java.net.URL; import java.util.Map; +import java.util.Optional; 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); + String name, String metastoreType, Map properties, TableMetaStore metaStore) { + Optional hiveSiteLocation = metaStore.getHiveSiteLocation(); + Map catalogProperties = Maps.newHashMap(); + catalogProperties.putAll(properties); + + hiveSiteLocation.ifPresent( + url -> + catalogProperties.put( + HiveCatalogOptions.HIVE_CONF_DIR.key(), new File(url.getPath()).getParent())); + return new PaimonCatalog( + paimonCatalog(metastoreType, catalogProperties, metaStore.getConfiguration()), name); } public static Catalog paimonCatalog( diff --git a/core/src/main/java/com/netease/arctic/mixed/BasicMixedIcebergCatalog.java b/core/src/main/java/com/netease/arctic/mixed/BasicMixedIcebergCatalog.java index 957cd6a57b..57190441ef 100644 --- a/core/src/main/java/com/netease/arctic/mixed/BasicMixedIcebergCatalog.java +++ b/core/src/main/java/com/netease/arctic/mixed/BasicMixedIcebergCatalog.java @@ -19,10 +19,9 @@ package com.netease.arctic.mixed; import com.netease.arctic.AmsClient; -import com.netease.arctic.ams.api.CatalogMeta; +import com.netease.arctic.PooledAmsClient; import com.netease.arctic.ams.api.properties.CatalogMetaProperties; import com.netease.arctic.catalog.ArcticCatalog; -import com.netease.arctic.catalog.CatalogLoader; import com.netease.arctic.io.ArcticFileIO; import com.netease.arctic.io.TableTrashManagers; import com.netease.arctic.op.CreateTableTransaction; @@ -34,8 +33,6 @@ import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.blocker.BasicTableBlockerManager; import com.netease.arctic.table.blocker.TableBlockerManager; -import com.netease.arctic.utils.CatalogUtil; -import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; @@ -48,7 +45,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.thrift.TException; import java.util.List; import java.util.Map; @@ -58,73 +54,43 @@ public class BasicMixedIcebergCatalog implements ArcticCatalog { - private Map clientSideProperties; private Catalog icebergCatalog; private TableMetaStore tableMetaStore; - private CatalogMeta meta; + private Map catalogProperties; + private String name; private Pattern databaseFilterPattern; private AmsClient client; private MixedTables tables; - public BasicMixedIcebergCatalog() {} - - public BasicMixedIcebergCatalog(CatalogMeta meta) { - this.initialize(meta); - } - @Override public String name() { - return meta.getCatalogName(); + return this.name; } @Override - public void initialize(AmsClient client, CatalogMeta meta, Map properties) { - this.client = client; - this.clientSideProperties = properties == null ? Maps.newHashMap() : properties; - this.initialize(meta); - } - - private void initialize(CatalogMeta catalogMeta) { - CatalogUtil.mergeCatalogProperties(catalogMeta, clientSideProperties); - - catalogMeta.putToCatalogProperties( - org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE, catalogMeta.getCatalogType()); - if (CatalogMetaProperties.CATALOG_TYPE_GLUE.equals(catalogMeta.getCatalogType())) { - catalogMeta - .getCatalogProperties() - .put(CatalogProperties.CATALOG_IMPL, CatalogLoader.GLUE_CATALOG_IMPL); - } - if (catalogMeta.getCatalogProperties().containsKey(CatalogProperties.CATALOG_IMPL)) { - catalogMeta - .getCatalogProperties() - .remove(org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE); - } - - TableMetaStore metaStore = CatalogUtil.buildMetaStore(catalogMeta); + public void initialize(String name, Map properties, TableMetaStore metaStore) { Catalog icebergCatalog = metaStore.doAs( () -> org.apache.iceberg.CatalogUtil.buildIcebergCatalog( - catalogMeta.getCatalogName(), - catalogMeta.getCatalogProperties(), - metaStore.getConfiguration())); + name, properties, metaStore.getConfiguration())); Pattern databaseFilterPattern = null; - if (catalogMeta - .getCatalogProperties() - .containsKey(CatalogMetaProperties.KEY_DATABASE_FILTER_REGULAR_EXPRESSION)) { + if (properties.containsKey(CatalogMetaProperties.KEY_DATABASE_FILTER_REGULAR_EXPRESSION)) { String databaseFilter = - catalogMeta - .getCatalogProperties() - .get(CatalogMetaProperties.KEY_DATABASE_FILTER_REGULAR_EXPRESSION); + properties.get(CatalogMetaProperties.KEY_DATABASE_FILTER_REGULAR_EXPRESSION); databaseFilterPattern = Pattern.compile(databaseFilter); } - MixedTables tables = new MixedTables(metaStore, catalogMeta, icebergCatalog); + MixedTables tables = new MixedTables(metaStore, properties, icebergCatalog); synchronized (this) { + this.name = name; this.tableMetaStore = metaStore; this.icebergCatalog = icebergCatalog; this.databaseFilterPattern = databaseFilterPattern; + this.catalogProperties = properties; this.tables = tables; - this.meta = catalogMeta; + if (properties.containsKey(CatalogMetaProperties.AMS_URI)) { + this.client = new PooledAmsClient(properties.get(CatalogMetaProperties.AMS_URI)); + } } } @@ -167,7 +133,7 @@ public List listTables(String database) { } Table table = tableMetaStore.doAs(() -> icebergCatalog.loadTable(identifier)); if (tables.isBaseStore(table)) { - mixedTables.add(TableIdentifier.of(meta.getCatalogName(), database, identifier.name())); + mixedTables.add(TableIdentifier.of(name(), database, identifier.name())); visited.add(identifier); PrimaryKeySpec keySpec = tables.getPrimaryKeySpec(table); if (keySpec.primaryKeyExisted()) { @@ -233,24 +199,17 @@ public TableBuilder newTableBuilder(TableIdentifier identifier, Schema schema) { return new MixedIcebergTableBuilder(identifier, schema); } - @Override - public void refresh() { - try { - CatalogMeta catalogMeta = client.getCatalog(this.name()); - this.initialize(catalogMeta); - } catch (TException e) { - throw new IllegalStateException(String.format("failed load catalog %s.", name()), e); - } - } - @Override public TableBlockerManager getTableBlockerManager(TableIdentifier tableIdentifier) { + if (client == null) { + throw new UnsupportedOperationException("AMSClient is not initialized"); + } return BasicTableBlockerManager.build(tableIdentifier, client); } @Override public Map properties() { - return Maps.newHashMap(this.meta.getCatalogProperties()); + return Maps.newHashMap(catalogProperties); } private org.apache.iceberg.catalog.TableIdentifier toIcebergTableIdentifier( diff --git a/core/src/main/java/com/netease/arctic/mixed/MixedTables.java b/core/src/main/java/com/netease/arctic/mixed/MixedTables.java index a3d1610215..111342f0cc 100644 --- a/core/src/main/java/com/netease/arctic/mixed/MixedTables.java +++ b/core/src/main/java/com/netease/arctic/mixed/MixedTables.java @@ -18,7 +18,6 @@ package com.netease.arctic.mixed; -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.io.ArcticFileIO; @@ -47,13 +46,15 @@ public class MixedTables { protected TableMetaStore tableMetaStore; - protected CatalogMeta catalogMeta; protected Catalog icebergCatalog; - public MixedTables(TableMetaStore tableMetaStore, CatalogMeta catalogMeta, Catalog catalog) { + protected Map catalogProperties; + + public MixedTables( + TableMetaStore tableMetaStore, Map catalogProperties, Catalog catalog) { this.tableMetaStore = tableMetaStore; - this.catalogMeta = catalogMeta; this.icebergCatalog = catalog; + this.catalogProperties = catalogProperties; } public boolean isBaseStore(Table table) { @@ -89,11 +90,9 @@ public TableIdentifier changeStoreIdentifier(Table base) { protected TableIdentifier changeStoreIdentifier(TableIdentifier baseIdentifier) { String separator = - catalogMeta - .getCatalogProperties() - .getOrDefault( - CatalogMetaProperties.MIXED_FORMAT_TABLE_STORE_SEPARATOR, - CatalogMetaProperties.MIXED_FORMAT_TABLE_STORE_SEPARATOR_DEFAULT); + catalogProperties.getOrDefault( + CatalogMetaProperties.MIXED_FORMAT_TABLE_STORE_SEPARATOR, + CatalogMetaProperties.MIXED_FORMAT_TABLE_STORE_SEPARATOR_DEFAULT); return TableIdentifier.of( baseIdentifier.namespace(), baseIdentifier.name() + separator + "change" + separator); } @@ -104,24 +103,18 @@ public ArcticTable loadTable( PrimaryKeySpec keySpec = getPrimaryKeySpec(base); if (!keySpec.primaryKeyExisted()) { return new BasicUnkeyedTable( - tableIdentifier, - useArcticTableOperation(base, io), - io, - catalogMeta.getCatalogProperties()); + tableIdentifier, useArcticTableOperation(base, io), io, catalogProperties); } Table changeIcebergTable = loadChangeStore(base); BaseTable baseStore = new BasicKeyedTable.BaseInternalTable( - tableIdentifier, - useArcticTableOperation(base, io), - io, - catalogMeta.getCatalogProperties()); + tableIdentifier, useArcticTableOperation(base, io), io, catalogProperties); ChangeTable changeStore = new BasicKeyedTable.ChangeInternalTable( tableIdentifier, useArcticTableOperation(changeIcebergTable, io), io, - catalogMeta.getCatalogProperties()); + catalogProperties); return new BasicKeyedTable(keySpec, baseStore, changeStore); } @@ -156,7 +149,7 @@ public ArcticTable createTable( if (!keySpec.primaryKeyExisted()) { return new BasicUnkeyedTable( - identifier, useArcticTableOperation(base, io), io, catalogMeta.getCatalogProperties()); + identifier, useArcticTableOperation(base, io), io, catalogProperties); } Catalog.TableBuilder changeBuilder = @@ -170,13 +163,10 @@ public ArcticTable createTable( Table change = tableMetaStore.doAs(changeBuilder::create); BaseTable baseStore = new BasicKeyedTable.BaseInternalTable( - identifier, useArcticTableOperation(base, io), io, catalogMeta.getCatalogProperties()); + identifier, useArcticTableOperation(base, io), io, catalogProperties); ChangeTable changeStore = new BasicKeyedTable.ChangeInternalTable( - identifier, - useArcticTableOperation(change, io), - io, - catalogMeta.getCatalogProperties()); + identifier, useArcticTableOperation(change, io), io, catalogProperties); return new BasicKeyedTable(keySpec, baseStore, changeStore); } diff --git a/core/src/main/java/com/netease/arctic/op/ArcticTransaction.java b/core/src/main/java/com/netease/arctic/op/ArcticTransaction.java index dab5f304ca..bfa3d2930b 100644 --- a/core/src/main/java/com/netease/arctic/op/ArcticTransaction.java +++ b/core/src/main/java/com/netease/arctic/op/ArcticTransaction.java @@ -45,6 +45,7 @@ import org.apache.iceberg.UpdatePartitionSpec; import org.apache.iceberg.UpdateProperties; import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.UpdateStatistics; import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; @@ -141,6 +142,11 @@ public ExpireSnapshots expireSnapshots() { return transaction.expireSnapshots(); } + @Override + public UpdateStatistics updateStatistics() { + return transaction.updateStatistics(); + } + @Override public void commitTransaction() { transaction.commitTransaction(); @@ -342,6 +348,11 @@ public Map refs() { return transactionTable.refs(); } + @Override + public UpdateStatistics updateStatistics() { + return transactionTable.updateStatistics(); + } + @Override public String toString() { return transactionTable.toString(); diff --git a/core/src/main/java/com/netease/arctic/op/KeyedPartitionRewrite.java b/core/src/main/java/com/netease/arctic/op/KeyedPartitionRewrite.java index 04e6ed6906..49f67dd94d 100644 --- a/core/src/main/java/com/netease/arctic/op/KeyedPartitionRewrite.java +++ b/core/src/main/java/com/netease/arctic/op/KeyedPartitionRewrite.java @@ -20,18 +20,21 @@ import com.netease.arctic.table.BaseTable; import com.netease.arctic.table.KeyedTable; -import com.netease.arctic.table.TableProperties; +import com.netease.arctic.utils.ArcticTableUtil; +import com.netease.arctic.utils.StatisticsFileUtil; import org.apache.iceberg.DataFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; import org.apache.iceberg.Transaction; +import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.StructLikeMap; +import java.util.Collections; import java.util.List; -import java.util.Map; /** Replace {@link BaseTable} partition files and change max transaction id map */ public class KeyedPartitionRewrite extends PartitionTransactionOperation @@ -57,29 +60,39 @@ public KeyedPartitionRewrite updateOptimizedSequenceDynamically(long sequence) { } @Override - protected StructLikeMap> apply(Transaction transaction) { + protected List apply(Transaction transaction) { PartitionSpec spec = transaction.table().spec(); - StructLikeMap> partitionProperties = - StructLikeMap.create(spec.partitionType()); if (this.addFiles.isEmpty()) { - return partitionProperties; + return Collections.emptyList(); } - Preconditions.checkNotNull(optimizedSequence, "optimized sequence must be set."); - Preconditions.checkArgument(optimizedSequence > 0, "optimized sequence must > 0."); + Preconditions.checkNotNull(this.optimizedSequence, "optimized sequence must be set."); + Preconditions.checkArgument(this.optimizedSequence > 0, "optimized sequence must > 0."); ReplacePartitions replacePartitions = transaction.newReplacePartitions(); + replacePartitions.set(ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE_EXIST, "true"); addFiles.forEach(replacePartitions::addFile); replacePartitions.commit(); + CreateSnapshotEvent newSnapshot = (CreateSnapshotEvent) replacePartitions.updateEvent(); - addFiles.forEach( - f -> - partitionProperties - .computeIfAbsent(f.partition(), k -> Maps.newHashMap()) - .put( - TableProperties.PARTITION_OPTIMIZED_SEQUENCE, - String.valueOf(optimizedSequence))); - return partitionProperties; + StructLikeMap oldOptimizedSequence = ArcticTableUtil.readOptimizedSequence(keyedTable); + StructLikeMap optimizedSequence = StructLikeMap.create(spec.partitionType()); + if (oldOptimizedSequence != null) { + optimizedSequence.putAll(oldOptimizedSequence); + } + addFiles.forEach(f -> optimizedSequence.put(f.partition(), this.optimizedSequence)); + + Table table = transaction.table(); + StatisticsFile statisticsFile = + StatisticsFileUtil.writerBuilder(table) + .withSnapshotId(newSnapshot.snapshotId()) + .build() + .add( + ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE, + optimizedSequence, + StatisticsFileUtil.createPartitionDataSerializer(table.spec(), Long.class)) + .complete(); + return Collections.singletonList(statisticsFile); } @Override diff --git a/core/src/main/java/com/netease/arctic/op/OverwriteBaseFiles.java b/core/src/main/java/com/netease/arctic/op/OverwriteBaseFiles.java index 386ff57ae0..56aca56fac 100644 --- a/core/src/main/java/com/netease/arctic/op/OverwriteBaseFiles.java +++ b/core/src/main/java/com/netease/arctic/op/OverwriteBaseFiles.java @@ -20,8 +20,9 @@ import com.netease.arctic.scan.CombinedScanTask; import com.netease.arctic.table.KeyedTable; -import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.ArcticTableUtil; +import com.netease.arctic.utils.StatisticsFileUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections.MapUtils; import org.apache.iceberg.DataFile; @@ -30,21 +31,22 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RewriteFiles; import org.apache.iceberg.RowDelta; +import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; import org.apache.iceberg.Transaction; +import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.StructLikeMap; import java.io.IOException; import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Map; /** Overwrite {@link com.netease.arctic.table.BaseTable} and change max transaction id map */ public class OverwriteBaseFiles extends PartitionTransactionOperation { @@ -156,7 +158,7 @@ protected boolean isEmptyCommit() { } @Override - protected StructLikeMap> apply(Transaction transaction) { + protected List apply(Transaction transaction) { Preconditions.checkState( this.dynamic != null, "updateOptimizedSequence() or updateOptimizedSequenceDynamically() must be invoked"); @@ -169,10 +171,13 @@ protected StructLikeMap> apply(Transaction transaction) { } UnkeyedTable baseTable = keyedTable.baseTable(); + List newSnapshots = Lists.newArrayList(); // step1: overwrite data files if (!this.addFiles.isEmpty() || !this.deleteFiles.isEmpty()) { OverwriteFiles overwriteFiles = transaction.newOverwrite(); + overwriteFiles.set(ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE_EXIST, "true"); + overwriteFiles.set(ArcticTableUtil.BLOB_TYPE_BASE_OPTIMIZED_TIME_EXIST, "true"); if (conflictDetectionFilter != null && baseTable.currentSnapshot() != null) { overwriteFiles.conflictDetectionFilter(conflictDetectionFilter).validateNoConflictingData(); @@ -196,6 +201,7 @@ protected StructLikeMap> apply(Transaction transaction) { properties.forEach(overwriteFiles::set); } overwriteFiles.commit(); + newSnapshots.add((CreateSnapshotEvent) overwriteFiles.updateEvent()); } // step2: RowDelta/Rewrite pos-delete files @@ -203,6 +209,8 @@ protected StructLikeMap> apply(Transaction transaction) { || CollectionUtils.isNotEmpty(deleteDeleteFiles)) { if (CollectionUtils.isEmpty(deleteDeleteFiles)) { RowDelta rowDelta = transaction.newRowDelta(); + rowDelta.set(ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE_EXIST, "true"); + rowDelta.set(ArcticTableUtil.BLOB_TYPE_BASE_OPTIMIZED_TIME_EXIST, "true"); if (baseTable.currentSnapshot() != null) { rowDelta.validateFromSnapshot(baseTable.currentSnapshot().snapshotId()); } @@ -218,8 +226,11 @@ protected StructLikeMap> apply(Transaction transaction) { properties.forEach(rowDelta::set); } rowDelta.commit(); + newSnapshots.add((CreateSnapshotEvent) rowDelta.updateEvent()); } else { RewriteFiles rewriteFiles = transaction.newRewrite(); + rewriteFiles.set(ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE_EXIST, "true"); + rewriteFiles.set(ArcticTableUtil.BLOB_TYPE_BASE_OPTIMIZED_TIME_EXIST, "true"); if (baseTable.currentSnapshot() != null) { rewriteFiles.validateFromSnapshot(baseTable.currentSnapshot().snapshotId()); } @@ -241,14 +252,26 @@ protected StructLikeMap> apply(Transaction transaction) { properties.forEach(rewriteFiles::set); } rewriteFiles.commit(); + newSnapshots.add((CreateSnapshotEvent) rewriteFiles.updateEvent()); } } + if (newSnapshots.isEmpty()) { + return Collections.emptyList(); + } // step3: set optimized sequence id, optimized time - String commitTime = String.valueOf(System.currentTimeMillis()); + long commitTime = System.currentTimeMillis(); PartitionSpec spec = transaction.table().spec(); - StructLikeMap> partitionProperties = - StructLikeMap.create(spec.partitionType()); + StructLikeMap oldOptimizedSequence = ArcticTableUtil.readOptimizedSequence(keyedTable); + StructLikeMap oldOptimizedTime = ArcticTableUtil.readBaseOptimizedTime(keyedTable); + StructLikeMap optimizedSequence = StructLikeMap.create(spec.partitionType()); + StructLikeMap optimizedTime = StructLikeMap.create(spec.partitionType()); + if (oldOptimizedSequence != null) { + optimizedSequence.putAll(oldOptimizedSequence); + } + if (oldOptimizedTime != null) { + optimizedTime.putAll(oldOptimizedTime); + } StructLikeMap toChangePartitionSequence; if (this.dynamic) { toChangePartitionSequence = sequenceForChangedPartitions; @@ -257,13 +280,31 @@ protected StructLikeMap> apply(Transaction transaction) { } toChangePartitionSequence.forEach( (partition, sequence) -> { - Map properties = - partitionProperties.computeIfAbsent(partition, k -> Maps.newHashMap()); - properties.put(TableProperties.PARTITION_OPTIMIZED_SEQUENCE, String.valueOf(sequence)); - properties.put(TableProperties.PARTITION_BASE_OPTIMIZED_TIME, commitTime); + optimizedSequence.put(partition, sequence); + optimizedTime.put(partition, commitTime); }); - return partitionProperties; + StatisticsFile statisticsFile = null; + List result = Lists.newArrayList(); + for (CreateSnapshotEvent newSnapshot : newSnapshots) { + if (statisticsFile != null) { + result.add(StatisticsFileUtil.copyToSnapshot(statisticsFile, newSnapshot.snapshotId())); + } else { + Table table = transaction.table(); + StatisticsFileUtil.PartitionDataSerializer dataSerializer = + StatisticsFileUtil.createPartitionDataSerializer(table.spec(), Long.class); + statisticsFile = + StatisticsFileUtil.writerBuilder(table) + .withSnapshotId(newSnapshot.snapshotId()) + .build() + .add( + ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE, optimizedSequence, dataSerializer) + .add(ArcticTableUtil.BLOB_TYPE_BASE_OPTIMIZED_TIME, optimizedTime, dataSerializer) + .complete(); + result.add(statisticsFile); + } + } + return result; } private void applyDeleteExpression() { diff --git a/core/src/main/java/com/netease/arctic/op/PartitionTransactionOperation.java b/core/src/main/java/com/netease/arctic/op/PartitionTransactionOperation.java index 2f506a2419..afc4d592db 100644 --- a/core/src/main/java/com/netease/arctic/op/PartitionTransactionOperation.java +++ b/core/src/main/java/com/netease/arctic/op/PartitionTransactionOperation.java @@ -21,15 +21,16 @@ import com.netease.arctic.table.BaseTable; import com.netease.arctic.table.KeyedTable; import org.apache.iceberg.PendingUpdate; +import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Transaction; -import org.apache.iceberg.util.StructLikeMap; +import org.apache.iceberg.UpdateStatistics; import java.util.HashMap; +import java.util.List; import java.util.Map; /** Abstract transaction operation on {@link BaseTable} which will change max transaction id map */ -public abstract class PartitionTransactionOperation - implements PendingUpdate>> { +public abstract class PartitionTransactionOperation implements PendingUpdate> { KeyedTable keyedTable; private Transaction tx; @@ -37,8 +38,8 @@ public abstract class PartitionTransactionOperation protected final Map properties; - public PartitionTransactionOperation(KeyedTable baseTable) { - this.keyedTable = baseTable; + public PartitionTransactionOperation(KeyedTable keyedTable) { + this.keyedTable = keyedTable; this.properties = new HashMap<>(); } @@ -60,10 +61,10 @@ public PartitionTransactionOperation set(String key, String value) { * @param transaction table transaction * @return changed partition properties */ - protected abstract StructLikeMap> apply(Transaction transaction); + protected abstract List apply(Transaction transaction); @Override - public StructLikeMap> apply() { + public List apply() { return apply(tx); } @@ -83,14 +84,12 @@ public void commit() { } this.tx = keyedTable.baseTable().newTransaction(); - StructLikeMap> changedPartitionProperties = apply(); - UpdatePartitionProperties updatePartitionProperties = - keyedTable.baseTable().updatePartitionProperties(tx); - changedPartitionProperties.forEach( - (partition, properties) -> - properties.forEach( - (key, value) -> updatePartitionProperties.set(partition, key, value))); - updatePartitionProperties.commit(); + List statisticsFiles = apply(); + if (statisticsFiles != null && !statisticsFiles.isEmpty()) { + UpdateStatistics updateStatistics = this.tx.updateStatistics(); + statisticsFiles.forEach(s -> updateStatistics.setStatistics(s.snapshotId(), s)); + updateStatistics.commit(); + } tx.commitTransaction(); } diff --git a/core/src/main/java/com/netease/arctic/op/RewritePartitions.java b/core/src/main/java/com/netease/arctic/op/RewritePartitions.java index 31d2d15a40..9b9d87ebbc 100644 --- a/core/src/main/java/com/netease/arctic/op/RewritePartitions.java +++ b/core/src/main/java/com/netease/arctic/op/RewritePartitions.java @@ -2,11 +2,11 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.PendingUpdate; -import org.apache.iceberg.util.StructLikeMap; +import org.apache.iceberg.StatisticsFile; -import java.util.Map; +import java.util.List; -public interface RewritePartitions extends PendingUpdate>> { +public interface RewritePartitions extends PendingUpdate> { /** * Add a {@link DataFile} to the table. diff --git a/core/src/main/java/com/netease/arctic/scan/BasicKeyedTableScan.java b/core/src/main/java/com/netease/arctic/scan/BasicKeyedTableScan.java index 50a6894b9f..313eabb827 100644 --- a/core/src/main/java/com/netease/arctic/scan/BasicKeyedTableScan.java +++ b/core/src/main/java/com/netease/arctic/scan/BasicKeyedTableScan.java @@ -23,7 +23,7 @@ import com.netease.arctic.scan.expressions.BasicPartitionEvaluator; import com.netease.arctic.table.BasicKeyedTable; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.utils.TablePropertyUtil; +import com.netease.arctic.utils.ArcticTableUtil; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; @@ -149,10 +149,7 @@ private CloseableIterable planBaseFiles() { } private CloseableIterable planChangeFiles() { - StructLikeMap partitionOptimizedSequence = - TablePropertyUtil.getPartitionOptimizedSequence(table); - StructLikeMap legacyPartitionMaxTransactionId = - TablePropertyUtil.getLegacyPartitionMaxTransactionId(table); + StructLikeMap partitionOptimizedSequence = ArcticTableUtil.readOptimizedSequence(table); Expression partitionExpressions = Expressions.alwaysTrue(); if (expression != null) { // Only push down filters related to partition @@ -160,11 +157,7 @@ private CloseableIterable planChangeFiles() { } ChangeTableIncrementalScan changeTableScan = - table - .changeTable() - .newScan() - .fromSequence(partitionOptimizedSequence) - .fromLegacyTransaction(legacyPartitionMaxTransactionId); + table.changeTable().newScan().fromSequence(partitionOptimizedSequence); changeTableScan = changeTableScan.filter(partitionExpressions); diff --git a/core/src/main/java/com/netease/arctic/scan/ChangeTableIncrementalScan.java b/core/src/main/java/com/netease/arctic/scan/ChangeTableIncrementalScan.java index 89d2b4b362..07844cbb5d 100644 --- a/core/src/main/java/com/netease/arctic/scan/ChangeTableIncrementalScan.java +++ b/core/src/main/java/com/netease/arctic/scan/ChangeTableIncrementalScan.java @@ -48,16 +48,6 @@ public interface ChangeTableIncrementalScan extends TableScan { */ ChangeTableIncrementalScan toSequence(long sequence); - /** - * Config this scan to read data from legacy {@code partitionTransactionId} exclusive to the - * current Transaction inclusive. For partitions set both TransactionId and LegacyTransactionId, - * LegacyTransactionId will be ignored. - * - * @param partitionTransactionId from TransactionId for each partition - * @return this for method chaining - */ - ChangeTableIncrementalScan fromLegacyTransaction(StructLikeMap partitionTransactionId); - @Override ChangeTableIncrementalScan useSnapshot(long snapshotId); diff --git a/core/src/main/java/com/netease/arctic/table/BasicUnkeyedTable.java b/core/src/main/java/com/netease/arctic/table/BasicUnkeyedTable.java index 14719be3be..9d0e5d1d9d 100644 --- a/core/src/main/java/com/netease/arctic/table/BasicUnkeyedTable.java +++ b/core/src/main/java/com/netease/arctic/table/BasicUnkeyedTable.java @@ -145,6 +145,11 @@ public Map sortOrders() { return icebergTable.sortOrders(); } + @Override + public UpdateStatistics updateStatistics() { + return icebergTable.updateStatistics(); + } + @Override public Map properties() { if (catalogProperties == null) { @@ -308,9 +313,4 @@ public StructLikeMap> partitionProperty() { public UpdatePartitionProperties updatePartitionProperties(Transaction transaction) { return new PartitionPropertiesUpdate(this, transaction); } - - @Override - public UpdateStatistics updateStatistics() { - return icebergTable.updateStatistics(); - } } diff --git a/core/src/main/java/com/netease/arctic/utils/ArcticTableUtil.java b/core/src/main/java/com/netease/arctic/utils/ArcticTableUtil.java index a85a768af1..25c93be24b 100644 --- a/core/src/main/java/com/netease/arctic/utils/ArcticTableUtil.java +++ b/core/src/main/java/com/netease/arctic/utils/ArcticTableUtil.java @@ -1,19 +1,26 @@ package com.netease.arctic.utils; -import com.netease.arctic.catalog.IcebergCatalogWrapper; +import com.netease.arctic.ams.api.TableFormat; 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 org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.StructLikeMap; + +import java.util.List; +import java.util.function.Predicate; public class ArcticTableUtil { - /** - * check arctic table is iceberg table format - * - * @param arcticTable target arctic table - * @return Whether iceberg table format - */ - public static boolean isIcebergTableFormat(ArcticTable arcticTable) { - return arcticTable instanceof IcebergCatalogWrapper.BasicIcebergTable; - } + public static final String BLOB_TYPE_OPTIMIZED_SEQUENCE = "optimized-sequence"; + public static final String BLOB_TYPE_BASE_OPTIMIZED_TIME = "base-optimized-time"; + + public static final String BLOB_TYPE_OPTIMIZED_SEQUENCE_EXIST = "optimized-sequence.exist"; + public static final String BLOB_TYPE_BASE_OPTIMIZED_TIME_EXIST = "base-optimized-time.exist"; /** Return the base store of the arctic table. */ public static UnkeyedTable baseStore(ArcticTable arcticTable) { @@ -27,11 +34,124 @@ public static UnkeyedTable baseStore(ArcticTable arcticTable) { /** Return the table root location of the arctic table. */ public static String tableRootLocation(ArcticTable arcticTable) { String tableRootLocation; - if (!ArcticTableUtil.isIcebergTableFormat(arcticTable) && arcticTable.isUnkeyedTable()) { + if (TableFormat.ICEBERG != arcticTable.format() && arcticTable.isUnkeyedTable()) { tableRootLocation = TableFileUtil.getFileDir(arcticTable.location()); } else { tableRootLocation = arcticTable.location(); } return tableRootLocation; } + + public static StructLikeMap readOptimizedSequence(KeyedTable table) { + return readWithLegacy(table.baseTable(), null, BLOB_TYPE_OPTIMIZED_SEQUENCE); + } + + public static StructLikeMap readOptimizedSequence(KeyedTable table, long snapshotId) { + return readWithLegacy(table.baseTable(), snapshotId, BLOB_TYPE_OPTIMIZED_SEQUENCE); + } + + public static StructLikeMap readBaseOptimizedTime(KeyedTable table) { + return readWithLegacy(table.baseTable(), null, BLOB_TYPE_BASE_OPTIMIZED_TIME); + } + + public static StructLikeMap readBaseOptimizedTime(KeyedTable table, long snapshotId) { + return readWithLegacy(table.baseTable(), snapshotId, BLOB_TYPE_BASE_OPTIMIZED_TIME); + } + + private static StructLikeMap readWithLegacy( + UnkeyedTable table, Long snapshotId, String type) { + if (snapshotId == null) { + Snapshot snapshot = table.currentSnapshot(); + if (snapshot == null) { + return readLegacyPartitionProperties(table, type); + } else { + snapshotId = snapshot.snapshotId(); + } + } + StructLikeMap result = readFromStatisticsFile(table, snapshotId, type); + return result != null ? result : readLegacyPartitionProperties(table, type); + } + + private static StructLikeMap readFromStatisticsFile( + UnkeyedTable table, long snapshotId, String type) { + Snapshot snapshot = table.snapshot(snapshotId); + Preconditions.checkArgument(snapshot != null, "Snapshot %s not found", snapshotId); + Snapshot snapshotContainsType = findLatestValidSnapshot(table, snapshotId, isTypeExist(type)); + if (snapshotContainsType == null) { + // Return null to read from legacy partition properties + return null; + } + List statisticsFiles = + StatisticsFileUtil.getStatisticsFiles(table, snapshotContainsType.snapshotId(), type); + Preconditions.checkState( + !statisticsFiles.isEmpty(), "Statistics file not found for snapshot %s", snapshotId); + Preconditions.checkState( + statisticsFiles.size() == 1, + "There should be only one statistics file for snapshot %s", + snapshotId); + List> result = + StatisticsFileUtil.reader(table) + .read( + statisticsFiles.get(0), + type, + StatisticsFileUtil.createPartitionDataSerializer(table.spec(), Long.class)); + if (result.size() != 1) { + throw new IllegalStateException( + "There should be only one partition data in statistics file for blob type " + type); + } + return result.get(0); + } + + /** + * Find the latest valid snapshot which satisfies the condition. + * + * @param table - Iceberg table + * @param currentSnapshotId - find from this snapshot + * @param condition - the condition to satisfy + * @return the latest valid snapshot or null if not exists + */ + public static Snapshot findLatestValidSnapshot( + Table table, long currentSnapshotId, Predicate condition) { + Long snapshotId = currentSnapshotId; + while (true) { + if (snapshotId == null || table.snapshot(snapshotId) == null) { + return null; + } + Snapshot snapshot = table.snapshot(snapshotId); + if (condition.test(snapshot)) { + return snapshot; + } else { + // seek parent snapshot + snapshotId = snapshot.parentId(); + } + } + } + + @VisibleForTesting + public static Predicate isTypeExist(String type) { + switch (type) { + case BLOB_TYPE_OPTIMIZED_SEQUENCE: + return snapshot -> snapshot.summary().containsKey(BLOB_TYPE_OPTIMIZED_SEQUENCE_EXIST); + case BLOB_TYPE_BASE_OPTIMIZED_TIME: + return snapshot -> snapshot.summary().containsKey(BLOB_TYPE_BASE_OPTIMIZED_TIME_EXIST); + default: + throw new IllegalArgumentException("Unknown type: " + type); + } + } + + private static StructLikeMap readLegacyPartitionProperties( + UnkeyedTable table, String type) { + // To be compatible with old Amoro version 0.5.0 which didn't use puffin file and stored the + // statistics in table properties + switch (type) { + case BLOB_TYPE_OPTIMIZED_SEQUENCE: + return TablePropertyUtil.getPartitionLongProperties( + table, TableProperties.PARTITION_OPTIMIZED_SEQUENCE); + case BLOB_TYPE_BASE_OPTIMIZED_TIME: + return TablePropertyUtil.getPartitionLongProperties( + table, TableProperties.PARTITION_BASE_OPTIMIZED_TIME); + default: + throw new IllegalArgumentException("Unknown type: " + type); + } + } } diff --git a/core/src/main/java/com/netease/arctic/utils/CatalogUtil.java b/core/src/main/java/com/netease/arctic/utils/CatalogUtil.java index 9759867539..19507fd7bd 100644 --- a/core/src/main/java/com/netease/arctic/utils/CatalogUtil.java +++ b/core/src/main/java/com/netease/arctic/utils/CatalogUtil.java @@ -31,8 +31,7 @@ import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.ams.api.TableMeta; import com.netease.arctic.ams.api.properties.CatalogMetaProperties; -import com.netease.arctic.catalog.ArcticCatalog; -import com.netease.arctic.catalog.BasicIcebergCatalog; +import com.netease.arctic.catalog.CatalogLoader; import com.netease.arctic.io.ArcticFileIO; import com.netease.arctic.op.ArcticHadoopTableOperations; import com.netease.arctic.op.ArcticTableOperations; @@ -43,10 +42,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.Table; import org.apache.iceberg.hadoop.HadoopTableOperations; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.rest.RESTCatalog; import org.apache.iceberg.util.PropertyUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,9 +65,15 @@ public class CatalogUtil { /** Return table format set catalog supported. */ public static Set tableFormats(CatalogMeta meta) { - if (meta.getCatalogProperties().containsKey(CatalogMetaProperties.TABLE_FORMATS)) { - String tableFormatsProperty = - meta.getCatalogProperties().get(CatalogMetaProperties.TABLE_FORMATS); + return tableFormats(meta.getCatalogType(), meta.getCatalogProperties()); + } + + /** Return table format set catalog supported. */ + public static Set tableFormats( + String metastoreType, Map catalogProperties) { + if (catalogProperties != null + && catalogProperties.containsKey(CatalogMetaProperties.TABLE_FORMATS)) { + String tableFormatsProperty = catalogProperties.get(CatalogMetaProperties.TABLE_FORMATS); return Arrays.stream(tableFormatsProperty.split(",")) .map( tableFormatString -> @@ -73,7 +81,7 @@ public static Set tableFormats(CatalogMeta meta) { .collect(Collectors.toSet()); } else { // Generate table format from catalog type for compatibility with older versions - switch (meta.getCatalogType()) { + switch (metastoreType) { case CATALOG_TYPE_AMS: return Sets.newHashSet(TableFormat.MIXED_ICEBERG); case CATALOG_TYPE_CUSTOM: @@ -83,7 +91,7 @@ public static Set tableFormats(CatalogMeta meta) { case CATALOG_TYPE_HIVE: return Sets.newHashSet(TableFormat.MIXED_HIVE); default: - throw new IllegalArgumentException("Unsupported catalog type:" + meta.getCatalogType()); + throw new IllegalArgumentException("Unsupported catalog type:" + metastoreType); } } } @@ -98,6 +106,42 @@ public static void mergeCatalogProperties(CatalogMeta meta, Map } } + /** + * add initialize properties for iceberg catalog + * + * @param catalogName - catalog name + * @param metastoreType - metastore type + * @param properties - catalog properties + * @return catalog properties with initialize properties. + */ + public static Map withIcebergCatalogInitializeProperties( + String catalogName, String metastoreType, Map properties) { + Map icebergCatalogProperties = Maps.newHashMap(properties); + icebergCatalogProperties.put( + org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE, metastoreType); + if (CatalogMetaProperties.CATALOG_TYPE_GLUE.equals(metastoreType)) { + icebergCatalogProperties.put(CatalogProperties.CATALOG_IMPL, CatalogLoader.GLUE_CATALOG_IMPL); + } + if (CATALOG_TYPE_AMS.equalsIgnoreCase(metastoreType)) { + icebergCatalogProperties.put(CatalogProperties.WAREHOUSE_LOCATION, catalogName); + if (!icebergCatalogProperties.containsKey(CatalogProperties.CATALOG_IMPL)) { + icebergCatalogProperties.put(CatalogProperties.CATALOG_IMPL, RESTCatalog.class.getName()); + } + } + + if (CATALOG_TYPE_CUSTOM.equalsIgnoreCase(metastoreType)) { + Preconditions.checkArgument( + icebergCatalogProperties.containsKey(CatalogProperties.CATALOG_IMPL), + "Custom catalog properties must contains " + CatalogProperties.CATALOG_IMPL); + } + + if (icebergCatalogProperties.containsKey(CatalogProperties.CATALOG_IMPL)) { + icebergCatalogProperties.remove(org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE); + } + + return icebergCatalogProperties; + } + /** Build {@link TableMetaStore} from catalog meta. */ public static TableMetaStore buildMetaStore(CatalogMeta catalogMeta) { // load storage configs @@ -187,17 +231,6 @@ public static Table useArcticTableOperations( return table; } - /** - * check arctic catalog is native iceberg catalog - * - * @param arcticCatalog target arctic catalog - * @return Whether native iceberg catalog. true is native iceberg catalog, false isn't native - * iceberg catalog. - */ - public static boolean isIcebergCatalog(ArcticCatalog arcticCatalog) { - return arcticCatalog instanceof BasicIcebergCatalog; - } - /** * merge properties of table level in catalog properties to table(properties key start with * table.) diff --git a/core/src/main/java/com/netease/arctic/utils/StatisticsFileUtil.java b/core/src/main/java/com/netease/arctic/utils/StatisticsFileUtil.java new file mode 100644 index 0000000000..72b8db285e --- /dev/null +++ b/core/src/main/java/com/netease/arctic/utils/StatisticsFileUtil.java @@ -0,0 +1,294 @@ +/* + * 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.utils; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.type.MapLikeType; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.BlobMetadata; +import org.apache.iceberg.puffin.FileMetadata; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinReader; +import org.apache.iceberg.puffin.PuffinWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.StructLikeMap; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; + +/** Util class for write and read Iceberg statistics file {@link StatisticsFile}. */ +public class StatisticsFileUtil { + + public static WriterBuilder writerBuilder(Table table) { + return new WriterBuilder(table); + } + + public static class WriterBuilder { + private final Table table; + private Long snapshotId; + + private WriterBuilder(Table table) { + this.table = table; + } + + public WriterBuilder withSnapshotId(long snapshotId) { + this.snapshotId = snapshotId; + return this; + } + + public Writer build() { + Snapshot snapshot; + if (snapshotId == null) { + snapshot = table.currentSnapshot(); + } else { + snapshot = table.snapshot(snapshotId); + } + Preconditions.checkArgument(snapshot != null, "Cannot find snapshot with id %s", snapshotId); + return new Writer(table, snapshot.snapshotId(), snapshot.sequenceNumber()); + } + } + + public static Reader reader(Table table) { + return new Reader(table); + } + + public static class Writer { + private final long snapshotId; + private final long sequenceNumber; + private final OutputFile outputFile; + private final PuffinWriter puffinWriter; + private boolean closed = false; + + private Writer(Table table, long snapshotId, long sequenceNumber) { + this.snapshotId = snapshotId; + this.sequenceNumber = sequenceNumber; + this.outputFile = + table + .io() + .newOutputFile( + table.location() + + "/data/puffin/" + + snapshotId + + "-" + + UUID.randomUUID() + + ".puffin"); + this.puffinWriter = Puffin.write(outputFile).build(); + } + + public Writer add(Blob blob) { + checkNotClosed(); + puffinWriter.add(blob); + return this; + } + + public Writer add(String type, ByteBuffer blobData) { + add(new Blob(type, Collections.emptyList(), snapshotId, sequenceNumber, blobData)); + return this; + } + + public Writer add(String type, T data, DataSerializer serializer) { + return add(type, serializer.serialize(data)); + } + + public StatisticsFile complete() { + checkNotClosed(); + try { + puffinWriter.finish(); + List blobMetadata = puffinWriter.writtenBlobsMetadata(); + long fileSize = puffinWriter.fileSize(); + long footerSize = puffinWriter.footerSize(); + List collect = + blobMetadata.stream().map(GenericBlobMetadata::from).collect(Collectors.toList()); + return new GenericStatisticsFile( + snapshotId, outputFile.location(), fileSize, footerSize, collect); + } catch (IOException e) { + throw new UncheckedIOException(e); + } finally { + close(); + } + } + + private void checkNotClosed() { + Preconditions.checkState(!closed, "Cannot operate on a closed writer"); + } + + private void close() { + if (!closed) { + closed = true; + try { + puffinWriter.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + } + + public static class Reader { + private final Table table; + + private Reader(Table table) { + this.table = table; + } + + public List read( + StatisticsFile statisticsFile, String type, DataSerializer deserializer) { + return read(statisticsFile, type).stream() + .map(deserializer::deserialize) + .collect(Collectors.toList()); + } + + public List read(StatisticsFile statisticsFile, String type) { + try (PuffinReader puffin = + Puffin.read(table.io().newInputFile(statisticsFile.path())).build()) { + FileMetadata fileMetadata = puffin.fileMetadata(); + List blobs = + fileMetadata.blobs().stream() + .filter(b -> type.equals(b.type())) + .collect(Collectors.toList()); + return Lists.newArrayList(Iterables.transform(puffin.readAll(blobs), Pair::second)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + /** + * Copy a statistic file with a new snapshot id, and it points to the same file in the file system + * as the original file. + * + * @param statisticsFile - original statistic file + * @param snapshotId - new snapshot id + * @return a new copied statistic file + */ + public static StatisticsFile copyToSnapshot(StatisticsFile statisticsFile, long snapshotId) { + return new GenericStatisticsFile( + snapshotId, + statisticsFile.path(), + statisticsFile.fileSizeInBytes(), + statisticsFile.fileFooterSizeInBytes(), + statisticsFile.blobMetadata()); + } + + /** + * Get the statistics files that belong to the given snapshot. + * + * @param table - Iceberg Table + * @param snapshotId - the snapshot id + * @param type - the type of the blob type + * @return the list of statistics files + */ + public static List getStatisticsFiles(Table table, long snapshotId, String type) { + List statisticsFiles = table.statisticsFiles(); + if (statisticsFiles.isEmpty()) { + return Collections.emptyList(); + } + return statisticsFiles.stream() + .filter(s -> s.blobMetadata().stream().anyMatch(b -> type.equals(b.type()))) + .collect(Collectors.groupingBy(StatisticsFile::snapshotId)) + .get(snapshotId); + } + + public static PartitionDataSerializer createPartitionDataSerializer( + PartitionSpec spec, Class valueClassType) { + return new PartitionDataSerializer<>(spec, valueClassType); + } + + /** + * A serializer to serialize and deserialize data between ByteBuffer and Type T. + * + * @param - the class type of the data + */ + public interface DataSerializer { + ByteBuffer serialize(T data); + + T deserialize(ByteBuffer buffer); + } + + public static class PartitionDataSerializer implements DataSerializer> { + + private final PartitionSpec spec; + private final Class valueClassType; + + public PartitionDataSerializer(PartitionSpec spec, Class valueClassType) { + this.spec = spec; + this.valueClassType = valueClassType; + } + + @Override + public ByteBuffer serialize(StructLikeMap data) { + Map stringKeyMap = Maps.newHashMap(); + for (StructLike pd : data.keySet()) { + String pathLike = spec.partitionToPath(pd); + stringKeyMap.put(pathLike, data.get(pd)); + } + String value; + try { + value = new ObjectMapper().writeValueAsString(stringKeyMap); + } catch (JsonProcessingException e) { + throw new UncheckedIOException(e); + } + return ByteBuffer.wrap(value.getBytes()); + } + + @Override + public StructLikeMap deserialize(ByteBuffer buffer) { + try { + StructLikeMap results = StructLikeMap.create(spec.partitionType()); + ObjectMapper objectMapper = new ObjectMapper(); + MapLikeType mapLikeType = + objectMapper + .getTypeFactory() + .constructMapLikeType(Map.class, String.class, valueClassType); + + Map map = objectMapper.readValue(new String(buffer.array()), mapLikeType); + for (String key : map.keySet()) { + if (spec.isUnpartitioned()) { + results.put(TablePropertyUtil.EMPTY_STRUCT, map.get(key)); + } else { + StructLike partitionData = ArcticDataFiles.data(spec, key); + results.put(partitionData, map.get(key)); + } + } + return results; + } catch (JsonProcessingException e) { + throw new UnsupportedOperationException("Failed to decode partition data ", e); + } + } + } +} diff --git a/core/src/main/java/com/netease/arctic/utils/TableFileUtil.java b/core/src/main/java/com/netease/arctic/utils/TableFileUtil.java index 14a9ca976e..a8b023725f 100644 --- a/core/src/main/java/com/netease/arctic/utils/TableFileUtil.java +++ b/core/src/main/java/com/netease/arctic/utils/TableFileUtil.java @@ -53,13 +53,6 @@ public static String getFileDir(String filePath) { return filePath.substring(0, lastSlash); } - public static String getPartitionPathFromFilePath( - String fileLocation, String tableLocation, String fileName) { - int tableIndex = fileLocation.indexOf(tableLocation); - int fileIndex = fileLocation.lastIndexOf(fileName); - return fileLocation.substring(tableIndex + tableLocation.length(), fileIndex - 1); - } - /** * Try to recursiveDelete the empty directory * diff --git a/core/src/main/java/com/netease/arctic/utils/TablePropertyUtil.java b/core/src/main/java/com/netease/arctic/utils/TablePropertyUtil.java index ec1b350361..b86c1c4bd5 100644 --- a/core/src/main/java/com/netease/arctic/utils/TablePropertyUtil.java +++ b/core/src/main/java/com/netease/arctic/utils/TablePropertyUtil.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; 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 org.apache.iceberg.PartitionSpec; @@ -77,16 +76,6 @@ public static String encodePartitionProperties( return value; } - public static StructLikeMap getPartitionBaseOptimizedTime(KeyedTable keyedTable) { - return getPartitionLongProperties( - keyedTable.baseTable(), TableProperties.PARTITION_BASE_OPTIMIZED_TIME); - } - - public static StructLikeMap getPartitionOptimizedSequence(KeyedTable keyedTable) { - return getPartitionLongProperties( - keyedTable.baseTable(), TableProperties.PARTITION_OPTIMIZED_SEQUENCE); - } - public static StructLikeMap getPartitionLongProperties( UnkeyedTable unkeyedTable, String key) { StructLikeMap result = StructLikeMap.create(unkeyedTable.spec().partitionType()); @@ -136,28 +125,6 @@ public static Map getPartitionProperties( return result; } - public static StructLikeMap getLegacyPartitionMaxTransactionId(KeyedTable keyedTable) { - StructLikeMap baseTableMaxTransactionId = - StructLikeMap.create(keyedTable.spec().partitionType()); - - StructLikeMap> partitionProperty = - keyedTable.asKeyedTable().baseTable().partitionProperty(); - partitionProperty.forEach( - (partitionKey, propertyValue) -> { - Long maxTxId = - (propertyValue == null - || propertyValue.get(TableProperties.BASE_TABLE_MAX_TRANSACTION_ID) == null) - ? null - : Long.parseLong( - propertyValue.get(TableProperties.BASE_TABLE_MAX_TRANSACTION_ID)); - if (maxTxId != null) { - baseTableMaxTransactionId.put(partitionKey, maxTxId); - } - }); - - return baseTableMaxTransactionId; - } - public static long getTableWatermark(Map properties) { String watermarkValue = properties.get(TableProperties.WATERMARK_TABLE); if (watermarkValue == null) { diff --git a/core/src/main/java/com/netease/arctic/utils/TableTypeUtil.java b/core/src/main/java/com/netease/arctic/utils/TableTypeUtil.java deleted file mode 100644 index 681d7d275b..0000000000 --- a/core/src/main/java/com/netease/arctic/utils/TableTypeUtil.java +++ /dev/null @@ -1,17 +0,0 @@ -package com.netease.arctic.utils; - -import com.netease.arctic.catalog.IcebergCatalogWrapper; -import com.netease.arctic.table.ArcticTable; - -/** Used for check arctic table type */ -public class TableTypeUtil { - /** - * check arctic table is iceberg table format - * - * @param arcticTable target arctic table - * @return Whether iceberg table format - */ - public static boolean isIcebergTableFormat(ArcticTable arcticTable) { - return arcticTable instanceof IcebergCatalogWrapper.BasicIcebergTable; - } -} diff --git a/core/src/main/java/org/apache/iceberg/MixedChangeTableScan.java b/core/src/main/java/org/apache/iceberg/MixedChangeTableScan.java index cd9b8c2591..483b23fe3e 100644 --- a/core/src/main/java/org/apache/iceberg/MixedChangeTableScan.java +++ b/core/src/main/java/org/apache/iceberg/MixedChangeTableScan.java @@ -19,7 +19,6 @@ package org.apache.iceberg; import com.netease.arctic.data.DefaultKeyedFile; -import com.netease.arctic.data.FileNameRules; import com.netease.arctic.scan.BasicArcticFileScanTask; import com.netease.arctic.scan.ChangeTableIncrementalScan; import com.netease.arctic.utils.TablePropertyUtil; @@ -33,7 +32,6 @@ */ public class MixedChangeTableScan extends DataTableScan implements ChangeTableIncrementalScan { private StructLikeMap fromPartitionSequence; - private StructLikeMap fromPartitionLegacyTransactionId; private Long toSequence; private Long fromSequence; @@ -62,7 +60,6 @@ protected MixedChangeTableScan newRefinedScan( Table table, Schema schema, TableScanContext context) { MixedChangeTableScan scan = new MixedChangeTableScan(table, schema, context); scan.fromPartitionSequence = this.fromPartitionSequence; - scan.fromPartitionLegacyTransactionId = this.fromPartitionLegacyTransactionId; scan.toSequence = this.toSequence; return scan; } @@ -88,14 +85,6 @@ public ChangeTableIncrementalScan toSequence(long sequence) { return scan; } - @Override - public ChangeTableIncrementalScan fromLegacyTransaction( - StructLikeMap partitionTransactionId) { - MixedChangeTableScan scan = newRefinedScan(table(), schema(), context()); - scan.fromPartitionLegacyTransactionId = partitionTransactionId; - return scan; - } - @Override public CloseableIterable doPlanFiles() { CloseableIterable filteredTasks = @@ -104,14 +93,7 @@ public CloseableIterable doPlanFiles() { fileScanTask -> { StructLike partition = fileScanTask.file().partition(); long sequenceNumber = fileScanTask.file().dataSequenceNumber(); - Boolean shouldKeep = shouldKeepFile(partition, sequenceNumber); - if (shouldKeep == null) { - String filePath = fileScanTask.file().path().toString(); - return shouldKeepFileWithLegacyTxId( - partition, FileNameRules.parseChange(filePath, sequenceNumber).transactionId()); - } else { - return shouldKeep; - } + return shouldKeepFile(partition, sequenceNumber); }); return CloseableIterable.transform( filteredTasks, @@ -120,14 +102,14 @@ public CloseableIterable doPlanFiles() { DefaultKeyedFile.parseChange(fileScanTask.file()), null, table().spec(), null)); } - private Boolean shouldKeepFile(StructLike partition, long sequence) { + private boolean shouldKeepFile(StructLike partition, long sequence) { if (biggerThanToSequence(sequence)) { return false; } if (fromSequence == null && (fromPartitionSequence == null || fromPartitionSequence.isEmpty())) { - // if fromPartitionSequence is not set or is empty, return null to check legacy transactionId - return null; + // if fromPartitionSequence is not set or is empty, return all change files + return true; } Long fromSequence; if (table().spec().isUnpartitioned()) { @@ -138,7 +120,7 @@ private Boolean shouldKeepFile(StructLike partition, long sequence) { if (fromSequence != null) { return sequence > fromSequence; } else { - return null; + return true; } } @@ -156,25 +138,4 @@ private Long scanFromSequence(StructLike partitionData) { private boolean biggerThanToSequence(long sequence) { return this.toSequence != null && sequence > this.toSequence; } - - private boolean shouldKeepFileWithLegacyTxId(StructLike partition, long legacyTxId) { - if (fromPartitionLegacyTransactionId == null || fromPartitionLegacyTransactionId.isEmpty()) { - // if fromPartitionLegacyTransactionId is not set or is empty, return all files - return true; - } - if (table().spec().isUnpartitioned()) { - Long fromTransactionId = - fromPartitionLegacyTransactionId.entrySet().iterator().next().getValue(); - return legacyTxId > fromTransactionId; - } else { - if (!fromPartitionLegacyTransactionId.containsKey(partition)) { - // if fromPartitionLegacyTransactionId not contains this partition, return all files of this - // partition - return true; - } else { - Long partitionTransactionId = fromPartitionLegacyTransactionId.get(partition); - return legacyTxId > partitionTransactionId; - } - } - } } 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 b97c258b2f..b4080917f7 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,4 +17,6 @@ # com.netease.arctic.formats.iceberg.IcebergCatalogFactory -com.netease.arctic.formats.paimon.PaimonCatalogFactory \ No newline at end of file +com.netease.arctic.formats.paimon.PaimonCatalogFactory +com.netease.arctic.formats.mixed.MixedIcebergCatalogFactory +com.netease.arctic.formats.mixed.MixedHiveCatalogFactory \ No newline at end of file diff --git a/core/src/test/java/com/netease/arctic/catalog/BasicCatalogTestHelper.java b/core/src/test/java/com/netease/arctic/catalog/BasicCatalogTestHelper.java index f56a4b2c30..4496b6d7f8 100644 --- a/core/src/test/java/com/netease/arctic/catalog/BasicCatalogTestHelper.java +++ b/core/src/test/java/com/netease/arctic/catalog/BasicCatalogTestHelper.java @@ -22,9 +22,12 @@ import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP; +import com.netease.arctic.CommonUnifiedCatalog; +import com.netease.arctic.UnifiedCatalog; 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.utils.CatalogUtil; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -75,6 +78,11 @@ public String metastoreType() { return metastoreType; } + @Override + public UnifiedCatalog buildUnifiedCatalog(CatalogMeta catalogMeta) { + return new CommonUnifiedCatalog(() -> catalogMeta, Maps.newHashMap()); + } + @Override public Catalog buildIcebergCatalog(CatalogMeta catalogMeta) { if (!TableFormat.ICEBERG.equals(tableFormat)) { @@ -93,7 +101,8 @@ public MixedTables buildMixedTables(CatalogMeta catalogMeta) { throw new UnsupportedOperationException( "Cannot build mixed-tables for table format:" + tableFormat); } - return new MixedTables(catalogMeta); + return new MixedTables( + catalogMeta.getCatalogProperties(), CatalogUtil.buildMetaStore(catalogMeta)); } @Override diff --git a/core/src/test/java/com/netease/arctic/catalog/CatalogTestBase.java b/core/src/test/java/com/netease/arctic/catalog/CatalogTestBase.java index 5357b895df..bc8d80c472 100644 --- a/core/src/test/java/com/netease/arctic/catalog/CatalogTestBase.java +++ b/core/src/test/java/com/netease/arctic/catalog/CatalogTestBase.java @@ -19,9 +19,11 @@ package com.netease.arctic.catalog; import com.netease.arctic.TestAms; +import com.netease.arctic.UnifiedCatalog; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.MockArcticMetastoreServer; import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.ams.api.properties.CatalogMetaProperties; import org.apache.commons.lang3.SystemUtils; import org.apache.iceberg.catalog.Catalog; import org.junit.After; @@ -37,7 +39,8 @@ public abstract class CatalogTestBase { @ClassRule public static TestAms TEST_AMS = new TestAms(); private final CatalogTestHelper testHelper; @Rule public TemporaryFolder temp = new TemporaryFolder(); - private ArcticCatalog catalog; + private UnifiedCatalog unifiedCatalog; + private ArcticCatalog mixedFormatCatalog; private CatalogMeta catalogMeta; private Catalog icebergCatalog; @@ -56,6 +59,7 @@ public void setupCatalog() throws IOException { baseDir = "file:/" + temp.newFolder().getPath().replace("\\", "/"); } catalogMeta = testHelper.buildCatalogMeta(baseDir); + catalogMeta.putToCatalogProperties(CatalogMetaProperties.AMS_URI, TEST_AMS.getServerUrl()); getAmsHandler().createCatalog(catalogMeta); } @@ -63,15 +67,19 @@ public void setupCatalog() throws IOException { public void dropCatalog() { if (catalogMeta != null) { getAmsHandler().dropCatalog(catalogMeta.getCatalogName()); - catalog = null; + mixedFormatCatalog = null; } } - protected ArcticCatalog getCatalog() { - if (catalog == null) { - catalog = CatalogLoader.load(getCatalogUrl()); + protected ArcticCatalog getMixedFormatCatalog() { + if (mixedFormatCatalog == null) { + mixedFormatCatalog = CatalogLoader.load(getCatalogUrl()); } - return catalog; + return mixedFormatCatalog; + } + + protected void refreshMixedFormatCatalog() { + this.mixedFormatCatalog = CatalogLoader.load(getCatalogUrl()); } protected String getCatalogUrl() { @@ -92,4 +100,11 @@ protected Catalog getIcebergCatalog() { } return icebergCatalog; } + + protected UnifiedCatalog getUnifiedCatalog() { + if (unifiedCatalog == null) { + unifiedCatalog = testHelper.buildUnifiedCatalog(catalogMeta); + } + return unifiedCatalog; + } } diff --git a/core/src/test/java/com/netease/arctic/catalog/CatalogTestHelper.java b/core/src/test/java/com/netease/arctic/catalog/CatalogTestHelper.java index 8b2062dff4..2c78ee6e40 100644 --- a/core/src/test/java/com/netease/arctic/catalog/CatalogTestHelper.java +++ b/core/src/test/java/com/netease/arctic/catalog/CatalogTestHelper.java @@ -19,6 +19,7 @@ package com.netease.arctic.catalog; import com.netease.arctic.TableTestHelper; +import com.netease.arctic.UnifiedCatalog; import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.ams.api.properties.CatalogMetaProperties; @@ -38,6 +39,8 @@ default boolean isInternalCatalog() { CatalogMeta buildCatalogMeta(String baseDir); + UnifiedCatalog buildUnifiedCatalog(CatalogMeta catalogMeta); + Catalog buildIcebergCatalog(CatalogMeta catalogMeta); MixedTables buildMixedTables(CatalogMeta catalogMeta); diff --git a/core/src/test/java/com/netease/arctic/catalog/TableTestBase.java b/core/src/test/java/com/netease/arctic/catalog/TableTestBase.java index 9e65986d61..cbf0d61aaf 100644 --- a/core/src/test/java/com/netease/arctic/catalog/TableTestBase.java +++ b/core/src/test/java/com/netease/arctic/catalog/TableTestBase.java @@ -43,7 +43,7 @@ public TableTestBase(CatalogTestHelper catalogTestHelper, TableTestHelper tableT public void setupTable() { this.tableMetaStore = CatalogUtil.buildMetaStore(getCatalogMeta()); - getCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); + getUnifiedCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); switch (getTestFormat()) { case MIXED_HIVE: case MIXED_ICEBERG: @@ -57,7 +57,8 @@ public void setupTable() { private void createMixedFormatTable() { TableBuilder tableBuilder = - getCatalog().newTableBuilder(TableTestHelper.TEST_TABLE_ID, tableTestHelper.tableSchema()); + getMixedFormatCatalog() + .newTableBuilder(TableTestHelper.TEST_TABLE_ID, tableTestHelper.tableSchema()); tableBuilder.withProperties(tableTestHelper.tableProperties()); if (isKeyedTable()) { tableBuilder.withPrimaryKeySpec(tableTestHelper.primaryKeySpec()); @@ -76,14 +77,19 @@ private void createIcebergFormatTable() { tableTestHelper.tableSchema(), tableTestHelper.partitionSpec(), tableTestHelper.tableProperties()); - arcticTable = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); + arcticTable = + (ArcticTable) + getUnifiedCatalog() + .loadTable(TableTestHelper.TEST_DB_NAME, TableTestHelper.TEST_TABLE_NAME) + .originalTable(); } @After public void dropTable() { - getCatalog().dropTable(tableTestHelper.id(), true); + getUnifiedCatalog() + .dropTable(tableTestHelper.id().getDatabase(), tableTestHelper.id().getTableName(), true); try { - getCatalog().dropDatabase(TableTestHelper.TEST_DB_NAME); + getUnifiedCatalog().dropDatabase(TableTestHelper.TEST_DB_NAME); } catch (Exception e) { e.printStackTrace(); } diff --git a/core/src/test/java/com/netease/arctic/catalog/TestBasicArcticCatalog.java b/core/src/test/java/com/netease/arctic/catalog/TestBasicArcticCatalog.java index eafc6d3c73..2fe29326d8 100644 --- a/core/src/test/java/com/netease/arctic/catalog/TestBasicArcticCatalog.java +++ b/core/src/test/java/com/netease/arctic/catalog/TestBasicArcticCatalog.java @@ -44,31 +44,28 @@ public TestBasicArcticCatalog(CatalogTestHelper catalogTestHelper) { @Parameterized.Parameters(name = "tableFormat = {0}") public static Object[] parameters() { - return new Object[] { - new BasicCatalogTestHelper(TableFormat.ICEBERG), - new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG) - }; + return new Object[] {new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG)}; } @Test public void testCreateAndDropDatabase() { String createDbName = TableTestHelper.TEST_DB_NAME; - Assert.assertFalse(getCatalog().listDatabases().contains(createDbName)); - getCatalog().createDatabase(createDbName); - Assert.assertTrue(getCatalog().listDatabases().contains(createDbName)); - getCatalog().dropDatabase(createDbName); - Assert.assertFalse(getCatalog().listDatabases().contains(createDbName)); + Assert.assertFalse(getMixedFormatCatalog().listDatabases().contains(createDbName)); + getMixedFormatCatalog().createDatabase(createDbName); + Assert.assertTrue(getMixedFormatCatalog().listDatabases().contains(createDbName)); + getMixedFormatCatalog().dropDatabase(createDbName); + Assert.assertFalse(getMixedFormatCatalog().listDatabases().contains(createDbName)); } @Test public void testCreateDuplicateDatabase() { String createDbName = TableTestHelper.TEST_DB_NAME; - Assert.assertFalse(getCatalog().listDatabases().contains(createDbName)); - getCatalog().createDatabase(createDbName); - Assert.assertTrue(getCatalog().listDatabases().contains(createDbName)); + Assert.assertFalse(getMixedFormatCatalog().listDatabases().contains(createDbName)); + getMixedFormatCatalog().createDatabase(createDbName); + Assert.assertTrue(getMixedFormatCatalog().listDatabases().contains(createDbName)); Assert.assertThrows( - AlreadyExistsException.class, () -> getCatalog().createDatabase(createDbName)); - getCatalog().dropDatabase(createDbName); + AlreadyExistsException.class, () -> getMixedFormatCatalog().createDatabase(createDbName)); + getMixedFormatCatalog().dropDatabase(createDbName); } @Test @@ -81,9 +78,9 @@ public void testCreateTableWithCatalogTableProperties() throws TException { testCatalogMeta, CatalogMetaProperties.TABLE_PROPERTIES_PREFIX + TableProperties.ENABLE_SELF_OPTIMIZING, "false"); - getCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); + getMixedFormatCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); createTestTable(); - ArcticTable createTable = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); + ArcticTable createTable = getMixedFormatCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); Assert.assertFalse( PropertyUtil.propertyAsBoolean( createTable.properties(), @@ -93,9 +90,9 @@ public void testCreateTableWithCatalogTableProperties() throws TException { @Test public void testLoadTableWithNewCatalogProperties() throws TException { - getCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); + getMixedFormatCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); createTestTable(); - ArcticTable createTable = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); + ArcticTable createTable = getMixedFormatCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); Assert.assertTrue( PropertyUtil.propertyAsBoolean( @@ -111,8 +108,8 @@ public void testLoadTableWithNewCatalogProperties() throws TException { testCatalogMeta, CatalogMetaProperties.TABLE_PROPERTIES_PREFIX + TableProperties.ENABLE_SELF_OPTIMIZING, "false"); - getCatalog().refresh(); - ArcticTable loadTable = getCatalog().loadTable(createTable.id()); + refreshMixedFormatCatalog(); + ArcticTable loadTable = getMixedFormatCatalog().loadTable(createTable.id()); Assert.assertFalse( PropertyUtil.propertyAsBoolean( loadTable.properties(), @@ -122,9 +119,9 @@ public void testLoadTableWithNewCatalogProperties() throws TException { @After public void after() { - getCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); - if (getCatalog().listDatabases().contains(TableTestHelper.TEST_DB_NAME)) { - getCatalog().dropDatabase(TableTestHelper.TEST_DB_NAME); + getMixedFormatCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); + if (getMixedFormatCatalog().listDatabases().contains(TableTestHelper.TEST_DB_NAME)) { + getMixedFormatCatalog().dropDatabase(TableTestHelper.TEST_DB_NAME); } } @@ -138,7 +135,7 @@ protected void createTestTable() { break; case MIXED_ICEBERG: case MIXED_HIVE: - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, BasicTableTestHelper.TABLE_SCHEMA) .create(); break; diff --git a/core/src/test/java/com/netease/arctic/catalog/TestCatalogLoader.java b/core/src/test/java/com/netease/arctic/catalog/TestCatalogLoader.java index 0f732deb59..07139c21a0 100644 --- a/core/src/test/java/com/netease/arctic/catalog/TestCatalogLoader.java +++ b/core/src/test/java/com/netease/arctic/catalog/TestCatalogLoader.java @@ -22,8 +22,6 @@ import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.ams.api.properties.CatalogMetaProperties; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.TestCatalogUtil; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.Assert; import org.junit.ClassRule; @@ -36,40 +34,6 @@ public class TestCatalogLoader { private static final String TEST_CATALOG_NAME = "test"; @ClassRule public static TestAms TEST_AMS = new TestAms(); - @Test - public void testLoadIcebergHadoopCatalog() { - Map properties = Maps.newHashMap(); - properties.put(CatalogMetaProperties.KEY_WAREHOUSE, "/temp"); - CatalogMeta catalogMeta = - CatalogTestHelpers.buildCatalogMeta( - TEST_CATALOG_NAME, - CatalogMetaProperties.CATALOG_TYPE_HADOOP, - properties, - TableFormat.ICEBERG); - TEST_AMS.getAmsHandler().createCatalog(catalogMeta); - ArcticCatalog loadCatalog = CatalogLoader.load(getCatalogUrl(TEST_CATALOG_NAME)); - Assert.assertEquals(TEST_CATALOG_NAME, loadCatalog.name()); - Assert.assertTrue(loadCatalog instanceof BasicIcebergCatalog); - TEST_AMS.getAmsHandler().dropCatalog(TEST_CATALOG_NAME); - } - - @Test - public void testLoadIcebergCustomCatalog() { - Map properties = Maps.newHashMap(); - properties.put(CatalogProperties.CATALOG_IMPL, TestCatalogUtil.TestCatalog.class.getName()); - CatalogMeta catalogMeta = - CatalogTestHelpers.buildCatalogMeta( - TEST_CATALOG_NAME, - CatalogMetaProperties.CATALOG_TYPE_CUSTOM, - properties, - TableFormat.ICEBERG); - TEST_AMS.getAmsHandler().createCatalog(catalogMeta); - ArcticCatalog loadCatalog = CatalogLoader.load(getCatalogUrl(TEST_CATALOG_NAME)); - Assert.assertEquals(TEST_CATALOG_NAME, loadCatalog.name()); - Assert.assertTrue(loadCatalog instanceof BasicIcebergCatalog); - TEST_AMS.getAmsHandler().dropCatalog(TEST_CATALOG_NAME); - } - @Test public void testLoadMixedIcebergCatalog() { Map properties = Maps.newHashMap(); @@ -105,6 +69,7 @@ public void testLoadCatalogWithErrorFormat() { properties, TableFormat.MIXED_ICEBERG); TEST_AMS.getAmsHandler().createCatalog(catalogMeta); + // lack warehouse Assert.assertThrows( "failed when load catalog test", IllegalStateException.class, diff --git a/core/src/test/java/com/netease/arctic/catalog/TestIcebergCatalog.java b/core/src/test/java/com/netease/arctic/catalog/TestIcebergCatalog.java deleted file mode 100644 index 7dcbc87a70..0000000000 --- a/core/src/test/java/com/netease/arctic/catalog/TestIcebergCatalog.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.catalog; - -import com.netease.arctic.BasicTableTestHelper; -import com.netease.arctic.TableTestHelper; -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.io.RecoverableHadoopFileIO; -import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.table.TableProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.thrift.TException; -import org.junit.After; -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 TestIcebergCatalog extends CatalogTestBase { - - @Parameterized.Parameters(name = "testFormat = {0}") - public static Object[] parameters() { - return new Object[] {new BasicCatalogTestHelper(TableFormat.ICEBERG)}; - } - - public TestIcebergCatalog(CatalogTestHelper catalogTestHelper) { - super(catalogTestHelper); - } - - @Before - public void before() { - if (!getCatalog().listDatabases().contains(TableTestHelper.TEST_DB_NAME)) { - getCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); - } - } - - @Test - public void testLoadIcebergTable() { - createIcebergTable(); - ArcticTable table = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); - Assert.assertTrue(table instanceof IcebergCatalogWrapper.BasicIcebergTable); - Assert.assertTrue(table.isUnkeyedTable()); - Assert.assertEquals(BasicTableTestHelper.TABLE_SCHEMA.asStruct(), table.schema().asStruct()); - } - - @Test - public void testRecoverableFileIO() throws TException { - createIcebergTable(); - ArcticTable table = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); - Assert.assertFalse(table.io() instanceof RecoverableHadoopFileIO); - - CatalogMeta testCatalogMeta = - TEST_AMS.getAmsHandler().getCatalog(CatalogTestHelper.TEST_CATALOG_NAME); - TEST_AMS - .getAmsHandler() - .updateMeta( - testCatalogMeta, - CatalogMetaProperties.TABLE_PROPERTIES_PREFIX + TableProperties.ENABLE_TABLE_TRASH, - "true"); - getCatalog().refresh(); - - table = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); - Assert.assertFalse(table.io() instanceof RecoverableHadoopFileIO); - - getCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); - createIcebergTable(); - table = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); - Assert.assertFalse(table.io() instanceof RecoverableHadoopFileIO); - } - - @After - public void after() { - getCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); - getCatalog().dropDatabase(TableTestHelper.TEST_DB_NAME); - } - - private void createIcebergTable() { - Catalog nativeIcebergCatalog = getIcebergCatalog(); - nativeIcebergCatalog.createTable( - TableIdentifier.of(TableTestHelper.TEST_DB_NAME, TableTestHelper.TEST_TABLE_NAME), - BasicTableTestHelper.TABLE_SCHEMA); - } -} diff --git a/core/src/test/java/com/netease/arctic/catalog/TestMixedCatalog.java b/core/src/test/java/com/netease/arctic/catalog/TestMixedCatalog.java index 61b4dc620b..f67b106a0d 100644 --- a/core/src/test/java/com/netease/arctic/catalog/TestMixedCatalog.java +++ b/core/src/test/java/com/netease/arctic/catalog/TestMixedCatalog.java @@ -59,8 +59,8 @@ public static Object[] parameters() { @Before public void before() { - if (!getCatalog().listDatabases().contains(TableTestHelper.TEST_DB_NAME)) { - getCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); + if (!getMixedFormatCatalog().listDatabases().contains(TableTestHelper.TEST_DB_NAME)) { + getMixedFormatCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); } } @@ -87,21 +87,22 @@ protected void validateCreatedTable(ArcticTable table) throws TException { @Test public void testCreateUnkeyedTable() throws TException { UnkeyedTable createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPartitionSpec(getCreateTableSpec()) .create() .asUnkeyedTable(); validateCreatedTable(createTable); - UnkeyedTable loadTable = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID).asUnkeyedTable(); + UnkeyedTable loadTable = + getMixedFormatCatalog().loadTable(TableTestHelper.TEST_TABLE_ID).asUnkeyedTable(); validateCreatedTable(loadTable); } @Test public void testCreateKeyedTable() throws TException { KeyedTable createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPartitionSpec(getCreateTableSpec()) .withPrimaryKeySpec(BasicTableTestHelper.PRIMARY_KEY_SPEC) @@ -109,14 +110,15 @@ public void testCreateKeyedTable() throws TException { .asKeyedTable(); validateCreatedTable(createTable); - KeyedTable loadTable = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID).asKeyedTable(); + KeyedTable loadTable = + getMixedFormatCatalog().loadTable(TableTestHelper.TEST_TABLE_ID).asKeyedTable(); validateCreatedTable(loadTable); } @Test public void testCreateTableWithNewCatalogProperties() throws TException { UnkeyedTable createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPartitionSpec(getCreateTableSpec()) .create() @@ -126,7 +128,7 @@ public void testCreateTableWithNewCatalogProperties() throws TException { createTable.properties(), TableProperties.ENABLE_SELF_OPTIMIZING, TableProperties.ENABLE_SELF_OPTIMIZING_DEFAULT)); - getCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); + getMixedFormatCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); CatalogMeta testCatalogMeta = TEST_AMS.getAmsHandler().getCatalog(CatalogTestHelper.TEST_CATALOG_NAME); @@ -136,9 +138,9 @@ public void testCreateTableWithNewCatalogProperties() throws TException { testCatalogMeta, CatalogMetaProperties.TABLE_PROPERTIES_PREFIX + TableProperties.ENABLE_SELF_OPTIMIZING, "false"); - getCatalog().refresh(); + refreshMixedFormatCatalog(); createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPartitionSpec(getCreateTableSpec()) .create() @@ -153,7 +155,7 @@ public void testCreateTableWithNewCatalogProperties() throws TException { @Test public void testCreateTableWithNewCatalogLogProperties() throws TException { UnkeyedTable createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPartitionSpec(getCreateTableSpec()) .create() @@ -163,7 +165,7 @@ public void testCreateTableWithNewCatalogLogProperties() throws TException { createTable.properties(), TableProperties.ENABLE_SELF_OPTIMIZING, TableProperties.ENABLE_SELF_OPTIMIZING_DEFAULT)); - getCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); + getMixedFormatCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); CatalogMeta testCatalogMeta = TEST_AMS.getAmsHandler().getCatalog(CatalogTestHelper.TEST_CATALOG_NAME); @@ -185,9 +187,9 @@ public void testCreateTableWithNewCatalogLogProperties() throws TException { testCatalogMeta, CatalogMetaProperties.TABLE_PROPERTIES_PREFIX + TableProperties.ENABLE_SELF_OPTIMIZING, "false"); - getCatalog().refresh(); + refreshMixedFormatCatalog(); createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPartitionSpec(getCreateTableSpec()) .withProperty(TableProperties.ENABLE_LOG_STORE, "true") @@ -203,7 +205,7 @@ public void testCreateTableWithNewCatalogLogProperties() throws TException { @Test public void testUnkeyedRecoverableFileIO() throws TException { UnkeyedTable createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPartitionSpec(getCreateTableSpec()) .create() @@ -218,14 +220,14 @@ public void testUnkeyedRecoverableFileIO() throws TException { testCatalogMeta, CatalogMetaProperties.TABLE_PROPERTIES_PREFIX + TableProperties.ENABLE_TABLE_TRASH, "true"); - getCatalog().refresh(); + refreshMixedFormatCatalog(); - ArcticTable table = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); + ArcticTable table = getMixedFormatCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); assertRecoverableFileIO(table); - getCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); + getMixedFormatCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPartitionSpec(getCreateTableSpec()) .create() @@ -236,7 +238,7 @@ public void testUnkeyedRecoverableFileIO() throws TException { @Test public void testKeyedRecoverableFileIO() throws TException { KeyedTable createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPartitionSpec(getCreateTableSpec()) .withPrimaryKeySpec(BasicTableTestHelper.PRIMARY_KEY_SPEC) @@ -254,16 +256,16 @@ public void testKeyedRecoverableFileIO() throws TException { testCatalogMeta, CatalogMetaProperties.TABLE_PROPERTIES_PREFIX + TableProperties.ENABLE_TABLE_TRASH, "true"); - getCatalog().refresh(); + refreshMixedFormatCatalog(); - ArcticTable table = getCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); + ArcticTable table = getMixedFormatCatalog().loadTable(TableTestHelper.TEST_TABLE_ID); assertRecoverableFileIO(table); assertRecoverableFileIO(table.asKeyedTable().changeTable()); assertRecoverableFileIO(table.asKeyedTable().baseTable()); - getCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); + getMixedFormatCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPartitionSpec(getCreateTableSpec()) .withPrimaryKeySpec(BasicTableTestHelper.PRIMARY_KEY_SPEC) @@ -296,21 +298,22 @@ private void assertRecoverableFileIO(ArcticTable arcticTable) { @Test public void testGetTableBlockerManager() { KeyedTable createTable = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(TableTestHelper.TEST_TABLE_ID, getCreateTableSchema()) .withPrimaryKeySpec(BasicTableTestHelper.PRIMARY_KEY_SPEC) .withPartitionSpec(getCreateTableSpec()) .create() .asKeyedTable(); - TableBlockerManager tableBlockerManager = getCatalog().getTableBlockerManager(createTable.id()); + TableBlockerManager tableBlockerManager = + getMixedFormatCatalog().getTableBlockerManager(createTable.id()); Assert.assertEquals(createTable.id(), tableBlockerManager.tableIdentifier()); Assert.assertTrue(tableBlockerManager.getBlockers().isEmpty()); } @After public void after() { - getCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); - getCatalog().dropDatabase(TableTestHelper.TEST_DB_NAME); + getMixedFormatCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); + getMixedFormatCatalog().dropDatabase(TableTestHelper.TEST_DB_NAME); } protected Schema getCreateTableSchema() { diff --git a/core/src/test/java/com/netease/arctic/formats/AbstractFormatCatalogTestHelper.java b/core/src/test/java/com/netease/arctic/formats/AbstractFormatCatalogTestHelper.java new file mode 100644 index 0000000000..e27bbf9946 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/formats/AbstractFormatCatalogTestHelper.java @@ -0,0 +1,59 @@ +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 org.apache.hadoop.conf.Configuration; + +import java.util.HashMap; +import java.util.Map; + +public abstract class AbstractFormatCatalogTestHelper implements AmoroCatalogTestHelper { + + protected final String catalogName; + + protected final Map catalogProperties; + + protected AbstractFormatCatalogTestHelper( + String catalogName, Map catalogProperties) { + this.catalogName = catalogName; + this.catalogProperties = catalogProperties == null ? new HashMap<>() : catalogProperties; + } + + @Override + public void initWarehouse(String warehouseLocation) { + catalogProperties.put(catalogWarehouseKey(), warehouseLocation); + } + + protected String catalogWarehouseKey() { + return "warehouse"; + } + + protected abstract TableFormat format(); + + protected String getMetastoreType() { + return CatalogMetaProperties.CATALOG_TYPE_HADOOP; + } + + @Override + public void initHiveConf(Configuration hiveConf) {} + + @Override + public CatalogMeta getCatalogMeta() { + return CatalogTestHelpers.buildCatalogMeta( + catalogName, getMetastoreType(), catalogProperties, format()); + } + + @Override + public abstract AmoroCatalog amoroCatalog(); + + @Override + public abstract C originalCatalog(); + + @Override + public String catalogName() { + return catalogName; + } +} diff --git a/core/src/test/java/com/netease/arctic/formats/IcebergHadoopCatalogTestHelper.java b/core/src/test/java/com/netease/arctic/formats/IcebergHadoopCatalogTestHelper.java index c3f219a012..f105391c70 100644 --- a/core/src/test/java/com/netease/arctic/formats/IcebergHadoopCatalogTestHelper.java +++ b/core/src/test/java/com/netease/arctic/formats/IcebergHadoopCatalogTestHelper.java @@ -21,11 +21,10 @@ 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 com.netease.arctic.table.TableMetaStore; +import com.netease.arctic.utils.CatalogUtil; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -38,7 +37,7 @@ import java.util.HashMap; import java.util.Map; -public class IcebergHadoopCatalogTestHelper implements AmoroCatalogTestHelper { +public class IcebergHadoopCatalogTestHelper extends AbstractFormatCatalogTestHelper { public static final Schema schema = new Schema( @@ -57,18 +56,13 @@ public class IcebergHadoopCatalogTestHelper implements AmoroCatalogTestHelper catalogProperties; - public IcebergHadoopCatalogTestHelper(String catalogName, Map catalogProperties) { - this.catalogName = catalogName; - this.catalogProperties = catalogProperties == null ? new HashMap<>() : catalogProperties; + super(catalogName, catalogProperties); } @Override - public void initWarehouse(String warehouseLocation) { - catalogProperties.put(CatalogMetaProperties.KEY_WAREHOUSE, warehouseLocation); + protected TableFormat format() { + return TableFormat.ICEBERG; } @Override @@ -76,28 +70,22 @@ 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(); + TableMetaStore metaStore = CatalogUtil.buildMetaStore(getCatalogMeta()); return icebergCatalogFactory.create( - catalogName, getMetastoreType(), catalogProperties, new Configuration()); + catalogName, getMetastoreType(), catalogProperties, metaStore); } @Override public Catalog originalCatalog() { - return IcebergCatalogFactory.icebergCatalog( - catalogName, getMetastoreType(), catalogProperties, new Configuration()); - } - - @Override - public String catalogName() { - return catalogName; + Map props = + CatalogUtil.withIcebergCatalogInitializeProperties( + catalogName, getMetastoreType(), catalogProperties); + TableMetaStore metaStore = CatalogUtil.buildMetaStore(getCatalogMeta()); + return org.apache.iceberg.CatalogUtil.buildIcebergCatalog( + catalogName, props, metaStore.getConfiguration()); } @Override @@ -139,10 +127,6 @@ public void createTable(String db, String tableName) throws Exception { 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/MixedIcebergHadoopCatalogTestHelper.java b/core/src/test/java/com/netease/arctic/formats/MixedIcebergHadoopCatalogTestHelper.java new file mode 100644 index 0000000000..f22f668b72 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/formats/MixedIcebergHadoopCatalogTestHelper.java @@ -0,0 +1,107 @@ +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.catalog.ArcticCatalog; +import com.netease.arctic.catalog.CatalogLoader; +import com.netease.arctic.formats.mixed.MixedIcebergCatalogFactory; +import com.netease.arctic.table.TableIdentifier; +import com.netease.arctic.table.TableMetaStore; +import com.netease.arctic.utils.CatalogUtil; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; + +import java.util.HashMap; +import java.util.Map; + +public class MixedIcebergHadoopCatalogTestHelper + extends AbstractFormatCatalogTestHelper { + + 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 MixedIcebergHadoopCatalogTestHelper( + String catalogName, Map catalogProperties) { + super(catalogName, catalogProperties); + } + + @Override + protected TableFormat format() { + return TableFormat.MIXED_ICEBERG; + } + + @Override + public AmoroCatalog amoroCatalog() { + MixedIcebergCatalogFactory icebergCatalogFactory = new MixedIcebergCatalogFactory(); + TableMetaStore metaStore = CatalogUtil.buildMetaStore(getCatalogMeta()); + return icebergCatalogFactory.create( + catalogName, getMetastoreType(), catalogProperties, metaStore); + } + + @Override + public ArcticCatalog originalCatalog() { + CatalogMeta meta = getCatalogMeta(); + TableMetaStore metaStore = CatalogUtil.buildMetaStore(meta); + return CatalogLoader.createCatalog( + catalogName(), meta.getCatalogType(), meta.getCatalogProperties(), metaStore); + } + + @Override + public void setTableProperties(String db, String tableName, String key, String value) { + originalCatalog() + .loadTable(TableIdentifier.of(catalogName(), db, tableName)) + .updateProperties() + .set(key, value) + .commit(); + } + + @Override + public void removeTableProperties(String db, String tableName, String key) { + originalCatalog() + .loadTable(TableIdentifier.of(catalogName(), db, tableName)) + .updateProperties() + .remove(key) + .commit(); + } + + @Override + public void clean() { + ArcticCatalog catalog = originalCatalog(); + catalog + .listDatabases() + .forEach( + db -> { + catalog.listTables(db).forEach(id -> catalog.dropTable(id, true)); + try { + catalog.dropDatabase(db); + } catch (Exception e) { + // pass + } + }); + } + + @Override + public void createTable(String db, String tableName) throws Exception { + ArcticCatalog catalog = originalCatalog(); + catalog + .newTableBuilder(TableIdentifier.of(catalogName(), db, tableName), schema) + .withPartitionSpec(spec) + .create(); + } + + public static MixedIcebergHadoopCatalogTestHelper defaultHelper() { + return new MixedIcebergHadoopCatalogTestHelper("test_mixed_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 index 4000e622a0..46418b76c2 100644 --- a/core/src/test/java/com/netease/arctic/formats/PaimonHadoopCatalogTestHelper.java +++ b/core/src/test/java/com/netease/arctic/formats/PaimonHadoopCatalogTestHelper.java @@ -19,12 +19,10 @@ 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 com.netease.arctic.table.TableMetaStore; +import com.netease.arctic.utils.CatalogUtil; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.options.CatalogOptions; @@ -35,7 +33,7 @@ import java.util.HashMap; import java.util.Map; -public class PaimonHadoopCatalogTestHelper implements AmoroCatalogTestHelper { +public class PaimonHadoopCatalogTestHelper extends AbstractFormatCatalogTestHelper { public static final Schema schema = Schema.newBuilder() @@ -47,13 +45,8 @@ public class PaimonHadoopCatalogTestHelper implements AmoroCatalogTestHelper catalogProperties; - public PaimonHadoopCatalogTestHelper(String catalogName, Map catalogProperties) { - this.catalogName = catalogName; - this.catalogProperties = catalogProperties == null ? new HashMap<>() : catalogProperties; + super(catalogName, catalogProperties); } public void initWarehouse(String warehouseLocation) { @@ -61,21 +54,16 @@ public void initWarehouse(String warehouseLocation) { } @Override - public void initHiveConf(Configuration hiveConf) { - // Do nothing - } - - @Override - public CatalogMeta getCatalogMeta() { - return CatalogTestHelpers.buildCatalogMeta( - catalogName, getMetastoreType(), catalogProperties, TableFormat.PAIMON); + protected TableFormat format() { + return TableFormat.PAIMON; } @Override public AmoroCatalog amoroCatalog() { PaimonCatalogFactory paimonCatalogFactory = new PaimonCatalogFactory(); + TableMetaStore metaStore = CatalogUtil.buildMetaStore(getCatalogMeta()); return paimonCatalogFactory.create( - catalogName, getMetastoreType(), catalogProperties, new Configuration()); + catalogName, getMetastoreType(), catalogProperties, metaStore); } @Override @@ -83,11 +71,6 @@ public Catalog originalCatalog() { return PaimonCatalogFactory.paimonCatalog(getMetastoreType(), catalogProperties, null); } - @Override - public String catalogName() { - return catalogName; - } - @Override public void setTableProperties(String db, String tableName, String key, String value) { try { @@ -138,10 +121,6 @@ public void createTable(String db, String tableName) throws Exception { } } - 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/TestMixedIcebergFormatCatalog.java b/core/src/test/java/com/netease/arctic/formats/TestMixedIcebergFormatCatalog.java new file mode 100644 index 0000000000..c052996732 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/formats/TestMixedIcebergFormatCatalog.java @@ -0,0 +1,68 @@ +/* + * 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.catalog.ArcticCatalog; +import com.netease.arctic.table.TableIdentifier; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.List; +import java.util.Map; + +@RunWith(Parameterized.class) +public class TestMixedIcebergFormatCatalog extends TestAmoroCatalogBase { + + public TestMixedIcebergFormatCatalog(AmoroCatalogTestHelper amoroCatalogTestHelper) { + super(amoroCatalogTestHelper); + } + + @Parameterized.Parameters(name = "{0}") + public static Object[] parameters() { + return new Object[] {MixedIcebergHadoopCatalogTestHelper.defaultHelper()}; + } + + @Override + protected void createDatabase(String dbName) { + catalog().createDatabase(dbName); + } + + @Override + protected void createTable(String dbName, String tableName, Map properties) { + Schema schema = + new Schema( + Types.NestedField.of(1, false, "id", Types.IntegerType.get()), + Types.NestedField.of(2, false, "data", Types.StringType.get())); + catalog() + .newTableBuilder(TableIdentifier.of(catalog().name(), dbName, tableName), schema) + .withProperties(properties) + .create(); + } + + @Override + protected List listDatabases() { + return catalog().listDatabases(); + } + + private ArcticCatalog catalog() { + return (ArcticCatalog) originalCatalog; + } +} diff --git a/core/src/test/java/com/netease/arctic/io/TestBasicTableTrashManager.java b/core/src/test/java/com/netease/arctic/io/TestBasicTableTrashManager.java index a0618b8166..152bbc3967 100644 --- a/core/src/test/java/com/netease/arctic/io/TestBasicTableTrashManager.java +++ b/core/src/test/java/com/netease/arctic/io/TestBasicTableTrashManager.java @@ -259,7 +259,7 @@ public void testDeleteTrashLocation() throws IOException { Assert.assertTrue(tableTrashManager.fileExistInTrash(file1)); String trashParentLocation = TableTrashManagers.getTrashParentLocation(getArcticTable().id(), customTrashLocation); - getCatalog().dropTable(getArcticTable().id(), true); + getMixedFormatCatalog().dropTable(getArcticTable().id(), true); Assert.assertFalse(getArcticTable().io().exists(trashParentLocation)); Assert.assertFalse(tableTrashManager.fileExistInTrash(file1)); } diff --git a/core/src/test/java/com/netease/arctic/op/TestOverwriteBaseFile.java b/core/src/test/java/com/netease/arctic/op/TestOverwriteBaseFile.java index 634f5ca54e..10658334df 100644 --- a/core/src/test/java/com/netease/arctic/op/TestOverwriteBaseFile.java +++ b/core/src/test/java/com/netease/arctic/op/TestOverwriteBaseFile.java @@ -20,7 +20,7 @@ import com.netease.arctic.io.MixedDataTestHelpers; import com.netease.arctic.io.TableDataTestBase; -import com.netease.arctic.utils.TablePropertyUtil; +import com.netease.arctic.utils.ArcticTableUtil; import org.apache.iceberg.DataFile; import org.apache.iceberg.data.Record; import org.apache.iceberg.expressions.Expressions; @@ -58,7 +58,7 @@ public void testOverwriteAllPartition() { // overwrite all partition and add new data file StructLikeMap partitionOptimizedSequence = - TablePropertyUtil.getPartitionOptimizedSequence(getArcticTable().asKeyedTable()); + ArcticTableUtil.readOptimizedSequence(getArcticTable().asKeyedTable()); // expect result: all partition with new txId Assert.assertEquals( txId, @@ -82,7 +82,7 @@ public void testOverwriteAllPartition() { .longValue()); StructLikeMap partitionOptimizedTime = - TablePropertyUtil.getPartitionBaseOptimizedTime(getArcticTable().asKeyedTable()); + ArcticTableUtil.readBaseOptimizedTime(getArcticTable().asKeyedTable()); // expect result: all partition with new optimized time assertRange( before, @@ -145,7 +145,7 @@ public void testOverwritePartitionByExpression() { long after = System.currentTimeMillis(); StructLikeMap partitionOptimizedSequence = - TablePropertyUtil.getPartitionOptimizedSequence(getArcticTable().asKeyedTable()); + ArcticTableUtil.readOptimizedSequence(getArcticTable().asKeyedTable()); // expect result: 1,2,4 partition with new txId, 3 partition is null Assert.assertEquals( txId, @@ -167,7 +167,7 @@ public void testOverwritePartitionByExpression() { .longValue()); StructLikeMap partitionOptimizedTime = - TablePropertyUtil.getPartitionBaseOptimizedTime(getArcticTable().asKeyedTable()); + ArcticTableUtil.readBaseOptimizedTime(getArcticTable().asKeyedTable()); // expect result: 1,2,4 partition with new optimized time, 3 partition is null assertRange( before, diff --git a/core/src/test/java/com/netease/arctic/op/TestRewritePartitions.java b/core/src/test/java/com/netease/arctic/op/TestRewritePartitions.java index 9bed9132a5..8253646749 100644 --- a/core/src/test/java/com/netease/arctic/op/TestRewritePartitions.java +++ b/core/src/test/java/com/netease/arctic/op/TestRewritePartitions.java @@ -20,7 +20,7 @@ import com.netease.arctic.io.MixedDataTestHelpers; import com.netease.arctic.io.TableDataTestBase; -import com.netease.arctic.utils.TablePropertyUtil; +import com.netease.arctic.utils.ArcticTableUtil; import org.apache.iceberg.DataFile; import org.apache.iceberg.data.Record; import org.apache.iceberg.expressions.Expressions; @@ -54,7 +54,7 @@ public void testDynamicOverwritePartition() { // rewrite 1 partition by data file StructLikeMap partitionOptimizedSequence = - TablePropertyUtil.getPartitionOptimizedSequence(getArcticTable().asKeyedTable()); + ArcticTableUtil.readOptimizedSequence(getArcticTable().asKeyedTable()); // expect result: 1 partition with new txId, 2,3 partition use old txId Assert.assertEquals( txId, diff --git a/core/src/test/java/com/netease/arctic/scan/TestMixedChangeTableScan.java b/core/src/test/java/com/netease/arctic/scan/TestMixedChangeTableScan.java index b0739a9128..9653ed4ec2 100644 --- a/core/src/test/java/com/netease/arctic/scan/TestMixedChangeTableScan.java +++ b/core/src/test/java/com/netease/arctic/scan/TestMixedChangeTableScan.java @@ -18,7 +18,6 @@ package com.netease.arctic.scan; -import com.netease.arctic.data.FileNameRules; import com.netease.arctic.io.TableDataTestBase; import com.netease.arctic.utils.ArcticDataFiles; import org.apache.iceberg.FileScanTask; @@ -93,49 +92,6 @@ public void testIncrementalScanFromTo() throws IOException { } } - @Test - public void testIgnoreLegacyTxId() throws IOException { - StructLikeMap fromSequence = - StructLikeMap.create(getArcticTable().spec().partitionType()); - StructLike partitionData = - ArcticDataFiles.data(getArcticTable().spec(), "op_time_day=2022-01-01"); - fromSequence.put(partitionData, 1L); - StructLikeMap fromLegacyTxId = - StructLikeMap.create(getArcticTable().spec().partitionType()); - StructLike partitionData1 = - ArcticDataFiles.data(getArcticTable().spec(), "op_time_day=2022-01-01"); - fromLegacyTxId.put(partitionData1, 100L); - ChangeTableIncrementalScan changeTableIncrementalScan = - getArcticTable() - .asKeyedTable() - .changeTable() - .newScan() - .fromSequence(fromSequence) - .fromLegacyTransaction(fromLegacyTxId); - try (CloseableIterable tasks = changeTableIncrementalScan.planFiles()) { - assertFilesSequence(tasks, 1, 2, 2); - } - } - - @Test - public void testUseLegacyId() throws IOException { - StructLikeMap fromLegacyTxId = - StructLikeMap.create(getArcticTable().spec().partitionType()); - StructLike partitionData1 = - ArcticDataFiles.data(getArcticTable().spec(), "op_time_day=2022-01-01"); - fromLegacyTxId.put(partitionData1, 2L); - ChangeTableIncrementalScan changeTableIncrementalScan = - getArcticTable() - .asKeyedTable() - .changeTable() - .newScan() - .fromLegacyTransaction(fromLegacyTxId); - try (CloseableIterable tasks = changeTableIncrementalScan.planFiles()) { - assertFiles( - tasks, 1, task -> FileNameRules.parseTransactionId(task.file().path().toString()) > 2L); - } - } - private void assertFiles( CloseableIterable tasks, int fileCnt, Predicate validator) { int taskCount = 0; diff --git a/core/src/test/java/com/netease/arctic/table/blocker/TestBasicTableBlockerManager.java b/core/src/test/java/com/netease/arctic/table/blocker/TestBasicTableBlockerManager.java index 11fe9b368f..cd88eb87a7 100644 --- a/core/src/test/java/com/netease/arctic/table/blocker/TestBasicTableBlockerManager.java +++ b/core/src/test/java/com/netease/arctic/table/blocker/TestBasicTableBlockerManager.java @@ -31,7 +31,7 @@ public TestBasicTableBlockerManager() { @Test public void testBlockAndRelease() throws OperationConflictException { TableBlockerManager tableBlockerManager = - getCatalog().getTableBlockerManager(TableTestHelper.TEST_TABLE_ID); + getMixedFormatCatalog().getTableBlockerManager(TableTestHelper.TEST_TABLE_ID); Assert.assertTrue(tableBlockerManager instanceof BasicTableBlockerManager); BasicTableBlockerManager blockerManager = (BasicTableBlockerManager) tableBlockerManager; diff --git a/core/src/test/java/com/netease/arctic/utils/TestCatalogUtil.java b/core/src/test/java/com/netease/arctic/utils/TestCatalogUtil.java index a6f0295e8a..e47bf356f4 100644 --- a/core/src/test/java/com/netease/arctic/utils/TestCatalogUtil.java +++ b/core/src/test/java/com/netease/arctic/utils/TestCatalogUtil.java @@ -1,5 +1,9 @@ package com.netease.arctic.utils; +import com.netease.arctic.ams.api.properties.CatalogMetaProperties; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.RESTCatalog; import org.junit.Assert; import org.junit.Test; @@ -247,4 +251,46 @@ public void testMergeCatalogPropertiesToTable7() { CatalogUtil.mergeCatalogPropertiesToTable(userDefined, catalogProperties); Assert.assertEquals(expected, result); } + + @Test + public void testWithIcebergCatalogInitializeProperties() { + Map props; + final String name = "test"; + final String typeHadoop = "hadoop"; + final String typeCustom = "custom"; + final String typeAms = CatalogMetaProperties.CATALOG_TYPE_AMS; + final String type = "type"; + final String keyWarehouse = CatalogProperties.WAREHOUSE_LOCATION; + final String path = "hdfs://test-cluster/warehouse"; + final String restImpl = RESTCatalog.class.getName(); + + // hive catalog + props = + CatalogUtil.withIcebergCatalogInitializeProperties( + name, typeHadoop, ImmutableMap.of(keyWarehouse, path)); + Assert.assertEquals(typeHadoop, props.get(type)); + + // custom + props = + CatalogUtil.withIcebergCatalogInitializeProperties( + name, + typeCustom, + ImmutableMap.of(keyWarehouse, path, CatalogProperties.CATALOG_IMPL, restImpl)); + Assert.assertFalse(props.containsKey(type)); + // custom args check + Assert.assertThrows( + IllegalArgumentException.class, + () -> { + CatalogUtil.withIcebergCatalogInitializeProperties( + name, typeCustom, ImmutableMap.of(keyWarehouse, path)); + }); + + // ams + props = + CatalogUtil.withIcebergCatalogInitializeProperties( + name, typeAms, ImmutableMap.of(keyWarehouse, path)); + Assert.assertEquals(name, props.get(keyWarehouse)); + Assert.assertFalse(props.containsKey(type)); + Assert.assertEquals(restImpl, props.get(CatalogProperties.CATALOG_IMPL)); + } } diff --git a/core/src/test/java/com/netease/arctic/utils/TestStatisticsFileUtil.java b/core/src/test/java/com/netease/arctic/utils/TestStatisticsFileUtil.java new file mode 100644 index 0000000000..d319a39cb9 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/utils/TestStatisticsFileUtil.java @@ -0,0 +1,156 @@ +/* + * 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.utils; + +import com.netease.arctic.BasicTableTestHelper; +import com.netease.arctic.TableTestHelper; +import com.netease.arctic.ams.api.TableFormat; +import com.netease.arctic.catalog.BasicCatalogTestHelper; +import com.netease.arctic.catalog.CatalogTestHelper; +import com.netease.arctic.catalog.TableTestBase; +import com.netease.arctic.io.MixedDataTestHelpers; +import com.netease.arctic.table.UnkeyedTable; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.StructLikeMap; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.List; + +@RunWith(Parameterized.class) +public class TestStatisticsFileUtil extends TableTestBase { + + @Parameterized.Parameters(name = "{0}, {1}") + public static Object[] parameters() { + return new Object[][] { + {new BasicCatalogTestHelper(TableFormat.ICEBERG), new BasicTableTestHelper(false, true)}, + {new BasicCatalogTestHelper(TableFormat.ICEBERG), new BasicTableTestHelper(false, false)}, + {new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), new BasicTableTestHelper(true, true)}, + {new BasicCatalogTestHelper(TableFormat.MIXED_ICEBERG), new BasicTableTestHelper(true, false)} + }; + } + + public TestStatisticsFileUtil( + CatalogTestHelper catalogTestHelper, TableTestHelper tableTestHelper) { + super(catalogTestHelper, tableTestHelper); + } + + @Test + public void testWriteAndReadPuffin() { + UnkeyedTable table = + getArcticTable().isKeyedTable() + ? getArcticTable().asKeyedTable().baseTable() + : getArcticTable().asUnkeyedTable(); + table + .newAppend() + .set(ArcticTableUtil.BLOB_TYPE_BASE_OPTIMIZED_TIME_EXIST, "true") + .set(ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE_EXIST, "true") + .commit(); + + Snapshot snapshot = table.currentSnapshot(); + StructLikeMap optimizedTime = buildPartitionOptimizedTime(); + StructLikeMap optimizedSequence = buildPartitionOptimizedSequence(); + + StatisticsFileUtil.PartitionDataSerializer dataSerializer = + StatisticsFileUtil.createPartitionDataSerializer(table.spec(), Long.class); + StatisticsFileUtil.Writer writer = + StatisticsFileUtil.writerBuilder(table) + .withSnapshotId(snapshot.snapshotId()) + .build() + .add(ArcticTableUtil.BLOB_TYPE_BASE_OPTIMIZED_TIME, optimizedTime, dataSerializer) + .add(ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE, optimizedSequence, dataSerializer); + StatisticsFile statisticsFile = writer.complete(); + table.updateStatistics().setStatistics(snapshot.snapshotId(), statisticsFile).commit(); + + assertStructLikeEquals( + optimizedTime, readPartitionData(table, ArcticTableUtil.BLOB_TYPE_BASE_OPTIMIZED_TIME)); + assertStructLikeEquals( + optimizedSequence, readPartitionData(table, ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE)); + + table.newAppend().commit(); + + assertStructLikeEquals( + optimizedTime, readPartitionData(table, ArcticTableUtil.BLOB_TYPE_BASE_OPTIMIZED_TIME)); + assertStructLikeEquals( + optimizedSequence, readPartitionData(table, ArcticTableUtil.BLOB_TYPE_OPTIMIZED_SEQUENCE)); + } + + private StatisticsFile findValidStatisticFile(Table table, String type) { + Snapshot latestValidSnapshot = + ArcticTableUtil.findLatestValidSnapshot( + table, table.currentSnapshot().snapshotId(), ArcticTableUtil.isTypeExist(type)); + Preconditions.checkState(latestValidSnapshot != null, "Expect one valid snapshot"); + List statisticsFiles = + StatisticsFileUtil.getStatisticsFiles(table, latestValidSnapshot.snapshotId(), type); + Preconditions.checkArgument(statisticsFiles.size() == 1, "Expect one valid statistics file"); + return statisticsFiles.get(0); + } + + private StructLikeMap readPartitionData(Table table, String type) { + StatisticsFileUtil.PartitionDataSerializer dataSerializer = + StatisticsFileUtil.createPartitionDataSerializer(table.spec(), Long.class); + List> result = + StatisticsFileUtil.reader(table) + .read(findValidStatisticFile(table, type), type, dataSerializer); + Assert.assertEquals(1, result.size()); + return result.get(0); + } + + private void assertStructLikeEquals(StructLikeMap expected, StructLikeMap actual) { + Assert.assertEquals(expected.size(), actual.size()); + for (StructLike structLike : expected.keySet()) { + Assert.assertEquals(expected.get(structLike), actual.get(structLike)); + } + } + + private StructLikeMap buildPartitionOptimizedSequence() { + PartitionSpec spec = getArcticTable().spec(); + StructLikeMap result = StructLikeMap.create(spec.partitionType()); + if (spec.isUnpartitioned()) { + result.put(TablePropertyUtil.EMPTY_STRUCT, 1L); + } else { + StructLike partition1 = MixedDataTestHelpers.recordPartition("2022-01-01T12:00:00"); + StructLike partition2 = MixedDataTestHelpers.recordPartition("2022-01-01T12:00:00"); + result.put(partition1, 1L); + result.put(partition2, 2L); + } + return result; + } + + private StructLikeMap buildPartitionOptimizedTime() { + PartitionSpec spec = getArcticTable().spec(); + StructLikeMap result = StructLikeMap.create(spec.partitionType()); + if (spec.isUnpartitioned()) { + result.put(TablePropertyUtil.EMPTY_STRUCT, 1000L); + } else { + StructLike partition1 = MixedDataTestHelpers.recordPartition("2022-01-01T12:00:00"); + StructLike partition2 = MixedDataTestHelpers.recordPartition("2022-01-01T12:00:00"); + result.put(partition1, 1000L); + result.put(partition2, 2000L); + } + return result; + } +} diff --git a/docs/formats/overview.md b/docs/formats/overview.md index 31b097071f..40bf21e143 100644 --- a/docs/formats/overview.md +++ b/docs/formats/overview.md @@ -23,9 +23,9 @@ In addition, new table formats such as [Iceberg](https://Iceberg.apache.org/) al For users, the design goal of Amoro is to provide an out-of-the-box data lake system. Internally, Amoro's design philosophy is to use different table formats as storage engines for data lakes. This design pattern is more common in open-source systems such as MySQL and ClickHouse. -Currently, Amoro mainly provides the following three table formats: +Currently, Amoro mainly provides the following four table formats: - **Iceberg format:** Users can directly entrust their Iceberg tables to Amoro for maintenance, so that users can not only use all the functions of Iceberg tables, but also enjoy the performance and stability improvements brought by Amoro. - **Mixed-Iceberg format:** Amoro provides a set of more optimized formats for streaming update scenarios on top of the Iceberg format. If users have high performance requirements for streaming updates or have demands for CDC incremental data reading functions, they can choose to use the Mixed-Iceberg format. - **Mixed-Hive format:** Many users do not want to affect the business originally built on Hive while using data lakes. Therefore, Amoro provides the Mixed-Hive format, which can upgrade Hive tables to Mixed-Hive format only through metadata migration, and the original Hive tables can still be used normally. This ensures business stability and benefits from the advantages of data lake computing. -- **Paimon format:** Amoro supports displaying metadata information in the Paimon format, including Schema, Options, Files, Snapshots, DDLs, and Compaction information. \ No newline at end of file +- **Paimon format:** Amoro supports displaying metadata information in the Paimon format, including Schema, Options, Files, Snapshots, DDLs, and Compaction information. diff --git a/flink/flink-common/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java b/flink/flink-common/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java index ace526864b..6c3d22d1f2 100644 --- a/flink/flink-common/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java +++ b/flink/flink-common/src/test/java/com/netease/arctic/flink/read/TestArcticSource.java @@ -127,7 +127,7 @@ public class TestArcticSource extends TestRowDataReaderFunction implements Seria @Before public void testSetup() throws IOException { - ArcticCatalog testCatalog = getCatalog(); + ArcticCatalog testCatalog = getMixedFormatCatalog(); String db = FAIL_TABLE_ID.getDatabase(); if (!testCatalog.listDatabases().contains(db)) { @@ -148,9 +148,9 @@ public void testSetup() throws IOException { @After public void dropTable() { miniClusterResource.cancelAllJobs(); - getCatalog().dropTable(FAIL_TABLE_ID, true); - getCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); - getCatalog().dropDatabase(TableTestHelper.TEST_DB_NAME); + getMixedFormatCatalog().dropTable(FAIL_TABLE_ID, true); + getMixedFormatCatalog().dropTable(TableTestHelper.TEST_TABLE_ID, true); + getMixedFormatCatalog().dropDatabase(TableTestHelper.TEST_DB_NAME); } @Test @@ -296,7 +296,7 @@ public void testArcticContinuousSourceWithEmptyChangeInInit() throws Exception { TableIdentifier tableId = TableIdentifier.of(TEST_CATALOG_NAME, TEST_DB_NAME, "test_empty_change"); KeyedTable table = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(tableId, TABLE_SCHEMA) .withPartitionSpec(BasicTableTestHelper.SPEC) .withPrimaryKeySpec(BasicTableTestHelper.PRIMARY_KEY_SPEC) @@ -365,7 +365,7 @@ public void testArcticContinuousSourceWithEmptyChangeInInit() throws Exception { jobClient.cancel(); Assert.assertEquals(new HashSet<>(updateRecords()), new HashSet<>(actualResult)); - getCatalog().dropTable(tableId, true); + getMixedFormatCatalog().dropTable(tableId, true); } @Test @@ -373,7 +373,7 @@ public void testArcticSourceEnumeratorWithChangeExpired() throws Exception { final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; TableIdentifier tableId = TableIdentifier.of(TEST_CATALOG_NAME, TEST_DB_NAME, "test_keyed_tb"); KeyedTable table = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(tableId, TABLE_SCHEMA) .withProperty(MAX_CONTINUOUS_EMPTY_COMMITS, "1") .withPrimaryKeySpec(BasicTableTestHelper.PRIMARY_KEY_SPEC) @@ -467,7 +467,7 @@ public void testArcticSourceEnumeratorWithChangeExpired() throws Exception { jobClient.cancel(); Assert.assertEquals(new HashSet<>(updateRecords()), new HashSet<>(actualResult)); - getCatalog().dropTable(tableId, true); + getMixedFormatCatalog().dropTable(tableId, true); } @Test @@ -475,7 +475,7 @@ public void testArcticSourceEnumeratorWithBaseExpired() throws Exception { final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; TableIdentifier tableId = TableIdentifier.of(TEST_CATALOG_NAME, TEST_DB_NAME, "test_keyed_tb"); KeyedTable table = - getCatalog() + getMixedFormatCatalog() .newTableBuilder(tableId, TABLE_SCHEMA) .withProperty(MAX_CONTINUOUS_EMPTY_COMMITS, "1") .withPrimaryKeySpec(BasicTableTestHelper.PRIMARY_KEY_SPEC) @@ -569,7 +569,7 @@ public void testArcticSourceEnumeratorWithBaseExpired() throws Exception { jobClient.cancel(); Assert.assertEquals(new HashSet<>(updateRecords()), new HashSet<>(actualResult)); - getCatalog().dropTable(tableId, true); + getMixedFormatCatalog().dropTable(tableId, true); } @Test diff --git a/flink/flink-common/src/test/java/com/netease/arctic/flink/read/hybrid/reader/MixedIncrementalLoaderTest.java b/flink/flink-common/src/test/java/com/netease/arctic/flink/read/hybrid/reader/MixedIncrementalLoaderTest.java index ff8ad1603e..ccdcb15f26 100644 --- a/flink/flink-common/src/test/java/com/netease/arctic/flink/read/hybrid/reader/MixedIncrementalLoaderTest.java +++ b/flink/flink-common/src/test/java/com/netease/arctic/flink/read/hybrid/reader/MixedIncrementalLoaderTest.java @@ -167,6 +167,6 @@ public String getMetastoreUrl() { @Override public String getCatalogName() { - return getCatalog().name(); + return getMixedFormatCatalog().name(); } } diff --git a/flink/flink-common/src/test/java/com/netease/arctic/flink/table/LookupITCase.java b/flink/flink-common/src/test/java/com/netease/arctic/flink/table/LookupITCase.java index 21bd0599dd..baa16858f3 100644 --- a/flink/flink-common/src/test/java/com/netease/arctic/flink/table/LookupITCase.java +++ b/flink/flink-common/src/test/java/com/netease/arctic/flink/table/LookupITCase.java @@ -56,10 +56,10 @@ public LookupITCase() { @Before public void setup() throws IOException { - List dbs = getCatalog().listDatabases(); + List dbs = getMixedFormatCatalog().listDatabases(); if (dbs.isEmpty()) { db = "test_db"; - getCatalog().createDatabase(db); + getMixedFormatCatalog().createDatabase(db); } else { db = dbs.get(0); } @@ -145,7 +145,7 @@ public String getMetastoreUrl() { @Override public String getCatalogName() { - return getCatalog().name(); + return getMixedFormatCatalog().name(); } @Override @@ -168,7 +168,7 @@ private void writeAndCommit( boolean writeToBaseStore, boolean upsertEnabled) throws IOException { - ArcticTable arcticTable = getCatalog().loadTable(table); + ArcticTable arcticTable = getMixedFormatCatalog().loadTable(table); Assert.assertNotNull(arcticTable); RowType rowType = FlinkSchemaUtil.convert(arcticTable.schema()); for (RowData rowData : expected) { diff --git a/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestJoin.java b/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestJoin.java index fbfb8bb903..c5fa7e1c82 100644 --- a/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestJoin.java +++ b/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestJoin.java @@ -88,7 +88,7 @@ public void before() throws Exception { @After public void after() { - getCatalog().dropTable(TABLE_ID, true); + getMixedFormatCatalog().dropTable(TABLE_ID, true); } @Test diff --git a/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestLookupSecondary.java b/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestLookupSecondary.java index dcd0a5fb20..1eb25fba24 100644 --- a/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestLookupSecondary.java +++ b/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestLookupSecondary.java @@ -59,10 +59,10 @@ public TestLookupSecondary() { @Before public void setup() throws IOException { - List dbs = getCatalog().listDatabases(); + List dbs = getMixedFormatCatalog().listDatabases(); if (dbs.isEmpty()) { db = "test_db"; - getCatalog().createDatabase(db); + getMixedFormatCatalog().createDatabase(db); } else { db = dbs.get(0); } @@ -147,7 +147,7 @@ public String getMetastoreUrl() { @Override public String getCatalogName() { - return getCatalog().name(); + return getMixedFormatCatalog().name(); } @Override @@ -170,7 +170,7 @@ private void writeAndCommit( boolean writeToBaseStore, boolean upsertEnabled) throws IOException { - ArcticTable arcticTable = getCatalog().loadTable(table); + ArcticTable arcticTable = getMixedFormatCatalog().loadTable(table); Assert.assertNotNull(arcticTable); RowType rowType = FlinkSchemaUtil.convert(arcticTable.schema()); for (RowData rowData : expected) { diff --git a/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestUnkeyed.java b/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestUnkeyed.java index 01d0f8f60d..e710ad47fa 100644 --- a/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestUnkeyed.java +++ b/flink/flink-common/src/test/java/com/netease/arctic/flink/table/TestUnkeyed.java @@ -137,7 +137,7 @@ public void before() throws Exception { db = DB; } super.before(); - arcticCatalog = getCatalog(); + arcticCatalog = getMixedFormatCatalog(); topic = String.join(".", catalog, db, TABLE); super.config(); } diff --git a/hive/src/main/java/com/netease/arctic/hive/catalog/ArcticHiveCatalog.java b/hive/src/main/java/com/netease/arctic/hive/catalog/ArcticHiveCatalog.java index cc4a96de00..828ddc94ca 100644 --- a/hive/src/main/java/com/netease/arctic/hive/catalog/ArcticHiveCatalog.java +++ b/hive/src/main/java/com/netease/arctic/hive/catalog/ArcticHiveCatalog.java @@ -18,8 +18,6 @@ package com.netease.arctic.hive.catalog; -import com.netease.arctic.AmsClient; -import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.ams.api.TableMeta; import com.netease.arctic.catalog.BasicArcticCatalog; @@ -32,6 +30,7 @@ import com.netease.arctic.table.PrimaryKeySpec; import com.netease.arctic.table.TableBuilder; import com.netease.arctic.table.TableIdentifier; +import com.netease.arctic.table.TableMetaStore; import com.netease.arctic.table.TableProperties; import com.netease.arctic.utils.ConvertStructUtil; import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; @@ -61,14 +60,15 @@ public class ArcticHiveCatalog extends BasicArcticCatalog { private CachedHiveClientPool hiveClientPool; @Override - public void initialize(AmsClient client, CatalogMeta meta, Map properties) { - super.initialize(client, meta, properties); + public void initialize(String name, Map properties, TableMetaStore metaStore) { + super.initialize(name, properties, metaStore); this.hiveClientPool = ((MixedHiveTables) tables).getHiveClientPool(); } @Override - protected MixedTables newMixedTables(CatalogMeta catalogMeta) { - return new MixedHiveTables(catalogMeta); + protected MixedTables newMixedTables( + Map catalogProperties, TableMetaStore metaStore) { + return new MixedHiveTables(catalogProperties, metaStore); } @Override diff --git a/hive/src/main/java/com/netease/arctic/hive/catalog/MixedHiveTables.java b/hive/src/main/java/com/netease/arctic/hive/catalog/MixedHiveTables.java index 88a843897d..1568fca3b7 100644 --- a/hive/src/main/java/com/netease/arctic/hive/catalog/MixedHiveTables.java +++ b/hive/src/main/java/com/netease/arctic/hive/catalog/MixedHiveTables.java @@ -1,6 +1,5 @@ package com.netease.arctic.hive.catalog; -import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableMeta; import com.netease.arctic.ams.api.properties.MetaTableProperties; import com.netease.arctic.catalog.MixedTables; @@ -17,6 +16,7 @@ import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.PrimaryKeySpec; import com.netease.arctic.table.TableIdentifier; +import com.netease.arctic.table.TableMetaStore; import com.netease.arctic.table.TableProperties; import com.netease.arctic.utils.CatalogUtil; import org.apache.hadoop.hive.metastore.TableType; @@ -39,10 +39,9 @@ public class MixedHiveTables extends MixedTables { private volatile CachedHiveClientPool hiveClientPool; - public MixedHiveTables(CatalogMeta catalogMeta) { - super(catalogMeta); - this.hiveClientPool = - new CachedHiveClientPool(getTableMetaStore(), catalogMeta.getCatalogProperties()); + public MixedHiveTables(Map catalogProperties, TableMetaStore metaStore) { + super(catalogProperties, metaStore); + this.hiveClientPool = new CachedHiveClientPool(getTableMetaStore(), catalogProperties); } public CachedHiveClientPool getHiveClientPool() { @@ -62,7 +61,7 @@ protected KeyedHiveTable loadKeyedTable(TableMeta tableMeta) { tableLocation, tableMeta.getProperties(), tableMetaStore, - catalogMeta.getCatalogProperties()); + catalogProperties); checkPrivilege(fileIO, baseLocation); Table baseIcebergTable = tableMetaStore.doAs(() -> tables.load(baseLocation)); UnkeyedHiveTable baseTable = @@ -73,7 +72,7 @@ protected KeyedHiveTable loadKeyedTable(TableMeta tableMeta) { fileIO, tableLocation, hiveClientPool, - catalogMeta.getCatalogProperties(), + catalogProperties, false); Table changeIcebergTable = tableMetaStore.doAs(() -> tables.load(changeLocation)); @@ -83,7 +82,7 @@ protected KeyedHiveTable loadKeyedTable(TableMeta tableMeta) { CatalogUtil.useArcticTableOperations( changeIcebergTable, changeLocation, fileIO, tableMetaStore.getConfiguration()), fileIO, - catalogMeta.getCatalogProperties()); + catalogProperties); return new KeyedHiveTable( tableMeta, tableLocation, @@ -114,7 +113,7 @@ protected UnkeyedHiveTable loadUnKeyedTable(TableMeta tableMeta) { tableLocation, tableMeta.getProperties(), tableMetaStore, - catalogMeta.getCatalogProperties()); + catalogProperties); checkPrivilege(fileIO, baseLocation); Table table = tableMetaStore.doAs(() -> tables.load(baseLocation)); return new UnkeyedHiveTable( @@ -124,7 +123,7 @@ protected UnkeyedHiveTable loadUnKeyedTable(TableMeta tableMeta) { fileIO, tableLocation, hiveClientPool, - catalogMeta.getCatalogProperties()); + catalogProperties); } @Override @@ -152,7 +151,7 @@ protected KeyedTable createKeyedTable( tableLocation, tableMeta.getProperties(), tableMetaStore, - catalogMeta.getCatalogProperties()); + catalogProperties); Table baseIcebergTable = tableMetaStore.doAs( () -> { @@ -178,7 +177,7 @@ protected KeyedTable createKeyedTable( fileIO, tableLocation, hiveClientPool, - catalogMeta.getCatalogProperties(), + catalogProperties, false); Table changeIcebergTable = @@ -204,7 +203,7 @@ protected KeyedTable createKeyedTable( CatalogUtil.useArcticTableOperations( changeIcebergTable, changeLocation, fileIO, tableMetaStore.getConfiguration()), fileIO, - catalogMeta.getCatalogProperties()); + catalogProperties); Map metaProperties = tableMeta.getProperties(); try { @@ -318,7 +317,7 @@ protected UnkeyedHiveTable createUnKeyedTable( tableLocation, tableMeta.getProperties(), tableMetaStore, - catalogMeta.getCatalogProperties()); + catalogProperties); return new UnkeyedHiveTable( tableIdentifier, CatalogUtil.useArcticTableOperations( @@ -326,7 +325,7 @@ protected UnkeyedHiveTable createUnKeyedTable( fileIO, tableLocation, hiveClientPool, - catalogMeta.getCatalogProperties()); + catalogProperties); } @Override @@ -433,11 +432,4 @@ private boolean allowExistedHiveTable(TableMeta tableMeta) { tableMeta.getProperties().remove(HiveTableProperties.ALLOW_HIVE_TABLE_EXISTED); return Boolean.parseBoolean(allowStringValue); } - - @Override - public void refreshCatalogMeta(CatalogMeta meta) { - super.refreshCatalogMeta(meta); - this.hiveClientPool = - new CachedHiveClientPool(getTableMetaStore(), catalogMeta.getCatalogProperties()); - } } diff --git a/hive/src/main/java/com/netease/arctic/hive/op/HiveOperationTransaction.java b/hive/src/main/java/com/netease/arctic/hive/op/HiveOperationTransaction.java index 6aad211ad9..5dc37d7b4b 100644 --- a/hive/src/main/java/com/netease/arctic/hive/op/HiveOperationTransaction.java +++ b/hive/src/main/java/com/netease/arctic/hive/op/HiveOperationTransaction.java @@ -45,6 +45,7 @@ import org.apache.iceberg.UpdatePartitionSpec; import org.apache.iceberg.UpdateProperties; import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.UpdateStatistics; import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; @@ -154,6 +155,16 @@ public void commitTransaction() { transactionalClient.commit(); } + @Override + public UpdateStatistics updateStatistics() { + return wrapped.updateStatistics(); + } + + @Override + public ManageSnapshots manageSnapshots() { + return wrapped.manageSnapshots(); + } + private class TransactionalHMSClient implements HMSClientPool { List> pendingActions = Lists.newArrayList(); diff --git a/hive/src/test/java/com/netease/arctic/hive/catalog/HiveCatalogTestHelper.java b/hive/src/test/java/com/netease/arctic/hive/catalog/HiveCatalogTestHelper.java index da64069e3d..3113691a56 100644 --- a/hive/src/test/java/com/netease/arctic/hive/catalog/HiveCatalogTestHelper.java +++ b/hive/src/test/java/com/netease/arctic/hive/catalog/HiveCatalogTestHelper.java @@ -21,12 +21,15 @@ import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE; +import com.netease.arctic.CommonUnifiedCatalog; +import com.netease.arctic.UnifiedCatalog; 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.CatalogTestHelper; import com.netease.arctic.catalog.CatalogTestHelpers; import com.netease.arctic.catalog.MixedTables; +import com.netease.arctic.utils.CatalogUtil; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.CatalogProperties; @@ -76,6 +79,11 @@ public CatalogMeta buildCatalogMeta(String baseDir) { TEST_CATALOG_NAME, properties, hiveConf, tableFormat); } + @Override + public UnifiedCatalog buildUnifiedCatalog(CatalogMeta catalogMeta) { + return new CommonUnifiedCatalog(() -> catalogMeta, Maps.newHashMap()); + } + @Override public Catalog buildIcebergCatalog(CatalogMeta catalogMeta) { Map catalogProperties = Maps.newHashMap(catalogMeta.getCatalogProperties()); @@ -90,7 +98,8 @@ public MixedTables buildMixedTables(CatalogMeta catalogMeta) { throw new UnsupportedOperationException( "Cannot build mixed-tables for table format:" + tableFormat); } - return new MixedHiveTables(catalogMeta); + return new MixedHiveTables( + catalogMeta.getCatalogProperties(), CatalogUtil.buildMetaStore(catalogMeta)); } @Override diff --git a/hive/src/test/java/com/netease/arctic/hive/catalog/TestArcticHiveCatalog.java b/hive/src/test/java/com/netease/arctic/hive/catalog/TestArcticHiveCatalog.java index 9af17c0834..b5903d80e9 100644 --- a/hive/src/test/java/com/netease/arctic/hive/catalog/TestArcticHiveCatalog.java +++ b/hive/src/test/java/com/netease/arctic/hive/catalog/TestArcticHiveCatalog.java @@ -44,21 +44,18 @@ public TestArcticHiveCatalog(CatalogTestHelper catalogTestHelper) { @Parameterized.Parameters(name = "testFormat = {0}") public static Object[] parameters() { - return new Object[] { - new HiveCatalogTestHelper(TableFormat.MIXED_HIVE, TEST_HMS.getHiveConf()), - new HiveCatalogTestHelper(TableFormat.ICEBERG, TEST_HMS.getHiveConf()) - }; + return new Object[] {new HiveCatalogTestHelper(TableFormat.MIXED_HIVE, TEST_HMS.getHiveConf())}; } @Test public void testDropTableButNotDropHiveTable() throws MetaException { - if (getCatalog() instanceof ArcticHiveCatalog) { - getCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); + if (getMixedFormatCatalog() instanceof ArcticHiveCatalog) { + getMixedFormatCatalog().createDatabase(TableTestHelper.TEST_DB_NAME); createTestTable(); - getCatalog() + getMixedFormatCatalog() .dropTable( TableIdentifier.of( - getCatalog().name(), + getMixedFormatCatalog().name(), TableTestHelper.TEST_DB_NAME, TableTestHelper.TEST_TABLE_NAME), false); @@ -72,7 +69,7 @@ public void testDropTableButNotDropHiveTable() throws MetaException { @After public void cleanUp() throws TException { - if (getCatalog() instanceof ArcticHiveCatalog) { + if (getMixedFormatCatalog() instanceof ArcticHiveCatalog) { TEST_HMS .getHiveClient() .dropTable(TableTestHelper.TEST_DB_NAME, TableTestHelper.TEST_TABLE_NAME, true, true); diff --git a/hive/src/test/java/com/netease/arctic/hive/formats/MixedIcebergHiveCatalogTestHelper.java b/hive/src/test/java/com/netease/arctic/hive/formats/MixedIcebergHiveCatalogTestHelper.java new file mode 100644 index 0000000000..e1c9bc84d3 --- /dev/null +++ b/hive/src/test/java/com/netease/arctic/hive/formats/MixedIcebergHiveCatalogTestHelper.java @@ -0,0 +1,51 @@ +/* + * 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.MixedIcebergHadoopCatalogTestHelper; +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 MixedIcebergHiveCatalogTestHelper extends MixedIcebergHadoopCatalogTestHelper { + + public MixedIcebergHiveCatalogTestHelper( + String catalogName, Map catalogProperties) { + super(catalogName, catalogProperties); + } + + @Override + public void initHiveConf(Configuration hiveConf) { + catalogProperties.put( + CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); + } + + protected String getMetastoreType() { + return CatalogMetaProperties.CATALOG_TYPE_HIVE; + } + + public static MixedIcebergHiveCatalogTestHelper defaultHelper() { + return new MixedIcebergHiveCatalogTestHelper( + "test_mixed_iceberg_hive_catalog", new HashMap<>()); + } +} diff --git a/hive/src/test/java/com/netease/arctic/hive/catalog/TestIcebergHiveCatalog.java b/hive/src/test/java/com/netease/arctic/hive/formats/TestMixedIcebergHiveAmoroCatalog.java similarity index 58% rename from hive/src/test/java/com/netease/arctic/hive/catalog/TestIcebergHiveCatalog.java rename to hive/src/test/java/com/netease/arctic/hive/formats/TestMixedIcebergHiveAmoroCatalog.java index f039bff769..150c0577ba 100644 --- a/hive/src/test/java/com/netease/arctic/hive/catalog/TestIcebergHiveCatalog.java +++ b/hive/src/test/java/com/netease/arctic/hive/formats/TestMixedIcebergHiveAmoroCatalog.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,27 +16,34 @@ * limitations under the License. */ -package com.netease.arctic.hive.catalog; +package com.netease.arctic.hive.formats; -import com.netease.arctic.ams.api.TableFormat; -import com.netease.arctic.catalog.CatalogTestHelper; -import com.netease.arctic.catalog.TestIcebergCatalog; +import com.netease.arctic.formats.AmoroCatalogTestHelper; +import com.netease.arctic.formats.TestMixedIcebergFormatCatalog; 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 TestIcebergHiveCatalog extends TestIcebergCatalog { +public class TestMixedIcebergHiveAmoroCatalog extends TestMixedIcebergFormatCatalog { @ClassRule public static TestHMS TEST_HMS = new TestHMS(); - public TestIcebergHiveCatalog(CatalogTestHelper catalogTestHelper) { - super(catalogTestHelper); + public TestMixedIcebergHiveAmoroCatalog(AmoroCatalogTestHelper amoroCatalogTestHelper) { + super(amoroCatalogTestHelper); } - @Parameterized.Parameters(name = "testFormat = {0}") + @Parameterized.Parameters(name = "{0}") public static Object[] parameters() { - return new Object[] {new HiveCatalogTestHelper(TableFormat.ICEBERG, TEST_HMS.getHiveConf())}; + return new Object[] {MixedIcebergHiveCatalogTestHelper.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/utils/TestUpgradeHiveTableUtil.java b/hive/src/test/java/com/netease/arctic/hive/utils/TestUpgradeHiveTableUtil.java index fe305ca1f1..4c8b85d381 100644 --- a/hive/src/test/java/com/netease/arctic/hive/utils/TestUpgradeHiveTableUtil.java +++ b/hive/src/test/java/com/netease/arctic/hive/utils/TestUpgradeHiveTableUtil.java @@ -97,12 +97,12 @@ public void setUp() throws TException, IOException { if (isPartitioned) { createPartition(); } - identifier = TableIdentifier.of(getCatalog().name(), db, table); + identifier = TableIdentifier.of(getMixedFormatCatalog().name(), db, table); } @After public void dropTable() throws TException { - getCatalog().dropTable(identifier, true); + getMixedFormatCatalog().dropTable(identifier, true); TEST_HMS.getHiveClient().dropDatabase(db); } @@ -135,8 +135,11 @@ public static Object[] parameters() { @Test public void upgradeHiveTable() throws Exception { UpgradeHiveTableUtil.upgradeHiveTable( - (ArcticHiveCatalog) getCatalog(), identifier, new ArrayList<>(), new HashMap<>()); - ArcticTable table = getCatalog().loadTable(identifier); + (ArcticHiveCatalog) getMixedFormatCatalog(), + identifier, + new ArrayList<>(), + new HashMap<>()); + ArcticTable table = getMixedFormatCatalog().loadTable(identifier); UnkeyedHiveTable baseTable = table.isKeyedTable() ? (UnkeyedHiveTable) table.asKeyedTable().baseTable() @@ -144,7 +147,7 @@ public void upgradeHiveTable() throws Exception { if (table.spec().isPartitioned()) { List partitions = HivePartitionUtil.getHiveAllPartitions( - ((ArcticHiveCatalog) getCatalog()).getHMSClient(), table.id()); + ((ArcticHiveCatalog) getMixedFormatCatalog()).getHMSClient(), table.id()); for (Partition partition : partitions) { StructLike partitionData = HivePartitionUtil.buildPartitionData(partition.getValues(), table.spec()); diff --git a/spark/v3.1/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java b/spark/v3.1/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java index 9b8e148fae..54e228b6f5 100644 --- a/spark/v3.1/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java +++ b/spark/v3.1/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java @@ -81,6 +81,7 @@ public class ArcticSparkCatalog implements TableCatalog, SupportsNamespaces { private String catalogName = null; private ArcticCatalog catalog; + private CaseInsensitiveStringMap options; /** * Build an Arctic {@link TableIdentifier} for the given Spark identifier. @@ -114,7 +115,7 @@ protected TableIdentifier buildInnerTableIdentifier(Identifier identifier) { @Override public Table loadTable(Identifier ident) throws NoSuchTableException { - checkAndRefreshCatalogMeta(catalog); + checkAndRefreshCatalogMeta(); TableIdentifier identifier; ArcticTable table; try { @@ -152,7 +153,7 @@ private boolean isInnerTableIdentifier(Identifier identifier) { public Table createTable( Identifier ident, StructType schema, Transform[] transforms, Map properties) throws TableAlreadyExistsException { - checkAndRefreshCatalogMeta(catalog); + checkAndRefreshCatalogMeta(); properties = Maps.newHashMap(properties); Schema finalSchema = checkAndConvertSchema(schema, properties); TableIdentifier identifier = buildIdentifier(ident); @@ -183,13 +184,13 @@ && isIdentifierLocation(properties.get(TableCatalog.PROP_LOCATION), ident)) { } } - private void checkAndRefreshCatalogMeta(ArcticCatalog catalog) { + private void checkAndRefreshCatalogMeta() { SparkSession sparkSession = SparkSession.active(); if (Boolean.parseBoolean( sparkSession .conf() .get(REFRESH_CATALOG_BEFORE_USAGE, REFRESH_CATALOG_BEFORE_USAGE_DEFAULT))) { - catalog.refresh(); + initialize(catalogName, options); } } @@ -413,6 +414,7 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { Preconditions.checkArgument( StringUtils.isNotBlank(catalogUrl), "lack required properties: url"); catalog = CatalogLoader.load(catalogUrl, options); + this.options = options; } @Override diff --git a/spark/v3.2/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java b/spark/v3.2/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java index 2d88e8038b..bcc4a0d9a2 100644 --- a/spark/v3.2/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java +++ b/spark/v3.2/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java @@ -81,6 +81,7 @@ public class ArcticSparkCatalog implements TableCatalog, SupportsNamespaces { private String catalogName = null; private ArcticCatalog catalog; + private CaseInsensitiveStringMap options; /** * Build an Arctic {@link com.netease.arctic.table.TableIdentifier} for the given Spark @@ -122,7 +123,7 @@ protected TableIdentifier buildInnerTableIdentifier(Identifier identifier) { @Override public Table loadTable(Identifier ident) throws NoSuchTableException { - checkAndRefreshCatalogMeta(catalog); + checkAndRefreshCatalogMeta(); TableIdentifier identifier; ArcticTable table; try { @@ -166,7 +167,7 @@ private boolean isInnerTableIdentifier(Identifier identifier) { public Table createTable( Identifier ident, StructType schema, Transform[] transforms, Map properties) throws TableAlreadyExistsException { - checkAndRefreshCatalogMeta(catalog); + checkAndRefreshCatalogMeta(); properties = Maps.newHashMap(properties); Schema finalSchema = checkAndConvertSchema(schema, properties); TableIdentifier identifier = buildIdentifier(ident); @@ -197,13 +198,13 @@ && isIdentifierLocation(properties.get(TableCatalog.PROP_LOCATION), ident)) { } } - private void checkAndRefreshCatalogMeta(ArcticCatalog catalog) { + private void checkAndRefreshCatalogMeta() { SparkSession sparkSession = SparkSession.active(); if (Boolean.parseBoolean( sparkSession .conf() .get(REFRESH_CATALOG_BEFORE_USAGE, REFRESH_CATALOG_BEFORE_USAGE_DEFAULT))) { - catalog.refresh(); + initialize(catalogName, options); } } @@ -428,7 +429,8 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { if (StringUtils.isBlank(catalogUrl)) { throw new IllegalArgumentException("lack required properties: url"); } - catalog = CatalogLoader.load(catalogUrl, Maps.newHashMap()); + catalog = CatalogLoader.load(catalogUrl, options); + this.options = options; } @Override diff --git a/spark/v3.3/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java b/spark/v3.3/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java index bc85c9e8e5..0ac920784b 100644 --- a/spark/v3.3/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java +++ b/spark/v3.3/spark/src/main/java/com/netease/arctic/spark/ArcticSparkCatalog.java @@ -82,6 +82,7 @@ public class ArcticSparkCatalog implements TableCatalog, SupportsNamespaces { private String catalogName = null; private ArcticCatalog catalog; + private CaseInsensitiveStringMap options; /** * Build an Arctic {@link TableIdentifier} for the given Spark identifier. @@ -122,7 +123,7 @@ protected TableIdentifier buildInnerTableIdentifier(Identifier identifier) { @Override public Table loadTable(Identifier ident) throws NoSuchTableException { - checkAndRefreshCatalogMeta(catalog); + checkAndRefreshCatalogMeta(); TableIdentifier identifier; ArcticTable table; try { @@ -166,7 +167,7 @@ private boolean isInnerTableIdentifier(Identifier identifier) { public Table createTable( Identifier ident, StructType schema, Transform[] transforms, Map properties) throws TableAlreadyExistsException { - checkAndRefreshCatalogMeta(catalog); + checkAndRefreshCatalogMeta(); properties = Maps.newHashMap(properties); Schema finalSchema = checkAndConvertSchema(schema, properties); TableIdentifier identifier = buildIdentifier(ident); @@ -197,13 +198,13 @@ && isIdentifierLocation(properties.get(TableCatalog.PROP_LOCATION), ident)) { } } - private void checkAndRefreshCatalogMeta(ArcticCatalog catalog) { + private void checkAndRefreshCatalogMeta() { SparkSession sparkSession = SparkSession.active(); if (Boolean.parseBoolean( sparkSession .conf() .get(REFRESH_CATALOG_BEFORE_USAGE, REFRESH_CATALOG_BEFORE_USAGE_DEFAULT))) { - catalog.refresh(); + initialize(catalogName, options); } } @@ -428,7 +429,8 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { if (StringUtils.isBlank(catalogUrl)) { throw new IllegalArgumentException("lack required properties: url"); } - catalog = CatalogLoader.load(catalogUrl, Maps.newHashMap()); + catalog = CatalogLoader.load(catalogUrl, options); + this.options = options; } @Override diff --git a/trino/src/main/java/com/netease/arctic/trino/ArcticCatalogSupportTableSuffix.java b/trino/src/main/java/com/netease/arctic/trino/ArcticCatalogSupportTableSuffix.java index 7759bf25bd..82aa332958 100644 --- a/trino/src/main/java/com/netease/arctic/trino/ArcticCatalogSupportTableSuffix.java +++ b/trino/src/main/java/com/netease/arctic/trino/ArcticCatalogSupportTableSuffix.java @@ -18,8 +18,6 @@ package com.netease.arctic.trino; -import com.netease.arctic.AmsClient; -import com.netease.arctic.ams.api.CatalogMeta; import com.netease.arctic.ams.api.TableFormat; import com.netease.arctic.catalog.ArcticCatalog; import com.netease.arctic.io.ArcticFileIO; @@ -32,6 +30,7 @@ import com.netease.arctic.table.MetadataColumns; import com.netease.arctic.table.TableBuilder; import com.netease.arctic.table.TableIdentifier; +import com.netease.arctic.table.TableMetaStore; import com.netease.arctic.table.blocker.TableBlockerManager; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DeleteFiles; @@ -84,8 +83,8 @@ public String name() { } @Override - public void initialize(AmsClient client, CatalogMeta meta, Map properties) { - arcticCatalog.initialize(client, meta, properties); + public void initialize(String name, Map properties, TableMetaStore metaStore) { + arcticCatalog.initialize(name, properties, metaStore); } @Override @@ -150,11 +149,6 @@ public TableBuilder newTableBuilder(TableIdentifier identifier, Schema schema) { return arcticCatalog.newTableBuilder(identifier, schema); } - @Override - public void refresh() { - arcticCatalog.refresh(); - } - @Override public TableBlockerManager getTableBlockerManager(TableIdentifier tableIdentifier) { return arcticCatalog.getTableBlockerManager(tableIdentifier);