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 2022/07/21 00:06:19 UTC

[GitHub] [iceberg] stevenzwu opened a new pull request, #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

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

   … added an opt-in config to use FLIP-27 source in Flink SQL


-- 
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 diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r926174163


##########
flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java:
##########
@@ -111,6 +118,32 @@ private DataStream<RowData> createDataStream(StreamExecutionEnvironment execEnv)
         .build();
   }
 
+  private DataStreamSource<RowData> createFLIP27Stream(StreamExecutionEnvironment env) {
+    SplitAssignerType assignerType = readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE);
+    IcebergSource<RowData> source = IcebergSource.forRowData()
+        .tableLoader(loader)
+        .assignerFactory(assignerType.factory())
+        .properties(properties)
+        .project(getProjectedSchema())
+        .limit(limit)
+        .filters(filters)
+        .flinkConfig(readableConfig)
+        .build();
+    DataStreamSource stream = env.fromSource(
+        source, WatermarkStrategy.noWatermarks(), source.name(), TypeInformation.of(RowData.class));
+
+    if (source.getBoundedness() == Boundedness.BOUNDED) {
+      int parallelism = SourceUtil.inferParallelism(readableConfig, limit, () -> source.planSplitsForBatch().size());

Review Comment:
   the expensive lambda function of `planSplitsForBatch` will only be executed if config is enabled



-- 
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 diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r926177543


##########
flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java:
##########
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iceberg.flink;
+package org.apache.iceberg.flink.source;

Review Comment:
   moved to source dir to be consistent with the move of `IcebergTableSource` 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.

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 closed pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu closed pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…
URL: https://github.com/apache/iceberg/pull/5318


-- 
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 diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r930218433


##########
flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java:
##########
@@ -68,26 +72,59 @@
   private final ReaderFunction<T> readerFunction;
   private final SplitAssignerFactory assignerFactory;
 
+  // Can't use SerializableTable as enumerator needs a regular table
+  // that can discover table changes
+  private transient Table table;
+
   IcebergSource(
       TableLoader tableLoader,
       ScanContext scanContext,
       ReaderFunction<T> readerFunction,
-      SplitAssignerFactory assignerFactory) {
+      SplitAssignerFactory assignerFactory,
+      Table table) {
     this.tableLoader = tableLoader;
     this.scanContext = scanContext;
     this.readerFunction = readerFunction;
     this.assignerFactory = assignerFactory;
+    this.table = table;

Review Comment:
   We are avoiding the feature of inferring parallelism. But I think this refactoring is still good. 
   
   - It avoid double loading of the table. a `Table` is loaded in the builder to get fields like `schema`, `io`, `encryption` etc. It will be loaded again in the `IcebergSource#createEnumerator` method, which also runs in the jobmanager/driver.
   - `table/lazyTable()` is used by the `name()` getter.
   



-- 
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 diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r926177222


##########
flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java:
##########
@@ -84,171 +65,7 @@ private TableEnvironment getTableEnv() {
   protected List<Row> run(FlinkSource.Builder formatBuilder, Map<String, String> sqlOptions, String sqlFilter,
                           String... sqlSelectedFields) {
     String select = String.join(",", sqlSelectedFields);
-
-    StringBuilder builder = new StringBuilder();
-    sqlOptions.forEach((key, value) -> builder.append(optionToKv(key, value)).append(","));
-
-    String optionStr = builder.toString();
-
-    if (optionStr.endsWith(",")) {
-      optionStr = optionStr.substring(0, optionStr.length() - 1);
-    }
-
-    if (!optionStr.isEmpty()) {
-      optionStr = String.format("/*+ OPTIONS(%s)*/", optionStr);
-    }
-
-    return sql("select %s from t %s %s", select, optionStr, sqlFilter);
-  }
-
-  @Test
-  public void testResiduals() throws Exception {

Review Comment:
   remaining test methods are refactored into `TestSqlBase` to share the test btw current and new FLIP-27 sources



-- 
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 diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r926176269


##########
flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.List;
+import java.util.Map;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SqlHelpers {

Review Comment:
   util methods extracted from `TestFlinkScanSql`



-- 
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] rdblue commented on pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#issuecomment-1197388846

   Looks good to me! I'll merge when tests are passing.


-- 
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] rdblue commented on pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#issuecomment-1197319971

   @stevenzwu, can you rebase?


-- 
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 pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

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

   @zhangjun0x01 @openinx @yittg like to get your input on the inferring parallelism feature. The current implementation in `FlinkSource` would require two split planning (1) get the splits to derive split count (2) split planning in source. This is obviously not ideal. Hence @rdblue and I are wondering how useful is this feature? Do we need to carry it over to the FLIP-27 source?
   


-- 
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 diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r926175964


##########
flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java:
##########
@@ -56,30 +46,11 @@ public StaticIcebergEnumerator(
     this.assigner = assigner;
     this.table = table;
     this.scanContext = scanContext;
-    // split enumeration is not needed during restore scenario
-    this.shouldEnumerate = enumState == null;
   }
 
   @Override
   public void start() {
     super.start();
-    if (shouldEnumerate) {

Review Comment:
   split discovery for static/batch enumerator is performed in the `IcebergSource` line 183 now. this is to consolidate the batch planning into a single 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.

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 diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r926176932


##########
flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java:
##########
@@ -88,23 +84,8 @@ public static Object[] parameters() {
     this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
   }
 
-  @Before
-  public void before() throws IOException {

Review Comment:
   this is refactored to `HadoopCatalogResource` to be reusable (e.g. by the new `TestSqlBase`)



-- 
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 diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r926173709


##########
flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java:
##########
@@ -17,14 +17,18 @@
  * under the License.
  */
 
-package org.apache.iceberg.flink;
+package org.apache.iceberg.flink.source;

Review Comment:
   moved this class inside the `source` dir so that it can use some package private methods from `IcebergSource`. and it also seems like a good home for this 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.

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] rdblue commented on a diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r930206793


##########
flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java:
##########
@@ -68,26 +72,59 @@
   private final ReaderFunction<T> readerFunction;
   private final SplitAssignerFactory assignerFactory;
 
+  // Can't use SerializableTable as enumerator needs a regular table
+  // that can discover table changes
+  private transient Table table;
+
   IcebergSource(
       TableLoader tableLoader,
       ScanContext scanContext,
       ReaderFunction<T> readerFunction,
-      SplitAssignerFactory assignerFactory) {
+      SplitAssignerFactory assignerFactory,
+      Table table) {
     this.tableLoader = tableLoader;
     this.scanContext = scanContext;
     this.readerFunction = readerFunction;
     this.assignerFactory = assignerFactory;
+    this.table = table;

Review Comment:
   I thought we were going to avoid these changes for now, since we don't know whether they will be needed to infer parallelism?



-- 
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 diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r930218433


##########
flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java:
##########
@@ -68,26 +72,59 @@
   private final ReaderFunction<T> readerFunction;
   private final SplitAssignerFactory assignerFactory;
 
+  // Can't use SerializableTable as enumerator needs a regular table
+  // that can discover table changes
+  private transient Table table;
+
   IcebergSource(
       TableLoader tableLoader,
       ScanContext scanContext,
       ReaderFunction<T> readerFunction,
-      SplitAssignerFactory assignerFactory) {
+      SplitAssignerFactory assignerFactory,
+      Table table) {
     this.tableLoader = tableLoader;
     this.scanContext = scanContext;
     this.readerFunction = readerFunction;
     this.assignerFactory = assignerFactory;
+    this.table = table;

Review Comment:
   We are avoiding the feature of inferring parallelism. But I think this refactoring is still good. 
   
   - It avoid double loading of the table. a `Table` is loaded in the builder to get fields like `schema`, `io`, `encryption` etc.
   - `table/lazyTable()` is used by the `name()` getter.
   



-- 
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 diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r926175348


##########
flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java:
##########
@@ -251,58 +250,6 @@ public DataStream<RowData> build() {
             .transform(readerOperatorName, typeInfo, StreamingReaderOperator.factory(format));
       }
     }
-
-    int inferParallelism(FlinkInputFormat format, ScanContext context) {

Review Comment:
   the following two methods are refactored to a new class `SourceUtil` so that the FLIP-27 `IcebergSource` can reuse them.



-- 
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] rdblue commented on a diff in pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5318:
URL: https://github.com/apache/iceberg/pull/5318#discussion_r927812551


##########
flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java:
##########
@@ -39,7 +43,10 @@
 import org.apache.flink.table.expressions.ResolvedExpression;
 import org.apache.flink.table.types.DataType;
 import org.apache.iceberg.expressions.Expression;
-import org.apache.iceberg.flink.source.FlinkSource;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.FlinkFilters;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.source.assigner.SplitAssignerType;

Review Comment:
   Should we mark this class internal so people don't use 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.

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 pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

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

   @rdblue this is ready to be merged. I reviewed the diff again and looks good after rebase.
   
   Previous two CI runs failed due to congestion of CI machines (probably overwhelmed by a lot of PRs rebased after the big-bang spotlessApply commit). Once it failed in the `TestS3OutputStream`. Another time it failed in the FLIP-27 `TestIcebergSourceFailover`, which because it wasn't able to make enough progress after 2 mins of waiting.


-- 
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] rdblue merged pull request #5318: Flink: bridge the gap btw FlinkSource and IcebergSource (FLIP-27) and…

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #5318:
URL: https://github.com/apache/iceberg/pull/5318


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