Kyo91 opened a new issue #3709:
URL: https://github.com/apache/iceberg/issues/3709
I encountered the following regression when trying to update from iceberg
0.11.1 to 0.12.1:
Create a partitioned table with at least one column named "partition"
(assume catalog/namespace already exists in Hive):
```sql
CREATE TABLE my.test.table (id int, partition int) USING iceberg PARTITIONED
BY (partition)
```
The associated metadata table now has the following schema:
```scala
scala> scala> spark.table("my.test.table.files").printSchema
root
|-- content: integer (nullable = true)
|-- file_path: string (nullable = false)
|-- file_format: string (nullable = false)
|-- partition: struct (nullable = false)
| |-- partition: integer (nullable = true)
|-- record_count: long (nullable = false)
|-- file_size_in_bytes: long (nullable = false)
|-- column_sizes: map (nullable = true)
| |-- key: integer
| |-- value: long (valueContainsNull = false)
|-- value_counts: map (nullable = true)
| |-- key: integer
| |-- value: long (valueContainsNull = false)
|-- null_value_counts: map (nullable = true)
| |-- key: integer
| |-- value: long (valueContainsNull = false)
|-- nan_value_counts: map (nullable = true)
| |-- key: integer
| |-- value: long (valueContainsNull = false)
|-- lower_bounds: map (nullable = true)
| |-- key: integer
| |-- value: binary (valueContainsNull = false)
|-- upper_bounds: map (nullable = true)
| |-- key: integer
| |-- value: binary (valueContainsNull = false)
|-- key_metadata: binary (nullable = true)
|-- split_offsets: array (nullable = true)
| |-- element: long (containsNull = false)
|-- equality_ids: array (nullable = true)
| |-- element: integer (containsNull = false)
|-- sort_order_id: integer (nullable = true)
```
While this table is empty, operations work as normal
```scala
scala> spark.table("my.test.table.files").show()
+-------+---------+-----------+---------+------------+------------------+------------+------------+-----------------+----------------+------------+------------+------------+-------------+------------+-------------+
|content|file_path|file_format|partition|record_count|file_size_in_bytes|column_sizes|value_counts|null_value_counts|nan_value_counts|lower_bounds|upper_bounds|key_metadata|split_offsets|equality_ids|sort_order_id|
+-------+---------+-----------+---------+------------+------------------+------------+------------+-----------------+----------------+------------+------------+------------+-------------+------------+-------------+
+-------+---------+-----------+---------+------------+------------------+------------+------------+-----------------+----------------+------------+------------+------------+-------------+------------+-------------+
```
Then create some sample data to add to the table:
```scala
scala> val test = List((1, 1), (2, 1), (3, 2), (2, 2)).toDF("id",
"partition")
test: org.apache.spark.sql.DataFrame = [id: int, partition: int]
scala> test.show()
+---+---------+
| id|partition|
+---+---------+
| 1| 1|
| 2| 1|
| 3| 2|
| 2| 2|
+---+---------+
scala> test.writeTo("my.test.table").append()
scala> spark.table("my.test.table").show()
+---+---------+
| id|partition|
+---+---------+
| 1| 1|
| 2| 1|
| 3| 2|
| 2| 2|
+---+---------+
```
But now trying any operation on the metadata table will fail with the
following error:
```scala
spark.table("my.test.table.files").show()
2021-12-10 15:18:22,405 [main] INFO org.apache.iceberg.BaseTableScan -
Scanning table my.test.table snapshot 1482056096771695160 created at 2021-12-10
15:18:15.323 with filter true
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:217)
at
org.apache.iceberg.PartitionSpec$Builder.checkAndAddPartitionName(PartitionSpec.java:344)
at
org.apache.iceberg.PartitionSpec$Builder.identity(PartitionSpec.java:380)
at
org.apache.iceberg.BaseMetadataTable.lambda$transformSpec$0(BaseMetadataTable.java:66)
at
org.apache.iceberg.relocated.com.google.common.collect.ImmutableList.forEach(ImmutableList.java:405)
at
org.apache.iceberg.BaseMetadataTable.transformSpec(BaseMetadataTable.java:66)
at
org.apache.iceberg.DataFilesTable$FilesTableScan.planFiles(DataFilesTable.java:118)
at org.apache.iceberg.BaseTableScan.planFiles(BaseTableScan.java:208)
at
org.apache.iceberg.DataFilesTable$FilesTableScan.planFiles(DataFilesTable.java:68)
at org.apache.iceberg.BaseTableScan.planTasks(BaseTableScan.java:241)
at
org.apache.iceberg.DataFilesTable$FilesTableScan.planTasks(DataFilesTable.java:68)
at
org.apache.iceberg.spark.source.SparkBatchQueryScan.tasks(SparkBatchQueryScan.java:122)
at
org.apache.iceberg.spark.source.SparkBatchScan.planInputPartitions(SparkBatchScan.java:143)
at
org.apache.spark.sql.execution.datasources.v2.BatchScanExec.partitions$lzycompute(BatchScanExec.scala:43)
at
org.apache.spark.sql.execution.datasources.v2.BatchScanExec.partitions(BatchScanExec.scala:43)
at
org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase.supportsColumnar(DataSourceV2ScanExecBase.scala:61)
at
org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase.supportsColumnar$(DataSourceV2ScanExecBase.scala:60)
at
org.apache.spark.sql.execution.datasources.v2.BatchScanExec.supportsColumnar(BatchScanExec.scala:29)
at
org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy.apply(DataSourceV2Strategy.scala:84)
at
org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$1(QueryPlanner.scala:63)
at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:489)
at
org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
at
org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:68)
at
org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$3(QueryPlanner.scala:78)
at
scala.collection.TraversableOnce.$anonfun$foldLeft$1(TraversableOnce.scala:162)
at
scala.collection.TraversableOnce.$anonfun$foldLeft$1$adapted(TraversableOnce.scala:162)
at scala.collection.Iterator.foreach(Iterator.scala:941)
at scala.collection.Iterator.foreach$(Iterator.scala:941)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
at scala.collection.TraversableOnce.foldLeft(TraversableOnce.scala:162)
at scala.collection.TraversableOnce.foldLeft$(TraversableOnce.scala:160)
at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1429)
at
org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$2(QueryPlanner.scala:75)
at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490)
at
org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
at
org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:68)
at
org.apache.spark.sql.execution.QueryExecution$.createSparkPlan(QueryExecution.scala:338)
at
org.apache.spark.sql.execution.QueryExecution.$anonfun$sparkPlan$1(QueryExecution.scala:99)
at
org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
at
org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:138)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:767)
at
org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:138)
at
org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:99)
at
org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:92)
at
org.apache.spark.sql.execution.QueryExecution.$anonfun$executedPlan$1(QueryExecution.scala:112)
at
org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
at
org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:138)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:767)
at
org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:138)
at
org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:112)
at
org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:105)
at
org.apache.spark.sql.execution.QueryExecution.$anonfun$writePlans$5(QueryExecution.scala:204)
at
org.apache.spark.sql.catalyst.plans.QueryPlan$.append(QueryPlan.scala:478)
at
org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$writePlans(QueryExecution.scala:204)
at
org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:212)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:95)
at
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:87)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:767)
at
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3616)
at org.apache.spark.sql.Dataset.head(Dataset.scala:2697)
at org.apache.spark.sql.Dataset.take(Dataset.scala:2904)
at org.apache.spark.sql.Dataset.getRows(Dataset.scala:300)
at org.apache.spark.sql.Dataset.showString(Dataset.scala:337)
at org.apache.spark.sql.Dataset.show(Dataset.scala:824)
at org.apache.spark.sql.Dataset.show(Dataset.scala:783)
at org.apache.spark.sql.Dataset.show(Dataset.scala:792)
... 47 elided
```
--
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]