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



##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
##########
@@ -100,8 +103,8 @@
       "iceberg-flink-job-id", BasicTypeInfo.STRING_TYPE_INFO);
   private transient ListState<String> jobIdState;
   // All pending checkpoints states for this function.
-  private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> 
STATE_DESCRIPTOR = buildStateDescriptor();
-  private transient ListState<SortedMap<Long, List<DataFile>>> 
checkpointsState;
+  private static final ListStateDescriptor<SortedMap<Long, Byte[]>> 
STATE_DESCRIPTOR = buildStateDescriptor();

Review comment:
       instead of a Byte[] value type, should we just define the a Flink 
ManifestFile state type? maybe an Avro record or use 
`SimpleVersionedSerializer` that Jingsong suggested before? In addition, we can 
have the state type include the metadata (like jobId). '
   
   ```
   {
       "type": "record",
       "name": "ManifestFileState",
       "namespace": "org.apache.iceberg....",
       "fields": [
           {"name":"path", "type":"string"},
           {"name":"length", "type":"long"},
           {"name":"specId", "type":"int"},
           { "name":"dataFileCount", "type": "long"},
           { "name":"recordCount", "type": "long"},
   ////////////// Flink specific fields
           {"name":"jobId", "type":"string"},
           { "name":"checkpointId", "type": "long"},
           { "name":"checkpointTimestamp", "type": "long"},
           { "name":"byteCount", "type": "long"}
       ]
   }
   ```
   
   Then we also have `FlinkManifestFile` interface extends from `ManifestFile` 
interface and can convert from/to `ManifestFileState`
   
   ```
   public interface FlinkManifestFile extends ManifestFile {
     String jobId();
     long checkpointId();
     long checkpointTimestamp();
     long byteCount();
     ManifestFileState toState();
   }




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

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