kbendick commented on a change in pull request #3501: URL: https://github.com/apache/iceberg/pull/3501#discussion_r746194600
########## File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java ########## @@ -0,0 +1,126 @@ +/* + * 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 { Review comment: Should we set `serialVersionUID` in case users refactor their DAGs? ########## File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java ########## @@ -0,0 +1,75 @@ +/* + * 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.serializedFormCache() == null) { + final byte[] result = serializeV1(split); + split.serializedFormCache(result); + } + return split.serializedFormCache(); + } + + @Override + public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + default: + throw new IOException("Unknown version: " + version); Review comment: Nit: It might help to mention the highest known version (assuming that we'll use monotonically increasing versioning). Or at the least, since most users won't be aware of `SimpleVersionedSerializer`, mentioning what the unknown version means (seems like this would happen in theory just with data written by a newer library version or maybe corrupted data / some kind of bug). Something like `Failed to deserialize IcebergSourceSplit. Encountered unknown version $version. The maximum version that can be handled is ${currentVersion}`. ########## File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java ########## @@ -0,0 +1,75 @@ +/* + * 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> { Review comment: Will this get used to serialize / deserialize the splits across task boundaries, or just for checkpoints? The comment in `IcebergSourceSplit` for `serializedFormCache` field mentions checkpoints, but curious about crossing task boundaries etc. ########## 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 don't think that we have a guarantee that `serializedFormCache` will be set unless the split has in fact been serialized via `IcebergSourceSplitSerializer`. ########## File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java ########## @@ -0,0 +1,126 @@ +/* + * 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; Review comment: Nit: This is a great idea. However, the name is a little confusing for me. Maybe `lzySerializedBytes` or something (sort of like @openinx's comment)? Just a nit as I don't want to bike shed on the name. Clever idea overall. ########## 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; + } Review comment: Correct me if I'm wrong, but it seems like it is lazy and only set via the first call to `IcebergSourceSplitSerializer`. Maybe the name is what looks a bit odd to you? I had to read it a few times myself, but I'm not as quick at Flink stuff as you two (or in general). ########## File path: flink/v1.13/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java ########## @@ -0,0 +1,75 @@ +/* + * 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> { Review comment: Question for my own understanding after looking through `SimpleVersionedSerializer` docs: It seems like `SimpleVersionedSerializer` can only handle one version at a time (hence the `getVersion()` function). Is that correct? Are there any best practices when working with `SimpleVersionedSerializer` to consider? When we evolve, will we have two classes or put all known classes into one instance? ########## 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: I believe the usage in `org.apache.iceberg.flink.source.split.SplitHelpers` is why this is now public instead of package-private. Is it abnormal for various `Context` objects in Flink to be public? -- 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]
