Copilot commented on code in PR #6311:
URL: https://github.com/apache/hive/pull/6311#discussion_r2785505308
##########
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java:
##########
@@ -125,7 +125,7 @@ public void testEndFunctionListener() throws Exception {
e = context.getException();
assertTrue((e!=null));
assertTrue((e instanceof NoSuchObjectException));
- assertEquals(context.getInputTableName(), tblName);
+ assertEquals(context.getInputTableName(), "hive.hive3524." + tblName);
Review Comment:
This assertion hard-codes the default catalog name ("hive"). Since the test
already uses Warehouse.DEFAULT_CATALOG_NAME when creating the DB, prefer
building the expected qualified name using that constant (and dbName) instead
of a literal string to avoid coupling the test to a specific default-catalog
value.
```suggestion
assertEquals(context.getInputTableName(),
Warehouse.DEFAULT_CATALOG_NAME + "." + dbName + "." + tblName);
```
##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java:
##########
@@ -2774,62 +2657,35 @@ public Partition
append_partition_with_environment_context(final String dbName,
final String tableName, final List<String> part_vals, final
EnvironmentContext envContext)
throws InvalidObjectException, AlreadyExistsException, MetaException {
String[] parsedDbName = parseDbName(dbName, conf);
- startPartitionFunction("append_partition_with_environment_context",
parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, part_vals);
- Partition ret = null;
- Exception ex = null;
- try {
- AppendPartitionsRequest appendPartitionsReq = new
AppendPartitionsRequest();
- appendPartitionsReq.setDbName(parsedDbName[DB_NAME]);
- appendPartitionsReq.setTableName(tableName);
- appendPartitionsReq.setPartVals(part_vals);
- appendPartitionsReq.setCatalogName(parsedDbName[CAT_NAME]);
- appendPartitionsReq.setEnvironmentContext(envContext);
- ret = append_partition_req(appendPartitionsReq);
- } catch (Exception e) {
- ex = e;
- throw handleException(e).throwIfInstance(MetaException.class,
InvalidObjectException.class, AlreadyExistsException.class)
- .defaultMetaException();
- } finally {
- endFunction("append_partition_with_environment_context", ret != null,
ex, tableName);
- }
- return ret;
+ AppendPartitionsRequest appendPartitionsReq = new
AppendPartitionsRequest();
+ appendPartitionsReq.setDbName(parsedDbName[DB_NAME]);
+ appendPartitionsReq.setTableName(tableName);
+ appendPartitionsReq.setPartVals(part_vals);
+ appendPartitionsReq.setCatalogName(parsedDbName[CAT_NAME]);
+ appendPartitionsReq.setEnvironmentContext(envContext);
+ return append_partition_req(appendPartitionsReq);
}
@Override
public Partition append_partition_req(final AppendPartitionsRequest
appendPartitionsReq)
throws InvalidObjectException, AlreadyExistsException, MetaException {
- List<String> part_vals = appendPartitionsReq.getPartVals();
String dbName = appendPartitionsReq.getDbName();
String catName = appendPartitionsReq.isSetCatalogName() ?
appendPartitionsReq.getCatalogName() : getDefaultCatalog(conf);
String tableName = appendPartitionsReq.getTableName();
- String partName = appendPartitionsReq.getName();
- if (partName == null && (part_vals == null || part_vals.isEmpty())) {
- throw new MetaException("The partition values must not be null or
empty.");
- }
- if (part_vals == null || part_vals.isEmpty()) {
- // partition name is set, get partition vals and then append partition
- part_vals = getPartValsFromName(getMS(), catName, dbName, tableName,
partName);
- }
- startPartitionFunction("append_partition_req", catName, dbName, tableName,
part_vals);
- if (LOG.isDebugEnabled()) {
- for (String part : part_vals) {
- LOG.debug(part);
- }
- }
- Partition ret = null;
+ startTableFunction("append_partition_req", catName, dbName, tableName);
Exception ex = null;
try {
- ret = append_partition_common(getMS(), catName, dbName, tableName,
part_vals, appendPartitionsReq.getEnvironmentContext());
+ AppendPartitionHandler appendPartition =
AbstractRequestHandler.offer(this, appendPartitionsReq);
+ return appendPartition.getResult().partition();
} catch (Exception e) {
ex = e;
throw handleException(e)
.throwIfInstance(MetaException.class, InvalidObjectException.class,
AlreadyExistsException.class)
.defaultMetaException();
} finally {
- endFunction("append_partition_req", ret != null, ex, tableName);
+ endFunction("append_partition_req", ex == null, ex, tableName);
}
Review Comment:
append_partition_req() reports success to endFunction based solely on (ex ==
null). If the underlying AppendPartitionHandler completes but returns
success=false (e.g., transaction commit fails or addPartition returns false
without throwing), endFunction listeners/metrics will incorrectly record this
as success. Capture the handler result and pass its success() (or call
appendPartition.success()) into endFunction instead.
##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java:
##########
@@ -5299,116 +4888,52 @@ public int get_num_partitions_by_filter(final String
dbName,
return ret;
}
- private int getNumPartitionsByPs(final String catName, final String dbName,
- final String tblName, List<String> partVals)
- throws TException {
- String[] parsedDbName = parseDbName(dbName, conf);
- startTableFunction("getNumPartitionsByPs", parsedDbName[CAT_NAME],
- parsedDbName[DB_NAME], tblName);
-
- int ret = -1;
- Exception ex = null;
- try {
- ret = getMS().getNumPartitionsByPs(catName, dbName, tblName, partVals);
- } catch (Exception e) {
- ex = e;
- rethrowException(e);
- } finally {
- endFunction("getNumPartitionsByPs", ret != -1, ex, tblName);
- }
- return ret;
- }
-
@Override
@Deprecated
public List<Partition> get_partitions_by_names(final String dbName, final
String tblName,
final List<String> partNames)
throws TException {
- return get_partitions_by_names(dbName, tblName, false, null, null, null,
- new
GetPartitionsArgs.GetPartitionsArgsBuilder().partNames(partNames).build());
+ if (partNames == null) {
+ throw new MetaException("The partNames is null");
+ }
+ String[] dbNameParts = parseDbName(dbName, conf);
+ TableName tableName = new TableName(dbNameParts[CAT_NAME],
dbNameParts[DB_NAME], tblName);
+ return GetPartitionsHandler.getPartitionsResult(
+ t -> startTableFunction("get_partitions_by_names", t.getCat(),
t.getDb(), t.getTable()),
+ ex -> endFunction("get_partitions_by_names", ex == null, ex,
tableName.toString()),
+ this, tableName,
+ new
GetPartitionsArgs.GetPartitionsArgsBuilder().partNames(partNames).build()).result();
}
@Override
public GetPartitionsByNamesResult
get_partitions_by_names_req(GetPartitionsByNamesRequest gpbnr)
throws TException {
- List<Partition> partitions = get_partitions_by_names(gpbnr.getDb_name(),
- gpbnr.getTbl_name(),
- gpbnr.isSetGet_col_stats() && gpbnr.isGet_col_stats(),
gpbnr.getEngine(),
- gpbnr.getProcessorCapabilities(), gpbnr.getProcessorIdentifier(),
- new GetPartitionsArgs.GetPartitionsArgsBuilder()
+ if (gpbnr.getNames() == null) {
+ throw new MetaException("The names in GetPartitionsByNamesRequest is
null");
+ }
+ String[] dbNameParts = parseDbName(gpbnr.getDb_name(), conf);
+ TableName tableName = new TableName(dbNameParts[CAT_NAME],
dbNameParts[DB_NAME], gpbnr.getTbl_name());
+ GetPartitionsHandler.GetPartitionsRequest request =
+ new GetPartitionsHandler.GetPartitionsRequest(tableName, new
GetPartitionsArgs.GetPartitionsArgsBuilder()
.partNames(gpbnr.getNames()).skipColumnSchemaForPartition(gpbnr.isSkipColumnSchemaForPartition())
.excludeParamKeyPattern(gpbnr.getExcludeParamKeyPattern())
- .includeParamKeyPattern(gpbnr.getIncludeParamKeyPattern())
- .build());
- GetPartitionsByNamesResult result = new
GetPartitionsByNamesResult(partitions);
- return result;
- }
-
- private List<Partition> get_partitions_by_names(final String dbName, final
String tblName,
- boolean getColStats, String engine,
- List<String> processorCapabilities, String processorId,
- GetPartitionsArgs args) throws TException {
-
- String[] dbNameParts = parseDbName(dbName, conf);
- String parsedCatName = dbNameParts[CAT_NAME];
- String parsedDbName = dbNameParts[DB_NAME];
- List<Partition> ret = null;
- Table table = null;
+
.includeParamKeyPattern(gpbnr.getIncludeParamKeyPattern()).build());
+ request.setEngine(gpbnr.getEngine());
+ request.setGetColStats(gpbnr.isSetGet_col_stats() &&
gpbnr.isGet_col_stats());
+ request.setProcessorId(gpbnr.getProcessorIdentifier());
+ request.setProcessorCapabilities(gpbnr.getProcessorCapabilities());
+ startTableFunction("get_partitions_by_names", tableName.getCat(),
tableName.getDb(), tableName.getTable());
Exception ex = null;
- boolean success = false;
- startTableFunction("get_partitions_by_names", parsedCatName, parsedDbName,
tblName);
- try {
- getMS().openTransaction();
- authorizeTableForPartitionMetadata(parsedCatName, parsedDbName, tblName);
-
- fireReadTablePreEvent(parsedCatName, parsedDbName, tblName);
-
- checkLimitNumberOfPartitions(tblName, args.getPartNames().size());
- ret = getMS().getPartitionsByNames(parsedCatName, parsedDbName, tblName,
args);
- ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled,
filterHook, ret);
- table = getTable(parsedCatName, parsedDbName, tblName);
-
- // If requested add column statistics in each of the partition objects
- if (getColStats) {
- // Since each partition may have stats collected for different set of
columns, we
- // request them separately.
- for (Partition part: ret) {
- String partName = Warehouse.makePartName(table.getPartitionKeys(),
part.getValues());
- List<ColumnStatistics> partColStatsList =
- getMS().getPartitionColumnStatistics(parsedCatName,
parsedDbName, tblName,
- Collections.singletonList(partName),
- StatsSetupConst.getColumnsHavingStats(part.getParameters()),
- engine);
- if (partColStatsList != null && !partColStatsList.isEmpty()) {
- ColumnStatistics partColStats = partColStatsList.get(0);
- if (partColStats != null) {
- part.setColStats(partColStats);
- }
- }
- }
- }
-
- if (processorCapabilities == null || processorCapabilities.size() == 0 ||
- processorCapabilities.contains("MANAGERAWMETADATA")) {
- LOG.info("Skipping translation for processor with " + processorId);
- } else {
- if (transformer != null) {
- ret = transformer.transformPartitions(ret, table,
processorCapabilities, processorId);
- }
- }
- success = getMS().commitTransaction();
+ try {
+ GetPartitionsHandler<Partition> getPartitionsHandler =
AbstractRequestHandler.offer(this, request);
+ List<Partition> partitions = getPartitionsHandler.getResult().result();
+ return new GetPartitionsByNamesResult(partitions);
} catch (Exception e) {
ex = e;
- throw handleException(e)
- .throwIfInstance(MetaException.class, NoSuchObjectException.class,
InvalidObjectException.class)
- .defaultMetaException();
+ throw handleException(e).defaultTException();
} finally {
- if (!success) {
- getMS().rollbackTransaction();
- }
- endFunction("get_partitions_by_names", ret != null, ex, tblName);
+ endFunction("get_partitions_by_names", ex == null, ex,
tableName.toString());
}
Review Comment:
get_partitions_by_names_req() calls endFunction with (ex == null) as the
success indicator. The underlying GetPartitionsHandler can return a Result with
success=false (e.g., transaction commit fails) without necessarily throwing,
which would be reported as success here. Consider using the offered handler's
success() / result.success() value for the endFunction success flag instead of
relying only on exception presence.
##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/handler/GetPartitionsHandler.java:
##########
@@ -0,0 +1,532 @@
+/*
+ * 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.handler;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.function.Consumer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.MetaStoreFilterHook;
+import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+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.PartitionValuesResponse;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
+import org.apache.hadoop.hive.metastore.utils.FilterUtils;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static
org.apache.hadoop.hive.metastore.ExceptionHandler.handleException;
+import static
org.apache.hadoop.hive.metastore.HMSHandler.PARTITION_NUMBER_EXCEED_LIMIT_MSG;
+import static
org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
+// Collect get partitions APIs together
+@SuppressWarnings({"unchecked", "rawtypes"})
+@RequestHandler(requestBody = GetPartitionsHandler.GetPartitionsRequest.class)
+public class GetPartitionsHandler<T> extends
AbstractRequestHandler<GetPartitionsHandler.GetPartitionsRequest,
+ GetPartitionsHandler.GetPartitionsResult<T>> {
+ private static final Logger LOG =
LoggerFactory.getLogger(GetPartitionsHandler.class);
+ private static final String NO_FILTER_STRING = "";
+ private RawStore rs;
+ private String catName;
+ private String dbName;
+ private String tblName;
+ private GetPartitionsArgs args;
+ private Table table;
+ private Configuration conf;
+ private GetPartitionsMethod getMethod;
+ private MetaStoreFilterHook filterHook;
+ private boolean isServerFilterEnabled;
+
+ enum GetPartitionsMethod {
+ EXPR, NAMES, FILTER, PART_VALS, ALL, VALUES
+ }
+
+ GetPartitionsHandler(IHMSHandler handler, GetPartitionsRequest request) {
+ super(handler, false, request);
+ }
+
+ @Override
+ protected void beforeExecute() throws TException, IOException {
+ this.args = request.getGetPartitionsArgs();
+ if (request.isGetPartitionValues()) {
+ getMethod = GetPartitionsMethod.VALUES;
+ } else if (args.getExpr() != null) {
+ getMethod = GetPartitionsMethod.EXPR;
+ } else if (args.getFilter() != null) {
+ getMethod = GetPartitionsMethod.FILTER;
+ } else if (args.getPartNames() != null) {
+ getMethod = GetPartitionsMethod.NAMES;
+ } else if (args.getPart_vals() != null) {
+ getMethod = GetPartitionsMethod.PART_VALS;
+ } else {
+ getMethod = GetPartitionsMethod.ALL;
+ }
+
+ this.catName = normalizeIdentifier(request.getTableName().getCat());
+ this.dbName = normalizeIdentifier(request.getTableName().getDb());
+ this.tblName = normalizeIdentifier(request.getTableName().getTable());
+ this.conf = handler.getConf();
+ this.rs = handler.getMS();
+ this.filterHook = handler.getMetaFilterHook();
+ this.isServerFilterEnabled = filterHook != null;
+ GetTableRequest getTableRequest = new GetTableRequest(dbName, tblName);
+ getTableRequest.setCatName(catName);
+ this.table = handler.get_table_core(getTableRequest);
+ ((HMSHandler) handler).firePreEvent(new PreReadTableEvent(table, handler));
+ authorizeTableForPartitionMetadata();
Review Comment:
beforeExecute() always loads the full Table via get_table_core() and fires a
PreReadTableEvent, even for requests that only need partition names. This
introduces an unconditional metastore table lookup (and pre-event overhead) on
all get-partitions code paths, which is a behavior/performance change from the
previous per-API implementations (some only loaded the table when needed).
Consider making table loading lazy and/or conditional (e.g., only when column
stats/transformations are requested or when preListeners exist), and performing
FilterUtils.checkDbAndTableFilters before any table fetch when possible.
##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/handler/GetPartitionsHandler.java:
##########
@@ -0,0 +1,532 @@
+/*
+ * 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.handler;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.function.Consumer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.MetaStoreFilterHook;
+import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+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.PartitionValuesResponse;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
+import org.apache.hadoop.hive.metastore.utils.FilterUtils;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static
org.apache.hadoop.hive.metastore.ExceptionHandler.handleException;
+import static
org.apache.hadoop.hive.metastore.HMSHandler.PARTITION_NUMBER_EXCEED_LIMIT_MSG;
+import static
org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
+// Collect get partitions APIs together
+@SuppressWarnings({"unchecked", "rawtypes"})
+@RequestHandler(requestBody = GetPartitionsHandler.GetPartitionsRequest.class)
+public class GetPartitionsHandler<T> extends
AbstractRequestHandler<GetPartitionsHandler.GetPartitionsRequest,
+ GetPartitionsHandler.GetPartitionsResult<T>> {
+ private static final Logger LOG =
LoggerFactory.getLogger(GetPartitionsHandler.class);
+ private static final String NO_FILTER_STRING = "";
+ private RawStore rs;
+ private String catName;
+ private String dbName;
+ private String tblName;
+ private GetPartitionsArgs args;
+ private Table table;
+ private Configuration conf;
+ private GetPartitionsMethod getMethod;
+ private MetaStoreFilterHook filterHook;
+ private boolean isServerFilterEnabled;
+
+ enum GetPartitionsMethod {
+ EXPR, NAMES, FILTER, PART_VALS, ALL, VALUES
+ }
+
+ GetPartitionsHandler(IHMSHandler handler, GetPartitionsRequest request) {
+ super(handler, false, request);
+ }
+
+ @Override
+ protected void beforeExecute() throws TException, IOException {
+ this.args = request.getGetPartitionsArgs();
+ if (request.isGetPartitionValues()) {
+ getMethod = GetPartitionsMethod.VALUES;
+ } else if (args.getExpr() != null) {
+ getMethod = GetPartitionsMethod.EXPR;
+ } else if (args.getFilter() != null) {
+ getMethod = GetPartitionsMethod.FILTER;
+ } else if (args.getPartNames() != null) {
+ getMethod = GetPartitionsMethod.NAMES;
+ } else if (args.getPart_vals() != null) {
+ getMethod = GetPartitionsMethod.PART_VALS;
+ } else {
+ getMethod = GetPartitionsMethod.ALL;
+ }
+
+ this.catName = normalizeIdentifier(request.getTableName().getCat());
+ this.dbName = normalizeIdentifier(request.getTableName().getDb());
+ this.tblName = normalizeIdentifier(request.getTableName().getTable());
+ this.conf = handler.getConf();
+ this.rs = handler.getMS();
+ this.filterHook = handler.getMetaFilterHook();
+ this.isServerFilterEnabled = filterHook != null;
+ GetTableRequest getTableRequest = new GetTableRequest(dbName, tblName);
+ getTableRequest.setCatName(catName);
+ this.table = handler.get_table_core(getTableRequest);
+ ((HMSHandler) handler).firePreEvent(new PreReadTableEvent(table, handler));
+ authorizeTableForPartitionMetadata();
+
+ LOG.info("Starting to get {} of {} using {}", request.isFetchPartNames() ?
"partition names" : "partitions",
+ TableName.getQualified(catName, dbName, tblName), getMethod);
+ }
+
+ @Override
+ protected GetPartitionsResult<T> execute() throws TException, IOException {
+ return (GetPartitionsResult<T>) switch (getMethod) {
+ case EXPR -> getPartitionsByExpr();
+ case FILTER -> getPartitionsByFilter();
+ case NAMES -> getPartitionsByNames();
+ case PART_VALS -> getPartitionsByVals();
+ case ALL -> getPartitions();
+ case VALUES -> getPartitionValues();
+ };
+ }
+
+ private GetPartitionsResult getPartitionsByVals() throws TException {
+ if (request.isFetchPartNames()) {
+ List<String> ret = rs.listPartitionNamesPs(catName, dbName, tblName,
+ args.getPart_vals(), (short) args.getMax());
+ ret = FilterUtils.filterPartitionNamesIfEnabled(isServerFilterEnabled,
+ filterHook, catName, dbName, tblName, ret);
+ return new GetPartitionsResult<>(ret, true);
+ } else {
+ List<Partition> ret;
+ if (args.getPart_vals() != null) {
+ checkLimitNumberOfPartitionsByPs(args.getPart_vals(), args.getMax());
+ } else {
+ checkLimitNumberOfPartitionsByFilter(NO_FILTER_STRING, args.getMax());
+ }
+ ret = rs.listPartitionsPsWithAuth(catName, dbName, tblName, args);
+ return new GetPartitionsResult(ret, true);
+ }
+ }
+
+ private GetPartitionsResult getPartitionValues() throws MetaException {
+ PartitionValuesResponse resp = rs.listPartitionValues(catName, dbName,
tblName, request.getPartitionKeys(),
+ request.isApplyDistinct(), args.getFilter(), request.isAscending(),
+ request.getPartitionOrders(), args.getMax());
+ return new GetPartitionsResult<>(Arrays.asList(resp), true);
+ }
+
+ private void checkLimitNumberOfPartitionsByPs(List<String> partVals, int
requestMax)
+ throws TException {
+ if (exceedsPartitionFetchLimit(requestMax)) {
+ checkLimitNumberOfPartitions(tblName, rs.getNumPartitionsByPs(catName,
dbName, tblName,
+ partVals));
+ }
+ }
+
+ private GetPartitionsResult<Partition> getPartitionsByFilter() throws
TException {
+ List<Partition> ret = null;
+ if (exceedsPartitionFetchLimit(args.getMax())) {
+ // Since partition limit is configured, we need fetch at most (limit +
1) partition names
+ int max = MetastoreConf.getIntVar(conf,
MetastoreConf.ConfVars.LIMIT_PARTITION_REQUEST) + 1;
+ args = new
GetPartitionsArgs.GetPartitionsArgsBuilder(args).max(max).build();
+ List<String> partNames = rs.listPartitionNamesByFilter(catName, dbName,
tblName, args);
+ checkLimitNumberOfPartitions(tblName, partNames.size());
+ ret = rs.getPartitionsByNames(catName, dbName, tblName,
+ new
GetPartitionsArgs.GetPartitionsArgsBuilder(args).partNames(partNames).build());
+ } else {
+ ret = rs.getPartitionsByFilter(catName, dbName, tblName, args);
+ }
+
+ ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled,
filterHook, ret);
+ return new GetPartitionsResult<>(ret, true);
+ }
+
+ /**
+ * Check if user can access the table associated with the partition. If not,
then throw exception
+ * so user cannot access partitions associated with this table
+ * We are not calling Pre event listener for authorization because it
requires getting the
+ * table object from DB, more overhead. Instead ,we call filter hook to
filter out table if user
+ * has no access. Filter hook only requires table name, not table object.
That saves DB access for
+ * table object, and still achieve the same purpose: checking if user can
access the specified
+ * table
+ *
+ * @throws NoSuchObjectException
+ * @throws MetaException
+ */
+ private void authorizeTableForPartitionMetadata()
+ throws NoSuchObjectException, MetaException {
+ FilterUtils.checkDbAndTableFilters(
+ isServerFilterEnabled, filterHook, catName, dbName, tblName);
+ }
+
+ private GetPartitionsResult<Partition> getPartitionsByNames() throws
TException {
+ List<Partition> ret = null;
+ boolean success = false;
+ rs.openTransaction();
+ try {
+ checkLimitNumberOfPartitions(tblName, args.getPartNames().size());
+ ret = rs.getPartitionsByNames(catName, dbName, tblName, args);
+ ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled,
filterHook, ret);
+
+ // If requested add column statistics in each of the partition objects
+ if (request.isGetColStats()) {
+ // Since each partition may have stats collected for different set of
columns, we
+ // request them separately.
+ for (Partition part: ret) {
+ String partName = Warehouse.makePartName(table.getPartitionKeys(),
part.getValues());
+ List<ColumnStatistics> partColStatsList =
+ rs.getPartitionColumnStatistics(catName, dbName, tblName,
+ Collections.singletonList(partName),
+ StatsSetupConst.getColumnsHavingStats(part.getParameters()),
+ request.getEngine());
+ if (partColStatsList != null && !partColStatsList.isEmpty()) {
+ ColumnStatistics partColStats = partColStatsList.get(0);
+ if (partColStats != null) {
+ part.setColStats(partColStats);
+ }
+ }
+ }
+ }
+
+ List<String> processorCapabilities = request.getProcessorCapabilities();
+ if (processorCapabilities == null || processorCapabilities.isEmpty() ||
+ processorCapabilities.contains("MANAGERAWMETADATA")) {
+ LOG.info("Skipping translation for processor with {}",
request.getProcessorId());
+ } else {
+ if (handler.getMetadataTransformer() != null) {
+ ret = handler.getMetadataTransformer().transformPartitions(ret,
table,
+ processorCapabilities, request.getProcessorId());
+ }
+ }
+ success = rs.commitTransaction();
+ } finally {
+ if (!success) {
+ rs.rollbackTransaction();
+ }
+ }
+ return new GetPartitionsResult<>(ret, success);
+ }
+
+ private GetPartitionsResult getPartitions() throws TException {
+ if (request.isFetchPartNames()) {
+ List<String> ret = rs.listPartitionNames(catName, dbName, tblName,
(short) args.getMax());
+ ret = FilterUtils.filterPartitionNamesIfEnabled(isServerFilterEnabled,
+ filterHook, catName, dbName, tblName, ret);
+ return new GetPartitionsResult<>(ret, true);
+ } else {
+ List<Partition> ret;
+ checkLimitNumberOfPartitionsByFilter(NO_FILTER_STRING, args.getMax());
+ ret = rs.listPartitionsPsWithAuth(catName, dbName, tblName, args);
+ ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled,
filterHook, ret);
+ return new GetPartitionsResult<>(ret, true);
+ }
+ }
+
+ private void checkLimitNumberOfPartitionsByFilter(String filterString, int
requestMax) throws TException {
+ if (exceedsPartitionFetchLimit(requestMax)) {
+ checkLimitNumberOfPartitions(tblName,
rs.getNumPartitionsByFilter(catName, dbName, tblName, filterString));
+ }
+ }
+
+ private GetPartitionsResult getPartitionsByExpr() throws TException {
+ if (request.isFetchPartNames()) {
+ List<String> ret = rs.listPartitionNames(catName, dbName, tblName,
+ args.getDefaultPartName(), args.getExpr(), args.getOrder(),
args.getMax());
+ ret = FilterUtils.filterPartitionNamesIfEnabled(isServerFilterEnabled,
+ filterHook, catName, dbName, tblName, ret);
+ return new GetPartitionsResult(ret, true);
+ } else {
+ List<Partition> partitions = new LinkedList<>();
+ boolean hasUnknownPartitions = false;
+ if (exceedsPartitionFetchLimit(args.getMax())) {
+ // Since partition limit is configured, we need fetch at most (limit +
1) partition names
+ int max = MetastoreConf.getIntVar(handler.getConf(),
MetastoreConf.ConfVars.LIMIT_PARTITION_REQUEST) + 1;
+ List<String> partNames = rs.listPartitionNames(catName, dbName,
tblName, args.getDefaultPartName(),
+ args.getExpr(), null, max);
Review Comment:
In the partition-fetch-limit branch of getPartitionsByExpr(),
listPartitionNames is called with a null "order" argument. This ignores
GetPartitionsArgs.order and can change the ordering/selection of the partition
names compared to the non-limited path (and compared to the fetchPartNames path
which passes args.getOrder()). Pass args.getOrder() instead of null here.
```suggestion
args.getExpr(), args.getOrder(), max);
```
##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/handler/GetPartitionsHandler.java:
##########
@@ -0,0 +1,532 @@
+/*
+ * 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.handler;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.function.Consumer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.MetaStoreFilterHook;
+import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+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.PartitionValuesResponse;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
+import org.apache.hadoop.hive.metastore.utils.FilterUtils;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static
org.apache.hadoop.hive.metastore.ExceptionHandler.handleException;
+import static
org.apache.hadoop.hive.metastore.HMSHandler.PARTITION_NUMBER_EXCEED_LIMIT_MSG;
+import static
org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
+// Collect get partitions APIs together
+@SuppressWarnings({"unchecked", "rawtypes"})
+@RequestHandler(requestBody = GetPartitionsHandler.GetPartitionsRequest.class)
+public class GetPartitionsHandler<T> extends
AbstractRequestHandler<GetPartitionsHandler.GetPartitionsRequest,
+ GetPartitionsHandler.GetPartitionsResult<T>> {
+ private static final Logger LOG =
LoggerFactory.getLogger(GetPartitionsHandler.class);
+ private static final String NO_FILTER_STRING = "";
+ private RawStore rs;
+ private String catName;
+ private String dbName;
+ private String tblName;
+ private GetPartitionsArgs args;
+ private Table table;
+ private Configuration conf;
+ private GetPartitionsMethod getMethod;
+ private MetaStoreFilterHook filterHook;
+ private boolean isServerFilterEnabled;
+
+ enum GetPartitionsMethod {
+ EXPR, NAMES, FILTER, PART_VALS, ALL, VALUES
+ }
+
+ GetPartitionsHandler(IHMSHandler handler, GetPartitionsRequest request) {
+ super(handler, false, request);
+ }
+
+ @Override
+ protected void beforeExecute() throws TException, IOException {
+ this.args = request.getGetPartitionsArgs();
+ if (request.isGetPartitionValues()) {
+ getMethod = GetPartitionsMethod.VALUES;
+ } else if (args.getExpr() != null) {
+ getMethod = GetPartitionsMethod.EXPR;
+ } else if (args.getFilter() != null) {
+ getMethod = GetPartitionsMethod.FILTER;
+ } else if (args.getPartNames() != null) {
+ getMethod = GetPartitionsMethod.NAMES;
+ } else if (args.getPart_vals() != null) {
+ getMethod = GetPartitionsMethod.PART_VALS;
+ } else {
+ getMethod = GetPartitionsMethod.ALL;
+ }
+
+ this.catName = normalizeIdentifier(request.getTableName().getCat());
+ this.dbName = normalizeIdentifier(request.getTableName().getDb());
+ this.tblName = normalizeIdentifier(request.getTableName().getTable());
+ this.conf = handler.getConf();
+ this.rs = handler.getMS();
+ this.filterHook = handler.getMetaFilterHook();
+ this.isServerFilterEnabled = filterHook != null;
+ GetTableRequest getTableRequest = new GetTableRequest(dbName, tblName);
+ getTableRequest.setCatName(catName);
+ this.table = handler.get_table_core(getTableRequest);
+ ((HMSHandler) handler).firePreEvent(new PreReadTableEvent(table, handler));
+ authorizeTableForPartitionMetadata();
+
+ LOG.info("Starting to get {} of {} using {}", request.isFetchPartNames() ?
"partition names" : "partitions",
+ TableName.getQualified(catName, dbName, tblName), getMethod);
+ }
+
+ @Override
+ protected GetPartitionsResult<T> execute() throws TException, IOException {
+ return (GetPartitionsResult<T>) switch (getMethod) {
+ case EXPR -> getPartitionsByExpr();
+ case FILTER -> getPartitionsByFilter();
+ case NAMES -> getPartitionsByNames();
+ case PART_VALS -> getPartitionsByVals();
+ case ALL -> getPartitions();
+ case VALUES -> getPartitionValues();
+ };
+ }
+
+ private GetPartitionsResult getPartitionsByVals() throws TException {
+ if (request.isFetchPartNames()) {
+ List<String> ret = rs.listPartitionNamesPs(catName, dbName, tblName,
+ args.getPart_vals(), (short) args.getMax());
+ ret = FilterUtils.filterPartitionNamesIfEnabled(isServerFilterEnabled,
+ filterHook, catName, dbName, tblName, ret);
+ return new GetPartitionsResult<>(ret, true);
+ } else {
+ List<Partition> ret;
+ if (args.getPart_vals() != null) {
+ checkLimitNumberOfPartitionsByPs(args.getPart_vals(), args.getMax());
+ } else {
+ checkLimitNumberOfPartitionsByFilter(NO_FILTER_STRING, args.getMax());
+ }
+ ret = rs.listPartitionsPsWithAuth(catName, dbName, tblName, args);
Review Comment:
In getPartitionsByVals() (partition objects path), the result from
listPartitionsPsWithAuth is returned without applying the server-side partition
filter hook. Other code paths (e.g., getPartitions(), getPartitionsByFilter(),
getPartitionsByNames()) call FilterUtils.filterPartitionsIfEnabled, so this
looks like an accidental omission that could expose partitions a user should
not see when a MetaStoreFilterHook is configured. Apply
FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret)
before returning.
```suggestion
ret = rs.listPartitionsPsWithAuth(catName, dbName, tblName, args);
ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled,
filterHook, ret);
```
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]