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