tweise commented on a change in pull request #2305: URL: https://github.com/apache/iceberg/pull/2305#discussion_r716115030
########## File path: flink/src/main/java/org/apache/iceberg/flink/source/reader/FileRecords.java ########## @@ -0,0 +1,103 @@ +/* + * 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.reader; + +import java.io.IOException; +import java.util.Collections; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.file.src.util.RecordAndPosition; +import org.apache.iceberg.io.CloseableIterator; + +/** + * A batch of recrods for one split + */ +@Internal +public class FileRecords<T> implements RecordsWithSplitIds<RecordAndPosition<T>> { Review comment: Found the class name not intuitive. From usage, this appears to be a "fetch result"? ########## File path: flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java ########## @@ -0,0 +1,74 @@ +/* + * 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 issue-1698. Review comment: Please add full link. With new version such subsequent change will be backward compatible? ########## 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: You mean by `IcebergSourceSplitSerializer`. Since that's same package it doesn't need to be public? ########## File path: flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java ########## @@ -0,0 +1,46 @@ +/* + * 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.reader; + +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.file.src.util.RecordAndPosition; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; + +public abstract class DataIteratorReaderFunction<T> implements ReaderFunction<T> { Review comment: internal or javadoc? ########## File path: flink/src/main/java/org/apache/iceberg/flink/source/reader/RecyclableArrayIterator.java ########## @@ -0,0 +1,84 @@ +/* + * 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.reader; + +import javax.annotation.Nullable; +import org.apache.flink.connector.file.src.util.ArrayResultIterator; +import org.apache.flink.connector.file.src.util.CheckpointedPosition; +import org.apache.flink.connector.file.src.util.MutableRecordAndPosition; +import org.apache.flink.connector.file.src.util.Pool; +import org.apache.flink.connector.file.src.util.RecordAndPosition; +import org.apache.iceberg.io.CloseableIterator; + +/** + * Similar to the {@link ArrayResultIterator}. + * Main difference is the records array can be recycled back to a pool. + */ +final class RecyclableArrayIterator<E> implements CloseableIterator<RecordAndPosition<E>> { + private final Pool.Recycler<E[]> recycler; + private final E[] records; + private final int num; + private final MutableRecordAndPosition<E> recordAndPosition; + + private int pos; + + RecyclableArrayIterator(Pool.Recycler<E[]> recycler) { + this(recycler, null, 0, CheckpointedPosition.NO_OFFSET, 0L); + } + + /** + * Each record's {@link RecordAndPosition} will have the same fileOffset (for {@link RecordAndPosition#getOffset()}. Review comment: should this move to class level doc? ########## File path: flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java ########## @@ -0,0 +1,118 @@ +/* + * 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.reader; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayDeque; +import java.util.Queue; +import javax.annotation.Nullable; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.file.src.util.RecordAndPosition; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class IcebergSourceSplitReader<T> implements SplitReader<RecordAndPosition<T>, IcebergSourceSplit> { Review comment: Did you consider the need to subclass the reader for customization? Maybe it should be protected? ########## File path: flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java ########## @@ -0,0 +1,31 @@ +/* + * 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.reader; + +import java.io.Serializable; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.file.src.util.RecordAndPosition; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; + +@FunctionalInterface Review comment: javadoc? -- 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]
