You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/01/18 02:46:41 UTC

[GitHub] [iceberg] stevenzwu opened a new pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

stevenzwu opened a new pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105


   Scope in the first version
   
   - simple split assigner (no ordering or locality aware)
   - support both batch and streaming read
   
   This is the uber PR for the reference of complete context. Will submit smaller PRs for the code review
   
   1. [refactor Flink tests so that new source implementation can reuse](https://github.com/apache/iceberg/pull/2047)
   2. [Upgrade Flink version to 1.12.1](https://github.com/apache/iceberg/pull/1956)
   3. IcebergSourceSplit (TBD)
   4. SplitAssigner (TBD)
   5. SplitEnumerator (TBD)
   6. IcebergSource where everything put together (TBD)
   
   The new `IcebergSource` will be marked as `@Experimental` as FLIP-27 source is maturing and we are making it production ready.


----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r574898819



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.enumerator;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.iceberg.flink.source.IcebergSourceEvents;
+import org.apache.iceberg.flink.source.assigner.GetSplitResult;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * TODO: publish enumerator monitor metrics like number of pending metrics
+ * after FLINK-21000 is resolved
+ */
+abstract class AbstractIcebergEnumerator implements
+    SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class);
+
+  private final SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext;
+  private final SplitAssigner assigner;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+  private final Map<Integer, String> readersAwaitingSplit;
+  private final AtomicReference<CompletableFuture<Void>> availableFuture;
+
+  AbstractIcebergEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext,
+      SplitAssigner assigner,
+      IcebergEnumeratorConfig enumeratorConfig) {
+    this.enumeratorContext = enumeratorContext;
+    this.assigner = assigner;
+    this.enumeratorConfig = enumeratorConfig;
+    this.readersAwaitingSplit = new LinkedHashMap<>();
+    this.availableFuture = new AtomicReference<>();
+  }
+
+  @Override
+  public void start() {
+    assigner.start();
+  }
+
+  @Override
+  public void close() throws IOException {
+    assigner.close();
+  }
+
+  @Override
+  public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+    // Iceberg source uses a custom event inside handleSourceEvent
+    // so that we can carry over the finishedSplitIds
+    throw new UnsupportedOperationException("Iceberg source uses its own SplitRequestEvent");
+  }
+
+  @Override
+  public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
+    if (sourceEvent instanceof IcebergSourceEvents.SplitRequestEvent) {
+      final IcebergSourceEvents.SplitRequestEvent splitRequestEvent =
+          (IcebergSourceEvents.SplitRequestEvent) sourceEvent;
+      LOG.error("Received request split event from subtask {}", subtaskId);
+      assigner.onCompletedSplits(splitRequestEvent.finishedSplitIds());
+      readersAwaitingSplit.put(subtaskId, splitRequestEvent.requesterHostname());
+      assignSplits();
+    } else {
+      LOG.error("Received unrecognized event from subtask {}: {}", subtaskId, sourceEvent);

Review comment:
       good question. throwing exception will cause job to fail and restart. explicit failure is probably better.




----------------------------------------------------------------
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:
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] tweise commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r575539431



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerStats.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;

Review comment:
       What's the purpose of this class? How will metrics from the enumerator/assigner be reported to Flink?




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632167819



##########
File path: build.gradle
##########
@@ -318,14 +318,24 @@ project(':iceberg-flink') {
     compileOnly "org.apache.flink:flink-table-api-java-bridge_2.12"
     compileOnly "org.apache.flink:flink-table-planner-blink_2.12"
     compileOnly "org.apache.flink:flink-table-planner_2.12"
+    compileOnly "org.apache.flink:flink-parquet_2.12"
     compileOnly "org.apache.hadoop:hadoop-hdfs"
     compileOnly "org.apache.hadoop:hadoop-common"
     compileOnly("org.apache.hadoop:hadoop-minicluster") {
       exclude group: 'org.apache.avro', module: 'avro'
     }
 
+    // flink-dist doesn't include these two modules (FLINK-20098).
+    // Hence we should add them as compile deps.
+    // This doesn't affect iceberg-flink-runtime shadow jar,
+    // since it excludes all flink jars.
+    compile "org.apache.flink:flink-connector-base"
+    compile "org.apache.flink:flink-connector-files"

Review comment:
       originally it was `compileOnly`. I changed it after the review comments/discussions: https://github.com/apache/iceberg/pull/2105#discussion_r571299927




-- 
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632164424



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.io.Serializable;
+import java.util.Map;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+
+public interface SplitAssignerFactory extends Serializable {

Review comment:
       that is correct. it is big enough already. other assigners will be added later separately.




-- 
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r574887339



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.enumerator;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.iceberg.flink.source.IcebergSourceEvents;
+import org.apache.iceberg.flink.source.assigner.GetSplitResult;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * TODO: publish enumerator monitor metrics like number of pending metrics
+ * after FLINK-21000 is resolved
+ */
+abstract class AbstractIcebergEnumerator implements
+    SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class);
+
+  private final SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext;
+  private final SplitAssigner assigner;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+  private final Map<Integer, String> readersAwaitingSplit;
+  private final AtomicReference<CompletableFuture<Void>> availableFuture;
+
+  AbstractIcebergEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext,
+      SplitAssigner assigner,
+      IcebergEnumeratorConfig enumeratorConfig) {
+    this.enumeratorContext = enumeratorContext;
+    this.assigner = assigner;
+    this.enumeratorConfig = enumeratorConfig;
+    this.readersAwaitingSplit = new LinkedHashMap<>();
+    this.availableFuture = new AtomicReference<>();
+  }
+
+  @Override
+  public void start() {
+    assigner.start();
+  }
+
+  @Override
+  public void close() throws IOException {
+    assigner.close();
+  }
+
+  @Override
+  public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+    // Iceberg source uses a custom event inside handleSourceEvent
+    // so that we can carry over the finishedSplitIds
+    throw new UnsupportedOperationException("Iceberg source uses its own SplitRequestEvent");
+  }
+
+  @Override
+  public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
+    if (sourceEvent instanceof IcebergSourceEvents.SplitRequestEvent) {
+      final IcebergSourceEvents.SplitRequestEvent splitRequestEvent =
+          (IcebergSourceEvents.SplitRequestEvent) sourceEvent;
+      LOG.error("Received request split event from subtask {}", subtaskId);

Review comment:
       yeah. thx for catching 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.

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] sundargates commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
sundargates commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r634814943



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,207 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final Table table = loadTable(tableLoader);
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+      // for batch jobs, discover splits eagerly during job initialization.
+      // As FLINK-16866 supports non-blocking job submission since 1.12,
+      // heavy job initialization won't lead to request timeout for job submission.
+      if (enumeratorConfig.splitDiscoveryInterval() == null) {

Review comment:
       Instead of adding it as a separate config, it might make sense to just add a method as part of IcebergEnumeratorConfig that provides information on whether the requested enumerating mode is continuous or static based on this field. This way this logic of config -> reading mode is moved to one place. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.enumerator;
+
+public interface ContinuousSplitPlanner {
+
+  ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition);

Review comment:
       Instead of passing in the position, do you want to pass the EnumeratorState instead? I'm asking this because there might be cases where the planner might decide to enumerate splits only when the previously returned splits have all been completely read. So, passing in the EnumeratorState will help the planner identify where the source is to decide if it wants to do the enumeration further.




-- 
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:
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] sundargates commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
sundargates commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r568381583



