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/05/19 00:13:21 UTC

[GitHub] [iceberg] sundargates commented on a change in pull request #2105: Flink: Initial implementation of Flink source with the new FLIP-27 source interface

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