IMPALA-14108: Add support for SHOW FILES IN table PARTITION for Iceberg

tables

This patch implements partition filtering support for the SHOW FILES
statement on Iceberg tables, based on the functionality added in
IMPALA-12243. Prior to this change, the syntax resulted in a
NullPointerException.

Key changes:
- Added ShowFilesStmt.analyzeIceberg() to validate and transform
  partition expressions using IcebergPartitionExpressionRewriter and
  IcebergPartitionPredicateConverter. After that, it collects matching
  file paths using IcebergUtil.planFiles().
- Added FeIcebergTable.Utils.getIcebergTableFilesFromPaths() to
  accept pre-filtered file lists from the analysis phase.
- Enhanced TShowFilesParams thrift struct with optional selected_files
  field to pass pre-filtered file paths from frontend to backend.

Testing:
- Analyzer tests for negative cases: non-existent partitions, invalid
  expressions, non-partition columns, unsupported transforms.
- Analyzer tests for positive cases: all transform types, complex
  expressions.
- Authorization tests for non-filtered and filtered syntaxes.
- E2E tests covering every partition transform type with various
  predicates.
- Schema evolution and rollback scenarios.

The implementation follows AlterTableDropPartition's pattern where the
analysis phase performs validation/metadata retrieval and the execution
phase handles result formatting and display.

Change-Id: Ibb9913e078e6842861bdbb004ed5d67286bd3152
Reviewed-on: http://gerrit.cloudera.org:8080/23455
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:
Mihaly Szjatinya
2025-10-04 15:53:24 +02:00
committed by Impala Public Jenkins
parent 275f03f10d
commit 087b715a2b
10 changed files with 675 additions and 17 deletions

View File

@@ -299,6 +299,10 @@ struct TShowFilesParams {
// An optional partition set. Set if this operation should apply to a list of
// partitions rather than the base table.
2: optional list<list<CatalogObjects.TPartitionKeyValue>> partition_set
// File paths for Iceberg tables (and potentially other formats),
// pre-collected during analysis phase
3: optional list<string> selected_files
}
// Parameters for SHOW [CURRENT] ROLES and SHOW ROLE GRANT GROUP <groupName> commands

View File