##########
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:
       How would the assigner know about the subtaskID that was assigned the split given that that information is never communicated to 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.

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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632185128



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,207 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final Table table = loadTable(tableLoader);
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+      // for batch jobs, discover splits eagerly during job initialization.
+      // As FLINK-16866 supports non-blocking job submission since 1.12,
+      // heavy job initialization won't lead to request timeout for job submission.
+      if (enumeratorConfig.splitDiscoveryInterval() == null) {
+        assigner.onDiscoveredSplits(FlinkSplitGenerator.planIcebergSourceSplits(table, scanContext));

Review comment:
       I haven't run into problems with unit test or when running it in our infrastructure at Apple. But I can definitely see your point. Instead of passing `Table`, I will pass the `TableLoader` to avoid the potential problem you mentioned




-- 
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r571326021



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?

Review comment:
       Here is from Javadoc. I think the scenario you described also falls into this CONTINUOUS_UNBOUNDED. I know it is not totally intuitive. 
   ```
   A CONTINUOUS_UNBOUNDED stream may also eventually stop at some point. But before that happens, Flink always assumes the sources are going to run forever.
   ```




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r571365754



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+    } else {
+      assigner = assignerFactory.createAssigner(enumState.pendingSplits());
+    }
+
+    final Table table = loadTable(tableLoader);
+    if (enumeratorConfig.splitDiscoveryInterval() == null) {
+      final List<IcebergSourceSplit> splits = FlinkSplitGenerator.planIcebergSourceSplits(table, scanContext);
+      assigner.onDiscoveredSplits(splits);

Review comment:
       actually this is done intentionally. if split planning failed, we will fail fast during the job initialization. Alternatively If we do the one-time planning in the start method, it will fail at task start in taskmanager. At least, we probably should add some comments to explain this.




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r564074591



##########
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:
       that is true. but I am not sure we need to document the enumerator behavior in the assigner interface/contract. Assigner is responsible of returning a split if available.




----------------------------------------------------------------
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:
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] sundargates commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
sundargates commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r568381583



##########
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:
       How would the assigner know about the subtaskID that was assigned the split given that that information is never communicated to it?

