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/12/03 11:51:22 UTC

[GitHub] [iceberg] openinx commented on a change in pull request #3501: Flink: FLIP-27 Iceberg source split

openinx commented on a change in pull request #3501:
URL: https://github.com/apache/iceberg/pull/3501#discussion_r761869611



##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+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;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private final CombinedScanTask task;
+  /**
+   * Position field is mutable
+   */
+  @Nullable
+  private final Position position;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable
+  private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, Position position) {
+    this.task = task;
+    this.position = position;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return fromCombinedScanTask(combinedScanTask, 0, 0L);
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(
+      CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) {
+    return new IcebergSourceSplit(combinedScanTask, new Position(fileOffset, recordOffset));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  byte[] serializedFormCache() {
+    return serializedFormCache;
+  }
+
+  void serializedFormCache(byte[] cachedBytes) {
+    this.serializedFormCache = cachedBytes;
+  }
+
+  @Override
+  public String splitId() {
+    return MoreObjects.toStringHelper(this)
+        .add("files", toString(task.files()))
+        .toString();
+  }
+
+  public void updatePosition(int newFileOffset, long newRecordOffset) {
+    position.update(newFileOffset, newRecordOffset);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    IcebergSourceSplit split = (IcebergSourceSplit) o;
+    return Objects.equal(splitId(), split.splitId()) &&
+        Objects.equal(position, split.position());
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(splitId());

Review comment:
       I still don't think it valid to make the hash code build on top of the `FileScanTask#toString` because this [comment](https://github.com/apache/iceberg/pull/3501/files#r745462545).

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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 java.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.Position;
+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;
+
+@Internal
+public class IcebergSourceSplit implements SourceSplit, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final CombinedScanTask task;
+  /**

Review comment:
       I think this minor comment need to be addressed.

##########
File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java
##########
@@ -22,33 +22,58 @@
 import java.io.IOException;
 import java.io.UncheckedIOException;
 import java.util.List;
+import org.apache.flink.annotation.Internal;
 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 {
-  private FlinkSplitGenerator() {
+@Internal
+public class FlinkSplitPlanner {
+  private FlinkSplitPlanner() {
   }
 
-  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));
+  static FlinkInputSplit[] planInputSplits(Table table, ScanContext context) {
+    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);
+    }
+  }
+
+  /**
+   * This returns splits for the FLIP-27 source
+   */
+  public static List<IcebergSourceSplit> planIcebergSourceSplits(
+      Table table, ScanContext context) {
+    try (CloseableIterable<CombinedScanTask> tasksIterable = planTasks(table, context)) {
+      List<IcebergSourceSplit> splits = Lists.newArrayList();
+      tasksIterable.forEach(task -> splits.add(IcebergSourceSplit.fromCombinedScanTask(task)));
+      return splits;
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to process task iterable: ", e);
     }
-    return splits;
   }
 
-  private static List<CombinedScanTask> tasks(Table table, ScanContext context) {
+  static CloseableIterable<CombinedScanTask> planTasks(Table table, ScanContext context) {
     TableScan scan = table
         .newScan()
         .caseSensitive(context.caseSensitive())
         .project(context.project());
 
+    if (context.includeColumnStats()) {
+      scan = scan.includeColumnStats();
+    }

Review comment:
       @stevenzwu How is your feeling for this comment ?




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