stevenzwu commented on a change in pull request #3015:
URL: https://github.com/apache/iceberg/pull/3015#discussion_r753631713



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -235,45 +235,48 @@ public void put(int i, Object value) {
         this.format = FileFormat.valueOf(value.toString());
         return;
       case 3:
-        this.partitionData = (PartitionData) value;
+        this.partitionSpecId = (value != null) ? (Integer) value : -1;

Review comment:
       @szehon-ho @RussellSpitzer @rdblue @openinx  FYI, shifting order is a 
breaking change that caused Flink failing to restore from checkpoint. It is a 
not big deal for us this time as we are still in testing phase. I just like to 
call out that we need to be more careful in the future.
   
   ```
   java.lang.ClassCastException: class org.apache.iceberg.PartitionData cannot 
be cast to class java.lang.Integer (org.apache.iceberg.PartitionData is in 
unnamed module of loader org.apache.flink.util.ChildFirstClassLoader @3e063fd4; 
java.lang.Integer is in module java.base of loader 'bootstrap')
        at org.apache.iceberg.BaseFile.put(BaseFile.java:238)
        at 
org.apache.iceberg.avro.ValueReaders$IndexedRecordReader.set(ValueReaders.java:746)
        at 
org.apache.iceberg.avro.ValueReaders$IndexedRecordReader.set(ValueReaders.java:715)
        at 
org.apache.iceberg.avro.ValueReaders$StructReader.read(ValueReaders.java:669)
        at 
org.apache.iceberg.avro.ValueReaders$StructReader.read(ValueReaders.java:669)
        at 
org.apache.iceberg.data.avro.DecoderResolver.resolveAndRead(DecoderResolver.java:48)
        at 
org.apache.iceberg.avro.GenericAvroReader.read(GenericAvroReader.java:69)
        at 
org.apache.iceberg.avro.ProjectionDatumReader.read(ProjectionDatumReader.java:74)
        at org.apache.avro.file.DataFileStream.next(DataFileStream.java:250)
        at 
org.apache.iceberg.avro.AvroIterable$AvroReuseIterator.next(AvroIterable.java:202)
        at 
org.apache.iceberg.io.CloseableIterable$4$1.next(CloseableIterable.java:113)
        at org.apache.iceberg.io.FilterIterator.advance(FilterIterator.java:66)
        at org.apache.iceberg.io.FilterIterator.hasNext(FilterIterator.java:50)
        at 
org.apache.iceberg.io.CloseableIterable$4$1.hasNext(CloseableIterable.java:108)
        at 
org.apache.iceberg.relocated.com.google.common.collect.Iterators.addAll(Iterators.java:355)
        at 
org.apache.iceberg.relocated.com.google.common.collect.Lists.newArrayList(Lists.java:143)
        at 
org.apache.iceberg.relocated.com.google.common.collect.Lists.newArrayList(Lists.java:130)
        at 
org.apache.iceberg.flink.sink.FlinkManifestUtil.readDataFiles(FlinkManifestUtil.java:60)
        at 
org.apache.iceberg.flink.sink.FlinkManifestUtil.readCompletedFiles(FlinkManifestUtil.java:105)
        at 
org.apache.iceberg.flink.sink.IcebergFilesCommitter.commitUpToCheckpoint(IcebergFilesCommitter.java:212)
        at 
org.apache.iceberg.flink.sink.IcebergFilesCommitter.initializeState(IcebergFilesCommitter.java:156)
        at 
org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.initializeOperatorState(StreamOperatorStateHandler.java:118)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:290)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain.initializeStateAndOpenOperators(OperatorChain.java:441)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:585)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.executeRestore(StreamTask.java:565)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:650)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:540)
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:759)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566)
        at java.base/java.lang.Thread.run(Thread.java:829)
   ```




-- 
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]

Reply via email to