You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "pvary (via GitHub)" <gi...@apache.org> on 2023/05/19 21:07:47 UTC

[GitHub] [iceberg] pvary opened a new pull request, #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

pvary opened a new pull request, #7661:
URL: https://github.com/apache/iceberg/pull/7661

   Flink users often depend on the fact that the IcebergSource reads the Iceberg snapshots in the same order as they were committed. This assumption could be broken on job start or job restart when multiple snapshots are read in one planning cycle.
   
   To ensure that the splits are ordered as expected the PR adds FileSequenceNumberBasedSplitAssignerFactory where the splits are ordered based on the ContentFile.fileSequenceNumber().


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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1224024679


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java:
##########
@@ -462,15 +475,22 @@ public IcebergSource<T> build() {
         }
       }
 
+      if (splitAssignerFactory == null) {
+        if (splitComparator == null) {
+          splitAssignerFactory = new SimpleSplitAssignerFactory();
+        } else {
+          splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator);
+        }
+      }
+
       checkRequired();
       // Since builder already load the table, pass it to the source to avoid double loading
       return new IcebergSource<T>(
-          tableLoader, context, readerFunction, splitAssignerFactory, table);
+          tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table);
     }
 
     private void checkRequired() {
       Preconditions.checkNotNull(tableLoader, "tableLoader is required.");
-      Preconditions.checkNotNull(splitAssignerFactory, "assignerFactory is required.");

Review Comment:
   Added back



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1222886844


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator

Review Comment:
   Changed, added, please check



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1218082380


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.Comparator;
+import java.util.Optional;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator implements Comparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    if (o1.splitId().equals(o2.splitId())) {
+      return 0;
+    }
+
+    Optional<Long> opt1 =

Review Comment:
   Done



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1222740144


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator
+    implements SerializableComparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber();
+    Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber();
+
+    Preconditions.checkNotNull(
+        seq1, "Invalid file sequence number for {}. Only V2 table format is supported", o1);

Review Comment:
   But printing out `null` here does not tell the user/administrator which split is causing the issue.
   Whereas adding the split will help fixing the problem



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1218620089


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SortedSplitAssigner.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.assigner;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitState;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link SplitAssigner} which assigns the splits ordered by the {@link
+ * ContentFile#fileSequenceNumber()}.
+ */
+class SortedSplitAssigner implements SplitAssigner {
+  private static final Logger LOG = LoggerFactory.getLogger(SortedSplitAssigner.class);
+
+  // CHECKSTYLE.OFF: IllegalTypeCheck - We use TreeSet.pollFirst
+  private final TreeSet<IcebergSourceSplit> pendingSplits;
+  // CHECKSTYLE.ON: IllegalTypeCheck
+  private CompletableFuture<Void> availableFuture;
+
+  SortedSplitAssigner(Comparator<IcebergSourceSplit> comparator) {
+    this(comparator, ImmutableList.of());
+  }
+
+  SortedSplitAssigner(
+      Comparator<IcebergSourceSplit> comparator,
+      Collection<IcebergSourceSplitState> assignerState) {
+    this.pendingSplits = Sets.newTreeSet(comparator);
+    // Because simple assigner only tracks unassigned splits,
+    // there is no need to filter splits based on status (unassigned) here.
+    assignerState.forEach(splitState -> pendingSplits.add(splitState.split()));
+  }
+
+  @Override
+  public synchronized GetSplitResult getNext(@Nullable String hostname) {
+    if (pendingSplits.isEmpty()) {
+      return GetSplitResult.unavailable();
+    } else {
+      IcebergSourceSplit split = pendingSplits.pollFirst();
+      LOG.info("Split served {}", split);
+      return GetSplitResult.forSplit(split);
+    }
+  }
+
+  @Override
+  public void onDiscoveredSplits(Collection<IcebergSourceSplit> splits) {
+    LOG.info("Splits discovered {}", splits);
+    addSplits(splits);
+  }
+
+  @Override
+  public void onUnassignedSplits(Collection<IcebergSourceSplit> splits) {
+    LOG.info("Splits unassigned {}", splits);
+    addSplits(splits);
+  }
+
+  private synchronized void addSplits(Collection<IcebergSourceSplit> splits) {
+    if (!splits.isEmpty()) {
+      pendingSplits.addAll(splits);
+      // only complete pending future if new splits are discovered
+      completeAvailableFuturesIfNeeded();
+    }
+  }
+
+  /** Simple assigner only tracks unassigned splits */
+  @Override
+  public synchronized Collection<IcebergSourceSplitState> state() {

Review Comment:
   `IcebergSourceSplitState` class just wrapped `IcebergSourceSplit` with a status flag (like `UNASSIGNED`, `ASSIGNED`). the flag was introduced for the watermark alignment assigner, as it needs to track the `ASSIGNED` (and not completed) splits in addition to the pending `UNASSIGNED` splits. With the new approach of leveraging Flink for watermark tracking/aggregation, we only need to track pending `UNASSIGNED` splits.



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1224024969


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator
+    implements SerializableComparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber();
+    Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber();
+
+    Preconditions.checkNotNull(
+        seq1, "Invalid file sequence number for {}. Only V2 table format is supported", o1);

Review Comment:
   Done



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


[GitHub] [iceberg] stevenzwu merged pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu merged PR #7661:
URL: https://github.com/apache/iceberg/pull/7661


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


[GitHub] [iceberg] pvary commented on pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#issuecomment-1603708342

   Thanks @stevenzwu for the review!
   Hopefully I will have some time to backport this to the other Flink versions soon


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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1222740144


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator
+    implements SerializableComparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber();
+    Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber();
+
+    Preconditions.checkNotNull(
+        seq1, "Invalid file sequence number for {}. Only V2 table format is supported", o1);

Review Comment:
   But printing out `null` here does not tell the user/administrator which split is causing the issue.
   Whereas adding the split will help fixing the problem, if not the whole table is problematic, just one old file.



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1222738538


##########
flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java:
##########
@@ -55,13 +56,67 @@ public void testReaderMetrics() throws Exception {
     TestingReaderOutput<RowData> readerOutput = new TestingReaderOutput<>();
     TestingMetricGroup metricGroup = new TestingMetricGroup();
     TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
-    IcebergSourceReader reader = createReader(metricGroup, readerContext);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, null);
     reader.start();
 
     testOneSplitFetcher(reader, readerOutput, metricGroup, 1);
     testOneSplitFetcher(reader, readerOutput, metricGroup, 2);
   }
 
+  @Test
+  public void testReaderOrder() throws Exception {
+    // Create 2 splits
+    List<List<Record>> recordBatchList1 =
+        ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task1 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    List<List<Record>> recordBatchList2 =
+        ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task2 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    // Sort the splits in one way
+    List<RowData> rowData1 =
+        read(
+            Arrays.asList(
+                IcebergSourceSplit.fromCombinedScanTask(task1),
+                IcebergSourceSplit.fromCombinedScanTask(task2)),
+            2);
+
+    // Reverse the splits
+    List<RowData> rowData2 =
+        read(
+            Arrays.asList(
+                IcebergSourceSplit.fromCombinedScanTask(task2),
+                IcebergSourceSplit.fromCombinedScanTask(task1)),
+            2);
+
+    // Check that the order of the elements is not changed
+    Assert.assertEquals(rowData1.get(0), rowData2.get(0));
+    Assert.assertEquals(rowData1.get(1), rowData2.get(1));
+  }
+
+  private List<RowData> read(List<IcebergSourceSplit> splits, long expected) throws Exception {
+    TestingMetricGroup metricGroup = new TestingMetricGroup();
+    TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator());

Review Comment:
   We would like to have a specific, well defined order of the splits. That is why I use a specific comparator instead of just leaving it `null`



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


[GitHub] [iceberg] stevenzwu commented on pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#issuecomment-1595273810

   thanks @pvary for the contribution


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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1221835400


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.assigner;
+
+import java.util.Collection;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitState;
+import org.apache.iceberg.flink.source.split.SerializableComparator;
+
+/** Create simple assigner that hands out splits without any guarantee in order or locality. */

Review Comment:
   nit: Javadoc needs to be changed



##########
flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java:
##########
@@ -62,8 +62,15 @@ public static List<IcebergSourceSplit> createSplitsFromTransientHadoopTable(
     final String warehouse = "file:" + warehouseFile;
     Configuration hadoopConf = new Configuration();
     final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse);
+    ImmutableMap<String, String> properties = ImmutableMap.of(TableProperties.FORMAT_VERSION, "2");

Review Comment:
   this hardcode the format to v2. what about adding a 2nd variation of `createSplitsFromTransientHadoopTable` with format version passed in?



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java:
##########
@@ -462,15 +475,22 @@ public IcebergSource<T> build() {
         }
       }
 
+      if (splitAssignerFactory == null) {
+        if (splitComparator == null) {
+          splitAssignerFactory = new SimpleSplitAssignerFactory();
+        } else {
+          splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator);
+        }
+      }
+
       checkRequired();
       // Since builder already load the table, pass it to the source to avoid double loading
       return new IcebergSource<T>(
-          tableLoader, context, readerFunction, splitAssignerFactory, table);
+          tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table);
     }
 
     private void checkRequired() {
       Preconditions.checkNotNull(tableLoader, "tableLoader is required.");
-      Preconditions.checkNotNull(splitAssignerFactory, "assignerFactory is required.");

Review Comment:
   why removing this check? it is still valid, right?



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator
+    implements SerializableComparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber();
+    Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber();
+
+    Preconditions.checkNotNull(
+        seq1, "Invalid file sequence number for {}. Only V2 table format is supported", o1);
+    Preconditions.checkNotNull(
+        seq2, "Invalid file sequence number for {}. Only V2 table format is supported", o2);
+
+    if (o1.splitId().equals(o2.splitId())) {

Review Comment:
   I think this if block is not necessarily



##########
flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java:
##########
@@ -55,13 +56,67 @@ public void testReaderMetrics() throws Exception {
     TestingReaderOutput<RowData> readerOutput = new TestingReaderOutput<>();
     TestingMetricGroup metricGroup = new TestingMetricGroup();
     TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
-    IcebergSourceReader reader = createReader(metricGroup, readerContext);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, null);
     reader.start();
 
     testOneSplitFetcher(reader, readerOutput, metricGroup, 1);
     testOneSplitFetcher(reader, readerOutput, metricGroup, 2);
   }
 
+  @Test
+  public void testReaderOrder() throws Exception {
+    // Create 2 splits
+    List<List<Record>> recordBatchList1 =
+        ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task1 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    List<List<Record>> recordBatchList2 =
+        ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task2 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    // Sort the splits in one way
+    List<RowData> rowData1 =

Review Comment:
   nit: `rowData1` sounds like a single `RowData` object (not a list)



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator

Review Comment:
   let's also provide some Javadoc for the factory method.



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java:
##########
@@ -23,14 +23,15 @@
 
 /** Create simple assigner that hands out splits without any guarantee in order or locality. */
 public class SimpleSplitAssignerFactory implements SplitAssignerFactory {
+  public SimpleSplitAssignerFactory() {}
 
   @Override
-  public SimpleSplitAssigner createAssigner() {
-    return new SimpleSplitAssigner();
+  public DefaultSplitAssigner createAssigner() {

Review Comment:
   just realized the previous mistake. this should just return the `SplitAssigner` interface (not the impl class)



##########
flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java:
##########
@@ -55,13 +56,67 @@ public void testReaderMetrics() throws Exception {
     TestingReaderOutput<RowData> readerOutput = new TestingReaderOutput<>();
     TestingMetricGroup metricGroup = new TestingMetricGroup();
     TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
-    IcebergSourceReader reader = createReader(metricGroup, readerContext);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, null);
     reader.start();
 
     testOneSplitFetcher(reader, readerOutput, metricGroup, 1);
     testOneSplitFetcher(reader, readerOutput, metricGroup, 2);
   }
 
+  @Test
+  public void testReaderOrder() throws Exception {
+    // Create 2 splits
+    List<List<Record>> recordBatchList1 =
+        ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task1 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    List<List<Record>> recordBatchList2 =
+        ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task2 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    // Sort the splits in one way
+    List<RowData> rowData1 =
+        read(
+            Arrays.asList(
+                IcebergSourceSplit.fromCombinedScanTask(task1),
+                IcebergSourceSplit.fromCombinedScanTask(task2)),
+            2);
+
+    // Reverse the splits
+    List<RowData> rowData2 =
+        read(
+            Arrays.asList(
+                IcebergSourceSplit.fromCombinedScanTask(task2),
+                IcebergSourceSplit.fromCombinedScanTask(task1)),
+            2);
+
+    // Check that the order of the elements is not changed
+    Assert.assertEquals(rowData1.get(0), rowData2.get(0));
+    Assert.assertEquals(rowData1.get(1), rowData2.get(1));
+  }
+
+  private List<RowData> read(List<IcebergSourceSplit> splits, long expected) throws Exception {
+    TestingMetricGroup metricGroup = new TestingMetricGroup();
+    TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator());

Review Comment:
   do we need `IdBasedComparator`? we can just pass in null here right?



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator

Review Comment:
   Following many of Iceberg code pattern, we can name this class as `SplitComparators` and provide an API as `public static SerializableComparator<IcebergSourceSplit> fileSequenceNumber()`



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1224384971


##########
flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java:
##########
@@ -55,13 +56,67 @@ public void testReaderMetrics() throws Exception {
     TestingReaderOutput<RowData> readerOutput = new TestingReaderOutput<>();
     TestingMetricGroup metricGroup = new TestingMetricGroup();
     TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
-    IcebergSourceReader reader = createReader(metricGroup, readerContext);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, null);
     reader.start();
 
     testOneSplitFetcher(reader, readerOutput, metricGroup, 1);
     testOneSplitFetcher(reader, readerOutput, metricGroup, 2);
   }
 
+  @Test
+  public void testReaderOrder() throws Exception {
+    // Create 2 splits
+    List<List<Record>> recordBatchList1 =
+        ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task1 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    List<List<Record>> recordBatchList2 =
+        ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task2 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    // Sort the splits in one way
+    List<RowData> rowData1 =
+        read(
+            Arrays.asList(
+                IcebergSourceSplit.fromCombinedScanTask(task1),
+                IcebergSourceSplit.fromCombinedScanTask(task2)),
+            2);
+
+    // Reverse the splits
+    List<RowData> rowData2 =
+        read(
+            Arrays.asList(
+                IcebergSourceSplit.fromCombinedScanTask(task2),
+                IcebergSourceSplit.fromCombinedScanTask(task1)),
+            2);
+
+    // Check that the order of the elements is not changed
+    Assert.assertEquals(rowData1.get(0), rowData2.get(0));
+    Assert.assertEquals(rowData1.get(1), rowData2.get(1));
+  }
+
+  private List<RowData> read(List<IcebergSourceSplit> splits, long expected) throws Exception {
+    TestingMetricGroup metricGroup = new TestingMetricGroup();
+    TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator());

Review Comment:
   the old behavior is null. don't we want to test that?



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1224025225


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator}
+ * which could be used for ordering splits. These are used by the {@link
+ * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link
+ * org.apache.iceberg.flink.source.reader.IcebergSourceReader}
+ */
+public class SplitComparators {
+  private SplitComparators() {}
+
+  /** Comparator which orders the splits based on the file sequence number of the data files */
+  public static SerializableComparator<IcebergSourceSplit> fileSequenceNumber() {
+    return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> {

Review Comment:
   Added a test



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1222888642


##########
flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java:
##########
@@ -62,8 +62,15 @@ public static List<IcebergSourceSplit> createSplitsFromTransientHadoopTable(
     final String warehouse = "file:" + warehouseFile;
     Configuration hadoopConf = new Configuration();
     final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse);
+    ImmutableMap<String, String> properties = ImmutableMap.of(TableProperties.FORMAT_VERSION, "2");

Review Comment:
   Since we are moving forward to make V2 the default, we might want to change the default to `2`, but kept it as `1` for now, since the goal of this PR is different



##########
flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java:
##########
@@ -55,13 +56,67 @@ public void testReaderMetrics() throws Exception {
     TestingReaderOutput<RowData> readerOutput = new TestingReaderOutput<>();
     TestingMetricGroup metricGroup = new TestingMetricGroup();
     TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
-    IcebergSourceReader reader = createReader(metricGroup, readerContext);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, null);
     reader.start();
 
     testOneSplitFetcher(reader, readerOutput, metricGroup, 1);
     testOneSplitFetcher(reader, readerOutput, metricGroup, 2);
   }
 
+  @Test
+  public void testReaderOrder() throws Exception {
+    // Create 2 splits
+    List<List<Record>> recordBatchList1 =
+        ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task1 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    List<List<Record>> recordBatchList2 =
+        ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task2 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    // Sort the splits in one way
+    List<RowData> rowData1 =

Review Comment:
   renamed



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1218080731


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SortedSplitAssigner.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.assigner;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitState;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link SplitAssigner} which assigns the splits ordered by the {@link
+ * ContentFile#fileSequenceNumber()}.
+ */
+class SortedSplitAssigner implements SplitAssigner {

Review Comment:
   Done.
   A bit concerned about the name change, as this is a backward incompatible change, but since this shows up in compile time, I am ok with it



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1218082073


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SortedSplitAssigner.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.assigner;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitState;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link SplitAssigner} which assigns the splits ordered by the {@link
+ * ContentFile#fileSequenceNumber()}.
+ */
+class SortedSplitAssigner implements SplitAssigner {
+  private static final Logger LOG = LoggerFactory.getLogger(SortedSplitAssigner.class);
+
+  // CHECKSTYLE.OFF: IllegalTypeCheck - We use TreeSet.pollFirst
+  private final TreeSet<IcebergSourceSplit> pendingSplits;
+  // CHECKSTYLE.ON: IllegalTypeCheck
+  private CompletableFuture<Void> availableFuture;
+
+  SortedSplitAssigner(Comparator<IcebergSourceSplit> comparator) {
+    this(comparator, ImmutableList.of());
+  }
+
+  SortedSplitAssigner(
+      Comparator<IcebergSourceSplit> comparator,
+      Collection<IcebergSourceSplitState> assignerState) {
+    this.pendingSplits = Sets.newTreeSet(comparator);
+    // Because simple assigner only tracks unassigned splits,
+    // there is no need to filter splits based on status (unassigned) here.
+    assignerState.forEach(splitState -> pendingSplits.add(splitState.split()));
+  }
+
+  @Override
+  public synchronized GetSplitResult getNext(@Nullable String hostname) {
+    if (pendingSplits.isEmpty()) {
+      return GetSplitResult.unavailable();
+    } else {
+      IcebergSourceSplit split = pendingSplits.pollFirst();
+      LOG.info("Split served {}", split);
+      return GetSplitResult.forSplit(split);
+    }
+  }
+
+  @Override
+  public void onDiscoveredSplits(Collection<IcebergSourceSplit> splits) {
+    LOG.info("Splits discovered {}", splits);
+    addSplits(splits);
+  }
+
+  @Override
+  public void onUnassignedSplits(Collection<IcebergSourceSplit> splits) {
+    LOG.info("Splits unassigned {}", splits);
+    addSplits(splits);
+  }
+
+  private synchronized void addSplits(Collection<IcebergSourceSplit> splits) {
+    if (!splits.isEmpty()) {
+      pendingSplits.addAll(splits);
+      // only complete pending future if new splits are discovered
+      completeAvailableFuturesIfNeeded();
+    }
+  }
+
+  /** Simple assigner only tracks unassigned splits */
+  @Override
+  public synchronized Collection<IcebergSourceSplitState> state() {

Review Comment:
   Wouldn't we still need the state to store the current read positions at savepoints/checkpoints?



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.Comparator;
+import java.util.Optional;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator implements Comparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    if (o1.splitId().equals(o2.splitId())) {
+      return 0;
+    }
+
+    Optional<Long> opt1 =
+        o1.task().files().stream().map(f -> f.file().fileSequenceNumber()).min(Long::compareTo);
+    Optional<Long> opt2 =
+        o2.task().files().stream().map(f -> f.file().fileSequenceNumber()).min(Long::compareTo);
+
+    Preconditions.checkArgument(opt1.isPresent(), "Sequence number should not be null for {}", o1);

Review Comment:
   Done



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1218616672


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SortedSplitAssigner.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.assigner;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitState;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link SplitAssigner} which assigns the splits ordered by the {@link
+ * ContentFile#fileSequenceNumber()}.
+ */
+class SortedSplitAssigner implements SplitAssigner {

Review Comment:
   assigners are non-public. assigner factory classes are public. we need to be more careful there.



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1226618791


##########
flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java:
##########
@@ -55,13 +56,67 @@ public void testReaderMetrics() throws Exception {
     TestingReaderOutput<RowData> readerOutput = new TestingReaderOutput<>();
     TestingMetricGroup metricGroup = new TestingMetricGroup();
     TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
-    IcebergSourceReader reader = createReader(metricGroup, readerContext);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, null);
     reader.start();
 
     testOneSplitFetcher(reader, readerOutput, metricGroup, 1);
     testOneSplitFetcher(reader, readerOutput, metricGroup, 2);
   }
 
+  @Test
+  public void testReaderOrder() throws Exception {
+    // Create 2 splits
+    List<List<Record>> recordBatchList1 =
+        ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task1 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    List<List<Record>> recordBatchList2 =
+        ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task2 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    // Sort the splits in one way
+    List<RowData> rowData1 =
+        read(
+            Arrays.asList(
+                IcebergSourceSplit.fromCombinedScanTask(task1),
+                IcebergSourceSplit.fromCombinedScanTask(task2)),
+            2);
+
+    // Reverse the splits
+    List<RowData> rowData2 =
+        read(
+            Arrays.asList(
+                IcebergSourceSplit.fromCombinedScanTask(task2),
+                IcebergSourceSplit.fromCombinedScanTask(task1)),
+            2);
+
+    // Check that the order of the elements is not changed
+    Assert.assertEquals(rowData1.get(0), rowData2.get(0));
+    Assert.assertEquals(rowData1.get(1), rowData2.get(1));
+  }
+
+  private List<RowData> read(List<IcebergSourceSplit> splits, long expected) throws Exception {
+    TestingMetricGroup metricGroup = new TestingMetricGroup();
+    TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator());

Review Comment:
   When there is no comparator there is no specification defining the read order, so I think adding a test for that is not needed.
   
   Also we have many other tests where the comparator is set to `null`, so I think we can rely on them if the goal is to check that the `IcebergSourceReader` is working with `null` comparator.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1218678324


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssignerFactory.java:
##########
@@ -19,18 +19,29 @@
 package org.apache.iceberg.flink.source.assigner;
 
 import java.util.Collection;
+import java.util.Comparator;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
 import org.apache.iceberg.flink.source.split.IcebergSourceSplitState;
 
 /** Create simple assigner that hands out splits without any guarantee in order or locality. */
-public class SimpleSplitAssignerFactory implements SplitAssignerFactory {
+public class DefaultSplitAssignerFactory implements SplitAssignerFactory {

Review Comment:
   maybe we don't have to change the factory class, since it is public. 
   
   earlier, I was thinking about just renaming the `SimpleSplitAssigner` to `DefaultSplitAssigner` with queue as constructor arg.
   
   We can keep the `SimpleSplitAssignerFactory` and add a new factory as `OrderedSplitAssignerFactory` like you had earlier.
   
   



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java:
##########
@@ -76,6 +78,7 @@
   private final ScanContext scanContext;
   private final ReaderFunction<T> readerFunction;
   private final SplitAssignerFactory assignerFactory;
+  private final Comparator<IcebergSourceSplit> splitComparator;

Review Comment:
   `Comparator` interface is not extended from `Serializable`. Although JDK doc recommends implementing comparator to implement serializable, it is not not part of the interface contract.
   
   We may want to introduce a `SerializableComparator` interface like `SerializableSupplier`.



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.Comparator;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator implements Comparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    Long opt1 = o1.task().files().iterator().next().file().fileSequenceNumber();
+    Long opt2 = o2.task().files().iterator().next().file().fileSequenceNumber();
+
+    Preconditions.checkNotNull(
+        opt1, "V2 table is needed. Sequence number should not be null for {}", o1);

Review Comment:
   nit: following the style of many error msgs in the Iceberg code. maybe `Invalid file sequence number: null. Only V2 table format is supported`



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.Comparator;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator implements Comparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    Long opt1 = o1.task().files().iterator().next().file().fileSequenceNumber();

Review Comment:
   nit: seq1?



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1223342802


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator}
+ * which could be used for ordering splits. These are used by the {@link
+ * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link
+ * org.apache.iceberg.flink.source.reader.IcebergSourceReader}
+ */
+public class SplitComparators {
+  private SplitComparators() {}
+
+  /** Comparator which orders the splits based on the file sequence number of the data files */
+  public static SerializableComparator<IcebergSourceSplit> fileSequenceNumber() {
+    return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> {

Review Comment:
   I thought anonymous class is not serializable. can we add a unit test to confirm?



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator
+    implements SerializableComparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber();
+    Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber();
+
+    Preconditions.checkNotNull(
+        seq1, "Invalid file sequence number for {}. Only V2 table format is supported", o1);

Review Comment:
   agree with what you side regarding which split causing the issue. but we will need to call out what exactly is invalid (`null` in this case). maybe print the split in the end of the formatted string
   ```
   Invalid file sequence number: null. Doesn't support splits written with V1 format: %s
   ```



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1223339467


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java:
##########
@@ -462,15 +475,22 @@ public IcebergSource<T> build() {
         }
       }
 
+      if (splitAssignerFactory == null) {
+        if (splitComparator == null) {
+          splitAssignerFactory = new SimpleSplitAssignerFactory();
+        } else {
+          splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator);
+        }
+      }
+
       checkRequired();
       // Since builder already load the table, pass it to the source to avoid double loading
       return new IcebergSource<T>(
-          tableLoader, context, readerFunction, splitAssignerFactory, table);
+          tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table);
     }
 
     private void checkRequired() {
       Preconditions.checkNotNull(tableLoader, "tableLoader is required.");
-      Preconditions.checkNotNull(splitAssignerFactory, "assignerFactory is required.");

Review Comment:
   I see that. but we can probably still keep the not null check



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1232729983


##########
flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java:
##########
@@ -55,13 +56,67 @@ public void testReaderMetrics() throws Exception {
     TestingReaderOutput<RowData> readerOutput = new TestingReaderOutput<>();
     TestingMetricGroup metricGroup = new TestingMetricGroup();
     TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
-    IcebergSourceReader reader = createReader(metricGroup, readerContext);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, null);
     reader.start();
 
     testOneSplitFetcher(reader, readerOutput, metricGroup, 1);
     testOneSplitFetcher(reader, readerOutput, metricGroup, 2);
   }
 
+  @Test
+  public void testReaderOrder() throws Exception {
+    // Create 2 splits
+    List<List<Record>> recordBatchList1 =
+        ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task1 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    List<List<Record>> recordBatchList2 =
+        ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1);
+    CombinedScanTask task2 =
+        ReaderUtil.createCombinedScanTask(
+            recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory);
+
+    // Sort the splits in one way
+    List<RowData> rowData1 =
+        read(
+            Arrays.asList(
+                IcebergSourceSplit.fromCombinedScanTask(task1),
+                IcebergSourceSplit.fromCombinedScanTask(task2)),
+            2);
+
+    // Reverse the splits
+    List<RowData> rowData2 =
+        read(
+            Arrays.asList(
+                IcebergSourceSplit.fromCombinedScanTask(task2),
+                IcebergSourceSplit.fromCombinedScanTask(task1)),
+            2);
+
+    // Check that the order of the elements is not changed
+    Assert.assertEquals(rowData1.get(0), rowData2.get(0));
+    Assert.assertEquals(rowData1.get(1), rowData2.get(1));
+  }
+
+  private List<RowData> read(List<IcebergSourceSplit> splits, long expected) throws Exception {
+    TestingMetricGroup metricGroup = new TestingMetricGroup();
+    TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup);
+    IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator());

Review Comment:
   got it. yes, my main intention is to check that reader works with `null` comparator.



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1222709592


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java:
##########
@@ -462,15 +475,22 @@ public IcebergSource<T> build() {
         }
       }
 
+      if (splitAssignerFactory == null) {
+        if (splitComparator == null) {
+          splitAssignerFactory = new SimpleSplitAssignerFactory();
+        } else {
+          splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator);
+        }
+      }
+
       checkRequired();
       // Since builder already load the table, pass it to the source to avoid double loading
       return new IcebergSource<T>(
-          tableLoader, context, readerFunction, splitAssignerFactory, table);
+          tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table);
     }
 
     private void checkRequired() {
       Preconditions.checkNotNull(tableLoader, "tableLoader is required.");
-      Preconditions.checkNotNull(splitAssignerFactory, "assignerFactory is required.");

Review Comment:
   We have added this code:
   ```
         if (splitAssignerFactory == null) {
           if (splitComparator == null) {
             splitAssignerFactory = new SimpleSplitAssignerFactory();
           } else {
             splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator);
           }
         }
   ```
   
   So the `splitAssignerFactory` never will be null



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1222885497


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator
+    implements SerializableComparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber();
+    Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber();
+
+    Preconditions.checkNotNull(
+        seq1, "Invalid file sequence number for {}. Only V2 table format is supported", o1);
+    Preconditions.checkNotNull(
+        seq2, "Invalid file sequence number for {}. Only V2 table format is supported", o2);
+
+    if (o1.splitId().equals(o2.splitId())) {

Review Comment:
   Removed



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1222884720


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.assigner;
+
+import java.util.Collection;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitState;
+import org.apache.iceberg.flink.source.split.SerializableComparator;
+
+/** Create simple assigner that hands out splits without any guarantee in order or locality. */

Review Comment:
   Fixed, please check



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


[GitHub] [iceberg] pvary commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1222886555


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java:
##########
@@ -23,14 +23,15 @@
 
 /** Create simple assigner that hands out splits without any guarantee in order or locality. */
 public class SimpleSplitAssignerFactory implements SplitAssignerFactory {
+  public SimpleSplitAssignerFactory() {}
 
   @Override
-  public SimpleSplitAssigner createAssigner() {
-    return new SimpleSplitAssigner();
+  public DefaultSplitAssigner createAssigner() {

Review Comment:
   Not really an issue as the `DefaultSplitAssigner` extends `SplitAssigner`, but changed the return type anyway



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1222438249


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator
+    implements SerializableComparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber();
+    Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber();
+
+    Preconditions.checkNotNull(
+        seq1, "Invalid file sequence number for {}. Only V2 table format is supported", o1);

Review Comment:
   convention is to call out what exactly is invalid (like null). here is what I have. 
   ```
   Invalid file sequence number: null. Expected splits from V2 table format with valid file sequence number
   ```



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7661: Flink: Add possibilit of ordering the splits based on the file sequence number

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7661:
URL: https://github.com/apache/iceberg/pull/7661#discussion_r1213412232


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.Comparator;
+import java.util.Optional;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator implements Comparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    if (o1.splitId().equals(o2.splitId())) {
+      return 0;
+    }
+
+    Optional<Long> opt1 =
+        o1.task().files().stream().map(f -> f.file().fileSequenceNumber()).min(Long::compareTo);
+    Optional<Long> opt2 =
+        o2.task().files().stream().map(f -> f.file().fileSequenceNumber()).min(Long::compareTo);
+
+    Preconditions.checkArgument(opt1.isPresent(), "Sequence number should not be null for {}", o1);

Review Comment:
   nit: in the error msg we can call out that this only works for v2 table.



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SortedSplitAssigner.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.assigner;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitState;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link SplitAssigner} which assigns the splits ordered by the {@link
+ * ContentFile#fileSequenceNumber()}.
+ */
+class SortedSplitAssigner implements SplitAssigner {
+  private static final Logger LOG = LoggerFactory.getLogger(SortedSplitAssigner.class);
+
+  // CHECKSTYLE.OFF: IllegalTypeCheck - We use TreeSet.pollFirst
+  private final TreeSet<IcebergSourceSplit> pendingSplits;
+  // CHECKSTYLE.ON: IllegalTypeCheck
+  private CompletableFuture<Void> availableFuture;
+
+  SortedSplitAssigner(Comparator<IcebergSourceSplit> comparator) {
+    this(comparator, ImmutableList.of());
+  }
+
+  SortedSplitAssigner(
+      Comparator<IcebergSourceSplit> comparator,
+      Collection<IcebergSourceSplitState> assignerState) {
+    this.pendingSplits = Sets.newTreeSet(comparator);
+    // Because simple assigner only tracks unassigned splits,
+    // there is no need to filter splits based on status (unassigned) here.
+    assignerState.forEach(splitState -> pendingSplits.add(splitState.split()));
+  }
+
+  @Override
+  public synchronized GetSplitResult getNext(@Nullable String hostname) {
+    if (pendingSplits.isEmpty()) {
+      return GetSplitResult.unavailable();
+    } else {
+      IcebergSourceSplit split = pendingSplits.pollFirst();
+      LOG.info("Split served {}", split);
+      return GetSplitResult.forSplit(split);
+    }
+  }
+
+  @Override
+  public void onDiscoveredSplits(Collection<IcebergSourceSplit> splits) {
+    LOG.info("Splits discovered {}", splits);
+    addSplits(splits);
+  }
+
+  @Override
+  public void onUnassignedSplits(Collection<IcebergSourceSplit> splits) {
+    LOG.info("Splits unassigned {}", splits);
+    addSplits(splits);
+  }
+
+  private synchronized void addSplits(Collection<IcebergSourceSplit> splits) {
+    if (!splits.isEmpty()) {
+      pendingSplits.addAll(splits);
+      // only complete pending future if new splits are discovered
+      completeAvailableFuturesIfNeeded();
+    }
+  }
+
+  /** Simple assigner only tracks unassigned splits */
+  @Override
+  public synchronized Collection<IcebergSourceSplitState> state() {

Review Comment:
   if we go with the new approach on watermark alignment, `IcebergSourceSplitState` is probably not needed. this can be `Collection<IcebergSourceSplit> pendingSplits()`. It can be a separate PR to not distract the purpose of this PR.



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SortedSplitAssigner.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.assigner;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitState;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link SplitAssigner} which assigns the splits ordered by the {@link
+ * ContentFile#fileSequenceNumber()}.
+ */
+class SortedSplitAssigner implements SplitAssigner {

Review Comment:
   Maybe we can probably avoid some duplication by rename/repurpose the `SimpleSplitAssigner` to `DefaultSplitAssigner`. `DefaultSplitAssigner` constructor can take a `Queue` arg? the queue could be a `Deque` or `PriorityQueue`.
   
   The assigner factory can calls the right constructor depends on if a comparator is provided or not.



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/FileSequenceNumberBasedComparator.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.Comparator;
+import java.util.Optional;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FileSequenceNumberBasedComparator implements Comparator<IcebergSourceSplit> {
+  @Override
+  public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) {
+    Preconditions.checkArgument(
+        o1.task().files().size() == 1 && o2.task().files().size() == 1,
+        "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split");
+
+    if (o1.splitId().equals(o2.splitId())) {
+      return 0;
+    }
+
+    Optional<Long> opt1 =

Review Comment:
   not sure if `stream/map` is needed since we already have Preconditions check on the files size.



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