rdblue commented on a change in pull request #3501:
URL: https://github.com/apache/iceberg/pull/3501#discussion_r766903328
##########
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 think we should just have the minimal set of changes to this file,
rather than trying to fix the duplication between this and
`FlinkSource.Builder`. Also, we should probably add `@Internal` because this is
becoming public.
##########
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) {
+ byte[] result = serializeV1(split);
+ split.serializedBytesCache(result);
Review comment:
Can we move serialization inside the split? We could then call
`split.serializeV1()` and it could cache internally.
##########
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) {
+ byte[] result = serializeV1(split);
+ split.serializedBytesCache(result);
+ }
+ return split.serializedBytesCache();
+ }
+
+ @Override
+ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws
IOException {
+ switch (version) {
+ case 1:
+ return deserializeV1(serialized);
+ default:
+ throw new IOException(String.format("Failed to deserialize
IcebergSourceSplit. " +
+ "Encountered unsupported version: %d. Supported version are [1]",
version));
+ }
+ }
+
+ @VisibleForTesting
+ byte[] serializeV1(IcebergSourceSplit split) throws IOException {
+ return InstantiationUtil.serializeObject(split);
+ }
+
+ @VisibleForTesting
+ IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException {
+ try {
+ return InstantiationUtil.deserializeObject(serialized,
getClass().getClassLoader());
Review comment:
Should the `ClassLoader` be `IcebergSourceSplit.class.getClassLoader()`?
##########
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;
Review comment:
As position advances, this needs to invalidate the cached bytes.
##########
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:
Invalidate serialized bytes here.
##########
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);
+ }
+
+ @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());
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(this)
+ .add("files", toString(task.files()))
+ .add("position", position)
+ .toString();
+ }
+
+ private String toString(Collection<FileScanTask> files) {
+ return Iterables.toString(files.stream().map(fileScanTask ->
+ MoreObjects.toStringHelper(fileScanTask)
+ .add("file", fileScanTask.file() != null ?
Review comment:
I think we should fix the mock instead of adding logic to the real code.
##########
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);
+ }
+
+ @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());
Review comment:
The `equals` and `hashCode` implementations are not consistent because
`equals` uses the position, but `hashCode` ignores it.
##########
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);
+ }
+
+ @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());
Review comment:
Seems like you might want to just remove this and `hashCode` and use
object identity.
##########
File path:
flink/v1.13/flink/src/test/java/org/apache/iceberg/flink/source/split/SplitHelpers.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseCombinedScanTask;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MockFileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.flink.TestFixtures;
+import org.apache.iceberg.flink.source.FlinkSplitPlanner;
+import org.apache.iceberg.flink.source.ScanContext;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+
+public class SplitHelpers {
+
+ private static final AtomicLong splitLengthIncrement = new AtomicLong();
+
+ private SplitHelpers() {
+
Review comment:
Nit: no need for a newline here.
##########
File path:
flink/v1.13/flink/src/test/java/org/apache/iceberg/flink/source/split/SplitHelpers.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseCombinedScanTask;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MockFileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.flink.TestFixtures;
+import org.apache.iceberg.flink.source.FlinkSplitPlanner;
+import org.apache.iceberg.flink.source.ScanContext;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+
+public class SplitHelpers {
+
+ private static final AtomicLong splitLengthIncrement = new AtomicLong();
+
+ private SplitHelpers() {
+
+ }
+
+ public static List<IcebergSourceSplit> createMockedSplits(int splitCount) {
+ List<IcebergSourceSplit> splits = new ArrayList<>();
+ for (int i = 0; i < splitCount; ++i) {
+ // make sure each task has a different length,
+ // as it is part of the splitId calculation.
+ // This way, we can make sure all generated splits have different
splitIds
+ FileScanTask fileScanTask = new MockFileScanTask(1024 +
splitLengthIncrement.incrementAndGet());
+ CombinedScanTask combinedScanTask = new
BaseCombinedScanTask(fileScanTask);
+ splits.add(IcebergSourceSplit.fromCombinedScanTask(combinedScanTask));
+ }
+ return splits;
+ }
+
+ /**
+ * Unlike {@link SplitHelpers#createMockedSplits(int)} above, this is to
generate some
+ * realistic IcebergSourceSplit with actual file paths.
+ *
+ * Actual data files are already deleted before return. Caller shouldn't
attempt to read the
+ * data files. They are intended for serializer or enumerator unit test
(without actual reading).
+ */
+ public static List<IcebergSourceSplit> createFileSplits(
+ TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit)
throws Exception {
+ File warehouseFile = temporaryFolder.newFolder();
+ Assert.assertTrue(warehouseFile.delete());
+ String warehouse = "file:" + warehouseFile;
+ org.apache.hadoop.conf.Configuration hadoopConf = new
org.apache.hadoop.conf.Configuration();
Review comment:
Can we remove the fully-qualified path?
##########
File path:
flink/v1.13/flink/src/test/java/org/apache/iceberg/flink/source/split/SplitHelpers.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseCombinedScanTask;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MockFileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.flink.TestFixtures;
+import org.apache.iceberg.flink.source.FlinkSplitPlanner;
+import org.apache.iceberg.flink.source.ScanContext;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+
+public class SplitHelpers {
+
+ private static final AtomicLong splitLengthIncrement = new AtomicLong();
+
+ private SplitHelpers() {
+
+ }
+
+ public static List<IcebergSourceSplit> createMockedSplits(int splitCount) {
+ List<IcebergSourceSplit> splits = new ArrayList<>();
+ for (int i = 0; i < splitCount; ++i) {
+ // make sure each task has a different length,
+ // as it is part of the splitId calculation.
+ // This way, we can make sure all generated splits have different
splitIds
+ FileScanTask fileScanTask = new MockFileScanTask(1024 +
splitLengthIncrement.incrementAndGet());
+ CombinedScanTask combinedScanTask = new
BaseCombinedScanTask(fileScanTask);
+ splits.add(IcebergSourceSplit.fromCombinedScanTask(combinedScanTask));
+ }
+ return splits;
+ }
+
+ /**
+ * Unlike {@link SplitHelpers#createMockedSplits(int)} above, this is to
generate some
+ * realistic IcebergSourceSplit with actual file paths.
+ *
+ * Actual data files are already deleted before return. Caller shouldn't
attempt to read the
+ * data files. They are intended for serializer or enumerator unit test
(without actual reading).
+ */
+ public static List<IcebergSourceSplit> createFileSplits(
+ TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit)
throws Exception {
Review comment:
Is there a better name that shows that this is creating a table, writing
data, and then planning a scan?
##########
File path:
flink/v1.13/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestIcebergSourceSplitSerializer {
Review comment:
Probably want a test for calling `advance` on the position and then
serializing the split.
##########
File path:
flink/v1.13/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestIcebergSourceSplitSerializer {
+
+ @ClassRule
+ public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder();
+
+ private final IcebergSourceSplitSerializer serializer =
IcebergSourceSplitSerializer.INSTANCE;
+
+ @Test
+ public void testLatestVersion() throws Exception {
+ serializeAndDeserialize(1, 1);
+ serializeAndDeserialize(10, 2);
+ }
+
+ private void serializeAndDeserialize(int splitCount, int filesPerSplit)
throws Exception {
+ List<IcebergSourceSplit> splits =
SplitHelpers.createFileSplits(TEMPORARY_FOLDER, splitCount, filesPerSplit);
+ for (IcebergSourceSplit split : splits) {
+ byte[] result = serializer.serialize(split);
+ IcebergSourceSplit deserialized =
serializer.deserialize(serializer.getVersion(), result);
+ Assert.assertEquals(split, deserialized);
+
+ byte[] cachedResult = serializer.serialize(split);
+ Assert.assertSame(result, cachedResult);
+ IcebergSourceSplit deserialized2 =
serializer.deserialize(serializer.getVersion(), cachedResult);
+ Assert.assertEquals(split, deserialized2);
+ }
+ }
+
+ @Test
+ public void testV1() throws Exception {
+ serializeAndDeserializeV1(1, 1);
+ serializeAndDeserializeV1(10, 2);
+ }
+
+ private void serializeAndDeserializeV1(int splitCount, int filesPerSplit)
throws Exception {
+ List<IcebergSourceSplit> splits =
SplitHelpers.createFileSplits(TEMPORARY_FOLDER, splitCount, filesPerSplit);
+ for (IcebergSourceSplit split : splits) {
+ byte[] result = serializer.serializeV1(split);
+ IcebergSourceSplit deserialized = serializer.deserializeV1(result);
+ Assert.assertEquals(split, deserialized);
+ }
+ }
+
+ @Test
+ public void testCheckpointedPosition() throws Exception {
+ AtomicInteger index = new AtomicInteger();
+ List<IcebergSourceSplit> splits =
SplitHelpers.createFileSplits(TEMPORARY_FOLDER, 10, 2).stream()
+ .map(split -> {
+ IcebergSourceSplit result;
+ if (index.get() % 2 == 0) {
+ result = IcebergSourceSplit.fromCombinedScanTask(split.task(),
index.get(), index.get());
Review comment:
Before this, call `serializer.serialize(...)` to test that serialization
is correct after changing the position.
Some of these should also call `advanceFile()` and `advanceRow()` as well.
##########
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;
+ }
+ Position that = (Position) o;
+ return Objects.equals(fileOffset, that.fileOffset) &&
+ Objects.equals(recordOffset, that.recordOffset);
Review comment:
These are primitives, so calling `equals(Object, Object)` will box both
values just to do a more expensive check.
--
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]