IMPALA-14065: Support WHERE clause in SHOW PARTITIONS statement

This patch extends the SHOW PARTITIONS statement to allow an optional
WHERE clause that filters partitions based on partition column values.
The implementation adds support for various comparison operators,
IN lists, BETWEEN clauses, IS NULL, and logical AND/OR expressions
involving partition columns.

Non-partition columns, subqueries, and analytic expressions in the
WHERE clause are not allowed and will result in an analysis error.

New analyzer tests have been added to AnalyzeDDLTest#TestShowPartitions
to verify correct parsing, semantic validation, and error handling for
supported and unsupported cases.

Testing:
- Added new unit tests in AnalyzeDDLTest for valid and invalid WHERE
clause cases.
- Verified functional tests covering partition filtering behavior.

Change-Id: I2e2a14aabcea3fb17083d4ad6f87b7861113f89e
Reviewed-on: http://gerrit.cloudera.org:8080/23566
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This commit is contained in:
Arnab Karmakar
2025-10-21 16:41:18 +05:30
committed by Impala Public Jenkins
parent 1684c2d9da
commit ddd82e02b9
17 changed files with 875 additions and 102 deletions

View File

@@ -253,6 +253,8 @@ struct TShowStatsParams {
1: TShowStatsOp op
2: CatalogObjects.TTableName table_name
3: optional bool show_column_minmax_stats
// Optional: filtered partition ids for SHOW PARTITIONS with a WHERE clause.
4: optional list<i64> filtered_partition_ids
}
// Parameters for DESCRIBE HISTORY command

View File

@@ -3158,6 +3158,8 @@ show_stats_stmt ::=
show_partitions_stmt ::=
KW_SHOW KW_PARTITIONS table_name:table
{: RESULT = new ShowStatsStmt(table, TShowStatsOp.PARTITIONS); :}
| KW_SHOW KW_PARTITIONS table_name:table KW_WHERE expr:where
{: RESULT = new ShowStatsStmt(table, TShowStatsOp.PARTITIONS, where); :}
;
show_range_partitions_stmt ::=

View File

