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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to