@@ -240,7 +240,7 @@ public class AnalysisContext {
public boolean isSingleColumnPrivStmt() {
return isDescribeTableStmt() || isResetMetadataStmt() || isUseStmt()
|| isShowTablesStmt() || isShowMetadataTablesStmt() || isShowViewsStmt()
|| isAlterTableStmt() || isShowFunctionsStmt();
|| isAlterTableStmt() || isShowFunctionsStmt() || isShowFilesStmt();
}
public boolean isConvertTableToIcebergStmt() {

View File

@@ -17,15 +17,31 @@
package org.apache.impala.analysis;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.iceberg.DeleteFile;
import org.apache.iceberg.FileScanTask;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.impala.authorization.Privilege;
import org.apache.impala.catalog.FeIcebergTable;
import org.apache.impala.catalog.FeFsTable;
import org.apache.impala.catalog.FeTable;
import org.apache.impala.catalog.TableLoadingException;
import org.apache.impala.catalog.paimon.FeShowFileStmtSupport;
import org.apache.impala.common.AnalysisException;
import org.apache.impala.common.IcebergPartitionPredicateConverter;
import org.apache.impala.common.IcebergPredicateConverter;
import org.apache.impala.common.ImpalaException;
import org.apache.impala.thrift.TShowFilesParams;
import org.apache.impala.thrift.TTableName;
import org.apache.impala.util.IcebergUtil;
import com.google.common.base.Preconditions;
@@ -45,6 +61,9 @@ public class ShowFilesStmt extends StatementBase implements SingleTableStmt {
// Set during analysis.
protected FeTable table_;
// File paths selected by Iceberg's partition filtering
private List<String> icebergFilePaths_;
public ShowFilesStmt(TableName tableName, PartitionSet partitionSet) {
tableName_ = Preconditions.checkNotNull(tableName);
partitionSet_ = partitionSet;
@@ -72,10 +91,13 @@ public class ShowFilesStmt extends StatementBase implements SingleTableStmt {
// and to allow us to evaluate partition predicates.
TableRef tableRef = new TableRef(tableName_.toPath(), null, Privilege.VIEW_METADATA);
tableRef = analyzer.resolveTableRef(tableRef);
if (tableRef instanceof InlineViewRef ||
tableRef instanceof CollectionTableRef) {
if (tableRef instanceof InlineViewRef) {
throw new AnalysisException(String.format(
"SHOW FILES not applicable to a non hdfs table: %s", tableName_));
"SHOW FILES not allowed on a view: %s", tableName_));
}
if (tableRef instanceof CollectionTableRef) {
throw new AnalysisException(String.format(
"SHOW FILES not allowed on a nested collection: %s", tableName_));
}
table_ = tableRef.getTable();
Preconditions.checkNotNull(table_);
@@ -101,14 +123,85 @@ public class ShowFilesStmt extends StatementBase implements SingleTableStmt {
partitionSet_.setPrivilegeRequirement(Privilege.VIEW_METADATA);
partitionSet_.analyze(analyzer);
}
if (table_ instanceof FeIcebergTable) { analyzeIceberg(analyzer); }
}
public void analyzeIceberg(Analyzer analyzer) throws AnalysisException {
if (partitionSet_ == null) {
icebergFilePaths_ = null;
return;
}
FeIcebergTable table = (FeIcebergTable) table_;
// To rewrite transforms and column references
IcebergPartitionExpressionRewriter rewriter =
new IcebergPartitionExpressionRewriter(analyzer,
table.getIcebergApiTable().spec());
// For Impala expression to Iceberg expression conversion
IcebergPredicateConverter converter =
new IcebergPartitionPredicateConverter(table.getIcebergSchema(), analyzer);
List<Expression> icebergPartitionExprs = new ArrayList<>();
for (Expr expr : partitionSet_.getPartitionExprs()) {
expr = rewriter.rewrite(expr);
expr.analyze(analyzer);
analyzer.getConstantFolder().rewrite(expr, analyzer);
try {
icebergPartitionExprs.add(converter.convert(expr));
} catch (ImpalaException e) {
throw new AnalysisException(
"Invalid partition filtering expression: " + expr.toSql());
}
}
try (CloseableIterable<FileScanTask> fileScanTasks = IcebergUtil.planFiles(table,
icebergPartitionExprs, null, null)) {
// Collect file paths without sorting - sorting will be done in execution phase
List<String> filePaths = new ArrayList<>();
Set<String> uniquePaths = new HashSet<>();
for (FileScanTask fileScanTask : fileScanTasks) {
if (fileScanTask.residual().isEquivalentTo(Expressions.alwaysTrue())) {
// Add delete files
for (DeleteFile deleteFile : fileScanTask.deletes()) {
String path = deleteFile.path().toString();
if (uniquePaths.add(path)) {
filePaths.add(path);
}
}
// Add data file
String dataFilePath = fileScanTask.file().path().toString();
if (uniquePaths.add(dataFilePath)) {
filePaths.add(dataFilePath);
}
}
}
// Store unsorted file paths - lexicographic sorting will be applied in execution
icebergFilePaths_ = filePaths;
if (icebergFilePaths_.isEmpty()) {
throw new AnalysisException("No matching partition(s) found.");
}
} catch (IOException | TableLoadingException e) {
throw new AnalysisException("Error loading metadata for Iceberg table", e);
}
}
public TShowFilesParams toThrift() {
TShowFilesParams params = new TShowFilesParams();
params.setTable_name(new TTableName(table_.getDb().getName(), table_.getName()));
if (partitionSet_ != null) {
// For Iceberg tables, we use selected_files instead of partition_set
if (partitionSet_ != null && !(table_ instanceof FeIcebergTable)) {
params.setPartition_set(partitionSet_.toThrift());
}
// For Iceberg tables: icebergFilePaths_ is null for unfiltered queries (no PARTITION
// clause) or contains the list of matching file paths for filtered queries.
if (table_ instanceof FeIcebergTable && icebergFilePaths_ != null) {
params.setSelected_files(icebergFilePaths_);
}
return params;
}
}

View File

@@ -648,7 +648,7 @@ public interface FeFsTable extends FeTable {
* @return partition file info, ordered by partition
*/
public static TResultSet getFiles(FeFsTable table,
List<List<TPartitionKeyValue>> partitionSet) {
List<List<TPartitionKeyValue>> partitionSet, List<String> icebergFiles) {
TResultSet result = new TResultSet();
TResultSetMetadata resultSchema = new TResultSetMetadata();
result.setSchema(resultSchema);
@@ -659,7 +659,8 @@ public interface FeFsTable extends FeTable {
result.setRows(new ArrayList<>());
if (table instanceof FeIcebergTable) {
return FeIcebergTable.Utils.getIcebergTableFiles((FeIcebergTable) table, result);
return FeIcebergTable.Utils.getIcebergTableFiles(
(FeIcebergTable) table, result, icebergFiles);
}
List<? extends FeFsPartition> orderedPartitions;

View File

@@ -457,8 +457,14 @@ public interface FeIcebergTable extends FeFsTable {
/**
* Get file info for the given fe iceberg table.
*/
public static TResultSet getIcebergTableFiles(FeIcebergTable table,
TResultSet result) {
public static TResultSet getIcebergTableFiles(
FeIcebergTable table, TResultSet result, List<String> icebergFiles) {
// If icebergFiles is provided (not null), use it for filtered results
// Note: Empty lists are handled in frontend analysis and converted to errors
if (icebergFiles != null) {
return getIcebergTableFilesFromPaths(result, icebergFiles, table);
}
// No partition filter was specified, show all files
List<FileDescriptor> orderedFds = Lists.newArrayList(
table.getContentFileStore().getAllFiles());
Collections.sort(orderedFds);
@@ -474,6 +480,40 @@ public interface FeIcebergTable extends FeFsTable {
return result;
}
// Method used when file paths are pre-collected during analysis
public static TResultSet getIcebergTableFilesFromPaths(
TResultSet result, List<String> filePaths, FeIcebergTable table) {
// Sort the file paths for consistent output
List<String> sortedFiles = new ArrayList<>(filePaths);
Collections.sort(sortedFiles);
IcebergContentFileStore contentFileStore = table.getContentFileStore();
for (String filePath : sortedFiles) {
TResultRowBuilder rowBuilder = new TResultRowBuilder();
rowBuilder.add(filePath);
// Try to get actual file size from content file store
String pathHash = IcebergUtil.getFilePathHash(filePath);
FileDescriptor fd = contentFileStore.getDataFileDescriptor(pathHash);
if (fd == null) {
fd = contentFileStore.getDeleteFileDescriptor(pathHash);
}
if (fd != null) {
rowBuilder.add(PrintUtils.printBytes(fd.getFileLength()));
} else {
rowBuilder.add("Unknown");
}
rowBuilder.add(""); // Partition - empty for now
rowBuilder.add(FileSystemUtil.getErasureCodingPolicy(new Path(filePath)));
result.addToRows(rowBuilder.get());
}
return result;
}
/**
* Get partition stats for the given fe iceberg table.
*/

View File

@@ -3466,10 +3466,13 @@ public class Frontend {
if (table instanceof FeShowFileStmtSupport) {
return ((FeShowFileStmtSupport) table).doGetTableFiles(request);
} else if (table instanceof FeFsTable) {
return FeFsTable.Utils.getFiles((FeFsTable)table, request.getPartition_set());
List<String> icebergFiles =
request.isSetSelected_files() ? request.getSelected_files() : null;
return FeFsTable.Utils.getFiles(
(FeFsTable) table, request.getPartition_set(), icebergFiles);
} else {
throw new InternalException("SHOW FILES only supports Hdfs table. " +
"Unsupported table class: " + table.getClass());
throw new InternalException("SHOW FILES only supports Hdfs and Iceberg tables. "
+ "Unsupported table class: " + table.getClass());
}
}

View File

@@ -4399,10 +4399,10 @@ public class AnalyzeDDLTest extends FrontendTestBase {
// Cannot show files on a non hdfs table.
AnalysisError(String.format("show files in functional.alltypes_view %s",
partition),
"SHOW FILES not applicable to a non hdfs table: functional.alltypes_view");
"SHOW FILES not allowed on a view: functional.alltypes_view");
AnalysisError(String.format("show files in allcomplextypes.int_array_col %s",
partition), createAnalysisCtx("functional"),
"SHOW FILES not applicable to a non hdfs table: allcomplextypes.int_array_col");
"SHOW FILES not allowed on a nested collection: allcomplextypes.int_array_col");
}
// Not a partition column.
@@ -4417,6 +4417,63 @@ public class AnalyzeDDLTest extends FrontendTestBase {
// Partition spec does not exist
AnalysisError("show files in functional.alltypes partition(year=2010,month=NULL)",
"No matching partition(s) found.");
// Iceberg SHOW FILES tests
String icebergPartitioned =
"show files in functional_parquet.iceberg_partitioned";
String icebergEvolution =
"show files in functional_parquet.iceberg_partition_evolution";
String icebergNonPartitioned =
"show files in functional_parquet.iceberg_non_partitioned";
// Non-partitioned Iceberg table
AnalysisError(icebergNonPartitioned + " partition (user = 'Alan')",
"Table is not partitioned: functional_parquet.iceberg_non_partitioned");
// Valid Iceberg partition filters
AnalyzesOk(icebergPartitioned + " partition (action = 'click')");
AnalyzesOk(
icebergPartitioned + " partition (action = 'click' or action = 'download')");
AnalyzesOk(icebergPartitioned + " partition (action in ('click', 'download'))");
AnalyzesOk(icebergPartitioned + " partition (hour(event_time) = '2020-01-01-9')");
AnalyzesOk(icebergPartitioned + " partition (hour(event_time) > '2020-01-01-01')");
AnalyzesOk(icebergPartitioned + " partition (hour(event_time) < '2020-02-01-01')");
AnalyzesOk(icebergPartitioned
+ " partition (hour(event_time) in ('2020-01-01-9', '2020-01-01-1'))");
AnalyzesOk(icebergPartitioned
+ " partition (hour(event_time) = '2020-01-01-9', action = 'click')");
AnalyzesOk(icebergEvolution + " partition (truncate(4,date_string_col,4) = '1231')");
AnalyzesOk(icebergEvolution + " partition (month = 12)");
// Error cases for Iceberg partition filters
// Non-existent partition should throw error (consistent with regular tables)
AnalysisError(icebergPartitioned + " partition (action = 'Foo')",
"No matching partition(s) found.");
AnalysisError(icebergPartitioned + " partition (user = 'Alan')",
"Partition exprs cannot contain non-partition column(s): `user`");
AnalysisError(
icebergPartitioned + " partition (user = 'Alan' or user = 'Lisa' and id > 10)",
"Partition exprs cannot contain non-partition column(s): `user`");
AnalysisError(icebergPartitioned + " partition (void(action) = 'click')",
"VOID transform is not supported for partition selection");
AnalysisError(icebergPartitioned + " partition (day(action) = 'Alan')",
"Can't filter column 'action' with transform type: 'DAY'");
AnalysisError(icebergPartitioned + " partition (action = action)",
"Invalid partition filtering expression: action = action");
AnalysisError(icebergPartitioned + " partition (action = action and action = 'click' "
+ "or hour(event_time) > '2020-01-01-01')",
"Invalid partition filtering expression: "
+ "action = action AND action = 'click' OR HOUR(event_time) > 438289");
AnalysisError(icebergPartitioned + " partition (action)",
"Invalid partition filtering expression: action");
AnalysisError(icebergPartitioned + " partition (2)",
"Invalid partition filtering expression: 2");
AnalysisError(icebergPartitioned + " partition (truncate(action))",
"BUCKET and TRUNCATE partition transforms should have a parameter");
AnalysisError(icebergPartitioned + " partition (truncate('string', action))",
"Invalid transform parameter value: string");
AnalysisError(icebergPartitioned + " partition (truncate(1, 2, action))",
"Invalid partition predicate: truncate(1, 2, action)");
}
@Test

View File

@@ -1346,6 +1346,30 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
authzTest.error(accessError("functional"));
}
// Show files for Iceberg tables
AuthzTest authzTestIce =
authorize("show files in functional_parquet.iceberg_partitioned");
for (TPrivilegeLevel privilege : viewMetadataPrivileges()) {
authzTestIce.ok(onServer(privilege))
.ok(onDatabase("functional_parquet", privilege))
.ok(onTable("functional_parquet", "iceberg_partitioned", privilege));
}
authzTestIce.error(accessError("functional_parquet"));
// Show files for Iceberg tables with partition filter; more restrictive
for (AuthzTest authzTest: new AuthzTest[]{
authorize("show files in functional_parquet.iceberg_partitioned " +
"partition(action='view')"),
authorize("show files in functional_parquet.iceberg_partitioned " +
"partition(action='click')")}) {
for (TPrivilegeLevel privilege : new TPrivilegeLevel[] {
TPrivilegeLevel.ALL, TPrivilegeLevel.OWNER, TPrivilegeLevel.SELECT}) {
authzTest.ok(onServer(privilege))
.ok(onDatabase("functional_parquet", privilege))
.ok(onTable("functional_parquet", "iceberg_partitioned", privilege));
}
}
// Show current roles should always be allowed.
authorize("show current roles").ok();
@@ -2644,9 +2668,7 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
.error(accessError(true, "nodb"), onServer(true, allExcept(
TPrivilegeLevel.ALL, TPrivilegeLevel.OWNER)));
}
} finally {
authzCatalog_.removeRole("foo");
}
} finally { authzCatalog_.removeRole("foo"); }
boolean exceptionThrown = false;
try {
parseAndAnalyze("alter database functional set owner role foo",

View File

@@ -0,0 +1,434 @@
====
---- QUERY
# Creating tables for each partition transform class
CREATE TABLE iceberg_identity_partitions_showfiles
(identity_boolean boolean, identity_int int, identity_bigint bigint,
identity_float float, identity_double double, identity_decimal decimal(20,10),
identity_date date, identity_timestamp timestamp, identity_string string)
PARTITIONED BY SPEC
(identity(identity_boolean), identity(identity_int), identity(identity_bigint),
identity(identity_float), identity(identity_double), identity(identity_decimal),
identity(identity_date), identity(identity_string))
STORED AS ICEBERG;
CREATE TABLE iceberg_bucket_partitions_showfiles
(bucket_int int, bucket_bigint bigint, bucket_decimal decimal(20,10),
bucket_date date, bucket_timestamp timestamp, bucket_string string)
PARTITIONED BY SPEC
(bucket(5,bucket_int), bucket(5,bucket_bigint), bucket(5,bucket_decimal),
bucket(5,bucket_date), bucket(5,bucket_timestamp), bucket(5,bucket_string))
STORED AS ICEBERG;
CREATE TABLE iceberg_truncate_partitions_showfiles
(truncate_int int, truncate_bigint bigint, truncate_decimal decimal(20,10),
truncate_string string)
PARTITIONED BY SPEC
(truncate(5,truncate_int), truncate(5,truncate_bigint), truncate(5,truncate_decimal),
truncate(5,truncate_string))
STORED AS ICEBERG;
CREATE TABLE iceberg_time_partitions_showfiles
(year_date date, year_timestamp timestamp, month_date date, month_timestamp timestamp,
day_date date, day_timestamp timestamp, hour_timestamp timestamp)
PARTITIONED BY SPEC
(year(year_date), year(year_timestamp),
month(month_date), month(month_timestamp),
day(day_date), day(day_timestamp),
hour(hour_timestamp))
STORED AS ICEBERG;
CREATE TABLE iceberg_mixed_partitions_showfiles
(identity_int int, identity_string string, hour_timestamp timestamp)
PARTITIONED BY SPEC
(identity(identity_int), identity(identity_string), hour(hour_timestamp))
STORED AS ICEBERG;
====
---- QUERY
# Failing implicit string to hour cast
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (hour(hour_timestamp) = "2012-12-12");
---- CATCH
AnalysisException: operands of type INT and STRING are not comparable: HOUR(hour_timestamp) = '2012-12-12'
====
---- QUERY
# Failing implicit string to day cast
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (day(day_date) = "2012-12");
---- CATCH
AnalysisException: operands of type INT and STRING are not comparable: DAY(day_date) = '2012-12'
====
---- QUERY
# Failing implicit string to month cast
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (month(month_date) = "2012");
---- CATCH
AnalysisException: operands of type INT and STRING are not comparable: MONTH(month_date) = '2012'
====
---- QUERY
# Failing implicit string to year cast
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (year(year_date) = "2012-12-12-20");
---- CATCH
AnalysisException: operands of type INT and STRING are not comparable: YEAR(year_date) = '2012-12-12-20'
====
---- QUERY
INSERT INTO iceberg_identity_partitions_showfiles(identity_boolean) VALUES (true);
INSERT INTO iceberg_identity_partitions_showfiles(identity_int) VALUES (1);
INSERT INTO iceberg_identity_partitions_showfiles(identity_bigint) VALUES (1);
INSERT INTO iceberg_identity_partitions_showfiles(identity_float) VALUES (1.0);
INSERT INTO iceberg_identity_partitions_showfiles(identity_double) VALUES (1.0);
INSERT INTO iceberg_identity_partitions_showfiles(identity_decimal) VALUES (1);
INSERT INTO iceberg_identity_partitions_showfiles(identity_date) VALUES ('2000-12-12');
INSERT INTO iceberg_identity_partitions_showfiles(identity_string) VALUES ("string-transform-omitted");
INSERT INTO iceberg_identity_partitions_showfiles(identity_string) VALUES ("string-transform-set");
INSERT INTO iceberg_identity_partitions_showfiles(identity_string) VALUES ("string"), ("another-string");
INSERT INTO iceberg_identity_partitions_showfiles(identity_string) VALUES ("string"), ("another-string");
INSERT INTO iceberg_bucket_partitions_showfiles(bucket_int) VALUES (100), (200);
INSERT INTO iceberg_bucket_partitions_showfiles(bucket_bigint) VALUES (100);
INSERT INTO iceberg_bucket_partitions_showfiles(bucket_decimal) VALUES (10);
INSERT INTO iceberg_bucket_partitions_showfiles(bucket_date) VALUES ("1526-01-12");
INSERT INTO iceberg_bucket_partitions_showfiles(bucket_string) VALUES ("string");
INSERT INTO iceberg_bucket_partitions_showfiles(bucket_timestamp) VALUES ("1583-04-02 03:00:00");
INSERT INTO iceberg_truncate_partitions_showfiles(truncate_int) VALUES (131072);
INSERT INTO iceberg_truncate_partitions_showfiles(truncate_bigint) VALUES (68719476736);
INSERT INTO iceberg_truncate_partitions_showfiles(truncate_decimal) VALUES (100000.1234567891);
INSERT INTO iceberg_truncate_partitions_showfiles(truncate_string) VALUES ('thisisalongstring');
INSERT INTO iceberg_time_partitions_showfiles(year_date) VALUES ('2077-05-06');
INSERT INTO iceberg_time_partitions_showfiles(month_date) VALUES ('2023-12-01');
INSERT INTO iceberg_time_partitions_showfiles(day_date) VALUES ('2023-12-01');
INSERT INTO iceberg_time_partitions_showfiles(year_timestamp) VALUES ('2023-12-02 00:00:00');
INSERT INTO iceberg_time_partitions_showfiles(month_timestamp) VALUES ('2023-12-02 00:00:00');
INSERT INTO iceberg_time_partitions_showfiles(day_timestamp) VALUES ('2023-03-02 00:00:00');
INSERT INTO iceberg_time_partitions_showfiles(hour_timestamp) VALUES ('2023-06-02 00:00:00');
INSERT INTO iceberg_mixed_partitions_showfiles(identity_string, hour_timestamp) VALUES ('string-hour','2023-03-02 00:00:00');
INSERT INTO iceberg_mixed_partitions_showfiles(identity_string, hour_timestamp) VALUES ('another-string-hour', '2023-03-02 00:00:00');
INSERT INTO iceberg_mixed_partitions_showfiles(identity_string, hour_timestamp) VALUES ('another-string-hour', '2023-03-02 10:00:00');
INSERT INTO iceberg_mixed_partitions_showfiles(identity_string, hour_timestamp) VALUES ('string-hour', '2023-03-02 10:00:00');
INSERT INTO iceberg_mixed_partitions_showfiles(identity_string, identity_int) VALUES ('string-comma', 567);
INSERT INTO iceberg_mixed_partitions_showfiles(identity_string, identity_int) VALUES ('string-comma', 568);
INSERT INTO iceberg_mixed_partitions_showfiles(identity_int) VALUES (NULL);
====
---- QUERY
# Show files for identity partitions - boolean
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity_boolean = true);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_boolean=true.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for identity partitions - boolean false (should return error)
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity_boolean = false);
---- CATCH
AnalysisException: No matching partition(s) found.
====
---- QUERY
# Show files for identity partitions - int
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity_int = 1);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_int=1.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for identity partitions - bigint
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity_bigint = 1);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_bigint=1.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for identity partitions - float range
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity_float < 3.0);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_float=1.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for identity partitions - double range
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity_double > 0.0);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_double=1.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for identity partitions - decimal range
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity_decimal < 3);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_decimal=1.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for identity partitions - date
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity_date = '2000-12-12');
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_date=2000-12-12.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for identity partitions - string without transform
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity_string = "string-transform-omitted");
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_string=string-transform-omitted.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for identity partitions - string with transform
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity(identity_string) = "string-transform-set");
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_string=string-transform-set.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for identity partitions - string "another-string"
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity(identity_string) = "another-string");
---- RESULTS: VERIFY_IS_EQUAL_SORTED
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_string=another-string.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_string=another-string.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for identity partitions - string "string"
SHOW FILES IN iceberg_identity_partitions_showfiles PARTITION (identity(identity_string) = "string");
---- RESULTS: VERIFY_IS_EQUAL_SORTED
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_string=string.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_identity_partitions_showfiles/data/.*identity_string=string.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for bucket partitions - int bucket range
SHOW FILES IN iceberg_bucket_partitions_showfiles PARTITION (bucket(5, bucket_int) in (1,2));
---- RESULTS: VERIFY_IS_EQUAL_SORTED
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_bucket_partitions_showfiles/data/.*bucket_int_bucket.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_bucket_partitions_showfiles/data/.*bucket_int_bucket.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for bucket partitions - bigint bucket
SHOW FILES IN iceberg_bucket_partitions_showfiles PARTITION (bucket(5, bucket_bigint) = 1);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_bucket_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for bucket partitions - decimal bucket
SHOW FILES IN iceberg_bucket_partitions_showfiles PARTITION (bucket(5, bucket_decimal) = 3);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_bucket_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for bucket partitions - date bucket
SHOW FILES IN iceberg_bucket_partitions_showfiles PARTITION (bucket(5, bucket_date) = 0);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_bucket_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for bucket partitions - timestamp bucket
SHOW FILES IN iceberg_bucket_partitions_showfiles PARTITION (bucket(5, bucket_timestamp) = 1);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_bucket_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for bucket partitions - string bucket
SHOW FILES IN iceberg_bucket_partitions_showfiles PARTITION (bucket(5, bucket_string) = 1);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_bucket_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for truncate partitions - int truncate
SHOW FILES IN iceberg_truncate_partitions_showfiles PARTITION (truncate(5, truncate_int) = 131070);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_truncate_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for truncate partitions - bigint truncate
SHOW FILES IN iceberg_truncate_partitions_showfiles PARTITION (truncate(5, truncate_bigint) = 68719476735);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_truncate_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for truncate partitions - decimal truncate
SHOW FILES IN iceberg_truncate_partitions_showfiles PARTITION (truncate(5, truncate_decimal) = 100000.1234567890);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_truncate_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for truncate partitions - string truncate
SHOW FILES IN iceberg_truncate_partitions_showfiles PARTITION (truncate(5, truncate_string) = 'thisi');
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_truncate_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for time partitions - year from date
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (year(year_date) = '2077');
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_time_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for time partitions - month from date
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (month(month_date) = '2023-12');
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_time_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for time partitions - day from date
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (day(day_date) = '2023-12-01');
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_time_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for time partitions - year from timestamp
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (year(year_timestamp) = '2023');
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_time_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for time partitions - month from timestamp
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (month(month_timestamp) = '2023-12');
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_time_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for time partitions - day from timestamp
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (day(day_timestamp) = '2023-03-02');
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_time_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for time partitions - hour from timestamp
SHOW FILES IN iceberg_time_partitions_showfiles PARTITION (hour(hour_timestamp) = '2023-06-02-0');
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_time_partitions_showfiles/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for mixed partitions - complex condition with AND
SHOW FILES IN iceberg_mixed_partitions_showfiles PARTITION (identity_string in ('string-hour', 'another-string-hour') and hour(hour_timestamp) = '2023-03-02-10');
---- RESULTS: VERIFY_IS_EQUAL_SORTED
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/.*hour_timestamp_hour=.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/.*hour_timestamp_hour=.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for mixed partitions - hour range
SHOW FILES IN iceberg_mixed_partitions_showfiles PARTITION (hour(hour_timestamp) < '2030-03-02-10');
---- RESULTS: VERIFY_IS_EQUAL_SORTED
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/.*hour_timestamp_hour=.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/.*hour_timestamp_hour=.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/.*hour_timestamp_hour=.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/.*hour_timestamp_hour=.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for mixed partitions - multiple values with IN
SHOW FILES IN iceberg_mixed_partitions_showfiles PARTITION (identity_string = "string-comma", identity_int in (567, 568));
---- RESULTS: VERIFY_IS_EQUAL_SORTED
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/.*identity_int=.*identity_string=string-comma.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/.*identity_int=.*identity_string=string-comma.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for mixed partitions - NULL partition
SHOW FILES IN iceberg_mixed_partitions_showfiles PARTITION (identity_int IS NULL);
---- RESULTS: VERIFY_IS_EQUAL_SORTED
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/identity_int=__HIVE_DEFAULT_PARTITION__/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/identity_int=__HIVE_DEFAULT_PARTITION__/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/identity_int=__HIVE_DEFAULT_PARTITION__/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/identity_int=__HIVE_DEFAULT_PARTITION__/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_mixed_partitions_showfiles/data/identity_int=__HIVE_DEFAULT_PARTITION__/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
====
---- QUERY
# Partition evolution - testing SHOW FILES across different partition specs
CREATE TABLE iceberg_showfiles_partition_evolution
(identity_int int, unpartitioned_int_to_identity_int int, year_date_col_to_month_date_col date)
PARTITIONED BY SPEC
(identity(identity_int), year(year_date_col_to_month_date_col)) STORED AS ICEBERG;
INSERT INTO iceberg_showfiles_partition_evolution VALUES (1, 2, "2023-10-11");
ALTER TABLE iceberg_showfiles_partition_evolution SET PARTITION SPEC(
identity(identity_int), identity(unpartitioned_int_to_identity_int), year(year_date_col_to_month_date_col));
INSERT INTO iceberg_showfiles_partition_evolution VALUES (1, 2, "2023-01-11");
# Show files for the newly partitioned column
SHOW FILES IN iceberg_showfiles_partition_evolution PARTITION (unpartitioned_int_to_identity_int = 2);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_showfiles_partition_evolution/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
ALTER TABLE iceberg_showfiles_partition_evolution SET PARTITION SPEC(
identity(identity_int), month(year_date_col_to_month_date_col));
# Should fail - trying to show files for a column that's no longer partitioned
SHOW FILES IN iceberg_showfiles_partition_evolution PARTITION (unpartitioned_int_to_identity_int = 2);
---- CATCH
AnalysisException: Partition exprs cannot contain non-partition column(s): unpartitioned_int_to_identity_int
====
---- QUERY
INSERT INTO iceberg_showfiles_partition_evolution VALUES (1, 2, "2023-11-11");
# Show files for the evolved partition (year -> month)
SHOW FILES IN iceberg_showfiles_partition_evolution PARTITION (month(year_date_col_to_month_date_col) = "2023-11");
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_showfiles_partition_evolution/data/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files with delete files (format-version=2)
CREATE TABLE iceberg_showfiles_with_deletes(identity_int int, unpartitioned_int int)
PARTITIONED BY SPEC (identity_int) STORED AS ICEBERG TBLPROPERTIES('format-version'='2');
INSERT INTO iceberg_showfiles_with_deletes VALUES (1,2);
INSERT INTO iceberg_showfiles_with_deletes VALUES (2,1);
DELETE FROM iceberg_showfiles_with_deletes WHERE identity_int = 1;
# Show files for partition with data file (no deletes)
SHOW FILES IN iceberg_showfiles_with_deletes PARTITION (identity_int = 2);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_showfiles_with_deletes/data/identity_int=2/.*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====
---- QUERY
# Show files for partition with both data and delete files
SHOW FILES IN iceberg_showfiles_with_deletes PARTITION (identity_int = 1);
---- RESULTS
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_showfiles_with_deletes/data/identity_int=1/delete-.*_data.*.parq','.*','','$ERASURECODE_POLICY'
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/iceberg_showfiles_with_deletes/data/identity_int=1/[^d].*_data.*.parq','.*','','$ERASURECODE_POLICY'
---- TYPES
STRING, STRING, STRING, STRING
====

View File

@@ -1564,6 +1564,10 @@ class TestIcebergTable(IcebergTestSuite):
assert snapshots[0].get_parent_id() == snapshots[2].get_parent_id()
assert snapshots[0].get_creation_time() < snapshots[2].get_creation_time()
def test_show_files_partition(self, vector, unique_database):
self.run_test_case('QueryTest/iceberg-show-files-partition', vector,
use_db=unique_database)
def test_scan_metrics_in_profile_basic(self, vector):
self.run_test_case('QueryTest/iceberg-scan-metrics-basic', vector)