openinx commented on a change in pull request #3501: URL: https://github.com/apache/iceberg/pull/3501#discussion_r750200159
########## 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: Nit: Could you pls leave a separate empty line between the two different block ? ########## 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: Why add this switch in this PR ? ########## File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java ########## @@ -0,0 +1,76 @@ +/* + * 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.IOException; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.InstantiationUtil; + +/** + * TODO: use Java serialization for now. + * Will switch to more stable serializer from + * <a href="https://github.com/apache/iceberg/issues/1698">issue-1698</a>. + */ +@Internal +public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer<IcebergSourceSplit> { + public static final IcebergSourceSplitSerializer INSTANCE = new IcebergSourceSplitSerializer(); + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergSourceSplit split) throws IOException { + if (split.serializedBytesCache() == null) { + final byte[] result = serializeV1(split); Review comment: Nit: remove the unnecessary `final` modifier. ########## 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: I will recommend to make this PR to add the flip-27 source split as focused as possible. So it will recommend to remove the unrelated changes. ########## 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( Review comment: Nit: I think we don't need to switch to a new line. ########## 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: Then why not assign an unique integer number to the `IcebergSourceSplit` as `splitId` when planing the tasks in `FlinkSplitPlanner#planIcebergSourceSplits` ? I think keeping the `toString` approach as the identifier did not answer the question I raised in the first comment. ########## 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; + /** + * 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[] serializedBytesCache; + + private 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[] serializedBytesCache() { + return serializedBytesCache; + } + + void serializedBytesCache(byte[] cachedBytes) { + this.serializedBytesCache = 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); + } Review comment: No usage for this method ? ########## File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/Position.java ########## @@ -0,0 +1,95 @@ +/* + * 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.Serializable; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * A mutable class that defines the read position + * <ul> + * <li>file offset in the list of files in a {@link CombinedScanTask}</li> + * <li>record offset within a file</li> + * </ul> + */ +@Internal +public class Position implements Serializable { + + private static final long serialVersionUID = 1L; + + private int fileOffset; + private long recordOffset; + + public Position(int fileOffset, long recordOffset) { + this.fileOffset = fileOffset; + this.recordOffset = recordOffset; + } + + void advanceFile() { + this.fileOffset += 1; + this.recordOffset = 0L; + } + + void advanceRecord() { + this.recordOffset += 1L; + } + + public void update(int newFileOffset, long newRecordOffset) { + this.fileOffset = newFileOffset; + this.recordOffset = newRecordOffset; + } + + public int fileOffset() { + return fileOffset; + } + + public long recordOffset() { + return recordOffset; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Position that = (Position) o; Review comment: Nit: we usually don't use `final` for a local variable in iceberg. ########## File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java ########## @@ -34,7 +34,7 @@ /** * Context object with optional arguments for a Flink Scan. */ -class ScanContext implements Serializable { +public class ScanContext implements Serializable { Review comment: It's okay to make a class from flink module to be public, but we need to choose them carefully because there are many flink users who are writing Java datastream job. Making a class to be public may imply that we need to consider the class compatibility for downstream users. So in theory I will recommend to not expose a class as `public` by default unless we have to. To avoid this ScanContext to be public, we even introduced many fields setters which are the same as the `ScanContext#Builder` in `FlinkSource`. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
