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

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



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

Reply via email to