##########
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:
       What's the rationale for this optimization? 
   
   I'm not sure if it's going to be expensive to serialize this DS, given that the number of splits the JM will deal with is probably going to be in the 1000s. Even if this was a really high number, there are other easier ways to deal with this, such as only serializing incomplete or un-checkpointed 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.

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] tweise commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r575578403



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+    } else {
+      assigner = assignerFactory.createAssigner(enumState.pendingSplits());
+    }
+
+    final Table table = loadTable(tableLoader);
+    if (enumeratorConfig.splitDiscoveryInterval() == null) {
+      // for batch jobs, discover splits eagerly during job initialization.
+      // As FLINK-16866 supports non-blocking job submission since 1.12,
+      // heavy job initialization won't lead to request timeout for job submission.
+      assigner.onDiscoveredSplits(FlinkSplitGenerator.planIcebergSourceSplits(table, scanContext));

Review comment:
       Maybe it would be better to rearrange this for clarity: When the assigner was created with `enumState.pendingSplits()`, then we shouldn't perform eager split discovery here? 




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
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`.




----------------------------------------------------------------
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:
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 #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#issuecomment-792534998


   @openinx yes, that is the plan as outlined in the description. I am actually preparing the next PR of split reader


----------------------------------------------------------------
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:
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] openinx commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r591009616



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {

Review comment:
       The `inputFiles` in DataIterator is a in-memory cache to get the given decrypted InputFile path by decrypted location.  We maintain those <location,  decryptedInputFile> into map because we are trying to get all the decrypted infos once ( Some `EncryptionManager` implementation will use this feature to request them in a batch RPC call).  It don't have relationship to row-level delete in format v2,  we could fetch the <location, decryptedInputFile> one by one but that may produce many RPC to a key server.  




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632186271



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.enumerator;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.iceberg.flink.source.IcebergSourceEvents;
+import org.apache.iceberg.flink.source.assigner.GetSplitResult;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * TODO: publish enumerator monitor metrics like number of pending metrics
+ * after FLINK-21000 is resolved
+ */
+abstract class AbstractIcebergEnumerator implements
+    SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class);
+
+  private final SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext;
+  private final SplitAssigner assigner;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+  private final Map<Integer, String> readersAwaitingSplit;
+  private final AtomicReference<CompletableFuture<Void>> availableFuture;
+
+  AbstractIcebergEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext,
+      SplitAssigner assigner,
+      IcebergEnumeratorConfig enumeratorConfig) {
+    this.enumeratorContext = enumeratorContext;
+    this.assigner = assigner;
+    this.enumeratorConfig = enumeratorConfig;
+    this.readersAwaitingSplit = new LinkedHashMap<>();
+    this.availableFuture = new AtomicReference<>();
+  }
+
+  @Override
+  public void start() {
+    assigner.start();
+  }
+
+  @Override
+  public void close() throws IOException {
+    assigner.close();
+  }
+
+  /**
+   * Iceberg source uses a custom event inside handleSourceEvent
+   * so that we can piggyback the finishedSplitIds in SplitRequestEvent.
+   * We can move the logic back into this method once FLINK-21364 is resolved.
+   */
+  @Override
+  public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+    throw new UnsupportedOperationException("Iceberg source uses its own SplitRequestEvent");
+  }
+
+  @Override
+  public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
+    if (sourceEvent instanceof IcebergSourceEvents.SplitRequestEvent) {
+      final IcebergSourceEvents.SplitRequestEvent splitRequestEvent =
+          (IcebergSourceEvents.SplitRequestEvent) sourceEvent;
+      LOG.info("Received request split event from subtask {}", subtaskId);
+      assigner.onCompletedSplits(splitRequestEvent.finishedSplitIds());
+      readersAwaitingSplit.put(subtaskId, splitRequestEvent.requesterHostname());
+      assignSplits();
+    } else {
+      throw new IllegalArgumentException(String.format(
+          "Received unknown event %s from subtask %d", sourceEvent, subtaskId));
+    }
+  }
+
+  @Override
+  public void addSplitsBack(List<IcebergSourceSplit> splits, int subtaskId) {
+    LOG.info("Add {} splits back to the pool for failed subtask {}: {}",
+        splits.size(), subtaskId, splits);
+    assigner.onUnassignedSplits(splits);
+    assignSplits();
+  }
+
+  @Override
+  public void addReader(int subtaskId) {
+    LOG.info("Added reader: {}", subtaskId);
+  }
+
+  private void assignSplits() {
+    LOG.info("Assigning splits for {} awaiting readers", readersAwaitingSplit.size());
+    final Iterator<Map.Entry<Integer, String>> awaitingReader =
+        readersAwaitingSplit.entrySet().iterator();
+    while (awaitingReader.hasNext()) {
+      final Map.Entry<Integer, String> nextAwaiting = awaitingReader.next();
+      // if the reader that requested another split has failed in the meantime, remove
+      // it from the list of waiting readers
+      if (!enumeratorContext.registeredReaders().containsKey(nextAwaiting.getKey())) {
+        awaitingReader.remove();
+        continue;
+      }
+
+      final int awaitingSubtask = nextAwaiting.getKey();
+      final String hostname = nextAwaiting.getValue();
+      final GetSplitResult getResult = assigner.getNext(hostname);
+      if (getResult.status() == GetSplitResult.Status.AVAILABLE) {
+        LOG.info("Assign split to subtask {}: {}", awaitingSubtask, getResult.split());
+        enumeratorContext.assignSplit(getResult.split(), awaitingSubtask);
+        awaitingReader.remove();
+      } else if (getResult.status() == GetSplitResult.Status.CONSTRAINED) {
+        getAvailableFutureIfNeeded();
+        break;
+      } else if (getResult.status() == GetSplitResult.Status.UNAVAILABLE) {
+        if (shouldWaitForMoreSplits()) {
+          getAvailableFutureIfNeeded();
+          break;
+        } else {
+          LOG.info("No more splits available for subtask {}", awaitingSubtask);
+          enumeratorContext.signalNoMoreSplits(awaitingSubtask);
+          awaitingReader.remove();
+        }
+      } else {
+        throw new IllegalArgumentException("Unsupported status: " + getResult.status());
+      }
+    }
+  }
+
+  /**
+   * @return true if enumerator should wait for splits
+   * like in the continuous enumerator case
+   */
+  protected abstract boolean shouldWaitForMoreSplits();
+
+  private synchronized void getAvailableFutureIfNeeded() {
+    if (availableFuture.get() != null) {
+      return;
+    }
+    CompletableFuture<Void> future = assigner.isAvailable();
+    future.thenAccept(ignore ->
+        // Must run assignSplits in coordinator thread
+        // because the future may be completed from other threads.
+        // E.g., in event time alignment assigner,
+        // watermark advancement from another source may
+        // cause the available future to be completed
+        enumeratorContext.runInCoordinatorThread(() -> {
+          LOG.debug("Executing callback of assignSplits");
+          availableFuture.set(null);
+          assignSplits();
+        }));
+    availableFuture.set(future);
+    LOG.debug("Registered callback for future available splits");
+  }

Review comment:
       I will email you a few keynote presentation slides. let's see if the animation helps understanding the interactions.




-- 
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r575600801



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerStats.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;

Review comment:
       Good catch. Originally, I was planning to have enumerator poll the assigner for the stats. This is for that purpose. @sundargates and I discussed this and think it is probably better to have the assigner directly publish metrics so that we don't have to force a single value class like this for all assigners. 
   
   I cleaned up the assigner/enumerator code for avoiding using this. but I forgot to remove this class. will delete 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.

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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632174299



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -114,4 +145,43 @@ public void close() throws IOException {
     currentIterator.close();
     tasks = null;
   }
+
+  public Position position() {
+    return position;
+  }
+
+  public static class Position {

Review comment:
       `CheckpointedPosition` is an immutable class




-- 
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r564082349



##########
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 was using this poll model of getter on stats, because I was trying to leverage the scheduler service from enumerator to poll the stats.
   ```
     @Override
     public void start() {
       assigner.start();
       enumeratorContext.callAsync(
           () -> null,
           this::updateAssignerStats,
           enumeratorConfig.metricRefreshInterval().toMillis(),
           enumeratorConfig.metricRefreshInterval().toMillis());
     }
   ```
   
   We can certainly go with listener model like
   ```
     void subscribeStats(SplitAssignerStatsListener listener);
   ```
   
   But that won't make difference on those two questions raised
   
   - each assigner implementation need to handle the stats logic
   - add new stat will require change in the listener interface too, right?




----------------------------------------------------------------
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:
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] sundargates commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r568804628



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




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632168659



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -62,11 +64,13 @@
     // decrypt with the batch call to avoid multiple RPCs to a key server, if possible
     Iterable<InputFile> decryptedFiles = encryption.decrypt(encrypted::iterator);
 
-    Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(task.files().size());
+    Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(combinedTask.files().size());

Review comment:
       thx, will fix 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.

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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r574885763



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSourceEvents.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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;
+
+import java.util.Collection;
+import java.util.Collections;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+
+/**
+ * These source events aren't included in Flink 1.11.2.
+ * Copying the definitions here.
+ *
+ * Once the latest flink-connector-base changes land in 1.11.3 or 1.12.0,

Review comment:
       thx for catching stale comment. will remove




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r575621315



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+    } else {
+      assigner = assignerFactory.createAssigner(enumState.pendingSplits());
+    }
+
+    final Table table = loadTable(tableLoader);
+    if (enumeratorConfig.splitDiscoveryInterval() == null) {
+      // for batch jobs, discover splits eagerly during job initialization.
+      // As FLINK-16866 supports non-blocking job submission since 1.12,
+      // heavy job initialization won't lead to request timeout for job submission.
+      assigner.onDiscoveredSplits(FlinkSplitGenerator.planIcebergSourceSplits(table, scanContext));

Review comment:
       great catch. this is actually a bug. let me fix it and add a unit 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.

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] openinx commented on pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#issuecomment-792500915


   @stevenzwu   would you mind to make this big PR into several small PRs for reviewing purpose ?


----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632165135



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSourceOptions.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+
+public interface IcebergSourceOptions {

Review comment:
       agree. a single `FlinkOptions` makes sense.




-- 
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r571325539



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+    } else {
+      assigner = assignerFactory.createAssigner(enumState.pendingSplits());
+    }
+
+    final Table table = loadTable(tableLoader);
+    if (enumeratorConfig.splitDiscoveryInterval() == null) {
+      final List<IcebergSourceSplit> splits = FlinkSplitGenerator.planIcebergSourceSplits(table, scanContext);
+      assigner.onDiscoveredSplits(splits);

Review comment:
       agree. will change




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632172339



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {

Review comment:
       I made the change in the child PR #2305 . In particular, this is the commit: https://github.com/apache/iceberg/pull/2305/commits/cec66f9148ffac666117125ca2af2139ccde839c
   
   I haven't merged it to this Uber PR yet, since I am hoping to get it reviewed first in the child PR #2305.




-- 
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:
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] kbendick commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r736786937



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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 javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+
+@Internal
+public class GetSplitResult {
+
+  public enum Status {
+
+    AVAILABLE,
+
+    /**
+     * There are pending splits. But they can't be assigned
+     * due to constraints (like event time alignment)
+     */
+    CONSTRAINED,
+
+    /**
+     * Assigner doesn't have pending splits.
+     */
+    UNAVAILABLE
+  }
+
+  private final Status status;
+  @Nullable
+  private final IcebergSourceSplit split;
+
+  public GetSplitResult(Status status) {
+    this(status, null);
+  }
+
+  public GetSplitResult(Status status, @Nullable IcebergSourceSplit split) {
+    if (null == split && status == Status.AVAILABLE) {
+      throw new IllegalArgumentException("Available status must have a non-null split");

Review comment:
       Nit: this error message is a bit confusing for me.
   
   Maybe `Splits cannot be null if their result is marked as Available`? I think my confusion stems from the phrasing `Available status must...`. Doesn't leave much for the user about what `Available status is`. Though if they encounter this error, then maybe it's fair to say that they should likely be digging through enough to find out.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java
##########
@@ -40,4 +40,10 @@ private FlinkConfigOptions() {
           .intType()
           .defaultValue(100)
           .withDescription("Sets max infer parallelism for source operator.");
+
+  public static final ConfigOption<Integer> SOURCE_READER_FETCH_BATCH_SIZE = ConfigOptions
+      .key("source.iceberg.reader.fetch-batch-size")
+      .intType()
+      .defaultValue(2048)
+      .withDescription("The target batch size for split reader fetch.");

Review comment:
       Is this target batch size for number of records per split, bytes, or what? I figure I'll find out further on in my reading but might be good to add to the description regardless 🙂 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -37,20 +38,46 @@
  */
 @Internal
 public class DataIterator<T> implements CloseableIterator<T> {
-
   private final FileScanTaskReader<T> fileScanTaskReader;
-
   private final InputFilesDecryptor inputFilesDecryptor;
+  private final CombinedScanTask combinedTask;
+
   private Iterator<FileScanTask> tasks;
   private CloseableIterator<T> currentIterator;
+  private Position position;
 
   public DataIterator(FileScanTaskReader<T> fileScanTaskReader, CombinedScanTask task,
                       FileIO io, EncryptionManager encryption) {
     this.fileScanTaskReader = fileScanTaskReader;
-
     this.inputFilesDecryptor = new InputFilesDecryptor(task, io, encryption);
+    this.combinedTask = task;
+
     this.tasks = task.files().iterator();
     this.currentIterator = CloseableIterator.empty();
+    // fileOffset starts at -1 because we started
+    // from an empty iterator that is not from the split files.
+    this.position = new Position(-1L, 0L);

Review comment:
       Nit: Maybe we can make this a named constant?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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 javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+
+@Internal

Review comment:
       +1 to the annotations.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
##########
@@ -68,6 +68,9 @@
   private static final ConfigOption<Duration> MONITOR_INTERVAL =
       ConfigOptions.key("monitor-interval").durationType().defaultValue(Duration.ofSeconds(10));
 
+  private static final ConfigOption<Boolean> INCLUDE_COLUMN_STATS =
+      ConfigOptions.key("include-column-stats").booleanType().defaultValue(false);

Review comment:
       What's the reason for defaulting this to `false`? Will it throw if we use that and there aren't column stats?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java
##########
@@ -40,4 +40,10 @@ private FlinkConfigOptions() {
           .intType()
           .defaultValue(100)
           .withDescription("Sets max infer parallelism for source operator.");
+
+  public static final ConfigOption<Integer> SOURCE_READER_FETCH_BATCH_SIZE = ConfigOptions
+      .key("source.iceberg.reader.fetch-batch-size")

Review comment:
       Eventually, we should consider making these constants. But unrelated to this PR.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java
##########
@@ -48,7 +48,6 @@
 
 @Internal
 public class RowDataFileScanTaskReader implements FileScanTaskReader<RowData> {
-

Review comment:
       Nit: Unnecessary diff

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?

Review comment:
       Yeah agree with Steven that it's not always intuitive but it does fall in line with their definition.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,207 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics;
+import org.apache.iceberg.flink.source.reader.ReaderFunction;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class);
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final ReaderFunction<T> readerFunction;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      ReaderFunction<T> readerFunction,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.readerFunction = readerFunction;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {

Review comment:
       Is there a reason not to put the call to `.open()` in the try-with-resources block? Like so:
   ```java
   try(TableLoader loader = tableLoader.open()) { ... }
   ```

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssigner.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.ArrayDeque;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus;
+
+/**
+ * This assigner hands out splits without any guarantee in order or locality.
+ * <p>
+ * Since all methods are called in the source coordinator thread by enumerator,
+ * there is no need for locking.
+ */
+public class SimpleSplitAssigner implements SplitAssigner {
+
+  private final Deque<IcebergSourceSplit> pendingSplits;
+  private CompletableFuture<Void> availableFuture;
+
+  SimpleSplitAssigner(Deque<IcebergSourceSplit> pendingSplits) {
+    this.pendingSplits = pendingSplits;
+  }
+
+  public SimpleSplitAssigner() {
+    this(new ArrayDeque<>());
+  }
+
+  public SimpleSplitAssigner(Map<IcebergSourceSplit, IcebergSourceSplitStatus> state) {
+    this(new ArrayDeque<>(state.keySet()));
+  }
+
+  @Override
+  public GetSplitResult getNext(@Nullable String hostname) {
+    if (pendingSplits.isEmpty()) {
+      return new GetSplitResult(GetSplitResult.Status.UNAVAILABLE);
+    } else {
+      IcebergSourceSplit split = pendingSplits.poll();
+      return new GetSplitResult(GetSplitResult.Status.AVAILABLE, split);
+    }
+  }
+
+  @Override
+  public void onDiscoveredSplits(Collection<IcebergSourceSplit> splits) {
+    pendingSplits.addAll(splits);
+    completeAvailableFuturesIfNeeded();
+  }
+
+  @Override
+  public void onUnassignedSplits(Collection<IcebergSourceSplit> splits) {
+    pendingSplits.addAll(splits);
+    completeAvailableFuturesIfNeeded();
+  }
+
+  /**
+   * Simple assigner only tracks unassigned splits

Review comment:
       Question: How does Simple assigner handle CONSTRAINED splits? In this model (for Simple / non-location aware) tracking / assigning, is `CONSTRAINED` synonymous with `UNASSIGNED`?




-- 
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r566931197



##########
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:
       side effect is a good point. Maybe we should also rename the method to `assignSplit`. sure, we can document the expectation as you described




----------------------------------------------------------------
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:
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] tweise commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r571326598



##########
File path: build.gradle
##########
@@ -317,19 +317,23 @@ project(':iceberg-flink') {
     compile project(':iceberg-parquet')
     compile project(':iceberg-hive-metastore')
 
+    compileOnly "org.apache.flink:flink-connector-base"

Review comment:
       This particular jar file won't be provided by the Flink dist. It should be a transitive dependency of the connector.




----------------------------------------------------------------
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:
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] sundargates commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
sundargates commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r566653337



##########
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:
       Yes but it also changes its internal state right? In other words, this method has a side-effect even though it's just a getter. Normally, getters are meant to be side-effect free. Thus, it might make sense to document that the assigner would assume the returned split will be assigned to some subtask; otherwise, it might lead to an inconsistent state between the enumerator & the assigner. 




----------------------------------------------------------------
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:
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] openinx commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r568303231



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {

Review comment:
       Currently,  we put those two-level iterators inside a single DataIterator, that makes the code a bit complex to read and understand. I'd prefer to make this into two different iterators: 
   1.    FileRecordIterator,  that will seek the provided row offset and then continue to read the following records.
   2.   CombinedTaskRecordIterator,  that will have multiple `FileRecordIterator`s,  it will locate the latest opening `FileRecordIterator` and seek to the given row offset to read the following records. 

##########
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 remember that there's an issue which have already made this `Table` to be serializable: https://github.com/apache/iceberg/pull/2046.  So we may don't need this serializable `TableInfo` any more. 

##########
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:
       Nit:  Lists.newArrayList ? 




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r568829175



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




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632178642



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {
+    // skip files
+    Preconditions.checkArgument(checkpointedPosition.getOffset() < combinedTask.files().size(),
+        String.format("Checkpointed file offset is %d, while CombinedScanTask has %d files",
+            checkpointedPosition.getOffset(), combinedTask.files().size()));
+    for (long i = 0L; i < checkpointedPosition.getOffset(); ++i) {
+      tasks.next();
+    }
+    updateCurrentIterator();
+    // skip records within the file
+    for (long i = 0; i < checkpointedPosition.getRecordsAfterOffset(); ++i) {
+      if (hasNext()) {
+        next();
+      } else {
+        throw new IllegalStateException("Not enough records to skip: " +

Review comment:
       yeah. this is already fixed in the child PR




-- 
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r568837059



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




----------------------------------------------------------------
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:
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] sundargates commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
sundargates commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r568383957



##########
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:
       What's the rationale for this optimization? 
   
   I'm not sure if it's going to be expensive to serialize this DS, given that the number of splits the JM will deal with is probably going to be in the 1000s. Even if this was a really high number, there are other easier ways to deal with this, such as only serializing incomplete or un-checkpointed 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.

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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r575600294



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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;
+
+/**
+ * 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
+   * when enumerator trying to assign splits to awaiting readers.
+   *
+   * If enumerator wasn't able to assign the split (e.g., reader disconnected),
+   * enumerator should call {@link SplitAssigner#onUnassignedSplits} to return the split.
+   */
+  GetSplitResult getNext(@Nullable String hostname);

Review comment:
       Originally, subtaskIndex was there. We removed it because we can't think of any use cases needing it. I am definitely open to add it back if there is a concrete use case. Can you elaborate a little? 




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r572620400



##########
File path: build.gradle
##########
@@ -317,19 +317,23 @@ project(':iceberg-flink') {
     compile project(':iceberg-parquet')
     compile project(':iceberg-hive-metastore')
 
+    compileOnly "org.apache.flink:flink-connector-base"

Review comment:
       @tweise thx a lot for the context. now it all makes sense to me. Also didn't notice that iceberg-flink-runtime actually exclude all flink jars. Updated with compile deps.
   
   In the future, if Flink decided to move `flink-connector-base` and `flink-connector-files` into `flink-dist` (as hinted in FLINK-20472), we can revisit the compile dep status. 




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r571325171



##########
File path: build.gradle
##########
@@ -317,19 +317,23 @@ project(':iceberg-flink') {
     compile project(':iceberg-parquet')
     compile project(':iceberg-hive-metastore')
 
+    compileOnly "org.apache.flink:flink-connector-base"

Review comment:
       All the Flink deps are defined as `compileOnly` in Iceberg. Yeah, it assumes Flink jars are provided at runtime.




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r571365754



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+    } else {
+      assigner = assignerFactory.createAssigner(enumState.pendingSplits());
+    }
+
+    final Table table = loadTable(tableLoader);
+    if (enumeratorConfig.splitDiscoveryInterval() == null) {
+      final List<IcebergSourceSplit> splits = FlinkSplitGenerator.planIcebergSourceSplits(table, scanContext);
+      assigner.onDiscoveredSplits(splits);

Review comment:
       actually this is done intentionally. if split planning failed, we will fail fast during the job initialization. Alternatively If we do the one-time planning in the start method, it will fail at task start in taskmanager.




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r741385771



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,207 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics;
+import org.apache.iceberg.flink.source.reader.ReaderFunction;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class);
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final ReaderFunction<T> readerFunction;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      ReaderFunction<T> readerFunction,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.readerFunction = readerFunction;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {

Review comment:
       `open` method returns `void`

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
##########
@@ -68,6 +68,9 @@
   private static final ConfigOption<Duration> MONITOR_INTERVAL =
       ConfigOptions.key("monitor-interval").durationType().defaultValue(Duration.ofSeconds(10));
 
+  private static final ConfigOption<Boolean> INCLUDE_COLUMN_STATS =
+      ConfigOptions.key("include-column-stats").booleanType().defaultValue(false);

Review comment:
       The current/default behavior is not including columns stats during planning




-- 
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] tweise commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r572487668



##########
File path: build.gradle
##########
@@ -317,19 +317,23 @@ project(':iceberg-flink') {
     compile project(':iceberg-parquet')
     compile project(':iceberg-hive-metastore')
 
+    compileOnly "org.apache.flink:flink-connector-base"

Review comment:
       https://issues.apache.org/jira/browse/FLINK-20098
   
   It is not desirable to place such dependencies into flink-dist.
   
   Regarding the transitive dependency: it would be surprising for the user to find that they have to add a `flink-connector-base` dependency to their project for the iceberg connector to work.
   




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r574885996



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSourceEvents.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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;
+
+import java.util.Collection;
+import java.util.Collections;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+
+/**
+ * These source events aren't included in Flink 1.11.2.
+ * Copying the definitions here.
+ *
+ * Once the latest flink-connector-base changes land in 1.11.3 or 1.12.0,
+ * we can remove at least NoMoreSplitsEvent and NoSplitAvailableEvent.
+ */
+public class IcebergSourceEvents {
+
+  /**
+   * A {@link SourceEvent} representing the request for a split, typically sent from the
+   * {@link SourceReader} to the {@link SplitEnumerator}.
+   *
+   * TODO: push change to Flink to carry the finished splitIds.

Review comment:
       Yeah, I forgot to follow up on the Flink side. Created the jira and attached a PR to it.
   https://issues.apache.org/jira/browse/FLINK-21364
   




----------------------------------------------------------------
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:
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] openinx commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r630840840



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {
+    // skip files
+    Preconditions.checkArgument(checkpointedPosition.getOffset() < combinedTask.files().size(),

Review comment:
       Nit:  The `checkArgument` could  __format__ the error message directly,  we don't need the extra `String.format` now.

##########
File path: build.gradle
##########
@@ -318,14 +318,24 @@ project(':iceberg-flink') {
     compileOnly "org.apache.flink:flink-table-api-java-bridge_2.12"
     compileOnly "org.apache.flink:flink-table-planner-blink_2.12"
     compileOnly "org.apache.flink:flink-table-planner_2.12"
+    compileOnly "org.apache.flink:flink-parquet_2.12"
     compileOnly "org.apache.hadoop:hadoop-hdfs"
     compileOnly "org.apache.hadoop:hadoop-common"
     compileOnly("org.apache.hadoop:hadoop-minicluster") {
       exclude group: 'org.apache.avro', module: 'avro'
     }
 
+    // flink-dist doesn't include these two modules (FLINK-20098).
+    // Hence we should add them as compile deps.
+    // This doesn't affect iceberg-flink-runtime shadow jar,
+    // since it excludes all flink jars.
+    compile "org.apache.flink:flink-connector-base"
+    compile "org.apache.flink:flink-connector-files"

Review comment:
       Personally, I think those two packages could be marked as `complieOnly` because people won't need to import those classes from iceberg-flink-xx.jar . ( The difference between `complile` and `compileOnly` is [here](https://liferay.dev/en/b/gradle-compile-vs-compileonly-vs-compileinclude)).

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {
+    // skip files
+    Preconditions.checkArgument(checkpointedPosition.getOffset() < combinedTask.files().size(),
+        String.format("Checkpointed file offset is %d, while CombinedScanTask has %d files",
+            checkpointedPosition.getOffset(), combinedTask.files().size()));
+    for (long i = 0L; i < checkpointedPosition.getOffset(); ++i) {
+      tasks.next();
+    }
+    updateCurrentIterator();
+    // skip records within the file
+    for (long i = 0; i < checkpointedPosition.getRecordsAfterOffset(); ++i) {
+      if (hasNext()) {
+        next();
+      } else {
+        throw new IllegalStateException("Not enough records to skip: " +

Review comment:
       Nit: It's good to provide the `filePath` in this error message.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -114,4 +145,43 @@ public void close() throws IOException {
     currentIterator.close();
     tasks = null;
   }
+
+  public Position position() {
+    return position;
+  }
+
+  public static class Position {

Review comment:
       Why need to introduce an extra `Position` class rather than reuse the flink's `CheckpointedPosition` (if we plan to keep the two-layer iterator in the same DataIterator class) ?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSourceOptions.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+
+public interface IcebergSourceOptions {

Review comment:
       I'm considering moving all those flink options inside a single `FlinkOptions` class because I don't want to define options everywhere ( now we have a [FlinkTableOptions](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/FlinkTableOptions.java#L26))

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -62,11 +64,13 @@
     // decrypt with the batch call to avoid multiple RPCs to a key server, if possible
     Iterable<InputFile> decryptedFiles = encryption.decrypt(encrypted::iterator);
 
-    Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(task.files().size());
+    Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(combinedTask.files().size());

Review comment:
       The expected size of hash map should be `keyMetadata.size()` because the `combinedTask` may have two `FileScanTask` that come from the same big data file ( That also means the `combinedTask.size() >= keyMetadata.size()` ).

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public class IcebergSourceSplit extends FileSourceSplit {
+
+  public enum Status {
+    UNASSIGNED,
+    ASSIGNED,
+    COMPLETED
+  }
+
+  private final CombinedScanTask task;
+  @Nullable
+  private final CheckpointedPosition checkpointedPosition;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, CheckpointedPosition checkpointedPosition) {
+    // Supply dummy values so that IcebergSourceSplit extend from FileSourceSplit,
+    // as required by using BulkFormat interface in IcebergSource.
+    // We are hoping to clean this up after FLINK-20174 is resolved.
+    super("", new Path("file:///dummy"), 0L, 0L);

Review comment:
       I am a little worried about how to ensure compatibility with iceberg fileio, if we plan to rely on the flink fs interfaces (such as the `Path`) in some places.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.enumerator;
+
+import java.io.Serializable;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+
+/**
+ * Enumerator state for checkpointing
+ */
+public class IcebergEnumeratorState implements Serializable {
+
+  @Nullable private final IcebergEnumeratorPosition enumeratorPosition;
+  private final Map<IcebergSourceSplit, IcebergSourceSplit.Status> pendingSplits;

Review comment:
       Nit: This map looks quite strange because if we get the `IcebergSourceSplit` then we could easily get its `Status`, maintaining those splits in a Set is enough I think.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.io.Serializable;
+import java.util.Map;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+
+public interface SplitAssignerFactory extends Serializable {

Review comment:
       Looks like we only provide the `simple` assigner in this PR.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {

Review comment:
       I think it's worth to follow the original [comment](https://github.com/apache/iceberg/pull/2105/files#r568303231)  because it clearly decouples the file and offset iterators code.

##########
File path: build.gradle
##########
@@ -318,14 +318,24 @@ project(':iceberg-flink') {
     compileOnly "org.apache.flink:flink-table-api-java-bridge_2.12"
     compileOnly "org.apache.flink:flink-table-planner-blink_2.12"
     compileOnly "org.apache.flink:flink-table-planner_2.12"
+    compileOnly "org.apache.flink:flink-parquet_2.12"
     compileOnly "org.apache.hadoop:hadoop-hdfs"
     compileOnly "org.apache.hadoop:hadoop-common"
     compileOnly("org.apache.hadoop:hadoop-minicluster") {
       exclude group: 'org.apache.avro', module: 'avro'
     }
 
+    // flink-dist doesn't include these two modules (FLINK-20098).
+    // Hence we should add them as compile deps.
+    // This doesn't affect iceberg-flink-runtime shadow jar,
+    // since it excludes all flink jars.
+    compile "org.apache.flink:flink-connector-base"
+    compile "org.apache.flink:flink-connector-files"

Review comment:
       We also don't need to provide a comment for the two jars because they should have the same meanings as the above flink jars.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public class IcebergSourceSplit extends FileSourceSplit {
+
+  public enum Status {
+    UNASSIGNED,
+    ASSIGNED,
+    COMPLETED
+  }
+
+  private final CombinedScanTask task;
+  @Nullable
+  private final CheckpointedPosition checkpointedPosition;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, CheckpointedPosition checkpointedPosition) {
+    // Supply dummy values so that IcebergSourceSplit extend from FileSourceSplit,
+    // as required by using BulkFormat interface in IcebergSource.
+    // We are hoping to clean this up after FLINK-20174 is resolved.
+    super("", new Path("file:///dummy"), 0L, 0L);

Review comment:
       Maybe we could just implement our iceberg own `SourceSplit`.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,207 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final Table table = loadTable(tableLoader);
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+      // for batch jobs, discover splits eagerly during job initialization.
+      // As FLINK-16866 supports non-blocking job submission since 1.12,
+      // heavy job initialization won't lead to request timeout for job submission.
+      if (enumeratorConfig.splitDiscoveryInterval() == null) {

Review comment:
       Looks like the `splitDiscoveryInterval` is the option to distingush whether it's a batch job or streaming job ? Literally it's a bit hard to understand. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,207 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final Table table = loadTable(tableLoader);
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+      // for batch jobs, discover splits eagerly during job initialization.
+      // As FLINK-16866 supports non-blocking job submission since 1.12,
+      // heavy job initialization won't lead to request timeout for job submission.
+      if (enumeratorConfig.splitDiscoveryInterval() == null) {
+        assigner.onDiscoveredSplits(FlinkSplitGenerator.planIcebergSourceSplits(table, scanContext));

Review comment:
       We cannot access this `table` instance ( especially doing planing tasks by using this table) again once we've closed the `tableLoader` because if we close the `tableLoader` that means we've closed the connection to the catalog (such as aws glue catalog or hive catalog),  re-access the resources from a closed RPC will lead to an exception.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.enumerator;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.iceberg.flink.source.IcebergSourceEvents;
+import org.apache.iceberg.flink.source.assigner.GetSplitResult;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * TODO: publish enumerator monitor metrics like number of pending metrics
+ * after FLINK-21000 is resolved
+ */
+abstract class AbstractIcebergEnumerator implements
+    SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class);
+
+  private final SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext;
+  private final SplitAssigner assigner;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+  private final Map<Integer, String> readersAwaitingSplit;
+  private final AtomicReference<CompletableFuture<Void>> availableFuture;
+
+  AbstractIcebergEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext,
+      SplitAssigner assigner,
+      IcebergEnumeratorConfig enumeratorConfig) {
+    this.enumeratorContext = enumeratorContext;
+    this.assigner = assigner;
+    this.enumeratorConfig = enumeratorConfig;
+    this.readersAwaitingSplit = new LinkedHashMap<>();
+    this.availableFuture = new AtomicReference<>();
+  }
+
+  @Override
+  public void start() {
+    assigner.start();
+  }
+
+  @Override
+  public void close() throws IOException {
+    assigner.close();
+  }
+
+  /**
+   * Iceberg source uses a custom event inside handleSourceEvent
+   * so that we can piggyback the finishedSplitIds in SplitRequestEvent.
+   * We can move the logic back into this method once FLINK-21364 is resolved.
+   */
+  @Override
+  public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+    throw new UnsupportedOperationException("Iceberg source uses its own SplitRequestEvent");
+  }
+
+  @Override
+  public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
+    if (sourceEvent instanceof IcebergSourceEvents.SplitRequestEvent) {
+      final IcebergSourceEvents.SplitRequestEvent splitRequestEvent =
+          (IcebergSourceEvents.SplitRequestEvent) sourceEvent;
+      LOG.info("Received request split event from subtask {}", subtaskId);
+      assigner.onCompletedSplits(splitRequestEvent.finishedSplitIds());
+      readersAwaitingSplit.put(subtaskId, splitRequestEvent.requesterHostname());
+      assignSplits();
+    } else {
+      throw new IllegalArgumentException(String.format(
+          "Received unknown event %s from subtask %d", sourceEvent, subtaskId));
+    }
+  }
+
+  @Override
+  public void addSplitsBack(List<IcebergSourceSplit> splits, int subtaskId) {
+    LOG.info("Add {} splits back to the pool for failed subtask {}: {}",
+        splits.size(), subtaskId, splits);
+    assigner.onUnassignedSplits(splits);
+    assignSplits();
+  }
+
+  @Override
+  public void addReader(int subtaskId) {
+    LOG.info("Added reader: {}", subtaskId);
+  }
+
+  private void assignSplits() {
+    LOG.info("Assigning splits for {} awaiting readers", readersAwaitingSplit.size());
+    final Iterator<Map.Entry<Integer, String>> awaitingReader =
+        readersAwaitingSplit.entrySet().iterator();
+    while (awaitingReader.hasNext()) {
+      final Map.Entry<Integer, String> nextAwaiting = awaitingReader.next();
+      // if the reader that requested another split has failed in the meantime, remove
+      // it from the list of waiting readers
+      if (!enumeratorContext.registeredReaders().containsKey(nextAwaiting.getKey())) {
+        awaitingReader.remove();
+        continue;
+      }
+
+      final int awaitingSubtask = nextAwaiting.getKey();
+      final String hostname = nextAwaiting.getValue();
+      final GetSplitResult getResult = assigner.getNext(hostname);
+      if (getResult.status() == GetSplitResult.Status.AVAILABLE) {
+        LOG.info("Assign split to subtask {}: {}", awaitingSubtask, getResult.split());
+        enumeratorContext.assignSplit(getResult.split(), awaitingSubtask);
+        awaitingReader.remove();
+      } else if (getResult.status() == GetSplitResult.Status.CONSTRAINED) {
+        getAvailableFutureIfNeeded();
+        break;
+      } else if (getResult.status() == GetSplitResult.Status.UNAVAILABLE) {
+        if (shouldWaitForMoreSplits()) {
+          getAvailableFutureIfNeeded();
+          break;
+        } else {
+          LOG.info("No more splits available for subtask {}", awaitingSubtask);
+          enumeratorContext.signalNoMoreSplits(awaitingSubtask);
+          awaitingReader.remove();
+        }
+      } else {
+        throw new IllegalArgumentException("Unsupported status: " + getResult.status());
+      }
+    }
+  }
+
+  /**
+   * @return true if enumerator should wait for splits
+   * like in the continuous enumerator case
+   */
+  protected abstract boolean shouldWaitForMoreSplits();
+
+  private synchronized void getAvailableFutureIfNeeded() {
+    if (availableFuture.get() != null) {
+      return;
+    }
+    CompletableFuture<Void> future = assigner.isAvailable();
+    future.thenAccept(ignore ->
+        // Must run assignSplits in coordinator thread
+        // because the future may be completed from other threads.
+        // E.g., in event time alignment assigner,
+        // watermark advancement from another source may
+        // cause the available future to be completed
+        enumeratorContext.runInCoordinatorThread(() -> {
+          LOG.debug("Executing callback of assignSplits");
+          availableFuture.set(null);
+          assignSplits();
+        }));
+    availableFuture.set(future);
+    LOG.debug("Registered callback for future available splits");
+  }

Review comment:
       This function is really hard to follow,  I guess it's addressing the issue from [google doc](https://docs.google.com/document/d/1q6xaBxUPFwYsW9aXWxYUh7die6O7rDeAPFQcTAMQ0GM/edit#heading=h.a8rlfx363xjc).




-- 
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r571475350



##########
File path: build.gradle
##########
@@ -317,19 +317,23 @@ project(':iceberg-flink') {
     compile project(':iceberg-parquet')
     compile project(':iceberg-hive-metastore')
 
+    compileOnly "org.apache.flink:flink-connector-base"

Review comment:
       Right now, the `iceberg-flink-runtime` shadow jar doesn't bring in any Flink deps. if we include `flink-connector-base` as `compile`, then it will be bundled in the `iceberg-flink-runtime` shadow. if a Flink app pulls in  `flink-connector-base` transitively via other deps (like Flink Kafka connector), then we can get dup classes in jars.
   
   @openinx maybe you can share some lights on how users get the Flink jars when using the Flink Iceberg connector.
   




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632182589



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.enumerator;
+
+import java.io.Serializable;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+
+/**
+ * Enumerator state for checkpointing
+ */
+public class IcebergEnumeratorState implements Serializable {
+
+  @Nullable private final IcebergEnumeratorPosition enumeratorPosition;
+  private final Map<IcebergSourceSplit, IcebergSourceSplit.Status> pendingSplits;

Review comment:
       I see your point. I did it this way so that `IcebergSourceSplit` can be immutable. If we embed `Status` in the `IcebergSourceSplit`, I guess we can do copy on write to maintain the immutability. 




-- 
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r569014186



##########
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:
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] tweise commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r574246563



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSourceEvents.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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;
+
+import java.util.Collection;
+import java.util.Collections;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+
+/**
+ * These source events aren't included in Flink 1.11.2.
+ * Copying the definitions here.
+ *
+ * Once the latest flink-connector-base changes land in 1.11.3 or 1.12.0,

Review comment:
       nit: Remove this comment: https://github.com/apache/flink/tree/release-1.12/flink-runtime/src/main/java/org/apache/flink/runtime/source/event




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r564077217



##########
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:
       I can't think of a use of the subtaskId. I will update both the design doc and APIs.

##########
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:
       make sense. will update




----------------------------------------------------------------
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:
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] sundargates commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
sundargates commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r571299927



##########
File path: build.gradle
##########
@@ -317,19 +317,23 @@ project(':iceberg-flink') {
     compile project(':iceberg-parquet')
     compile project(':iceberg-hive-metastore')
 
+    compileOnly "org.apache.flink:flink-connector-base"

Review comment:
       Why compileOnly? Does this assume that flink-connector-base would be supplied somehow? If so, what's recommended to users of the library given that flink-dist doesn't bundle flink-connector-base. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?

Review comment:
       Shouldn't boundedness be based on whether the data that's being read has finite bounds to it, i.e., if there's an end timestamp at which the source has to stop reading? You can have finite bounds but still have continuous discovery enabled if the end timestamp is sometime in the future. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+    } else {
+      assigner = assignerFactory.createAssigner(enumState.pendingSplits());
+    }
+
+    final Table table = loadTable(tableLoader);
+    if (enumeratorConfig.splitDiscoveryInterval() == null) {
+      final List<IcebergSourceSplit> splits = FlinkSplitGenerator.planIcebergSourceSplits(table, scanContext);
+      assigner.onDiscoveredSplits(splits);

Review comment:
       nit: can we move this to within the StaticIcebergEnumerator so that we can keep the consistency on the interactions between the enumerator and assigner? 




----------------------------------------------------------------
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:
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] tweise commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r575537492



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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;
+
+/**
+ * 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
+   * when enumerator trying to assign splits to awaiting readers.
+   *
+   * If enumerator wasn't able to assign the split (e.g., reader disconnected),
+   * enumerator should call {@link SplitAssigner#onUnassignedSplits} to return the split.
+   */
+  GetSplitResult getNext(@Nullable String hostname);

Review comment:
       Also pass the subtask index so that it is possible for an implementation to assign splits to subtasks in a particular order? Multiple subtasks can share a host.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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;
+
+/**
+ * Enumerator should call the assigner APIs from the coordinator thread.

Review comment:
       Maybe expand the javadoc a little to explain why this is a separate component (from the design doc)?




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632165830



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,207 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final BulkFormat<T, IcebergSourceSplit> bulkFormat;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      BulkFormat<T, IcebergSourceSplit> bulkFormat,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.bulkFormat = bulkFormat;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    return new IcebergSourceReader<>(
+        readerContext,
+        bulkFormat);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final Table table = loadTable(tableLoader);
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+      // for batch jobs, discover splits eagerly during job initialization.
+      // As FLINK-16866 supports non-blocking job submission since 1.12,
+      // heavy job initialization won't lead to request timeout for job submission.
+      if (enumeratorConfig.splitDiscoveryInterval() == null) {

Review comment:
       that is correct. if `splitDiscoveryInterval` is not configured, it means one-time split discovery during job startup (as for batch job). If it is more clear to have a separate config as `enumerationMode=static/continuous`, I can add 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.

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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r568723796



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




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r632179578



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public class IcebergSourceSplit extends FileSourceSplit {
+
+  public enum Status {
+    UNASSIGNED,
+    ASSIGNED,
+    COMPLETED
+  }
+
+  private final CombinedScanTask task;
+  @Nullable
+  private final CheckpointedPosition checkpointedPosition;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, CheckpointedPosition checkpointedPosition) {
+    // Supply dummy values so that IcebergSourceSplit extend from FileSourceSplit,
+    // as required by using BulkFormat interface in IcebergSource.
+    // We are hoping to clean this up after FLINK-20174 is resolved.
+    super("", new Path("file:///dummy"), 0L, 0L);

Review comment:
       that is also the big question/concern I have. please see my latest comment in the child PR: https://github.com/apache/iceberg/pull/2305#issuecomment-840902665




-- 
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:
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] tweise commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r574246758



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSourceEvents.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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;
+
+import java.util.Collection;
+import java.util.Collections;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+
+/**
+ * These source events aren't included in Flink 1.11.2.
+ * Copying the definitions here.
+ *
+ * Once the latest flink-connector-base changes land in 1.11.3 or 1.12.0,
+ * we can remove at least NoMoreSplitsEvent and NoSplitAvailableEvent.
+ */
+public class IcebergSourceEvents {
+
+  /**
+   * A {@link SourceEvent} representing the request for a split, typically sent from the
+   * {@link SourceReader} to the {@link SplitEnumerator}.
+   *
+   * TODO: push change to Flink to carry the finished splitIds.

Review comment:
       Is there a JIRA for this?




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r741385771



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,207 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics;
+import org.apache.iceberg.flink.source.reader.ReaderFunction;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class);
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final ReaderFunction<T> readerFunction;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      ReaderFunction<T> readerFunction,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.readerFunction = readerFunction;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {

Review comment:
       `open` method returns `void`




-- 
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] sundargates commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
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:
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] tweise commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r572488364



##########
File path: build.gradle
##########
@@ -317,19 +317,23 @@ project(':iceberg-flink') {
     compile project(':iceberg-parquet')
     compile project(':iceberg-hive-metastore')
 
+    compileOnly "org.apache.flink:flink-connector-base"

Review comment:
       Regarding the dup classes: The the user still has control over the transitive dependency if there is a version mismatch.




----------------------------------------------------------------
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:
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] tweise commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r572488364



##########
File path: build.gradle
##########
@@ -317,19 +317,23 @@ project(':iceberg-flink') {
     compile project(':iceberg-parquet')
     compile project(':iceberg-hive-metastore')
 
+    compileOnly "org.apache.flink:flink-connector-base"

Review comment:
       Regarding the dup classes: The the user still has control over the transitive dependency if there is a version mismatch (which is why it should be a transitive dependency and not included via shadow).




----------------------------------------------------------------
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:
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] tweise commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r574254659



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.enumerator;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.iceberg.flink.source.IcebergSourceEvents;
+import org.apache.iceberg.flink.source.assigner.GetSplitResult;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * TODO: publish enumerator monitor metrics like number of pending metrics
+ * after FLINK-21000 is resolved
+ */
+abstract class AbstractIcebergEnumerator implements
+    SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class);
+
+  private final SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext;
+  private final SplitAssigner assigner;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+  private final Map<Integer, String> readersAwaitingSplit;
+  private final AtomicReference<CompletableFuture<Void>> availableFuture;
+
+  AbstractIcebergEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext,
+      SplitAssigner assigner,
+      IcebergEnumeratorConfig enumeratorConfig) {
+    this.enumeratorContext = enumeratorContext;
+    this.assigner = assigner;
+    this.enumeratorConfig = enumeratorConfig;
+    this.readersAwaitingSplit = new LinkedHashMap<>();
+    this.availableFuture = new AtomicReference<>();
+  }
+
+  @Override
+  public void start() {
+    assigner.start();
+  }
+
+  @Override
+  public void close() throws IOException {
+    assigner.close();
+  }
+
+  @Override
+  public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+    // Iceberg source uses a custom event inside handleSourceEvent
+    // so that we can carry over the finishedSplitIds
+    throw new UnsupportedOperationException("Iceberg source uses its own SplitRequestEvent");
+  }
+
+  @Override
+  public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
+    if (sourceEvent instanceof IcebergSourceEvents.SplitRequestEvent) {
+      final IcebergSourceEvents.SplitRequestEvent splitRequestEvent =
+          (IcebergSourceEvents.SplitRequestEvent) sourceEvent;
+      LOG.error("Received request split event from subtask {}", subtaskId);
+      assigner.onCompletedSplits(splitRequestEvent.finishedSplitIds());
+      readersAwaitingSplit.put(subtaskId, splitRequestEvent.requesterHostname());
+      assignSplits();
+    } else {
+      LOG.error("Received unrecognized event from subtask {}: {}", subtaskId, sourceEvent);

Review comment:
       Should this throw exception?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.enumerator;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.iceberg.flink.source.IcebergSourceEvents;
+import org.apache.iceberg.flink.source.assigner.GetSplitResult;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * TODO: publish enumerator monitor metrics like number of pending metrics
+ * after FLINK-21000 is resolved
+ */
+abstract class AbstractIcebergEnumerator implements
+    SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class);
+
+  private final SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext;
+  private final SplitAssigner assigner;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+  private final Map<Integer, String> readersAwaitingSplit;
+  private final AtomicReference<CompletableFuture<Void>> availableFuture;
+
+  AbstractIcebergEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumeratorContext,
+      SplitAssigner assigner,
+      IcebergEnumeratorConfig enumeratorConfig) {
+    this.enumeratorContext = enumeratorContext;
+    this.assigner = assigner;
+    this.enumeratorConfig = enumeratorConfig;
+    this.readersAwaitingSplit = new LinkedHashMap<>();
+    this.availableFuture = new AtomicReference<>();
+  }
+
+  @Override
+  public void start() {
+    assigner.start();
+  }
+
+  @Override
+  public void close() throws IOException {
+    assigner.close();
+  }
+
+  @Override
+  public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+    // Iceberg source uses a custom event inside handleSourceEvent
+    // so that we can carry over the finishedSplitIds
+    throw new UnsupportedOperationException("Iceberg source uses its own SplitRequestEvent");
+  }
+
+  @Override
+  public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
+    if (sourceEvent instanceof IcebergSourceEvents.SplitRequestEvent) {
+      final IcebergSourceEvents.SplitRequestEvent splitRequestEvent =
+          (IcebergSourceEvents.SplitRequestEvent) sourceEvent;
+      LOG.error("Received request split event from subtask {}", subtaskId);

Review comment:
       LOG.info?




----------------------------------------------------------------
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:
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] openinx commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r568303231



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {

Review comment:
       Currently,  we put those two-level iterators inside a single DataIterator, that makes the code a bit complex to read and understand. I'd prefer to make this into two different iterators: 
   1.    FileRecordIterator,  that will seek the provided row offset and then continue to read the following records.
   2.   CombinedTaskRecordIterator,  that will have multiple `FileRecordIterator`s,  it will locate the latest opening `FileRecordIterator` and seek to the given row offset to read the following records. 

##########
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 remember that there's an issue which have already made this `Table` to be serializable: https://github.com/apache/iceberg/pull/2046.  So we may don't need this serializable `TableInfo` any more. 

##########
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:
       Nit:  Lists.newArrayList ? 




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r571491417



##########
File path: build.gradle
##########
@@ -317,19 +317,23 @@ project(':iceberg-flink') {
     compile project(':iceberg-parquet')
     compile project(':iceberg-hive-metastore')
 
+    compileOnly "org.apache.flink:flink-connector-base"

Review comment:
       I am also wondering if flink-dist should actually include `flink-connector-base`




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r564074591



##########
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:
       that is true. but I am not sure we need to document the enumerator behavior in the assigner interface/contract. Assigner is responsible of returning a split if available.

##########
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:
       I can't think of a use of the subtaskId. I will update both the design doc and APIs.

##########
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:
       make sense. will update

##########
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 was using this poll model of getter on stats, because I was trying to leverage the scheduler service from enumerator to poll the stats.
   ```
     @Override
     public void start() {
       assigner.start();
       enumeratorContext.callAsync(
           () -> null,
           this::updateAssignerStats,
           enumeratorConfig.metricRefreshInterval().toMillis(),
           enumeratorConfig.metricRefreshInterval().toMillis());
     }
   ```
   
   We can certainly go with listener model like
   ```
     void subscribeStats(SplitAssignerStatsListener listener);
   ```
   
   But that won't make difference on those two questions raised
   
   - each assigner implementation need to handle the stats logic
   - add new stat will require change in the listener interface too, right?




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r569695589



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {

Review comment:
       That makes sense to me.
   
   I have a question about the `Map<String, InputFile> inputFiles`. Right now, it is constructed per `CombinedScanTask`. Would it be ok to do it at each individual `FileScanTask`? I tried the change and delete tests work fine. But I am not sure if I could miss anything since I am not familiar on read merge on deleted rows.




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r589218333



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {

Review comment:
       @openinx can you take a look at my question in the comment above?




----------------------------------------------------------------
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:
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r575599803



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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;
+
+/**
+ * Enumerator should call the assigner APIs from the coordinator thread.

Review comment:
       good suggestion. will add




----------------------------------------------------------------
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:
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] sjwiesman commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
sjwiesman commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r709618800



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssigner;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory;
+import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner;
+import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorConfig;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState;
+import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer;
+import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
+import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics;
+import org.apache.iceberg.flink.source.reader.ReaderFunction;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Experimental
+public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class);
+
+  private final TableLoader tableLoader;
+  private final ScanContext scanContext;
+  private final ReaderFunction<T> readerFunction;
+  private final SplitAssignerFactory assignerFactory;
+  private final IcebergEnumeratorConfig enumeratorConfig;
+
+  IcebergSource(
+      TableLoader tableLoader,
+      ScanContext scanContext,
+      ReaderFunction<T> readerFunction,
+      SplitAssignerFactory assignerFactory,
+      IcebergEnumeratorConfig enumeratorConfig) {
+
+    this.tableLoader = tableLoader;
+    this.enumeratorConfig = enumeratorConfig;
+    this.scanContext = scanContext;
+    this.readerFunction = readerFunction;
+    this.assignerFactory = assignerFactory;
+  }
+
+  private static Table loadTable(TableLoader tableLoader) {
+    tableLoader.open();
+    try (TableLoader loader = tableLoader) {
+      return loader.loadTable();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to close table loader", e);
+    }
+  }
+
+  @Override
+  public Boundedness getBoundedness() {
+    return enumeratorConfig.splitDiscoveryInterval() == null ?
+        Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED;
+  }
+
+  @Override
+  public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext readerContext) {
+    final IcebergSourceReaderMetrics readerMetrics = new IcebergSourceReaderMetrics(readerContext.metricGroup());
+    return new IcebergSourceReader<>(
+        readerFunction,
+        readerContext,
+        readerMetrics);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext) {
+    return createEnumerator(enumContext, null);
+  }
+
+  @Override
+  public SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> restoreEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext, IcebergEnumeratorState enumState) {
+    return createEnumerator(enumContext, enumState);
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergSourceSplit> getSplitSerializer() {
+    return IcebergSourceSplitSerializer.INSTANCE;
+  }
+
+  @Override
+  public SimpleVersionedSerializer<IcebergEnumeratorState> getEnumeratorCheckpointSerializer() {
+    return IcebergEnumeratorStateSerializer.INSTANCE;
+  }
+
+  private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumerator(
+      SplitEnumeratorContext<IcebergSourceSplit> enumContext,
+      @Nullable IcebergEnumeratorState enumState) {
+
+    final Table table = loadTable(tableLoader);
+
+    final SplitAssigner assigner;
+    if (enumState == null) {
+      assigner = assignerFactory.createAssigner();
+      // for batch jobs, discover splits eagerly during job initialization.
+      // As FLINK-16866 supports non-blocking job submission since 1.12,
+      // heavy job initialization won't lead to request timeout for job submission.
+      if (enumeratorConfig.splitDiscoveryInterval() == null) {
+        final List<IcebergSourceSplit> splits = FlinkSplitGenerator.planIcebergSourceSplits(table, scanContext);

Review comment:
       Eagerly discovering all splits in this method is not a safe operation. Throwing an exception here will cause JobManager failover which can have repercussions when running on a session cluster. 
   
   Once this method returns, the SourceCoordinator, which runs the enumerator, executes all operations within a sandbox. If the enumerator throws an exception, the coordinator will fail the Job but keep the JM healthy[1].
   
   If you want to eagerly discover all spits in batch execution, I suggest having a StaticSplitAssigner that does so inside of start. 
   
   https://github.com/apache/flink/blob/583e20dcb97ed56eae76cdc949895086965d54c6/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java#L306-L332  




-- 
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 change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2105:
URL: https://github.com/apache/iceberg/pull/2105#discussion_r741386353



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
##########
@@ -68,6 +68,9 @@
   private static final ConfigOption<Duration> MONITOR_INTERVAL =
       ConfigOptions.key("monitor-interval").durationType().defaultValue(Duration.ofSeconds(10));
 
+  private static final ConfigOption<Boolean> INCLUDE_COLUMN_STATS =
+      ConfigOptions.key("include-column-stats").booleanType().defaultValue(false);

Review comment:
       The current/default behavior is not including columns stats during planning




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