Open
Description
Apache Iceberg version
1.5.2
Query engine
Spark
Please describe the bug 🐞
Hi, I tried to get data files list from specified branch, but it occurs error.
Stack trace
org.apache.iceberg.exceptions.ValidationException: Cannot find source column for partition field: 1000: ts_day: identity(1000)
at org.apache.iceberg.exceptions.ValidationException.check(ValidationException.java:49)
at org.apache.iceberg.PartitionSpec.checkCompatibility(PartitionSpec.java:562)
at org.apache.iceberg.PartitionSpec$Builder.build(PartitionSpec.java:543)
at org.apache.iceberg.BaseMetadataTable.transformSpec(BaseMetadataTable.java:80)
at org.apache.iceberg.BaseFilesTable.lambda$planFiles$0(BaseFilesTable.java:75)
at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalLoadingCache.lambda$newMappingFunction$2(LocalLoadingCache.java:145)
at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.UnboundedLocalCache.lambda$computeIfAbsent$2(UnboundedLocalCache.java:239)
at java.base/java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1705)
at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.UnboundedLocalCache.computeIfAbsent(UnboundedLocalCache.java:235)
at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:108)
at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:56)
at org.apache.iceberg.BaseFilesTable.lambda$planFiles$1(BaseFilesTable.java:81)
at org.apache.iceberg.io.CloseableIterable$4.shouldKeep(CloseableIterable.java:112)
at org.apache.iceberg.io.FilterIterator.advance(FilterIterator.java:66)
at org.apache.iceberg.io.FilterIterator.hasNext(FilterIterator.java:49)
at org.apache.iceberg.io.CloseableIterable$7$1.hasNext(CloseableIterable.java:197)
at org.apache.iceberg.spark.source.SparkPartitioningAwareScan.tasks(SparkPartitioningAwareScan.java:177)
at org.apache.iceberg.spark.source.SparkPartitioningAwareScan.taskGroups(SparkPartitioningAwareScan.java:202)
at org.apache.iceberg.spark.source.SparkPartitioningAwareScan.outputPartitioning(SparkPartitioningAwareScan.java:104)
at org.apache.spark.sql.execution.datasources.v2.V2ScanPartitioningAndOrdering$$anonfun$partitioning$1.applyOrElse(V2ScanPartitioningAndOrdering.scala:44)
at org.apache.spark.sql.execution.datasources.v2.V2ScanPartitioningAndOrdering$$anonfun$partitioning$1.applyOrElse(V2ScanPartitioningAndOrdering.scala:42)
at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:461)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:461)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:466)
at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1215)
at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1214)
at org.apache.spark.sql.catalyst.plans.logical.Project.mapChildren(basicLogicalOperators.scala:71)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:466)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:466)
at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1215)
at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1214)
at org.apache.spark.sql.catalyst.plans.logical.LocalLimit.mapChildren(basicLogicalOperators.scala:1608)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:466)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:466)
at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1215)
at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1214)
at org.apache.spark.sql.catalyst.plans.logical.GlobalLimit.mapChildren(basicLogicalOperators.scala:1587)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:466)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:437)
at org.apache.spark.sql.execution.datasources.v2.V2ScanPartitioningAndOrdering$.partitioning(V2ScanPartitioningAndOrdering.scala:42)
at org.apache.spark.sql.execution.datasources.v2.V2ScanPartitioningAndOrdering$.$anonfun$apply$1(V2ScanPartitioningAndOrdering.scala:35)
at org.apache.spark.sql.execution.datasources.v2.V2ScanPartitioningAndOrdering$.$anonfun$apply$3(V2ScanPartitioningAndOrdering.scala:38)
at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
at scala.collection.immutable.List.foldLeft(List.scala:91)
at org.apache.spark.sql.execution.datasources.v2.V2ScanPartitioningAndOrdering$.apply(V2ScanPartitioningAndOrdering.scala:37)
at org.apache.spark.sql.execution.datasources.v2.V2ScanPartitioningAndOrdering$.apply(V2ScanPartitioningAndOrdering.scala:33)
at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:222)
at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
at scala.collection.immutable.List.foldLeft(List.scala:91)
at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:219)
at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:211)
at scala.collection.immutable.List.foreach(List.scala:431)
at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:211)
at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:182)
at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89)
at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:182)
at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:152)
at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138)
at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:219)
at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:546)
at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:219)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:218)
at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:148)
at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:144)
at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:162)
at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:182)
at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:179)
at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:238)
at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:284)
at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:252)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:117)
at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
at org.apache.spark.sql.Dataset.withAction(Dataset.scala:4320)
at org.apache.spark.sql.Dataset.head(Dataset.scala:3314)
at org.apache.spark.sql.Dataset.take(Dataset.scala:3537)
at org.apache.spark.sql.Dataset.getRows(Dataset.scala:280)
at org.apache.spark.sql.Dataset.showString(Dataset.scala:315)
The problem only occurs on partitioned table.
I represented the issue with this pyspark code. It works with tag, but not on branch.
spark.sql("CREATE DATABASE IF NOT EXISTS exam_db")
spark.sql("""CREATE TABLE IF NOT EXISTS exam_db.exam_table (
ts timestamp NOT NULL,
id bigint NOT NULL,
name string)
USING iceberg
PARTITIONED BY (days(ts))""")
spark.sql("INSERT INTO exam_db.exam_table VALUES (current_timestamp(), 1, 'andy')")
spark.sql("ALTER TABLE exam_db.exam_table CREATE BRANCH `exam-branch`")
spark.sql("ALTER TABLE exam_db.exam_table CREATE TAG `exam-tag`")
# these are ok
df = spark.sql("SELECT * FROM exam_db.exam_table.files")
df.show()
df = spark.sql("SELECT * FROM exam_db.exam_table.files VERSION AS OF 'exam-tag'")
df.show()
# problem
df = spark.sql("SELECT * FROM exam_db.exam_table.files VERSION AS OF 'exam-branch'")
df.show()
Willingness to contribute
- I can contribute a fix for this bug independently
- I would be willing to contribute a fix for this bug with guidance from the Iceberg community
- I cannot contribute a fix for this bug at this time