Fokko opened a new issue, #8247:
URL: https://github.com/apache/iceberg/issues/8247
### Feature Request / Improvement
If you create an UUID table with a partition:
```python
from pyiceberg.catalog import load_catalog
from pyiceberg.schema import Schema
from pyiceberg.types import UUIDType, NestedField
schema = Schema(
NestedField(
field_id=1, name="c1", field_type=UUIDType(), required=False
),
)
from pyiceberg.partitioning import PartitionSpec, PartitionField
from pyiceberg.transforms import BucketTransform
partition_spec = PartitionSpec(
PartitionField(
source_id=1, field_id=1000, transform=BucketTransform(16),
name="c1_by_bucket"
)
)
cat.create_table(
identifier="default.uuid",
schema=schema,
partition_spec=partition_spec,
)
```
And you try to write to it, you'll get:
```
spark-sql (default)> insert into uuid
VALUES('102cb62f-e6f8-4eb0-9973-d9b012ff0967');
23/08/07 09:20:31 ERROR Utils: Aborting task
java.lang.ClassCastException: Cannot cast java.lang.String to java.util.UUID
at java.base/java.lang.Class.cast(Class.java:3605)
at
org.apache.iceberg.spark.source.InternalRowWrapper.get(InternalRowWrapper.java:63)
at org.apache.iceberg.Accessors$PositionAccessor.get(Accessors.java:71)
at org.apache.iceberg.Accessors$PositionAccessor.get(Accessors.java:58)
at org.apache.iceberg.PartitionKey.partition(PartitionKey.java:106)
at
org.apache.iceberg.spark.source.SparkWrite$PartitionedDataWriter.write(SparkWrite.java:752)
at
org.apache.iceberg.spark.source.SparkWrite$PartitionedDataWriter.write(SparkWrite.java:723)
at
org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.write(WriteToDataSourceV2Exec.scala:516)
at
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.$anonfun$run$1(WriteToDataSourceV2Exec.scala:471)
at
org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1563)
at
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run(WriteToDataSourceV2Exec.scala:509)
at
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run$(WriteToDataSourceV2Exec.scala:448)
at
org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.run(WriteToDataSourceV2Exec.scala:514)
at
org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.$anonfun$writeWithV2$2(WriteToDataSourceV2Exec.scala:411)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)
at
org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)
at org.apache.spark.scheduler.Task.run(Task.scala:139)
at
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)
23/08/07 09:20:31 ERROR DataWritingSparkTask: Aborting commit for partition
0 (task 1, attempt 0, stage 2.0)
23/08/07 09:20:31 ERROR DataWritingSparkTask: Aborted commit for partition 0
(task 1, attempt 0, stage 2.0)
23/08/07 09:20:31 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 1)
java.lang.ClassCastException: Cannot cast java.lang.String to java.util.UUID
at java.base/java.lang.Class.cast(Class.java:3605)
at
org.apache.iceberg.spark.source.InternalRowWrapper.get(InternalRowWrapper.java:63)
at org.apache.iceberg.Accessors$PositionAccessor.get(Accessors.java:71)
at org.apache.iceberg.Accessors$PositionAccessor.get(Accessors.java:58)
at org.apache.iceberg.PartitionKey.partition(PartitionKey.java:106)
at
org.apache.iceberg.spark.source.SparkWrite$PartitionedDataWriter.write(SparkWrite.java:752)
at
org.apache.iceberg.spark.source.SparkWrite$PartitionedDataWriter.write(SparkWrite.java:723)
at
org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.write(WriteToDataSourceV2Exec.scala:516)
at
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.$anonfun$run$1(WriteToDataSourceV2Exec.scala:471)
at
org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1563)
at
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run(WriteToDataSourceV2Exec.scala:509)
at
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run$(WriteToDataSourceV2Exec.scala:448)
at
org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.run(WriteToDataSourceV2Exec.scala:514)
at
org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.$anonfun$writeWithV2$2(WriteToDataSourceV2Exec.scala:411)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)
at
org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)
at org.apache.spark.scheduler.Task.run(Task.scala:139)
at
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)
23/08/07 09:20:31 WARN TaskSetManager: Lost task 0.0 in stage 2.0 (TID 1)
(192.168.1.61 executor driver): java.lang.ClassCastException: Cannot cast
java.lang.String to java.util.UUID
at java.base/java.lang.Class.cast(Class.java:3605)
at
org.apache.iceberg.spark.source.InternalRowWrapper.get(InternalRowWrapper.java:63)
at org.apache.iceberg.Accessors$PositionAccessor.get(Accessors.java:71)
at org.apache.iceberg.Accessors$PositionAccessor.get(Accessors.java:58)
at org.apache.iceberg.PartitionKey.partition(PartitionKey.java:106)
at
org.apache.iceberg.spark.source.SparkWrite$PartitionedDataWriter.write(SparkWrite.java:752)
at
org.apache.iceberg.spark.source.SparkWrite$PartitionedDataWriter.write(SparkWrite.java:723)
at
org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.write(WriteToDataSourceV2Exec.scala:516)
at
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.$anonfun$run$1(WriteToDataSourceV2Exec.scala:471)
at
org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1563)
at
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run(WriteToDataSourceV2Exec.scala:509)
at
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run$(WriteToDataSourceV2Exec.scala:448)
at
org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.run(WriteToDataSourceV2Exec.scala:514)
at
org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.$anonfun$writeWithV2$2(WriteToDataSourceV2Exec.scala:411)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)
at
org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)
at org.apache.spark.scheduler.Task.run(Task.scala:139)
at
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)
```
Tables without a UUID partition work great
### Query engine
None
--
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]