vedantpuri opened a new issue, #14056:
URL: https://github.com/apache/iceberg/issues/14056
### Apache Iceberg version
1.8.1
### Query engine
Spark
### Please describe the bug 🐞
Hello,
I was trying to run the `rewrite_position_delete_files` procedure on one of
my tables that has a column named `partition` and it was failing with the
following trace
<details>
<summary>Click to expand/collapse</summary>
```
java.lang.IllegalArgumentException: Cannot create identity partition sourced
from different field in schema: partition
at
org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument(Preconditions.java:218)
at
org.apache.iceberg.PartitionSpec$Builder.checkAndAddPartitionName(PartitionSpec.java:401)
at org.apache.iceberg.PartitionSpec$Builder.add(PartitionSpec.java:609)
at
org.apache.iceberg.UnboundPartitionSpec.copyToBuilder(UnboundPartitionSpec.java:65)
at
org.apache.iceberg.UnboundPartitionSpec.bind(UnboundPartitionSpec.java:46)
at
org.apache.iceberg.PartitionSpecParser.fromJson(PartitionSpecParser.java:71)
at
org.apache.iceberg.PartitionSpecParser.lambda$fromJson$1(PartitionSpecParser.java:88)
at org.apache.iceberg.util.JsonUtil.parse(JsonUtil.java:104)
at
org.apache.iceberg.PartitionSpecParser.lambda$fromJson$2(PartitionSpecParser.java:88)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$14(BoundedLocalCache.java:2406)
at
java.base/java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1916)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2404)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2387)
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.LocalManualCache.get(LocalManualCache.java:62)
at
org.apache.iceberg.PartitionSpecParser.fromJson(PartitionSpecParser.java:86)
at
org.apache.iceberg.BaseContentScanTask.spec(BaseContentScanTask.java:72)
at
org.apache.iceberg.spark.actions.RewritePositionDeleteFilesSparkAction.coercePartition(RewritePositionDeleteFilesSparkAction.java:467)
at
org.apache.iceberg.spark.actions.RewritePositionDeleteFilesSparkAction.groupByPartition(RewritePositionDeleteFilesSparkAction.java:175)
at
org.apache.iceberg.spark.actions.RewritePositionDeleteFilesSparkAction.planFileGroups(RewritePositionDeleteFilesSparkAction.java:151)
at
org.apache.iceberg.spark.actions.RewritePositionDeleteFilesSparkAction.execute(RewritePositionDeleteFilesSparkAction.java:126)
at
org.apache.iceberg.spark.actions.RewritePositionDeleteFilesSparkAction.execute(RewritePositionDeleteFilesSparkAction.java:74)
at
org.apache.iceberg.spark.procedures.RewritePositionDeleteFilesProcedure.lambda$call$0(RewritePositionDeleteFilesProcedure.java:105)
at
org.apache.iceberg.spark.procedures.BaseProcedure.execute(BaseProcedure.java:107)
at
org.apache.iceberg.spark.procedures.BaseProcedure.modifyIcebergTable(BaseProcedure.java:88)
at
org.apache.iceberg.spark.procedures.RewritePositionDeleteFilesProcedure.call(RewritePositionDeleteFilesProcedure.java:94)
at
org.apache.spark.sql.execution.datasources.v2.CallExec.run(CallExec.scala:34)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:126)
at
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:108)
at
org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:384)
at
org.apache.spark.sql.execution.SQLExecution$.executeQuery$1(SQLExecution.scala:157)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$10(SQLExecution.scala:220)
at
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:108)
at
org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:384)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$9(SQLExecution.scala:220)
at
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:405)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:219)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:901)
at
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:83)
at
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:74)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:123)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:114)
at
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:521)
at
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:77)
at
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:521)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:34)
at
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:303)
at
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:299)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:34)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:34)
at
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:497)
at
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:114)
at
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:101)
at
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:99)
at org.apache.spark.sql.Dataset.<init>(Dataset.scala:223)
at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:103)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:901)
at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:99)
at
org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:639)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:901)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:630)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:660)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
at
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:569)
at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:374)
at py4j.Gateway.invoke(Gateway.java:282)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at
py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
at java.base/java.lang.Thread.run(Thread.java:840)
```
</details>
Table properties:
```
TBLPROPERTIES (
'format' = 'iceberg/parquet',
'format-version' = '2',
'sort-order' = 'REDACTED',
'write.delete.mode' = 'merge-on-read',
'write.format.default' = 'parquet',
'write.merge.isolation-level' = 'snapshot',
'write.merge.mode' = 'merge-on-read',
'write.parquet.bloom-filter-enabled.REDACTED' = 'true',
'write.parquet.compression-codec' = 'zstd',
'write.update.isolation-level' = 'snapshot',
'write.update.mode' = 'merge-on-read')
```
The table I ran this on used that column in the partitioning spec as well
with a simple `identity` transform. In fact, I wasn't even able to select from
the `position_deletes` meta table, and it was erroring out similarly. I suspect
that is where the core issue is.
These were the jars in use:
```
"spark.jars.packages":
"org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1,org.apache.iceberg:iceberg-aws-bundle:1.8.1",
```
Based on some reading online and [this
issue](https://github.com/apache/iceberg/issues/3709) (which is now closed), I
considered renaming the column on one of my test tables as well. I renamed it
to `partition_key`
Renaming also didn't work and I still had the same error when selecting from
that table. I suspect this is because the old partition spec is around
(probably rightfully so, since there was data written using the old column
name).
Then, I tried to create a brand new table with the renamed column, inserted
some data, updated that data (in a way that would generate a positional delete
file) and reran the select, which did work. Running the procedure also worked
on this new table.
Please let me know if I can provide more context on this, but it should be
easy to repro:
1. Make an MoR table with a column named `partition` and use it in the
partitioning scheme
2. Insert some data and update it to generate positional deletes (using
something like ts_column > val)
3. Verify a delete file exists in the data location
4. Try selecting from `table.position_deletes` or running the
`rewrite_position_delete_files` procedure on it to see the error
### 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
- [x] I cannot contribute a fix for this bug at this time
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]