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



##########
File path: flink/src/main/java/org/apache/iceberg/flink/TableInfo.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * This util class holds the serializable parts of {@link 
org.apache.iceberg.Table}
+ */
+public class TableInfo implements Serializable {

Review comment:
       Yeah. I was planing to switch to the `SerializableTable`.

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.Map;
+import java.util.concurrent.CompletableFuture;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus;
+
+/**
+ * Enumerator should call the assigner APIs from the coordinator thread.
+ * This is to simplify the thread safety for assigner implementation.
+ */
+public interface SplitAssigner extends AutoCloseable {
+
+  /**
+   * Some assigners may need to start background threads or perform other 
activity such as
+   * registering as listeners to updates from other event sources e.g., 
watermark tracker.
+   */
+  default void start() {
+  }
+
+  /**
+   * Some assigners may need to perform certain actions
+   * when their corresponding enumerators are closed
+   */
+  @Override
+  default void close() {
+  }
+
+  /**
+   * Request a new split from the assigner
+   * as enumerator trying to assign splits to awaiting readers
+   */
+  GetSplitResult getNext(@Nullable String hostname);
+
+  /**
+   * Add new splits discovered by enumerator
+   */
+  void onDiscoveredSplits(Collection<IcebergSourceSplit> splits);
+
+  /**
+   *   Forward addSplitsBack event (for failed reader) to assigner
+   */
+  void onUnassignedSplits(Collection<IcebergSourceSplit> splits, int 
subtaskId);
+
+  /**
+   * Some assigner (like event time alignment) may rack in-progress splits
+   * to advance watermark upon completed splits
+   */
+  default void onCompletedSplits(Collection<String> completedSplitIds, int 
subtaskId) {
+  }
+
+  /**
+   * Get assigner state for checkpointing.
+   * This is a super-set API that works for all currently imagined assigners.
+   */
+  Map<IcebergSourceSplit, IcebergSourceSplitStatus> snapshotState();
+
+  /**
+   * Enumerator can get a notification via CompletableFuture
+   * when the assigner has more splits available later.
+   * Enumerator should schedule assignment in the thenAccept action of the 
future.
+   *
+   * Assigner will return the same future if this method is called again
+   * before the previous future is completed.
+   *
+   * The future can be completed from other thread,
+   * e.g. the coordinator thread from another thread
+   * for event time alignment.
+   *
+   * If enumerator need to trigger action upon the future completion,
+   * it may want to run it in the coordinator thread
+   * using {@link SplitEnumeratorContext#runInCoordinatorThread(Runnable)}.
+   */
+  CompletableFuture<Void> isAvailable();
+
+  /**
+   * @return assigner stats for monitoring purpose
+   */
+  SplitAssignerStats stats();

Review comment:
       As we discussed offline, we will remove the `stats()` API from assigner. 
If an assigner needs to publish metrics, it should get a `MetricGroup` arg from 
constructor.

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

Review comment:
       yeah. I should have removed it from here, as we removed the subtaskId 
from assigner API

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.source.split;
+
+import java.io.Serializable;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+
+public class IcebergSourceSplitStatus implements Serializable {
+
+  public enum Status {
+    UNASSIGNED,
+    ASSIGNED,
+    COMPLETED
+  }
+
+  private final Status status;
+  @Nullable
+  private final Integer assignedSubtaskId;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.

Review comment:
       This become irrelevant as `IcebergSourceSplitStatus` is now a simple 
enum.

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

Review comment:
       thx. fixed two places in this PR

##########
File path: flink/src/main/java/org/apache/iceberg/flink/TableInfo.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * This util class holds the serializable parts of {@link 
org.apache.iceberg.Table}
+ */
+public class TableInfo implements Serializable {

Review comment:
       I removed the `TableInfo` in the last commit. `Table` interface itself 
is not marked as Serializable, while `BaseTable` implements the Serializable 
interface. So I am using `Table` directly now.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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

Reply via email to