You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/12/03 17:10:13 UTC

[GitHub] [iceberg] stevenzwu commented on a change in pull request #3501: Flink: FLIP-27 Iceberg source split

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



##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedBytesCache;
+
+  private IcebergSourceSplit(CombinedScanTask task, Position position) {
+    this.task = task;
+    this.position = position;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return fromCombinedScanTask(combinedScanTask, 0, 0L);
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(
+      CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) {
+    return new IcebergSourceSplit(combinedScanTask, new Position(fileOffset, recordOffset));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  byte[] serializedBytesCache() {
+    return serializedBytesCache;
+  }
+
+  void serializedBytesCache(byte[] cachedBytes) {
+    this.serializedBytesCache = cachedBytes;
+  }
+
+  @Override
+  public String splitId() {
+    return MoreObjects.toStringHelper(this)
+        .add("files", toString(task.files()))
+        .toString();
+  }
+
+  public void updatePosition(int newFileOffset, long newRecordOffset) {
+    position.update(newFileOffset, newRecordOffset);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    IcebergSourceSplit split = (IcebergSourceSplit) o;
+    return Objects.equal(splitId(), split.splitId()) &&
+        Objects.equal(position, split.position());
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(splitId());
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("files", toString(task.files()))
+        .add("position", position)
+        .toString();
+  }
+
+  private String toString(Collection<FileScanTask> files) {
+    return Iterables.toString(files.stream().map(fileScanTask ->
+        MoreObjects.toStringHelper(fileScanTask)
+            .add("file", fileScanTask.file() != null ?

Review comment:
       @openinx I have to add back the null check, because in some other tests (outside this sub PR) we are using `MockFileScanTask` with just length. In that case, file will be null.

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java
##########
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+public enum IcebergSourceSplitStatus {

Review comment:
       This is not used by this PR. will remove

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
##########
@@ -34,7 +34,7 @@
 /**
  * Context object with optional arguments for a Flink Scan.
  */
-class ScanContext implements Serializable {
+public class ScanContext implements Serializable {

Review comment:
       We currently hide `ScanContext` by duplicating all 10+ setters in `FlinkSource`. I am not sure the value of hiding `ScanContext` in this case. Eventually, users need to deal with the same setters via `FlinkSource` indirectly. Hence in the new FLIP-27 source, I made `ScanContext` a user-facing public API.
   
   If most people think it is better to hide `ScanContext` and duplicate the setters in `IcebergSource` (FLIP-27), I will make `IcebergSouce` the same style as the current `FlinkSource`. cc @rdblue @kbendick for additional inputs.

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, Position position) {
+    this.task = task;
+    this.position = position;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return fromCombinedScanTask(combinedScanTask, 0, 0L);
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(
+      CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) {
+    return new IcebergSourceSplit(combinedScanTask, new Position(fileOffset, recordOffset));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  byte[] serializedFormCache() {
+    return serializedFormCache;
+  }
+
+  void serializedFormCache(byte[] cachedBytes) {
+    this.serializedFormCache = cachedBytes;
+  }
+
+  @Override
+  public String splitId() {
+    return MoreObjects.toStringHelper(this)
+        .add("files", toString(task.files()))
+        .toString();
+  }
+
+  public void updatePosition(int newFileOffset, long newRecordOffset) {
+    position.update(newFileOffset, newRecordOffset);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    IcebergSourceSplit split = (IcebergSourceSplit) o;
+    return Objects.equal(splitId(), split.splitId()) &&
+        Objects.equal(position, split.position());
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(splitId());

Review comment:
       I am not sure we want to assign an unique integer number as `splitId` (especially for long-running streaming jobs).
   We need to checkpoint the splitId counter. what if we can't restore from checkpoint (e.g. due to corrupted checkpoint state)? It is probably better to compute the `splitId` based on the intrinsic properties of `IcebergSourceSplit` (like path, start, length of data files). 
   
   I thought your first comment is about depending on `FileScanTask#toString`, which I explained that it is not the case. We don't call `FileScanTask#toString`. Instead, we depend on `fileScanTask.file().path().toString()`. Maybe I misunderstood your first comment. Can you elaborate a little more on your concern?

##########
File path: flink/v1.13/flink/src/test/java/org/apache/iceberg/flink/source/split/SplitHelpers.java
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseCombinedScanTask;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MockFileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.flink.TestFixtures;
+import org.apache.iceberg.flink.source.FlinkSplitPlanner;
+import org.apache.iceberg.flink.source.ScanContext;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+
+public class SplitHelpers {
+
+  private static final AtomicLong splitLengthIncrement = new AtomicLong();
+
+  private SplitHelpers() {
+
+  }
+
+  public static List<IcebergSourceSplit> createMockedSplits(int splitCount) {
+    final List<IcebergSourceSplit> splits = new ArrayList<>();
+    for (int i = 0; i < splitCount; ++i) {
+      // make sure each task has a different length,
+      // as it is part of the splitId calculation.
+      // This way, we can make sure all generated splits have different splitIds
+      FileScanTask fileScanTask = new MockFileScanTask(1024 + splitLengthIncrement.incrementAndGet());
+      CombinedScanTask combinedScanTask = new BaseCombinedScanTask(fileScanTask);
+      splits.add(IcebergSourceSplit.fromCombinedScanTask(combinedScanTask));
+    }
+    return splits;
+  }
+
+  public static List<IcebergSourceSplit> createFileSplits(
+      TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception {
+    final File warehouseFile = temporaryFolder.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    final String warehouse = "file:" + warehouseFile;
+    org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration();
+    final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse);
+    try {
+      final Table table = catalog.createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA);
+      final GenericAppenderHelper dataAppender = new GenericAppenderHelper(
+          table, FileFormat.PARQUET, temporaryFolder);
+      for (int i = 0; i < fileCount; ++i) {
+        List<Record> records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i);
+        dataAppender.appendToTable(records);
+      }
+
+      final ScanContext scanContext = ScanContext.builder().build();
+      final List<IcebergSourceSplit> splits = FlinkSplitPlanner.planIcebergSourceSplits(table, scanContext);
+      return splits.stream()
+          .flatMap(split -> {
+            List<List<FileScanTask>> filesList = Lists.partition(new ArrayList<>(split.task().files()), filesPerSplit);
+            return filesList.stream()
+                .map(files ->  new BaseCombinedScanTask(files))
+                .map(combinedScanTask -> IcebergSourceSplit.fromCombinedScanTask(combinedScanTask));
+          })
+          .collect(Collectors.toList());
+    } finally {
+      catalog.dropTable(TestFixtures.TABLE_IDENTIFIER);

Review comment:
       Good question. I see why this can be confusing. 
   
   Right now, this method is used only by `TestIcebergSourceSplitSerializer` to generate some realistic splits with actual paths (unlike the `createMockedSplits`). I will add some comments both at method level and here. Let me know if I should move this method into the `TestIcebergSourceSplitSerializer`. I kept it here so that it might be useful for other testing

##########
File path: flink/v1.13/flink/src/test/java/org/apache/iceberg/flink/source/split/SplitHelpers.java
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseCombinedScanTask;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MockFileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.flink.TestFixtures;
+import org.apache.iceberg.flink.source.FlinkSplitPlanner;
+import org.apache.iceberg.flink.source.ScanContext;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+
+public class SplitHelpers {
+
+  private static final AtomicLong splitLengthIncrement = new AtomicLong();
+
+  private SplitHelpers() {
+
+  }
+
+  public static List<IcebergSourceSplit> createMockedSplits(int splitCount) {
+    final List<IcebergSourceSplit> splits = new ArrayList<>();
+    for (int i = 0; i < splitCount; ++i) {
+      // make sure each task has a different length,
+      // as it is part of the splitId calculation.
+      // This way, we can make sure all generated splits have different splitIds
+      FileScanTask fileScanTask = new MockFileScanTask(1024 + splitLengthIncrement.incrementAndGet());
+      CombinedScanTask combinedScanTask = new BaseCombinedScanTask(fileScanTask);
+      splits.add(IcebergSourceSplit.fromCombinedScanTask(combinedScanTask));
+    }
+    return splits;
+  }
+
+  public static List<IcebergSourceSplit> createFileSplits(
+      TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception {
+    final File warehouseFile = temporaryFolder.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    final String warehouse = "file:" + warehouseFile;
+    org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration();
+    final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse);
+    try {
+      final Table table = catalog.createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA);
+      final GenericAppenderHelper dataAppender = new GenericAppenderHelper(
+          table, FileFormat.PARQUET, temporaryFolder);
+      for (int i = 0; i < fileCount; ++i) {
+        List<Record> records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i);
+        dataAppender.appendToTable(records);
+      }
+
+      final ScanContext scanContext = ScanContext.builder().build();
+      final List<IcebergSourceSplit> splits = FlinkSplitPlanner.planIcebergSourceSplits(table, scanContext);
+      return splits.stream()
+          .flatMap(split -> {
+            List<List<FileScanTask>> filesList = Lists.partition(new ArrayList<>(split.task().files()), filesPerSplit);
+            return filesList.stream()
+                .map(files ->  new BaseCombinedScanTask(files))
+                .map(combinedScanTask -> IcebergSourceSplit.fromCombinedScanTask(combinedScanTask));

Review comment:
       will do

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
##########
@@ -34,7 +34,7 @@
 /**
  * Context object with optional arguments for a Flink Scan.
  */
-class ScanContext implements Serializable {
+public class ScanContext implements Serializable {

Review comment:
       Once SplitEnumerator classes (not included in this PR) will also use ScanContext and FlinkSplitPlanner. enumerator will be in a diff sub package. Hence FlinkSplitPlanner will need to be public too. But FlinkSplitPlanner is marked as @Internal.
   
   Current [FlinkSource](https://github.com/stevenzwu/iceberg/blob/flip27IcebergSource/flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java) clone all `ScanContext` APIs in its builder, which I think is unnecessary. 
   
   `ScanContext` will be a public API for the FLIP-27 [IcebergSource](https://github.com/stevenzwu/iceberg/blob/flip27IcebergSource/flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java#L181)

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {

Review comment:
       good catch. will add it

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, Position position) {
+    this.task = task;
+    this.position = position;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return fromCombinedScanTask(combinedScanTask, 0, 0L);
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(
+      CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) {
+    return new IcebergSourceSplit(combinedScanTask, new Position(fileOffset, recordOffset));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  byte[] serializedFormCache() {
+    return serializedFormCache;
+  }
+
+  void serializedFormCache(byte[] cachedBytes) {
+    this.serializedFormCache = cachedBytes;
+  }
+
+  @Override
+  public String splitId() {
+    return MoreObjects.toStringHelper(this)
+        .add("files", toString(task.files()))
+        .toString();
+  }
+
+  public void updatePosition(int newFileOffset, long newRecordOffset) {
+    position.update(newFileOffset, newRecordOffset);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    IcebergSourceSplit split = (IcebergSourceSplit) o;
+    return Objects.equal(splitId(), split.splitId()) &&
+        Objects.equal(position, split.position());
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(splitId());

Review comment:
       @kbendick you are correct. I had a typo in my earlier comment. should be "can't use the lazy approach because ..."

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, Position position) {
+    this.task = task;
+    this.position = position;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return fromCombinedScanTask(combinedScanTask, 0, 0L);
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(
+      CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) {
+    return new IcebergSourceSplit(combinedScanTask, new Position(fileOffset, recordOffset));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  byte[] serializedFormCache() {
+    return serializedFormCache;
+  }
+
+  void serializedFormCache(byte[] cachedBytes) {
+    this.serializedFormCache = cachedBytes;
+  }

Review comment:
       Serialization is done by another class `IcebergSourceSplitSerializer`. It can't be computed and cached internally. Hence this setter provides a way for `IcebergSourceSplitSerializer` to cache the serialized bytes.
   
   `lazyIdToField ` can work because everything is encapsulated within the `Schema` class.

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, Position position) {
+    this.task = task;
+    this.position = position;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return fromCombinedScanTask(combinedScanTask, 0, 0L);
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(
+      CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) {
+    return new IcebergSourceSplit(combinedScanTask, new Position(fileOffset, recordOffset));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  byte[] serializedFormCache() {
+    return serializedFormCache;
+  }
+
+  void serializedFormCache(byte[] cachedBytes) {
+    this.serializedFormCache = cachedBytes;
+  }
+
+  @Override
+  public String splitId() {
+    return MoreObjects.toStringHelper(this)
+        .add("files", toString(task.files()))
+        .toString();
+  }
+
+  public void updatePosition(int newFileOffset, long newRecordOffset) {
+    position.update(newFileOffset, newRecordOffset);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    IcebergSourceSplit split = (IcebergSourceSplit) o;
+    return Objects.equal(splitId(), split.splitId()) &&
+        Objects.equal(position, split.position());
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(splitId());
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("files", toString(task.files()))
+        .add("position", position)
+        .toString();
+  }
+
+  private String toString(Collection<FileScanTask> files) {
+    return Iterables.toString(files.stream().map(fileScanTask ->
+        MoreObjects.toStringHelper(fileScanTask)
+            .add("file", fileScanTask.file() != null ?
+                fileScanTask.file().path().toString() :
+                "NoFile")

Review comment:
       will remove the null check here

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, Position position) {
+    this.task = task;
+    this.position = position;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return fromCombinedScanTask(combinedScanTask, 0, 0L);
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(
+      CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) {
+    return new IcebergSourceSplit(combinedScanTask, new Position(fileOffset, recordOffset));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  byte[] serializedFormCache() {
+    return serializedFormCache;
+  }
+
+  void serializedFormCache(byte[] cachedBytes) {
+    this.serializedFormCache = cachedBytes;
+  }
+
+  @Override
+  public String splitId() {
+    return MoreObjects.toStringHelper(this)
+        .add("files", toString(task.files()))
+        .toString();
+  }
+
+  public void updatePosition(int newFileOffset, long newRecordOffset) {
+    position.update(newFileOffset, newRecordOffset);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    IcebergSourceSplit split = (IcebergSourceSplit) o;
+    return Objects.equal(splitId(), split.splitId()) &&
+        Objects.equal(position, split.position());
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(splitId());

Review comment:
       as mentioned in the other comment, `serializedFormCache` can't use the lazy approach because the serialization is done by a separate serializer class. `IcebergSourceSplit` doesn't know how to serialize itself. Hence we can't calculate the byte array's hashCode.
   
   We aren't not using the `toString` from `FileScanTask`. We are only using `fileScanTask.file().path().toString()`.

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, Position position) {
+    this.task = task;
+    this.position = position;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return fromCombinedScanTask(combinedScanTask, 0, 0L);
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(
+      CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) {
+    return new IcebergSourceSplit(combinedScanTask, new Position(fileOffset, recordOffset));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  byte[] serializedFormCache() {
+    return serializedFormCache;
+  }
+
+  void serializedFormCache(byte[] cachedBytes) {
+    this.serializedFormCache = cachedBytes;
+  }
+
+  @Override
+  public String splitId() {
+    return MoreObjects.toStringHelper(this)
+        .add("files", toString(task.files()))
+        .toString();
+  }
+
+  public void updatePosition(int newFileOffset, long newRecordOffset) {
+    position.update(newFileOffset, newRecordOffset);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    IcebergSourceSplit split = (IcebergSourceSplit) o;
+    return Objects.equal(splitId(), split.splitId()) &&
+        Objects.equal(position, split.position());
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(splitId());

Review comment:
       @openinx not sure where is the miscommunication. but we don't call `FileScanTask#toString` here. Instead, we only call `FileScanTask.file().path().toString()`. Please see this code snippet below.
   
   ```
     private String toString(Collection<FileScanTask> files) {
       return Iterables.toString(files.stream().map(fileScanTask ->
           MoreObjects.toStringHelper(fileScanTask)
               .add("file", fileScanTask.file() != null ?
                   fileScanTask.file().path().toString() :
                   "NoDataFile")
               .add("start", fileScanTask.start())
               .add("length", fileScanTask.length())
               .toString()).collect(Collectors.toList()));
     }
   ```

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, Position position) {
+    this.task = task;
+    this.position = position;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return fromCombinedScanTask(combinedScanTask, 0, 0L);
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(
+      CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) {
+    return new IcebergSourceSplit(combinedScanTask, new Position(fileOffset, recordOffset));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  byte[] serializedFormCache() {
+    return serializedFormCache;
+  }
+
+  void serializedFormCache(byte[] cachedBytes) {
+    this.serializedFormCache = cachedBytes;
+  }

Review comment:
       To me, lazy pattern is encapsulated within the class. E.g., `Schema` class knows how to compute the IdToField mapping in the `lazyIdToField` method. Here, we have a more getter/setter protocol btw two classes for caching serialized bytes.

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.IOException;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.InstantiationUtil;
+
+/**
+ * TODO: use Java serialization for now.
+ * Will switch to more stable serializer from
+ * <a href="https://github.com/apache/iceberg/issues/1698">issue-1698</a>.
+ */
+@Internal
+public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer<IcebergSourceSplit> {

Review comment:
       `SimpleVersionedSerializer` will always use one/latest version to serialize. But during deserialization, it should handle multiple versions to support evolution (e.g. when we switch from Java serializable to some Avro serialization for FileScanTask)
   
   ```
     @Override
     public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOException {
       switch (version) {
         case 1:
           return deserializeV1(serialized);
         default:
           throw new IOException("Unknown version: " + version);
       }
     }
   ```

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final CombinedScanTask task;
+  /**

Review comment:
       There are two `private final` variables. Hence I thought it is one block. it is just that the 2nd variable has a Javadoc. If it is Iceberg's style convention to always start a Javadoc comment with an empty line, I am very happy to conform

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
##########
@@ -34,7 +34,7 @@
 /**
  * Context object with optional arguments for a Flink Scan.
  */
-class ScanContext implements Serializable {
+public class ScanContext implements Serializable {

Review comment:
       I basically included all classes (including test) connected to the `IcebergSourceSplit`. I then copied those classes from the uber `flip27IcebergSource` branch to this new branch.
   
   `ScanContext` is used by `FlinkSplitPlanner`.
   ```
       if (context.includeColumnStats()) {
         scan = scan.includeColumnStats();
       }
   ```
   
   `FlinkSplitPlanner` (renamed from `FlinkSplitGenerator`) is used by `SplitHelper` test class

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.IOException;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.InstantiationUtil;
+
+/**
+ * TODO: use Java serialization for now.
+ * Will switch to more stable serializer from
+ * <a href="https://github.com/apache/iceberg/issues/1698">issue-1698</a>.
+ */
+@Internal
+public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer<IcebergSourceSplit> {
+  public static final IcebergSourceSplitSerializer INSTANCE = new IcebergSourceSplitSerializer();
+  private static final int VERSION = 1;
+
+  @Override
+  public int getVersion() {
+    return VERSION;
+  }
+
+  @Override
+  public byte[] serialize(IcebergSourceSplit split) throws IOException {
+    if (split.serializedFormCache() == null) {
+      final byte[] result = serializeV1(split);
+      split.serializedFormCache(result);
+    }
+    return split.serializedFormCache();
+  }
+
+  @Override
+  public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOException {
+    switch (version) {
+      case 1:
+        return deserializeV1(serialized);
+      default:
+        throw new IOException("Unknown version: " + version);

Review comment:
       "maximum version" is probably also not accurate. it implies all versions below this number are supported. We may drop support of deserializing older versions. I will change the error msg to sth like
   ```
   Failed to deserialize IcebergSourceSplit. Encountered unsupported version: $version. Supported version are [1]"
   ```

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedFormCache;

Review comment:
       will change it to `serializedBytesCache`. This is not a "lazy" pattern, which is typically encapsulated within the class. this is a getter/setter interaction btw `IcebergSourceSplit` and `IcebergSourceSplitSerializer`

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/Position.java
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.flink.annotation.Internal;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+/**
+ * A mutable class that defines the read position
+ * <ul>
+ *   <li>file offset in the list of files in a {@link CombinedScanTask}</li>
+ *   <li>record offset within a file</li>
+ * </ul>
+ */
+@Internal
+public class Position implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  private int fileOffset;
+  private long recordOffset;
+
+  public Position(int fileOffset, long recordOffset) {
+    this.fileOffset = fileOffset;
+    this.recordOffset = recordOffset;
+  }
+
+  void advanceFile() {
+    this.fileOffset += 1;
+    this.recordOffset = 0L;
+  }
+
+  void advanceRecord() {
+    this.recordOffset += 1L;
+  }
+
+  public void update(int newFileOffset, long newRecordOffset) {
+    this.fileOffset = newFileOffset;
+    this.recordOffset = newRecordOffset;
+  }
+
+  public int fileOffset() {
+    return fileOffset;
+  }
+
+  public long recordOffset() {
+    return recordOffset;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    final Position that = (Position) o;

Review comment:
       yes. will remove

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java
##########
@@ -22,33 +22,58 @@
 import java.io.IOException;
 import java.io.UncheckedIOException;
 import java.util.List;
+import org.apache.flink.annotation.Internal;
 import org.apache.iceberg.CombinedScanTask;
 import org.apache.iceberg.Table;
 import org.apache.iceberg.TableProperties;
 import org.apache.iceberg.TableScan;
 import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
 import org.apache.iceberg.io.CloseableIterable;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 
-class FlinkSplitGenerator {
-  private FlinkSplitGenerator() {
+@Internal
+public class FlinkSplitPlanner {
+  private FlinkSplitPlanner() {
   }
 
-  static FlinkInputSplit[] createInputSplits(Table table, ScanContext context) {
-    List<CombinedScanTask> tasks = tasks(table, context);
-    FlinkInputSplit[] splits = new FlinkInputSplit[tasks.size()];
-    for (int i = 0; i < tasks.size(); i++) {
-      splits[i] = new FlinkInputSplit(i, tasks.get(i));
+  static FlinkInputSplit[] planInputSplits(Table table, ScanContext context) {
+    try (CloseableIterable<CombinedScanTask> tasksIterable = planTasks(table, context)) {
+      List<CombinedScanTask> tasks = Lists.newArrayList(tasksIterable);
+      FlinkInputSplit[] splits = new FlinkInputSplit[tasks.size()];
+      for (int i = 0; i < tasks.size(); i++) {
+        splits[i] = new FlinkInputSplit(i, tasks.get(i));
+      }
+      return splits;
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to process tasks iterable", e);
+    }
+  }
+
+  /**
+   * This returns splits for the FLIP-27 source
+   */
+  public static List<IcebergSourceSplit> planIcebergSourceSplits(
+      Table table, ScanContext context) {
+    try (CloseableIterable<CombinedScanTask> tasksIterable = planTasks(table, context)) {
+      List<IcebergSourceSplit> splits = Lists.newArrayList();
+      tasksIterable.forEach(task -> splits.add(IcebergSourceSplit.fromCombinedScanTask(task)));
+      return splits;
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to process task iterable: ", e);
     }
-    return splits;
   }
 
-  private static List<CombinedScanTask> tasks(Table table, ScanContext context) {
+  static CloseableIterable<CombinedScanTask> planTasks(Table table, ScanContext context) {
     TableScan scan = table
         .newScan()
         .caseSensitive(context.caseSensitive())
         .project(context.project());
 
+    if (context.includeColumnStats()) {
+      scan = scan.includeColumnStats();
+    }

Review comment:
       column stats are needed for event time/watermark aligned assigner. You are correct that it is directly used by this PR. Right now, I am taking the approach of splitting sub PRs at minimally connected files for easier creation of the sub PRs. if you think it is important to avoid unrelated changes inside a file, I can revert the piece of change.

##########
File path: flink/v1.13/flink/src/test/java/org/apache/iceberg/flink/source/split/SplitHelpers.java
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseCombinedScanTask;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MockFileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.flink.TestFixtures;
+import org.apache.iceberg.flink.source.FlinkSplitPlanner;
+import org.apache.iceberg.flink.source.ScanContext;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+
+public class SplitHelpers {
+
+  private static final AtomicLong splitLengthIncrement = new AtomicLong();
+
+  private SplitHelpers() {
+
+  }
+
+  public static List<IcebergSourceSplit> createMockedSplits(int splitCount) {
+    final List<IcebergSourceSplit> splits = new ArrayList<>();
+    for (int i = 0; i < splitCount; ++i) {
+      // make sure each task has a different length,
+      // as it is part of the splitId calculation.
+      // This way, we can make sure all generated splits have different splitIds
+      FileScanTask fileScanTask = new MockFileScanTask(1024 + splitLengthIncrement.incrementAndGet());
+      CombinedScanTask combinedScanTask = new BaseCombinedScanTask(fileScanTask);
+      splits.add(IcebergSourceSplit.fromCombinedScanTask(combinedScanTask));
+    }
+    return splits;
+  }
+
+  public static List<IcebergSourceSplit> createFileSplits(
+      TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception {
+    final File warehouseFile = temporaryFolder.newFolder();

Review comment:
       thx. will do a complete pass on finding and removing the unnecessary `final` modifiers

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.IOException;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.InstantiationUtil;
+
+/**
+ * TODO: use Java serialization for now.
+ * Will switch to more stable serializer from
+ * <a href="https://github.com/apache/iceberg/issues/1698">issue-1698</a>.
+ */
+@Internal
+public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer<IcebergSourceSplit> {
+  public static final IcebergSourceSplitSerializer INSTANCE = new IcebergSourceSplitSerializer();
+  private static final int VERSION = 1;
+
+  @Override
+  public int getVersion() {
+    return VERSION;
+  }
+
+  @Override
+  public byte[] serialize(IcebergSourceSplit split) throws IOException {
+    if (split.serializedBytesCache() == null) {
+      final byte[] result = serializeV1(split);

Review comment:
       will fix

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java
##########
@@ -22,33 +22,58 @@
 import java.io.IOException;
 import java.io.UncheckedIOException;
 import java.util.List;
+import org.apache.flink.annotation.Internal;
 import org.apache.iceberg.CombinedScanTask;
 import org.apache.iceberg.Table;
 import org.apache.iceberg.TableProperties;
 import org.apache.iceberg.TableScan;
 import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
 import org.apache.iceberg.io.CloseableIterable;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 
-class FlinkSplitGenerator {
-  private FlinkSplitGenerator() {
+@Internal
+public class FlinkSplitPlanner {
+  private FlinkSplitPlanner() {
   }
 
-  static FlinkInputSplit[] createInputSplits(Table table, ScanContext context) {
-    List<CombinedScanTask> tasks = tasks(table, context);
-    FlinkInputSplit[] splits = new FlinkInputSplit[tasks.size()];
-    for (int i = 0; i < tasks.size(); i++) {
-      splits[i] = new FlinkInputSplit(i, tasks.get(i));
+  static FlinkInputSplit[] planInputSplits(Table table, ScanContext context) {
+    try (CloseableIterable<CombinedScanTask> tasksIterable = planTasks(table, context)) {
+      List<CombinedScanTask> tasks = Lists.newArrayList(tasksIterable);
+      FlinkInputSplit[] splits = new FlinkInputSplit[tasks.size()];
+      for (int i = 0; i < tasks.size(); i++) {
+        splits[i] = new FlinkInputSplit(i, tasks.get(i));
+      }
+      return splits;
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to process tasks iterable", e);
+    }
+  }
+
+  /**
+   * This returns splits for the FLIP-27 source
+   */
+  public static List<IcebergSourceSplit> planIcebergSourceSplits(

Review comment:
       will fix

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedBytesCache;
+
+  private IcebergSourceSplit(CombinedScanTask task, Position position) {
+    this.task = task;
+    this.position = position;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return fromCombinedScanTask(combinedScanTask, 0, 0L);
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(
+      CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) {
+    return new IcebergSourceSplit(combinedScanTask, new Position(fileOffset, recordOffset));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  byte[] serializedBytesCache() {
+    return serializedBytesCache;
+  }
+
+  void serializedBytesCache(byte[] cachedBytes) {
+    this.serializedBytesCache = cachedBytes;
+  }
+
+  @Override
+  public String splitId() {
+    return MoreObjects.toStringHelper(this)
+        .add("files", toString(task.files()))
+        .toString();
+  }
+
+  public void updatePosition(int newFileOffset, long newRecordOffset) {
+    position.update(newFileOffset, newRecordOffset);
+  }

Review comment:
       it is not used here. But it is used by `IcebergSourceRecordEmitter`. 
   
   Again, I am creating the PR for minimal connected files (not minimally connected code within files). 

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.IOException;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.InstantiationUtil;
+
+/**
+ * TODO: use Java serialization for now.
+ * Will switch to more stable serializer from
+ * <a href="https://github.com/apache/iceberg/issues/1698">issue-1698</a>.
+ */
+@Internal
+public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer<IcebergSourceSplit> {

Review comment:
       This class is used for checkpoint state serializer. cross-process (JM->TM) communication is via Java serializable. Currently, we are using the Java serializable inside this class too for simpler start. This is not ideal, as we know Java serializable is not good with schema evolution. Schema evolution would be important for long-running streaming jobs (not so much for batch jobs).
   
   In the class Javadoc, we linked to an issue for future improvement. Note that this is already an issue for the current `FlinkSource` in streaming mode.
   https://github.com/apache/iceberg/issues/1698

##########
File path: flink/v1.13/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java
##########
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestIcebergSourceSplitSerializer {
+
+  @ClassRule
+  public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder();
+
+  private final IcebergSourceSplitSerializer serializer = IcebergSourceSplitSerializer.INSTANCE;
+
+  @Test
+  public void testLatestVersion() throws Exception {
+    serializeAndDeserialize(1, 1);
+    serializeAndDeserialize(10, 2);
+  }
+
+  private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws Exception {
+    final List<IcebergSourceSplit> splits = SplitHelpers.createFileSplits(TEMPORARY_FOLDER, splitCount, filesPerSplit);
+    for (IcebergSourceSplit split : splits) {
+      final byte[] result = serializer.serialize(split);
+      final IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result);
+      Assert.assertEquals(split, deserialized);
+
+      final byte[] cachedResult = serializer.serialize(split);
+      Assert.assertSame(result, cachedResult);
+      final IcebergSourceSplit deserialized2 = serializer.deserialize(serializer.getVersion(), cachedResult);
+      Assert.assertEquals(split, deserialized2);

Review comment:
       we already asserted `deserialized ` in line 48. this is to make sure `deserialized2` from the 2nd call from `serializer.deserialize` (cached bytes) still gets the same split.
   
   




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org