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/06/29 21:21:36 UTC

[GitHub] [iceberg] sundargates commented on a change in pull request #2305: Flink: FLIP-27 source split and reader

sundargates commented on a change in pull request #2305:
URL: https://github.com/apache/iceberg/pull/2305#discussion_r659247283



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+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;
+
+public class IcebergSourceSplit extends FileSourceSplit {

Review comment:
       what's the rationale for extending from FileSourceSplit given that it is used for representing a file itself and based on the passed args from the constructor in line 56, it doesn't seem to map well to the iceberg file split?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+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;
+
+public class IcebergSourceSplit extends FileSourceSplit {
+
+  public enum Status {

Review comment:
       nit: Given that this Status is not a field in the Split itself, wondering if we should move this to a separate file?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.util.Collection;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+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;
+
+public class IcebergSourceSplit extends FileSourceSplit {
+
+  public enum Status {
+    UNASSIGNED,
+    ASSIGNED,
+    COMPLETED
+  }
+
+  private final CombinedScanTask task;
+  @Nullable
+  private final CheckpointedPosition checkpointedPosition;
+
+  /**
+   * The splits are frequently serialized into checkpoints.
+   * Caching the byte representation makes repeated serialization cheap.
+   */
+  @Nullable private transient byte[] serializedFormCache;
+
+  public IcebergSourceSplit(CombinedScanTask task, CheckpointedPosition checkpointedPosition) {
+    // Supply dummy values so that IcebergSourceSplit extend from FileSourceSplit,
+    // as required by using BulkFormat interface in IcebergSource.
+    // We are hoping to clean this up after FLINK-20174 is resolved.
+    super("", new Path("file:///dummy"), 0L, 0L);
+    this.task = task;
+    this.checkpointedPosition = checkpointedPosition;
+  }
+
+  public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) {
+    return new IcebergSourceSplit(combinedScanTask, null);
+  }
+
+  public static IcebergSourceSplit fromSplitState(MutableIcebergSourceSplit state) {
+    return new IcebergSourceSplit(state.task(), new CheckpointedPosition(
+        state.offset(), state.recordsToSkipAfterOffset()));
+  }
+
+  public CombinedScanTask task() {
+    return task;
+  }
+
+  public CheckpointedPosition checkpointedPosition() {
+    return checkpointedPosition;
+  }
+
+  public byte[] serializedFormCache() {

Review comment:
       nit: I'm guessing this need not be public given that it's an optimization primarily around making serialization cheap?




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