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



##########
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);

Review comment:
       Add a comment here saying that this function assumes that the split 
returned for the host will be assigned by the SplitEnumerator. otherwise, this 
could lead to an inconsistent state as the assigner would assume the split was 
assigned but the reader may not have received it. 

##########
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:
       I'm not sure if it makes sense to have it in SplitAssigner as this would 
mean every one of the implementations will need to worry about handling this 
logic. Apart from that, when we need to keep track of other metrics such as how 
long it took for a split to get assigned etc... we might need to rewrite this 
value class and have all assigners handle that logic. It would be better if we 
instead use composition here and have a listener that calculates these metrics 
based on certain events happening in the assigner/enumerator. 

##########
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();

Review comment:
       nit: s/snapshotState/getState. "snapshotState" indicates an action and 
actions are generally side-effects. In this case, however, this is a pure 
function returning the current state of the assigner. 

##########
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) {

Review comment:
       same as above.

##########
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);

Review comment:
       what's the need for subtaskId parameter here considering that the 
assigner doesn't know anything about which subtask ID requested/assigned the 
split in the first place?




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