okumin commented on code in PR #5771: URL: https://github.com/apache/hive/pull/5771#discussion_r2186943232
########## standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/client/HookEnabledMetaStoreClientProxy.java: ########## @@ -0,0 +1,812 @@ +/* + * 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 org.apache.hadoop.hive.metastore.client; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.DefaultHiveMetaHook; +import org.apache.hadoop.hive.metastore.DefaultMetaStoreFilterHookImpl; +import org.apache.hadoop.hive.metastore.HiveMetaHook; +import org.apache.hadoop.hive.metastore.HiveMetaHookLoader; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetaStoreFilterHook; +import org.apache.hadoop.hive.metastore.PartitionDropOptions; +import org.apache.hadoop.hive.metastore.TableIterable; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.Catalog; +import org.apache.hadoop.hive.metastore.api.CreateTableRequest; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.DropDatabaseRequest; +import org.apache.hadoop.hive.metastore.api.EnvironmentContext; +import org.apache.hadoop.hive.metastore.api.GetDatabaseObjectsRequest; +import org.apache.hadoop.hive.metastore.api.GetDatabaseObjectsResponse; +import org.apache.hadoop.hive.metastore.api.GetLatestCommittedCompactionInfoRequest; +import org.apache.hadoop.hive.metastore.api.GetLatestCommittedCompactionInfoResponse; +import org.apache.hadoop.hive.metastore.api.GetPartitionNamesPsRequest; +import org.apache.hadoop.hive.metastore.api.GetPartitionNamesPsResponse; +import org.apache.hadoop.hive.metastore.api.GetPartitionRequest; +import org.apache.hadoop.hive.metastore.api.GetPartitionResponse; +import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest; +import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesResult; +import org.apache.hadoop.hive.metastore.api.GetPartitionsPsWithAuthRequest; +import org.apache.hadoop.hive.metastore.api.GetPartitionsPsWithAuthResponse; +import org.apache.hadoop.hive.metastore.api.GetProjectionsSpec; +import org.apache.hadoop.hive.metastore.api.GetTableRequest; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.PartitionSpec; +import org.apache.hadoop.hive.metastore.api.PartitionValuesRequest; +import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse; +import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest; +import org.apache.hadoop.hive.metastore.api.PartitionsRequest; +import org.apache.hadoop.hive.metastore.api.PartitionsResponse; +import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint; +import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint; +import org.apache.hadoop.hive.metastore.api.SQLForeignKey; +import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint; +import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey; +import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint; +import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; +import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.TableMeta; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; +import org.apache.hadoop.hive.metastore.utils.FilterUtils; +import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.hadoop.hive.common.AcidConstants.SOFT_DELETE_TABLE; +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; + +public class HookEnabledMetaStoreClientProxy extends BaseMetaStoreClientProxy { Review Comment: Let's decouple non-clearly answerable problems from core issues 👍 . I believe this PR is already one of the most significant achievements in 2025, provided we can correctly restructure the class dependencies. https://issues.apache.org/jira/browse/HIVE-29070 ########## standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java: ########## @@ -761,7 +761,7 @@ public void otherCatalog() throws TException { Assert.assertFalse("Table not exists", client.tableExists(catName, dbName, "non_existing_table")); } - @Test(expected = UnknownDBException.class) + @Test(expected = MetaException.class) Review Comment: We may keep this exception because some features catch `UnknownDBException`. I've not looked into how it is critical. ########## standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/client/BaseMetaStoreClientProxy.java: ########## @@ -0,0 +1,1525 @@ +/* + * 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 org.apache.hadoop.hive.metastore.client; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.common.ValidTxnList; +import org.apache.hadoop.hive.common.ValidWriteIdList; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.PartitionDropOptions; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.hadoop.hive.metastore.api.Package; +import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; +import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; +import org.apache.thrift.TException; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; + +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; + +public abstract class BaseMetaStoreClientProxy extends BaseMetaStoreClient { + + protected final IMetaStoreClient delegate; + + public BaseMetaStoreClientProxy(IMetaStoreClient delegate, Configuration conf) { + super(conf); + this.delegate = delegate; + } + + @Override + public boolean isCompatibleWith(Configuration conf) { + return delegate.isCompatibleWith(conf); + } + + @Override + public void setHiveAddedJars(String addedJars) { + delegate.setHiveAddedJars(addedJars); + } + + @Override + public boolean isLocalMetaStore() { + return delegate.isLocalMetaStore(); + } + + @Override + public void reconnect() throws MetaException { + delegate.reconnect(); + } + + @Override + public void close() { + delegate.close(); + } + + @Override + public void setMetaConf(String key, String value) throws MetaException, TException { + delegate.setMetaConf(key, value); + } + + @Override + public String getMetaConf(String key) throws MetaException, TException { + return delegate.getMetaConf(key); + } + + @Override + public void createCatalog(Catalog catalog) + throws AlreadyExistsException, InvalidObjectException, MetaException, TException { + delegate.createCatalog(catalog); + } + + @Override + public void alterCatalog(String catalogName, Catalog newCatalog) + throws NoSuchObjectException, InvalidObjectException, MetaException, TException { + delegate.alterCatalog(catalogName, newCatalog); + } + + @Override + public Catalog getCatalog(String catName) throws NoSuchObjectException, MetaException, TException { + return delegate.getCatalog(catName); + } + + @Override + public List<String> getCatalogs() throws MetaException, TException { + return delegate.getCatalogs(); + } + + @Override + public void dropCatalog(String catName, boolean ifExists) throws TException { + delegate.dropCatalog(catName, ifExists); + } + + @Override + public List<String> getDatabases(String catName, String databasePattern) throws MetaException, TException { + return delegate.getDatabases(catName, databasePattern); + } + + @Override + public List<String> getAllDatabases(String catName) throws MetaException, TException { + return delegate.getAllDatabases(catName); + } + + @Override + public List<String> getTables(String catName, String dbName, String tablePattern) + throws MetaException, TException, UnknownDBException { + return delegate.getTables(catName, dbName, tablePattern); + } + + @Override + public List<String> getTables(String catName, String dbName, String tablePattern, TableType tableType) + throws MetaException, TException, UnknownDBException { + return delegate.getTables(catName, dbName, tablePattern, tableType); + } + + @Override + public List<Table> getAllMaterializedViewObjectsForRewriting() + throws MetaException, TException, UnknownDBException { + return delegate.getAllMaterializedViewObjectsForRewriting(); + } + + @Override + public List<ExtendedTableInfo> getTablesExt(String catName, String dbName, String tablePattern, + int requestedFields, int limit) throws MetaException, TException { + return delegate.getTablesExt(catName, dbName, tablePattern, requestedFields, limit); + } + + @Override + public List<String> getMaterializedViewsForRewriting(String catName, String dbName) + throws MetaException, TException, UnknownDBException { + return delegate.getMaterializedViewsForRewriting(catName, dbName); + } + + @Override + public List<TableMeta> getTableMeta(String catName, String dbPatterns, String tablePatterns, + List<String> tableTypes) throws MetaException, TException, UnknownDBException { + return delegate.getTableMeta(catName, dbPatterns, tablePatterns, tableTypes); + } + + @Override + public List<String> getAllTables(String catName, String dbName) + throws MetaException, TException, UnknownDBException { + return delegate.getAllTables(catName, dbName); + } + + @Override + public List<String> listTableNamesByFilter(String catName, String dbName, String filter, int maxTables) + throws TException, InvalidOperationException, UnknownDBException { + return delegate.listTableNamesByFilter(catName, dbName, filter, maxTables); + } + + @Override + public void dropTable(Table table, boolean deleteData, boolean ignoreUnknownTab, boolean ifPurge) + throws TException { + delegate.dropTable(table, deleteData, ignoreUnknownTab, ifPurge); + } + + @Override + public void truncateTable(String catName, String dbName, String tableName, String ref, + List<String> partNames, String validWriteIds, long writeId, boolean deleteData, + EnvironmentContext context) throws TException { + delegate.truncateTable(catName, dbName, tableName, ref, partNames, validWriteIds, writeId, deleteData, + context); + } + + @Override + public CmRecycleResponse recycleDirToCmPath(CmRecycleRequest request) throws MetaException, TException { + return delegate.recycleDirToCmPath(request); + } + + @Override + public boolean tableExists(String catName, String dbName, String tableName) + throws MetaException, TException, UnknownDBException { + return delegate.tableExists(catName, dbName, tableName); + } + + @Override + public Database getDatabase(String catalogName, String databaseName) + throws NoSuchObjectException, MetaException, TException { + return delegate.getDatabase(catalogName, databaseName); + } + + @Override + public Table getTable(GetTableRequest getTableRequest) + throws MetaException, TException, NoSuchObjectException { + return delegate.getTable(getTableRequest); + } + + @Override + public List<Table> getTables(String catName, String dbName, List<String> tableNames, + GetProjectionsSpec projectionsSpec) + throws MetaException, InvalidOperationException, UnknownDBException, TException { + return delegate.getTables(catName, dbName, tableNames, projectionsSpec); + } + + @Override + public Materialization getMaterializationInvalidationInfo(CreationMetadata cm, String validTxnList) + throws MetaException, InvalidOperationException, UnknownDBException, TException { + return delegate.getMaterializationInvalidationInfo(cm, validTxnList); + } + + @Override + public void updateCreationMetadata(String catName, String dbName, String tableName, CreationMetadata cm) + throws MetaException, TException { + delegate.updateCreationMetadata(catName, dbName, tableName, cm); + } + + @Override + public Partition appendPartition(String catName, String dbName, String tableName, List<String> partVals) + throws InvalidObjectException, AlreadyExistsException, MetaException, TException { + return delegate.appendPartition(catName, dbName, tableName, partVals); + } + + @Override + public Partition appendPartition(String catName, String dbName, String tableName, String name) + throws InvalidObjectException, AlreadyExistsException, MetaException, TException { + return delegate.appendPartition(catName, dbName, tableName, name); + } + + @Override + public Partition add_partition(Partition partition) + throws InvalidObjectException, AlreadyExistsException, MetaException, TException { + return delegate.add_partition(partition); + } + + @Override + public int add_partitions_pspec(PartitionSpecProxy partitionSpec) + throws InvalidObjectException, AlreadyExistsException, MetaException, TException { + return delegate.add_partitions_pspec(partitionSpec); + } + + @Override + public List<Partition> add_partitions(List<Partition> partitions, boolean ifNotExists, boolean needResults) + throws InvalidObjectException, AlreadyExistsException, MetaException, TException { + return delegate.add_partitions(partitions, ifNotExists, needResults); + } + + @Override + public GetPartitionResponse getPartitionRequest(GetPartitionRequest req) + throws NoSuchObjectException, MetaException, TException { + return delegate.getPartitionRequest(req); + } + + @Override + public Partition getPartition(String catName, String dbName, String tblName, List<String> partVals) + throws NoSuchObjectException, MetaException, TException { + return delegate.getPartition(catName, dbName, tblName, partVals); + } + + @Override + public Partition exchange_partition(Map<String, String> partitionSpecs, String sourceCat, String sourceDb, + String sourceTable, String destCat, String destdb, String destTableName) + throws MetaException, NoSuchObjectException, InvalidObjectException, TException { + return delegate.exchange_partition(partitionSpecs, sourceCat, sourceDb, sourceTable, destCat, destdb, + destTableName); + } + + @Override + public List<Partition> exchange_partitions(Map<String, String> partitionSpecs, String sourceCat, + String sourceDb, String sourceTable, String destCat, String destdb, String destTableName) + throws MetaException, NoSuchObjectException, InvalidObjectException, TException { + return delegate.exchange_partitions(partitionSpecs, sourceCat, sourceDb, sourceTable, destCat, destdb, + destTableName); + } + + @Override + public Partition getPartition(String catName, String dbName, String tblName, String name) + throws MetaException, UnknownTableException, NoSuchObjectException, TException { + return delegate.getPartition(catName, dbName, tblName, name); + } + + @Override + public Partition getPartitionWithAuthInfo(String catName, String dbName, String tableName, + List<String> pvals, String userName, List<String> groupNames) + throws MetaException, UnknownTableException, NoSuchObjectException, TException { + return delegate.getPartitionWithAuthInfo(catName, dbName, tableName, pvals, userName, groupNames); + } + + @Override + public List<Partition> listPartitions(String catName, String db_name, String tbl_name, int max_parts) + throws NoSuchObjectException, MetaException, TException { + return delegate.listPartitions(catName, db_name, tbl_name, max_parts); + } + + @Override + public PartitionSpecProxy listPartitionSpecs(String catName, String dbName, String tableName, int maxParts) + throws TException { + return delegate.listPartitionSpecs(catName, dbName, tableName, maxParts); + } + + @Override + public List<Partition> listPartitions(String catName, String db_name, String tbl_name, + List<String> part_vals, int max_parts) throws NoSuchObjectException, MetaException, TException { + return delegate.listPartitions(catName, db_name, tbl_name, part_vals, max_parts); + } + + @Override + public List<String> listPartitionNames(String catName, String db_name, String tbl_name, int max_parts) + throws NoSuchObjectException, MetaException, TException { + // cf. This uses fetch_partition_names_req, not get_partition_names_req. + return delegate.listPartitionNames(catName, db_name, tbl_name, max_parts); + } + + @Override + public List<String> listPartitionNames(PartitionsByExprRequest request) + throws MetaException, TException, NoSuchObjectException { + return delegate.listPartitionNames(request); + } + + @Override + public GetPartitionNamesPsResponse listPartitionNamesRequest(GetPartitionNamesPsRequest req) + throws NoSuchObjectException, MetaException, TException { + // each proxy should modify validWriteIdList. + return delegate.listPartitionNamesRequest(req); + } + + @Override + public PartitionValuesResponse listPartitionValues(PartitionValuesRequest request) + throws MetaException, TException, NoSuchObjectException { + return delegate.listPartitionValues(request); + } + + @Override + public int getNumPartitionsByFilter(String catName, String dbName, String tableName, String filter) + throws MetaException, NoSuchObjectException, TException { + return delegate.getNumPartitionsByFilter(catName, dbName, tableName, filter); + } + + @Override + public List<Partition> listPartitionsByFilter(String catName, String db_name, String tbl_name, + String filter, int max_parts) throws MetaException, NoSuchObjectException, TException { + return delegate.listPartitionsByFilter(catName, db_name, tbl_name, filter, max_parts); + } + + @Override + public PartitionSpecProxy listPartitionSpecsByFilter(String catName, String db_name, String tbl_name, + String filter, int max_parts) throws MetaException, NoSuchObjectException, TException { + return delegate.listPartitionSpecsByFilter(catName, db_name, tbl_name, filter, max_parts); + } + + @Override + public boolean listPartitionsSpecByExpr(PartitionsByExprRequest req, List<PartitionSpec> result) + throws TException { + return delegate.listPartitionsSpecByExpr(req, result); + } + + @Override + public boolean listPartitionsByExpr(PartitionsByExprRequest req, List<Partition> result) throws TException { + return delegate.listPartitionsByExpr(req, result); + } + + @Override + public GetPartitionsPsWithAuthResponse listPartitionsWithAuthInfoRequest( + GetPartitionsPsWithAuthRequest req) throws MetaException, TException, NoSuchObjectException { + return delegate.listPartitionsWithAuthInfoRequest(req); + } + + @Override + public List<Partition> listPartitionsWithAuthInfo(String catName, String dbName, String tableName, + int maxParts, String userName, List<String> groupNames) + throws MetaException, TException, NoSuchObjectException { + return delegate.listPartitionsWithAuthInfo(catName, dbName, tableName, maxParts, userName, groupNames); + } + + @Override + public PartitionsResponse getPartitionsRequest(PartitionsRequest req) + throws NoSuchObjectException, MetaException, TException { + return delegate.getPartitionsRequest(req); + } + + @Override + public GetPartitionsByNamesResult getPartitionsByNames(GetPartitionsByNamesRequest req) throws TException { + return delegate.getPartitionsByNames(req); + } + + @Override + public List<Partition> listPartitionsWithAuthInfo(String catName, String dbName, String tableName, + List<String> partialPvals, int maxParts, String userName, List<String> groupNames) + throws MetaException, TException, NoSuchObjectException { + return delegate.listPartitionsWithAuthInfo(catName, dbName, tableName, partialPvals, maxParts, userName, + groupNames); + } + + @Override + public void markPartitionForEvent(String catName, String db_name, String tbl_name, + Map<String, String> partKVs, PartitionEventType eventType) throws MetaException, NoSuchObjectException, + TException, UnknownTableException, UnknownDBException, UnknownPartitionException, + InvalidPartitionException { + delegate.markPartitionForEvent(catName, db_name, tbl_name, partKVs, eventType); + } + + @Override + public boolean isPartitionMarkedForEvent(String catName, String db_name, String tbl_name, + Map<String, String> partKVs, PartitionEventType eventType) throws MetaException, NoSuchObjectException, + TException, UnknownTableException, UnknownDBException, UnknownPartitionException, + InvalidPartitionException { + return delegate.isPartitionMarkedForEvent(catName, db_name, tbl_name, partKVs, eventType); + } + + @Override + public void validatePartitionNameCharacters(List<String> partVals) throws TException, MetaException { + delegate.validatePartitionNameCharacters(partVals); + } + + @Override + public Table getTranslateTableDryrun(Table tbl) throws AlreadyExistsException, InvalidObjectException, + MetaException, NoSuchObjectException, TException { + return delegate.getTranslateTableDryrun(tbl); + } + + @Override + public void createTable(CreateTableRequest request) throws AlreadyExistsException, InvalidObjectException, + MetaException, NoSuchObjectException, TException { + delegate.createTable(request); + } + + @Override + public void alter_table(String catName, String databaseName, String tblName, Table table, + EnvironmentContext environmentContext, String validWriteIdList) + throws InvalidOperationException, MetaException, TException { + delegate.alter_table(catName, databaseName, tblName, table, environmentContext, validWriteIdList); + } + + @Override + public void createDatabase(Database db) + throws InvalidObjectException, AlreadyExistsException, MetaException, TException { + delegate.createDatabase(db); + } + + @Override + public void dropDatabase(DropDatabaseRequest req) throws TException { + delegate.dropDatabase(req); + } + + @Override + public void alterDatabase(String catName, String dbName, Database newDb) + throws NoSuchObjectException, MetaException, TException { + delegate.alterDatabase(catName, dbName, newDb); + } + + @Override + public void createDataConnector(DataConnector connector) + throws InvalidObjectException, AlreadyExistsException, MetaException, TException { + delegate.createDataConnector(connector); + } + + @Override + public void dropDataConnector(String name, boolean ifNotExists, boolean checkReferences) + throws NoSuchObjectException, InvalidOperationException, MetaException, TException { + delegate.dropDataConnector(name, ifNotExists, checkReferences); + } + + @Override + public void alterDataConnector(String name, DataConnector connector) + throws NoSuchObjectException, MetaException, TException { + delegate.alterDataConnector(name, connector); + } + + @Override + public DataConnector getDataConnector(String name) throws MetaException, TException { + return delegate.getDataConnector(name); + } + + @Override + public List<String> getAllDataConnectorNames() throws MetaException, TException { + return delegate.getAllDataConnectorNames(); + } + + @Override + public boolean dropPartition(String catName, String db_name, String tbl_name, List<String> part_vals, + PartitionDropOptions options) throws NoSuchObjectException, MetaException, TException { + return delegate.dropPartition(catName, db_name, tbl_name, part_vals, options); + } + + @Override + public List<Partition> dropPartitions(String catName, String dbName, String tblName, + List<Pair<Integer, byte[]>> partExprs, PartitionDropOptions options, EnvironmentContext context) + throws NoSuchObjectException, MetaException, TException { + return delegate.dropPartitions(catName, dbName, tblName, partExprs, options, context); + } + + @Override + public boolean dropPartition(String catName, String db_name, String tbl_name, String name, + boolean deleteData) throws NoSuchObjectException, MetaException, TException { + return delegate.dropPartition(catName, db_name, tbl_name, name, deleteData); + } + + @Override + public void alter_partition(String catName, String dbName, String tblName, Partition newPart, + EnvironmentContext environmentContext, String writeIdList) + throws InvalidOperationException, MetaException, TException { + delegate.alter_partition(catName, dbName, tblName, newPart, environmentContext, writeIdList); + } + + @Override + public void alter_partitions(String dbName, String tblName, List<Partition> newParts) + throws InvalidOperationException, MetaException, TException { + alter_partitions(getDefaultCatalog(conf), dbName, tblName, newParts, null, null, -1); Review Comment: Do we need this method here? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org For additional commands, e-mail: gitbox-h...@hive.apache.org