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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]