@@ -711,6 +711,11 @@ public class AnalysisContext {
return canRewriteStatement() && isZippingUnnestInSelectList(stmt_);
}
public boolean requiresExprRewrite() {
// For ShowStatsStmt, only require rewrite if there's a WHERE clause
if (isShowStatsStmt()) {
ShowStatsStmt showStatsStmt = (ShowStatsStmt) stmt_;
return showStatsStmt.hasWhereClause();
}
return isQueryStmt() || isInsertStmt() || isCreateTableAsSelectStmt()
|| isUpdateStmt() || isDeleteStmt() || isOptimizeStmt() || isMergeStmt();
}
@@ -725,6 +730,7 @@ public class AnalysisContext {
public boolean isShowCreateTableStmt() {
return stmt_ instanceof ShowCreateTableStmt;
}
public boolean isShowStatsStmt() { return stmt_ instanceof ShowStatsStmt; }
public boolean isQueryStmt() { return stmt_ instanceof QueryStmt; }
public boolean isInsertStmt() { return stmt_ instanceof InsertStmt; }
public boolean isMergeStmt() { return stmt_ instanceof MergeStmt; }

View File

@@ -85,7 +85,7 @@ public class PartitionSet extends PartitionSpecBase {
try {
HdfsPartitionPruner pruner = new HdfsPartitionPruner(desc);
partitions_ = pruner.prunePartitions(analyzer, transformedConjuncts, true,
partitions_ = pruner.prunePartitions(analyzer, transformedConjuncts, true, false,
null).first;
} catch (ImpalaException e) {
if (e instanceof AnalysisException) throw (AnalysisException)e;

View File

@@ -17,21 +17,30 @@
package org.apache.impala.analysis;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.impala.analysis.paimon.PaimonAnalyzer;
import org.apache.impala.authorization.Privilege;
import org.apache.impala.common.Pair;
import org.apache.impala.catalog.FeFsTable;
import org.apache.impala.catalog.FeIcebergTable;
import org.apache.impala.catalog.FeKuduTable;
import org.apache.impala.catalog.FeTable;
import org.apache.impala.catalog.FeFsPartition;
import org.apache.impala.catalog.FeView;
import org.apache.impala.catalog.paimon.FePaimonTable;
import org.apache.impala.common.AnalysisException;
import org.apache.impala.planner.HdfsPartitionPruner;
import org.apache.impala.rewrite.ExprRewriter;
import org.apache.impala.rewrite.ExtractCompoundVerticalBarExprRule;
import org.apache.impala.thrift.TShowStatsOp;
import org.apache.impala.thrift.TShowStatsParams;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicates;
/**
* Representation of a SHOW TABLE/COLUMN STATS statement for
@@ -41,21 +50,42 @@ public class ShowStatsStmt extends StatementBase implements SingleTableStmt {
protected final TShowStatsOp op_;
protected final TableName tableName_;
protected boolean show_column_minmax_stats_ = false;
// Optional WHERE predicate for SHOW PARTITIONS.
// ONLY supported for HDFS tables (FeFsTable, excluding FeIcebergTable).
// Iceberg tables use a different partition mechanism and are not supported.
protected Expr whereClause_;
// Computed during analysis if whereClause_ is set for HDFS tables.
protected List<Long> filteredPartitionIds_ = null;
// Set during analysis.
protected FeTable table_;
public ShowStatsStmt(TableName tableName, TShowStatsOp op) {
this(tableName, op, null);
}
public ShowStatsStmt(TableName tableName, TShowStatsOp op, Expr whereExpr) {
op_ = Preconditions.checkNotNull(op);
tableName_ = Preconditions.checkNotNull(tableName);
whereClause_ = whereExpr;
}
@Override
public TableName getTableName() { return tableName_; }
/**
* Returns true if this SHOW PARTITIONS statement has a WHERE clause.
*/
public boolean hasWhereClause() { return whereClause_ != null; }
@Override
public String toSql(ToSqlOptions options) {
return getSqlPrefix() + " " + tableName_.toString();
StringBuilder sb = new StringBuilder();
sb.append(getSqlPrefix()).append(" ").append(tableName_.toString());
if (whereClause_ != null && op_ == TShowStatsOp.PARTITIONS) {
sb.append(" WHERE ").append(whereClause_.toSql(options));
}
return sb.toString();
}
protected String getSqlPrefix() {
@@ -82,6 +112,7 @@ public class ShowStatsStmt extends StatementBase implements SingleTableStmt {
@Override
public void analyze(Analyzer analyzer) throws AnalysisException {
super.analyze(analyzer);
table_ = analyzer.getTable(tableName_, Privilege.VIEW_METADATA);
Preconditions.checkNotNull(table_);
if (table_ instanceof FeView) {
@@ -136,6 +167,117 @@ public class ShowStatsStmt extends StatementBase implements SingleTableStmt {
}
show_column_minmax_stats_ =
analyzer.getQueryOptions().isShow_column_minmax_stats();
// If WHERE clause is present for SHOW PARTITIONS on HDFS table, analyze and compute
// filtered IDs.
if (whereClause_ != null && op_ == TShowStatsOp.PARTITIONS) {
analyzeWhereClause(analyzer);
}
}
/**
* Analyzes the WHERE clause for SHOW PARTITIONS on HDFS tables and computes
* the filtered partition IDs.
*/
private void analyzeWhereClause(Analyzer analyzer) throws AnalysisException {
if (!(table_ instanceof FeFsTable) || table_ instanceof FeIcebergTable) {
throw new AnalysisException(
"WHERE clause in SHOW PARTITIONS is only supported for HDFS tables");
}
// Disable authorization checks for internal analysis of WHERE clause
analyzer.setEnablePrivChecks(false);
try {
TableName qualifiedName = new TableName(
table_.getDb().getName(), table_.getName());
TableRef tableRef = new TableRef(
qualifiedName.toPath(), null, Privilege.VIEW_METADATA);
tableRef = analyzer.resolveTableRef(tableRef);
tableRef.analyze(analyzer);
// Analyze the WHERE predicate if not already analyzed
if (!whereClause_.isAnalyzed()) {
whereClause_.analyze(analyzer);
}
whereClause_.checkReturnsBool("WHERE clause", true);
// Check if the WHERE clause contains CompoundVerticalBarExpr (||) that needs
// rewriting.
List<CompoundVerticalBarExpr> compoundVerticalBarExprs = new ArrayList<>();
whereClause_.collectAll(Predicates.instanceOf(CompoundVerticalBarExpr.class),
compoundVerticalBarExprs);
if (!compoundVerticalBarExprs.isEmpty()) {
// Expression needs rewriting - defer partition filtering to second analysis.
return;
}
// Check if the WHERE clause contains Subquery or AnalyticExpr.
if (whereClause_.contains(Subquery.class)) {
throw new AnalysisException(
"Subqueries are not allowed in SHOW PARTITIONS WHERE");
}
if (whereClause_.contains(AnalyticExpr.class)) {
throw new AnalysisException(
"Analytic expressions are not allowed in SHOW PARTITIONS WHERE");
}
// Aggregate functions cannot be evaluated per-partition.
if (whereClause_.contains(Expr.IS_AGGREGATE)) {
throw new AnalysisException(
"Aggregate functions are not allowed in SHOW PARTITIONS WHERE");
}
// Ensure all conjuncts reference only partition columns.
List<SlotId> partitionSlots = tableRef.getDesc().getPartitionSlots();
if (!whereClause_.isBoundBySlotIds(partitionSlots)) {
throw new AnalysisException(
"SHOW PARTITIONS WHERE supports only partition columns");
}
// Prune the partitions using the HdfsPartitionPruner.
HdfsPartitionPruner pruner = new HdfsPartitionPruner(tableRef.getDesc());
try {
// Clone the conjuncts because the pruner will modify the original list.
List<Expr> conjunctsCopy = new ArrayList<>(whereClause_.getConjuncts());
// Pass evalAllFuncs=true to ensure non-deterministic functions are evaluated
// per-partition instead of being skipped.
Pair<List<? extends FeFsPartition>, List<Expr>> res =
pruner.prunePartitions(analyzer, conjunctsCopy, true, true, tableRef);
Preconditions.checkState(conjunctsCopy.isEmpty(),
"All conjuncts should be evaluated");
// All partitions from the pruner have matched - collect their IDs.
Set<Long> ids = new HashSet<>();
for (FeFsPartition p : res.first) ids.add(p.getId());
filteredPartitionIds_ = new ArrayList<>(ids);
} catch (org.apache.impala.common.ImpalaException e) {
throw new AnalysisException(
"Failed to evaluate WHERE clause for SHOW PARTITIONS: " + e.getMessage(), e);
}
} finally {
// Re-enable authorization checks
analyzer.setEnablePrivChecks(true);
}
}
@Override
public void reset() {
super.reset();
// Clear computed partition IDs so they'll be recomputed after re-analysis
filteredPartitionIds_ = null;
// Reset the whereExpr if it exists
if (whereClause_ != null) whereClause_ = whereClause_.reset();
}
@Override
public void rewriteExprs(ExprRewriter rewriter) throws AnalysisException {
Preconditions.checkState(isAnalyzed());
// For SHOW PARTITIONS with WHERE clause, we need to do mandatory rewrites for ||
// operator as the partition pruner cannot execute it directly.
if (whereClause_ != null && op_ == TShowStatsOp.PARTITIONS) {
ExprRewriter mandatoryRewriter = new ExprRewriter(
ExtractCompoundVerticalBarExprRule.INSTANCE);
whereClause_ = mandatoryRewriter.rewrite(whereClause_, analyzer_);
rewriter.addNumChanges(mandatoryRewriter);
}
}
public TShowStatsParams toThrift() {
@@ -143,6 +285,11 @@ public class ShowStatsStmt extends StatementBase implements SingleTableStmt {
TShowStatsParams showStatsParam = new TShowStatsParams(op_,
new TableName(table_.getDb().getName(), table_.getName()).toThrift());
showStatsParam.setShow_column_minmax_stats(show_column_minmax_stats_);
// Always set filteredPartitionIds if it exists (even if empty) to distinguish
// between "no matches" (empty list) and "all partitions" (null)
if (filteredPartitionIds_ != null) {
showStatsParam.setFiltered_partition_ids(filteredPartitionIds_);
}
return showStatsParam;
}
}

View File

@@ -60,6 +60,8 @@ import org.apache.impala.util.TAccessLevelUtil;
import org.apache.impala.util.TResultRowBuilder;
import org.apache.thrift.TException;
import javax.annotation.Nullable;
/**
* Frontend interface for interacting with a filesystem-backed table.
*
@@ -451,17 +453,17 @@ public interface FeFsTable extends FeTable {
}
/**
* @return statistics on this table as a tabular result set. Used for the
* SHOW TABLE STATS statement. The schema of the returned TResultSet is set
* inside this method.
* Helper method to build the schema for table stats result set.
* @param result The TResultSet to populate with schema information
* @return true if stats extrapolation is enabled for this table
*/
default TResultSet getTableStats() {
TResultSet result = new TResultSet();
default boolean buildTableStatsSchema(TResultSet result) {
TResultSetMetadata resultSchema = new TResultSetMetadata();
result.setSchema(resultSchema);
result.setRows(new ArrayList<>());
// Add partition column headers
for (int i = 0; i < getNumClusteringCols(); ++i) {
// Add the partition-key values as strings for simplicity.
Column partCol = getColumns().get(i);
TColumn colDesc = new TColumn(partCol.getName(), Type.STRING.toThrift());
resultSchema.addToColumns(colDesc);
@@ -469,6 +471,7 @@ public interface FeFsTable extends FeTable {
boolean statsExtrap = Utils.isStatsExtrapolationEnabled(this);
// Add stats column headers
resultSchema.addToColumns(new TColumn("#Rows", Type.BIGINT.toThrift()));
if (statsExtrap) {
resultSchema.addToColumns(new TColumn("Extrap #Rows", Type.BIGINT.toThrift()));
@@ -482,93 +485,170 @@ public interface FeFsTable extends FeTable {
resultSchema.addToColumns(new TColumn("Location", Type.STRING.toThrift()));
resultSchema.addToColumns(new TColumn("EC Policy", Type.STRING.toThrift()));
// Pretty print partitions and their stats.
List<FeFsPartition> orderedPartitions = new ArrayList<>(loadAllPartitions());
return statsExtrap;
}
/**
* Helper method to build a stats row for a single partition.
* @param p The partition to build stats for
* @param statsExtrap Whether stats extrapolation is enabled
* @return An array containing [cachedBytes, totalBytes, numFiles, numRows]
* for aggregation
*/
default long[] buildPartitionStatsRow(
FeFsPartition p, boolean statsExtrap, TResultSet result) {
long numFiles = p.getNumFileDescriptors();
long size = p.getSize();
long cachedBytes = 0L;
long numRows = p.getNumRows();
TResultRowBuilder rowBuilder = new TResultRowBuilder();
// Add partition key values
for (LiteralExpr expr: p.getPartitionValues()) {
rowBuilder.add(expr.getStringValue());
}
// Add row counts
rowBuilder.add(numRows);
// Compute and report the extrapolated row count because the set of files could
// have changed since we last computed stats for this partition. We also follow
// this policy during scan-cardinality estimation.
if (statsExtrap) {
rowBuilder.add(Utils.getExtrapolatedNumRows(this, size));
}
// Add file stats
rowBuilder.add(numFiles).addBytes(size);
// Add cache stats
if (!p.isMarkedCached()) {
// Helps to differentiate partitions that have 0B cached versus partitions
// that are not marked as cached.
rowBuilder.add("NOT CACHED");
rowBuilder.add("NOT CACHED");
} else {
// Calculate cached bytes
for (FileDescriptor fd: p.getFileDescriptors()) {
int numBlocks = fd.getNumFileBlocks();
for (int i = 0; i < numBlocks; ++i) {
FbFileBlock block = fd.getFbFileBlock(i);
if (FileBlock.hasCachedReplica(block)) {
cachedBytes += FileBlock.getLength(block);
}
}
}
rowBuilder.addBytes(cachedBytes);
// Extract cache replication factor from the parameters of the table
// if the table is not partitioned or directly from the partition.
Short rep = HdfsCachingUtil.getCachedCacheReplication(
getNumClusteringCols() == 0 ?
p.getTable().getMetaStoreTable().getParameters() :
p.getParameters());
rowBuilder.add(rep.toString());
}
// Add format, incremental stats, location, and EC policy
rowBuilder.add(p.getFileFormat().toString());
rowBuilder.add(String.valueOf(p.hasIncrementalStats()));
rowBuilder.add(p.getLocation());
rowBuilder.add(FileSystemUtil.getErasureCodingPolicy(p.getLocationPath()));
result.addToRows(rowBuilder.get());
return new long[] { cachedBytes, size, numFiles, numRows };
}
/**
* Helper method to build the "Total" summary row for partitioned tables.
* @param totalCachedBytes Total cached bytes across all partitions
* @param totalBytes Total bytes across all partitions
* @param totalNumFiles Total number of files across all partitions
* @param totalNumRows Total number of rows across all partitions
* @param statsExtrap Whether stats extrapolation is enabled
* @param result The result set to add the row to
*/
default void buildTotalStatsRow(long totalCachedBytes, long totalBytes,
long totalNumFiles, long totalNumRows, boolean statsExtrap, TResultSet result) {
TResultRowBuilder rowBuilder = new TResultRowBuilder();
int numEmptyCells = getNumClusteringCols() - 1;
rowBuilder.add("Total");
for (int i = 0; i < numEmptyCells; ++i) {
rowBuilder.add("");
}
// Total rows and extrapolated rows
rowBuilder.add(totalNumRows);
// Compute and report the extrapolated row count because the set of files could
// have changed since we last computed stats for this partition. We also follow
// this policy during scan-cardinality estimation.
if (statsExtrap) {
rowBuilder.add(Utils.getExtrapolatedNumRows(this, getTotalHdfsBytes()));
}
// Total files, bytes, cache stats, and empty fields for format/location/etc
rowBuilder.add(totalNumFiles)
.addBytes(totalBytes)
.addBytes(totalCachedBytes).add("").add("").add("").add("").add("");
result.addToRows(rowBuilder.get());
}
/**
* @return statistics on this table as a tabular result set. Used for the
* SHOW TABLE STATS statement. The schema of the returned TResultSet is set
* inside this method.
*/
default TResultSet getTableStats() {
return getTableStats(null);
}
/**
* @return statistics on the specified subset of partitions of this table as a tabular
* result set. If partitionIds is null, returns stats for all partitions.
* Schema is the same as getTableStats().
*/
default TResultSet getTableStats(@Nullable Collection<Long> partitionIds) {
TResultSet result = new TResultSet();
boolean statsExtrap = buildTableStatsSchema(result);
// Load and sort partitions (all or filtered based on partitionIds)
List<FeFsPartition> orderedPartitions = partitionIds == null ?
new ArrayList<>(loadAllPartitions()) :
new ArrayList<>(loadPartitions(partitionIds));
orderedPartitions.sort(HdfsPartition.KV_COMPARATOR);
// Build rows for each partition and accumulate totals
long totalCachedBytes = 0L;
long totalBytes = 0L;
long totalNumFiles = 0L;
long totalNumRows = 0L;
boolean hasInvalidStats = false;
for (FeFsPartition p: orderedPartitions) {
long numFiles = p.getNumFileDescriptors();
long size = p.getSize();
totalNumFiles += numFiles;
totalBytes += size;
TResultRowBuilder rowBuilder = new TResultRowBuilder();
// Add the partition-key values (as strings for simplicity).
for (LiteralExpr expr: p.getPartitionValues()) {
rowBuilder.add(expr.getStringValue());
}
// Add rows, extrapolated rows, files, bytes, cache stats, and file format.
rowBuilder.add(p.getNumRows());
// Compute and report the extrapolated row count because the set of files could
// have changed since we last computed stats for this partition. We also follow
// this policy during scan-cardinality estimation.
if (statsExtrap) {
rowBuilder.add(Utils.getExtrapolatedNumRows(this, size));
}
rowBuilder.add(numFiles).addBytes(size);
if (!p.isMarkedCached()) {
// Helps to differentiate partitions that have 0B cached versus partitions
// that are not marked as cached.
rowBuilder.add("NOT CACHED");
rowBuilder.add("NOT CACHED");
} else {
// Calculate the number the number of bytes that are cached.
long cachedBytes = 0L;
for (FileDescriptor fd: p.getFileDescriptors()) {
int numBlocks = fd.getNumFileBlocks();
for (int i = 0; i < numBlocks; ++i) {
FbFileBlock block = fd.getFbFileBlock(i);
if (FileBlock.hasCachedReplica(block)) {
cachedBytes += FileBlock.getLength(block);
}
}
long[] stats = buildPartitionStatsRow(p, statsExtrap, result);
totalCachedBytes += stats[0];
totalBytes += stats[1];
totalNumFiles += stats[2];
// For filtered queries, accumulate partition-level row counts
if (partitionIds != null) {
// If any partition has -1 (no stats), mark the total as invalid
if (stats[3] < 0) {
hasInvalidStats = true;
} else {
totalNumRows += stats[3];
}
totalCachedBytes += cachedBytes;
rowBuilder.addBytes(cachedBytes);
// Extract cache replication factor from the parameters of the table
// if the table is not partitioned or directly from the partition.
Short rep = HdfsCachingUtil.getCachedCacheReplication(
getNumClusteringCols() == 0 ?
p.getTable().getMetaStoreTable().getParameters() :
p.getParameters());
rowBuilder.add(rep.toString());
}
rowBuilder.add(p.getFileFormat().toString());
rowBuilder.add(String.valueOf(p.hasIncrementalStats()));
rowBuilder.add(p.getLocation());
rowBuilder.add(FileSystemUtil.getErasureCodingPolicy(p.getLocationPath()));
result.addToRows(rowBuilder.get());
}
// For partitioned tables add a summary row at the bottom.
if (getNumClusteringCols() > 0) {
TResultRowBuilder rowBuilder = new TResultRowBuilder();
int numEmptyCells = getNumClusteringCols() - 1;
rowBuilder.add("Total");
for (int i = 0; i < numEmptyCells; ++i) {
rowBuilder.add("");
}
// Total rows, extrapolated rows, files, bytes, cache stats.
// Leave format empty.
rowBuilder.add(getNumRows());
// Compute and report the extrapolated row count because the set of files could
// have changed since we last computed stats for this partition. We also follow
// this policy during scan-cardinality estimation.
if (statsExtrap) {
rowBuilder.add(Utils.getExtrapolatedNumRows(this, getTotalHdfsBytes()));
}
rowBuilder.add(totalNumFiles)
.addBytes(totalBytes)
.addBytes(totalCachedBytes).add("").add("").add("").add("").add("");
result.addToRows(rowBuilder.get());
// Add summary row for partitioned tables.
// When filtering by partitions, only show Total row if there are matching partitions
if (getNumClusteringCols() > 0 &&
(partitionIds == null || !orderedPartitions.isEmpty())) {
// When showing all partitions, use table-level stats.
// When filtering, sum partition stats.
long totalRows = partitionIds == null ? getNumRows() :
(hasInvalidStats ? -1 : totalNumRows);
buildTotalStatsRow(totalCachedBytes, totalBytes,
totalNumFiles, totalRows, statsExtrap, result);
}
return result;
}

View File

@@ -125,9 +125,12 @@ public class HdfsPartitionFilter {
LOG.trace("buildPartitionPredicate: " + literalPredicate.toSql() + " " +
literalPredicate.debugString());
}
if (!literalPredicate.isConstant()) {
// After substitution, the predicate should not contain any SlotRefs.
// Non-deterministic functions are allowed and will be evaluated by the backend.
if (literalPredicate.contains(SlotRef.class)) {
throw new NotImplementedException(
"Unsupported non-deterministic predicate: " + predicate_.toSql());
"Unsupported predicate with unsubstituted slot references: " +
predicate_.toSql());
}
return literalPredicate;
}

View File

@@ -110,9 +110,13 @@ public class HdfsPartitionPruner {
* shown in the EXPLAIN output.
*
* If 'allowEmpty' is False, empty partitions are not returned.
* If 'evalAllFuncs' is True, non-deterministic functions will be evaluated
* per-partition instead of being skipped. Callers should ensure that all
* conjuncts are evaluable in the backend to avoid crashes (e.g., aggregate
* functions will crash the backend and must be rejected before calling this).
*/
public Pair<List<? extends FeFsPartition>, List<Expr>> prunePartitions(
Analyzer analyzer, List<Expr> conjuncts, boolean allowEmpty,
Analyzer analyzer, List<Expr> conjuncts, boolean allowEmpty, boolean evalAllFuncs,
TableRef hdfsTblRef)
throws ImpalaException {
// Start with creating a collection of partition filters for the applicable conjuncts.
@@ -129,7 +133,7 @@ public class HdfsPartitionPruner {
Iterator<Expr> it = conjuncts.iterator();
while (it.hasNext()) {
Expr conjunct = it.next();
if (isPartitionPrunedFilterConjunct(partitionSlots_, conjunct)) {
if (isPartitionPrunedFilterConjunct(partitionSlots_, conjunct, evalAllFuncs)) {
// Check if the conjunct can be evaluated from the partition metadata.
// Use a cloned conjunct to rewrite BetweenPredicates and allow
// canEvalUsingPartitionMd() to fold constant expressions without modifying
@@ -189,9 +193,14 @@ public class HdfsPartitionPruner {
}
public static boolean isPartitionPrunedFilterConjunct(List<SlotId> partitionSlots,
Expr conjunct) {
return conjunct.isBoundBySlotIds(partitionSlots) &&
!conjunct.contains(Expr.IS_NONDETERMINISTIC_BUILTIN_FN_PREDICATE);
Expr conjunct, boolean evalAllFuncs) {
if (!conjunct.isBoundBySlotIds(partitionSlots)) return false;
// Aggregate functions cannot be evaluated per-partition and will crash the backend.
if (conjunct.contains(Expr.IS_AGGREGATE)) return false;
// If evalAllFuncs is true, allow non-deterministic functions to be evaluated
// per-partition instead of skipping them.
if (evalAllFuncs) return true;
return !conjunct.contains(Expr.IS_NONDETERMINISTIC_BUILTIN_FN_PREDICATE);
}
/**

View File

@@ -1611,7 +1611,7 @@ public class SingleNodePlanner implements SingleNodePlannerIntf {
// end up removing some predicates.
HdfsPartitionPruner pruner = new HdfsPartitionPruner(tupleDesc);
Pair<List<? extends FeFsPartition>, List<Expr>> pair =
pruner.prunePartitions(analyzer, conjuncts, false, hdfsTblRef);
pruner.prunePartitions(analyzer, conjuncts, false, false, hdfsTblRef);
List<? extends FeFsPartition> partitions = pair.first;
// Mark all slots referenced by the remaining conjuncts as materialized.

View File

@@ -1737,20 +1737,22 @@ public class Frontend {
/**
* Generate result set and schema for a SHOW TABLE STATS command.
*/
public TResultSet getTableStats(String dbName, String tableName, TShowStatsOp op)
public TResultSet getTableStats(String dbName, String tableName, TShowStatsOp op,
List<Long> filteredPartitionIds)
throws ImpalaException {
RetryTracker retries = new RetryTracker(
String.format("fetching table stats from %s.%s", dbName, tableName));
while (true) {
try {
return doGetTableStats(dbName, tableName, op);
return doGetTableStats(dbName, tableName, op, filteredPartitionIds);
} catch(InconsistentMetadataFetchException e) {
retries.handleRetryOrThrow(e);
}
}
}
private TResultSet doGetTableStats(String dbName, String tableName, TShowStatsOp op)
private TResultSet doGetTableStats(String dbName, String tableName, TShowStatsOp op,
List<Long> filteredPartitionIds)
throws ImpalaException {
FeTable table = getCatalog().getTable(dbName, tableName);
if (table instanceof FeFsTable) {
@@ -1760,6 +1762,10 @@ public class Frontend {
if (table instanceof FeIcebergTable && op == TShowStatsOp.TABLE_STATS) {
return FeIcebergTable.Utils.getTableStats((FeIcebergTable) table);
}
if (op == TShowStatsOp.PARTITIONS && filteredPartitionIds != null) {
// When filteredPartitionIds is set (even if empty), use it to filter results.
return ((FeFsTable) table).getTableStats(filteredPartitionIds);
}
return ((FeFsTable) table).getTableStats();
} else if (table instanceof FeHBaseTable) {
return ((FeHBaseTable) table).getTableStats();

View File

@@ -486,7 +486,9 @@ public class JniFrontend {
params.isSetShow_column_minmax_stats() && params.show_column_minmax_stats);
} else {
result = frontend_.getTableStats(params.getTable_name().getDb_name(),
params.getTable_name().getTable_name(), params.op);
params.getTable_name().getTable_name(), params.op,
params.isSetFiltered_partition_ids() ?
params.getFiltered_partition_ids() : null);
}
try {
TSerializer serializer = new TSerializer(protocolFactory_);

View File

@@ -4506,6 +4506,88 @@ public class AnalyzeDDLTest extends FrontendTestBase {
"SHOW PARTITIONS not applicable to a view: functional.view_view");
AnalysisError("show partitions functional_hbase.alltypes",
"SHOW PARTITIONS must target an HDFS or Kudu table: functional_hbase.alltypes");
// Tests for WHERE clause in SHOW PARTITIONS (IMPALA-14065)
// Valid WHERE clauses with partition columns on HDFS tables
AnalyzesOk("show partitions functional.alltypes where year = 2009");
AnalyzesOk("show partitions functional.alltypes where year = 2009 and month = 1");
AnalyzesOk("show partitions functional.alltypes where year > 2009");
AnalyzesOk("show partitions functional.alltypes where year >= 2009 and month < 10");
AnalyzesOk("show partitions functional.alltypes where year in (2009, 2010)");
AnalyzesOk("show partitions functional.alltypes where year between 2009 and 2010");
AnalyzesOk("show partitions functional.alltypes where month is null");
AnalyzesOk("show partitions functional.alltypes where month is not null");
AnalyzesOk("show partitions functional.alltypes where year = 2009 or month = 5");
// WHERE clause with non-partition columns should fail
AnalysisError("show partitions functional.alltypes where id = 1",
"SHOW PARTITIONS WHERE supports only partition columns");
AnalysisError("show partitions functional.alltypes where year = 2009 and id = 1",
"SHOW PARTITIONS WHERE supports only partition columns");
// WHERE clause with subqueries should fail
AnalysisError("show partitions functional.alltypes where year in " +
"(select year from functional.alltypes)",
"Subqueries are not allowed in SHOW PARTITIONS WHERE");
// WHERE clause with analytic functions should fail
AnalysisError("show partitions functional.alltypes where " +
"row_number() over (order by year) = 1",
"Analytic expressions are not allowed in SHOW PARTITIONS WHERE");
// WHERE clause with Kudu table should fail (non-HDFS tables don't support WHERE)
AnalysisError("show partitions functional_kudu.alltypes where year = 2009",
"WHERE clause in SHOW PARTITIONS is only supported for HDFS tables");
// WHERE clause with Iceberg table should fail
AnalysisError("show partitions functional_parquet.iceberg_int_partitioned " +
"where i = 1",
"WHERE clause in SHOW PARTITIONS is only supported for HDFS tables");
// WHERE clause must be a boolean expression
AnalysisError("show partitions functional.alltypes where year",
"WHERE clause '`year`' requires return type 'BOOLEAN'. Actual type is 'INT'.");
// Tests for more complex WHERE clause predicates
// String operations: LIKE and REGEXP
AnalyzesOk("show partitions functional.stringpartitionkey where " +
"string_col like '2%'");
AnalyzesOk("show partitions functional.stringpartitionkey where " +
"string_col regexp '^partition'");
// Arithmetic expressions
AnalyzesOk("show partitions functional.alltypes where month + 1 = 2");
AnalyzesOk("show partitions functional.alltypes where year * 2 > 4000");
// CAST expressions
AnalyzesOk("show partitions functional.stringpartitionkey where " +
"cast(string_col as timestamp) > '2009-01-01'");
// CASE expressions
AnalyzesOk("show partitions functional.alltypes where " +
"case when year > 2009 then 1 else 0 end = 1");
AnalyzesOk("show partitions functional.alltypes where " +
"case when month <= 6 then 'H1' else 'H2' end = 'H1'");
// Non-deterministic functions
// rand() - should evaluate per partition
AnalyzesOk("show partitions functional.alltypes where month = rand()");
AnalyzesOk("show partitions functional.alltypes where " +
"month <= floor(rand() * 12) + 1");
AnalyzesOk("show partitions functional.alltypes where rand() < 0.5");
// uuid() - should evaluate per partition
AnalyzesOk("show partitions functional.alltypes where month = length(uuid()) / 3");
AnalyzesOk("show partitions functional.alltypes where length(uuid()) > 30");
// now() - should evaluate per partition
AnalyzesOk("show partitions functional.alltypes where month = month(now())");
AnalyzesOk("show partitions functional.alltypes where year <= year(now())");
// Complex expressions with non-deterministic functions
AnalyzesOk("show partitions functional.stringpartitionkey where " +
"string_col like cast(concat(cast(floor(rand() * 10) as string), '%') " +
"as string)");
}
@Test

View File

@@ -2130,6 +2130,26 @@ public class ParserTest extends FrontendTestBase {
ParsesOk("SHOW PARTITIONS tbl");
ParsesOk("SHOW PARTITIONS db.tbl");
ParsesOk("SHOW PARTITIONS `db`.`tbl`");
ParsesOk("SHOW PARTITIONS tbl WHERE year = 2025 and month < 6");
// Show partitions with various WHERE clause expressions (IMPALA-14065)
ParsesOk("SHOW PARTITIONS tbl WHERE year IN (2009, 2010)");
ParsesOk("SHOW PARTITIONS tbl WHERE year BETWEEN 2009 AND 2010");
ParsesOk("SHOW PARTITIONS tbl WHERE month IS NULL");
ParsesOk("SHOW PARTITIONS tbl WHERE month IS NOT NULL");
ParsesOk("SHOW PARTITIONS tbl WHERE string_col LIKE '2%'");
ParsesOk("SHOW PARTITIONS tbl WHERE string_col REGEXP '^partition'");
ParsesOk("SHOW PARTITIONS tbl WHERE month + 1 = 2");
ParsesOk("SHOW PARTITIONS tbl WHERE CAST(string_col AS INT) > 0");
ParsesOk("SHOW PARTITIONS tbl WHERE CASE WHEN year > 2009 THEN 1 ELSE 0 END = 1");
// Show partitions with non-deterministic functions (IMPALA-14065)
ParsesOk("SHOW PARTITIONS tbl WHERE month = rand()");
ParsesOk("SHOW PARTITIONS tbl WHERE month <= floor(rand() * 12) + 1");
ParsesOk("SHOW PARTITIONS tbl WHERE rand() < 0.5");
ParsesOk("SHOW PARTITIONS tbl WHERE month = length(uuid()) / 3");
ParsesOk("SHOW PARTITIONS tbl WHERE year <= year(now())");
ParsesOk("SHOW PARTITIONS tbl WHERE month = month(now())");
// Show range partitions
ParsesOk("SHOW RANGE PARTITIONS tbl");

View File

@@ -61,8 +61,7 @@ public class PrunedPartitionHelper {
List<Expr> conjuncts = converter.getImpalaConjuncts();
// IMPALA-13849: tblref is null. Tablesampling is disabled.
Pair<List<? extends FeFsPartition>, List<Expr>> impalaPair =
pruner.prunePartitions(analyzer, new ArrayList<>(conjuncts), true,
null);
pruner.prunePartitions(analyzer, new ArrayList<>(conjuncts), true, false, null);
prunedPartitions_ = impalaPair.first;
@@ -73,7 +72,8 @@ public class PrunedPartitionHelper {
List<SlotId> partitionSlots = tupleDesc.getPartitionSlots();
for (Expr conjunct : conjuncts) {
if (HdfsPartitionPruner.isPartitionPrunedFilterConjunct(partitionSlots, conjunct)) {
if (HdfsPartitionPruner.isPartitionPrunedFilterConjunct(partitionSlots, conjunct,
false)) {
partitionedConjBuilder.add(conjunct);
} else {
nonPartitionedConjBuilder.add(conjunct);

View File

@@ -162,8 +162,7 @@ public class CalciteTable extends RelOptAbstractTable
// TODO: pass in the conjuncts needed. An empty conjunct will return all partitions.
List<Expr> conjuncts = new ArrayList<>();
Pair<List<? extends FeFsPartition>, List<Expr>> impalaPair =
pruner.prunePartitions(analyzer, conjuncts, true,
null);
pruner.prunePartitions(analyzer, conjuncts, true, false, null);
return impalaPair.first;
}

View File

@@ -186,3 +186,379 @@ p, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STA
'Total',4,4,'8B','0B','','','','',''
---- TYPES
STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - filter by year
show partitions alltypes where year = 2009
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','1',310,1,'19.95KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=1','$ERASURECODE_POLICY'
'2009','2',280,1,'18.12KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=2','$ERASURECODE_POLICY'
'2009','3',310,1,'20.06KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=3','$ERASURECODE_POLICY'
'2009','4',300,1,'19.61KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=4','$ERASURECODE_POLICY'
'2009','5',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=5','$ERASURECODE_POLICY'
'2009','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=6','$ERASURECODE_POLICY'
'2009','7',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=7','$ERASURECODE_POLICY'
'2009','8',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=8','$ERASURECODE_POLICY'
'2009','9',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=9','$ERASURECODE_POLICY'
'2009','10',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=10','$ERASURECODE_POLICY'
'2009','11',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=11','$ERASURECODE_POLICY'
'2009','12',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=12','$ERASURECODE_POLICY'
'Total','',3650,12,'238.68KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - filter by year and month
show partitions alltypes where year = 2009 and month <= 3
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','1',310,1,'19.95KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=1','$ERASURECODE_POLICY'
'2009','2',280,1,'18.12KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=2','$ERASURECODE_POLICY'
'2009','3',310,1,'20.06KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=3','$ERASURECODE_POLICY'
'Total','',900,3,'58.14KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - filter with IN predicate
show partitions alltypes where year = 2010 and month in (1, 6, 12)
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2010','1',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=1','$ERASURECODE_POLICY'
'2010','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=6','$ERASURECODE_POLICY'
'2010','12',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=12','$ERASURECODE_POLICY'
'Total','',920,3,'60.43KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - no matches returns empty result
show partitions alltypes where year = 2025
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - using BETWEEN
show partitions alltypes where year = 2009 and month between 10 and 12
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','10',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=10','$ERASURECODE_POLICY'
'2009','11',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=11','$ERASURECODE_POLICY'
'2009','12',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=12','$ERASURECODE_POLICY'
'Total','',920,3,'60.43KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - using OR
show partitions alltypes where (year = 2009 and month = 1) or (year = 2010 and month = 12)
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','1',310,1,'19.95KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=1','$ERASURECODE_POLICY'
'2010','12',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=12','$ERASURECODE_POLICY'
'Total','',620,2,'40.32KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
---- QUERY
# SHOW PARTITIONS with WHERE clause - NOT operator
show partitions alltypes where year = 2009 and not (month <= 2 or month >= 11)
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','3',310,1,'20.06KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=3','$ERASURECODE_POLICY'
'2009','4',300,1,'19.61KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=4','$ERASURECODE_POLICY'
'2009','5',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=5','$ERASURECODE_POLICY'
'2009','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=6','$ERASURECODE_POLICY'
'2009','7',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=7','$ERASURECODE_POLICY'
'2009','8',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=8','$ERASURECODE_POLICY'
'2009','9',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=9','$ERASURECODE_POLICY'
'2009','10',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=10','$ERASURECODE_POLICY'
'Total','',2450,8,'160.54KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - NOT EQUAL operator
show partitions alltypes where year = 2009 and month != 12
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','1',310,1,'19.95KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=1','$ERASURECODE_POLICY'
'2009','2',280,1,'18.12KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=2','$ERASURECODE_POLICY'
'2009','3',310,1,'20.06KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=3','$ERASURECODE_POLICY'
'2009','4',300,1,'19.61KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=4','$ERASURECODE_POLICY'
'2009','5',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=5','$ERASURECODE_POLICY'
'2009','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=6','$ERASURECODE_POLICY'
'2009','7',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=7','$ERASURECODE_POLICY'
'2009','8',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=8','$ERASURECODE_POLICY'
'2009','9',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=9','$ERASURECODE_POLICY'
'2009','10',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=10','$ERASURECODE_POLICY'
'2009','11',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=11','$ERASURECODE_POLICY'
'Total','',3340,11,'218.32KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - using OR conditions
show partitions alltypes where month = 1 or month = 6
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','1',310,1,'19.95KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=1','$ERASURECODE_POLICY'
'2009','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=6','$ERASURECODE_POLICY'
'2010','1',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=1','$ERASURECODE_POLICY'
'2010','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=6','$ERASURECODE_POLICY'
'Total','',1220,4,'79.73KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - using OR conditions but using "||" operator
show partitions alltypes where month = 1 || month = 6
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','1',310,1,'19.95KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=1','$ERASURECODE_POLICY'
'2009','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=6','$ERASURECODE_POLICY'
'2010','1',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=1','$ERASURECODE_POLICY'
'2010','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=6','$ERASURECODE_POLICY'
'Total','',1220,4,'79.73KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - complex nested conditions
show partitions alltypes where (year = 2009 and (month = 1 or month = 12)) or (year = 2010 and month between 6 and 7)
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','1',310,1,'19.95KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=1','$ERASURECODE_POLICY'
'2009','12',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=12','$ERASURECODE_POLICY'
'2010','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=6','$ERASURECODE_POLICY'
'2010','7',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=7','$ERASURECODE_POLICY'
'Total','',1230,4,'80.39KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - testing with string partition column
show partitions functional.part_strings_with_quotes where p = '"'
---- LABELS
p, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'"',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/part_strings_with_quotes/p=%22','$ERASURECODE_POLICY'
'Total',1,1,'2B','0B','','','','',''
---- TYPES
STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - IS NULL predicate
show partitions alltypes where year IS NULL
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - LIKE predicate on string partition
show partitions functional.stringpartitionkey where string_col LIKE 'p%'
---- LABELS
STRING_COL, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'partition1',0,0,'0B','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/stringpartitionkey/string_col=partition1','$ERASURECODE_POLICY'
'Total',0,0,'0B','0B','','','','',''
---- TYPES
STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - REGEXP predicate on string partition
show partitions functional.stringpartitionkey where string_col REGEXP '^2'
---- LABELS
STRING_COL, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009-01-01 00:00:00',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/stringpartitionkey/string_col=2009-01-01 00%3A00%3A00','$ERASURECODE_POLICY'
'Total',1,1,'2B','0B','','','','',''
---- TYPES
STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - arithmetic expression
show partitions alltypes where month + 1 = 2
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','1',310,1,'19.95KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=1','$ERASURECODE_POLICY'
'2010','1',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=1','$ERASURECODE_POLICY'
'Total','',620,2,'40.32KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - CAST to TIMESTAMP
show partitions functional.stringpartitionkey where CAST(string_col AS TIMESTAMP) = CAST('2009-01-01 00:00:00' AS TIMESTAMP)
---- LABELS
STRING_COL, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009-01-01 00:00:00',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/stringpartitionkey/string_col=2009-01-01 00%3A00%3A00','$ERASURECODE_POLICY'
'Total',1,1,'2B','0B','','','','',''
---- TYPES
STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - CASE expression
show partitions alltypes where CASE WHEN year > 2009 THEN 1 ELSE 0 END = 1
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2010','1',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=1','$ERASURECODE_POLICY'
'2010','2',280,1,'18.39KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=2','$ERASURECODE_POLICY'
'2010','3',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=3','$ERASURECODE_POLICY'
'2010','4',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=4','$ERASURECODE_POLICY'
'2010','5',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=5','$ERASURECODE_POLICY'
'2010','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=6','$ERASURECODE_POLICY'
'2010','7',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=7','$ERASURECODE_POLICY'
'2010','8',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=8','$ERASURECODE_POLICY'
'2010','9',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=9','$ERASURECODE_POLICY'
'2010','10',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=10','$ERASURECODE_POLICY'
'2010','11',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=11','$ERASURECODE_POLICY'
'2010','12',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=12','$ERASURECODE_POLICY'
'Total','',3650,12,'239.77KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - uuid() non-deterministic function
show partitions alltypes where month = length(uuid()) / 3
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','12',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=12','$ERASURECODE_POLICY'
'2010','12',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=12','$ERASURECODE_POLICY'
'Total','',620,2,'40.73KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# SHOW PARTITIONS with WHERE clause - LIKE with CAST and concat
show partitions functional.stringpartitionkey where string_col LIKE CAST(concat('2', '%') AS STRING)
---- LABELS
STRING_COL, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009-01-01 00:00:00',1,1,'2B','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/stringpartitionkey/string_col=2009-01-01 00%3A00%3A00','$ERASURECODE_POLICY'
'Total',1,1,'2B','0B','','','','',''
---- TYPES
STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# Negative test: SHOW PARTITIONS WHERE not supported on Iceberg tables
show partitions functional_parquet.iceberg_int_partitioned where i = 1
---- CATCH
WHERE clause in SHOW PARTITIONS is only supported for HDFS tables
====
---- QUERY
# Negative test: SHOW PARTITIONS WHERE with non-partition column
show partitions alltypes where id = 1
---- CATCH
SHOW PARTITIONS WHERE supports only partition columns
====
---- QUERY
# Negative test: SHOW PARTITIONS WHERE with non-existent column
show partitions alltypes where nonexistent_col = 1
---- CATCH
Could not resolve column/field reference: 'nonexistent_col'
====
---- QUERY
# Negative test: SHOW PARTITIONS WHERE with non-boolean expression
show partitions alltypes where year
---- CATCH
WHERE clause '`year`' requires return type 'BOOLEAN'. Actual type is 'INT'.
====
---- QUERY
# Negative test: SHOW PARTITIONS WHERE with aggregate function
show partitions alltypes where count(year) > 0
---- CATCH
Aggregate functions are not allowed in SHOW PARTITIONS WHERE
====
---- QUERY
# Negative test: SHOW PARTITIONS WHERE with analytic function
show partitions alltypes where row_number() over (order by year) = 1
---- CATCH
Analytic expressions are not allowed in SHOW PARTITIONS WHERE
====
---- QUERY
# Negative test: SHOW PARTITIONS WHERE on Kudu table
show partitions functional_kudu.alltypes where year = 2009
---- CATCH
WHERE clause in SHOW PARTITIONS is only supported for HDFS tables
====
---- QUERY
# Negative test: SHOW PARTITIONS WHERE with subquery
show partitions alltypes where year in (select year from alltypes limit 1)
---- CATCH
Subqueries are not allowed in SHOW PARTITIONS WHERE
====
---- QUERY
# Negative test: SHOW PARTITIONS WHERE on non-partitioned table
show partitions alltypesaggmultifilesnopart where 1=1
---- CATCH
Table is not partitioned: functional.alltypesaggmultifilesnopart
====
---- QUERY
# Negative test: SHOW PARTITIONS WHERE with type mismatch
show partitions alltypes where year = 'abc'
---- CATCH
operands of type INT and STRING are not comparable: `year` = 'abc'
====
---- QUERY
# Edge case: SHOW PARTITIONS WHERE with always false predicate
show partitions alltypes where 1 = 0
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====
---- QUERY
# Edge case: SHOW PARTITIONS WHERE with always true predicate (shows all partitions)
show partitions alltypes where 1 = 1
---- LABELS
YEAR, MONTH, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION, EC POLICY
---- RESULTS
'2009','1',310,1,'19.95KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=1','$ERASURECODE_POLICY'
'2009','2',280,1,'18.12KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=2','$ERASURECODE_POLICY'
'2009','3',310,1,'20.06KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=3','$ERASURECODE_POLICY'
'2009','4',300,1,'19.61KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=4','$ERASURECODE_POLICY'
'2009','5',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=5','$ERASURECODE_POLICY'
'2009','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=6','$ERASURECODE_POLICY'
'2009','7',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=7','$ERASURECODE_POLICY'
'2009','8',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=8','$ERASURECODE_POLICY'
'2009','9',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=9','$ERASURECODE_POLICY'
'2009','10',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=10','$ERASURECODE_POLICY'
'2009','11',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=11','$ERASURECODE_POLICY'
'2009','12',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2009/month=12','$ERASURECODE_POLICY'
'2010','1',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=1','$ERASURECODE_POLICY'
'2010','2',280,1,'18.39KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=2','$ERASURECODE_POLICY'
'2010','3',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=3','$ERASURECODE_POLICY'
'2010','4',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=4','$ERASURECODE_POLICY'
'2010','5',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=5','$ERASURECODE_POLICY'
'2010','6',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=6','$ERASURECODE_POLICY'
'2010','7',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=7','$ERASURECODE_POLICY'
'2010','8',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=8','$ERASURECODE_POLICY'
'2010','9',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=9','$ERASURECODE_POLICY'
'2010','10',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=10','$ERASURECODE_POLICY'
'2010','11',300,1,'19.71KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=11','$ERASURECODE_POLICY'
'2010','12',310,1,'20.36KB','NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/alltypes/year=2010/month=12','$ERASURECODE_POLICY'
'Total','',7300,24,'478.45KB','0B','','','','',''
---- TYPES
STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING, STRING
====

View File

@@ -20,6 +20,7 @@
from __future__ import absolute_import, division, print_function
import pytest
import re
import datetime
from tests.common.impala_test_suite import ImpalaTestSuite
from tests.common.skip import SkipIfFS, SkipIfCatalogV2
@@ -66,6 +67,44 @@ class TestMetadataQueryStatements(ImpalaTestSuite):
def test_show_stats(self, vector):
self.run_test_case('QueryTest/show-stats', vector, "functional")
def test_show_partitions_with_nondeterministic_functions(self):
"""Test SHOW PARTITIONS WHERE with non-deterministic functions
like rand() and now().
"""
# Test rand() - just verify the statement succeeds without errors
result = self.execute_query("show partitions functional.alltypes where rand() < 0.5")
assert result.success, "SHOW PARTITIONS with rand() should succeed"
# Verify we got some partitions back (rand() typically returns ~0.47 without seed)
assert len(result.data) > 0, "SHOW PARTITIONS with rand() should return some \
partitions"
# Test now() - verify the statement succeeds and returns partitions for current month
current_month = datetime.datetime.now().month
result = self.execute_query(
"show partitions functional.alltypes where month = month(now())")
assert result.success, "SHOW PARTITIONS with now() should succeed"
# Verify we got exactly the partitions for the current month
# alltypes has 2 years (2009, 2010) with all 12 months
# So we should get 2 partitions (one per year) for the current month
partition_months = []
for row in result.data:
# Skip the 'Total' row
if 'Total' not in row:
parts = row.split('\t')
if len(parts) >= 2:
partition_months.append(int(parts[1]))
# All returned partitions should be for the current month
for month in partition_months:
assert month == current_month, \
"Expected month {0}, got {1}".format(current_month, month)
# We should have 2 partitions (year=2009/month=N and year=2010/month=N)
assert len(partition_months) == 2, \
"Expected 2 partitions for current month, got {0}".format(len(partition_months))
def test_describe_path(self, vector, unique_database):
self.run_test_case('QueryTest/describe-path', vector, unique_database)