StephanEwen commented on a change in pull request #13401: URL: https://github.com/apache/flink/pull/13401#discussion_r491699521
########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/FileSourceSplitReader.java ########## @@ -0,0 +1,110 @@ +/* + * 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.flink.connector.file.src.impl; + +import org.apache.flink.configuration.Configuration; +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.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connector.file.src.util.RecordAndPosition; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Queue; + +/** + * The {@link SplitReader} implementation for the file source. + */ +final class FileSourceSplitReader<T> implements SplitReader<RecordAndPosition<T>, FileSourceSplit> { + + private static final Logger LOG = LoggerFactory.getLogger(FileSourceSplitReader.class); + + private final Configuration config; + private final BulkFormat<T> readerFactory; + + private final Queue<FileSourceSplit> splits; + + @Nullable + private BulkFormat.Reader<T> currentReader; + @Nullable + private String currentSplitId; + + public FileSourceSplitReader(Configuration config, BulkFormat<T> readerFactory) { + this.config = config; + this.readerFactory = readerFactory; + this.splits = new ArrayDeque<>(); + } + + @Override + public RecordsWithSplitIds<RecordAndPosition<T>> fetch() throws IOException { + checkSplitOrStartNext(); + + final BulkFormat.RecordIterator<T> nextBatch = currentReader.readBatch(); + return nextBatch == null ? finishSplit() : FileRecords.forRecords(currentSplitId, nextBatch); + } + + @Override + public void handleSplitsChanges(final SplitsChange<FileSourceSplit> splitChange) { + if (!(splitChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException(String.format( + "The SplitChange type of %s is not supported.", splitChange.getClass())); + } + + LOG.debug("Handling split change {}", splitChange); + splits.addAll(splitChange.splits()); + } + + @Override + public void wakeUp() {} Review comment: This is a good question. TBH, I am not sure how to implement waking up here, or if we need it. - **Do we need wakeup?** I think not necessarily, because this method only stalls for a limited amount of time, until the I/O stream has delivered the next records. There is guaranteed to be something coming soon. Not waking up means some other operation (split changes) may get processed a second later. - **Can we implement wakeup?** I would not know how. Many I/O streams and decoders or parsers are not well suited for interruptions (don't react or become corrupt) and there is no common way to interrupt an I/O request. - **How can we speed up cancallation?** The only case where I am worried about a speedy response is cancellation. But we can speed this up by closing the reader directly. Closing reader/stream means that any reader on that stream will immediately exit with an IOException. Which is fine, because we ignore exceptions once cancellation has been initiated. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/FileSourceSplit.java ########## @@ -0,0 +1,190 @@ +/* + * 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.flink.connector.file.src; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Arrays; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link SourceSplit} that represents a file, or a region of a file. + */ +@PublicEvolving +public class FileSourceSplit implements SourceSplit, Serializable { + + private static final long serialVersionUID = 1L; + + private static final String[] NO_HOSTS = new String[0]; + + /** The unique ID of the split. Unique within the scope of this source. */ + private final String id; + + /** The path of the file referenced by this split. */ + private final Path filePath; + + /** The position of the first byte in the file to process. */ + private final long offset; + + /** The number of bytes in the file to process. */ + private final long length; + + /** The number of records to be skipped from the beginning of the split. + * This is for file formats that cannot pinpoint every exact record position via an offset, + * due to read buffers or bulk encoding or compression. */ + private final long skippedRecordCount; + + /** The names of the hosts storing this range of the file. Empty, if no host information is available. */ + private final String[] hostnames; + + /** The splits are frequently serialized into checkpoints. + * Caching the byte representation makes repeated serialization cheap. + * This field is used by {@link FileSourceSplitSerializer}. */ + @Nullable + transient byte[] serializedFormCache; Review comment: It is used by the `FileSourceSplitSerializer`. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/reader/FileRecordFormatAdapter.java ########## @@ -0,0 +1,143 @@ +/* + * 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.flink.connector.file.src.reader; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.util.CheckpointablePosition; +import org.apache.flink.connector.file.src.util.IteratorResultIterator; +import org.apache.flink.core.fs.Path; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.IOUtils; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The FormatReaderAdapter turns a {@link FileRecordFormat} into a {@link BulkFormat}. + */ +public final class FileRecordFormatAdapter<T> implements BulkFormat<T> { + + /** + * Config option for the number of records to hand over in each fetch. + * + * <p>The number should be large enough so that the thread-to-thread handover overhead + * is amortized across the records, but small enough so that the these records together do + * not consume too memory to be feasible. + */ + public static final ConfigOption<Integer> RECORDS_PER_FETCH = ConfigOptions Review comment: Good point, I'll move this. Same for the `StreamFormat`. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/ContinuousFileSplitEnumerator.java ########## @@ -0,0 +1,156 @@ +/* + * 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.flink.connector.file.src.impl; + +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.base.source.event.RequestSplitEvent; +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.PendingSplitsCheckpoint; +import org.apache.flink.connector.file.src.assigners.FileSplitAssigner; +import org.apache.flink.connector.file.src.enumerate.FileEnumerator; +import org.apache.flink.core.fs.Path; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A continuously monitoring enumerator. + */ +public class ContinuousFileSplitEnumerator implements SplitEnumerator<FileSourceSplit, PendingSplitsCheckpoint> { + + private static final Logger LOG = LoggerFactory.getLogger(ContinuousFileSplitEnumerator.class); + + private final SplitEnumeratorContext<FileSourceSplit> context; + + private final FileSplitAssigner splitAssigner; + + private final FileEnumerator enumerator; + + private final HashSet<Path> pathsAlreadyProcessed; + + private final HashSet<Integer> readersAwaitingSplit; + + private final Path[] paths; + + private final long discoveryInterval; + + // ------------------------------------------------------------------------ + + public ContinuousFileSplitEnumerator( + SplitEnumeratorContext<FileSourceSplit> context, + FileEnumerator enumerator, + FileSplitAssigner splitAssigner, + Path[] paths, + Collection<Path> alreadyDiscoveredPaths, + long discoveryInterval) { + + checkArgument(discoveryInterval > 0L); + this.context = checkNotNull(context); + this.enumerator = checkNotNull(enumerator); + this.splitAssigner = checkNotNull(splitAssigner); + this.paths = paths; + this.discoveryInterval = discoveryInterval; + this.pathsAlreadyProcessed = new HashSet<>(alreadyDiscoveredPaths); + this.readersAwaitingSplit = new HashSet<>(); + } + + @Override + public void start() { + context.callAsync( + () -> enumerator.enumerateSplits(paths, 1), + this::processDiscoveredSplits, + discoveryInterval, discoveryInterval); + } + + @Override + public void close() throws IOException { + // no resources to close + } + + @Override + public void addReader(int subtaskId) { + // this source is purely lazy-pull-based, nothing to do upon registration + } + + @Override + public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { + if (sourceEvent instanceof RequestSplitEvent) { + readersAwaitingSplit.add(subtaskId); + assignSplits(); + } + else { + LOG.error("Received unrecognized event: {}", sourceEvent); + } + } + + @Override + public void addSplitsBack(List<FileSourceSplit> splits, int subtaskId) { + LOG.debug("File Source Enumerator adds splits back: {}", splits); + splitAssigner.addSplits(splits); + } + + @Override + public PendingSplitsCheckpoint snapshotState() throws Exception { + return PendingSplitsCheckpoint.fromCollectionSnapshot(splitAssigner.remainingSplits(), pathsAlreadyProcessed); + } + + // ------------------------------------------------------------------------ + + private void processDiscoveredSplits(Collection<FileSourceSplit> splits, Throwable error) { + if (error != null) { + LOG.error("Failed to enumerate files", error); + return; + } + + final Collection<FileSourceSplit> newSplits = splits.stream() + .filter((split) -> pathsAlreadyProcessed.add(split.path())) + .collect(Collectors.toList()); + splitAssigner.addSplits(newSplits); + + assignSplits(); + } + + private void assignSplits() { + final Iterator<Integer> awaitingReader = readersAwaitingSplit.iterator(); + + while (awaitingReader.hasNext()) { + final int awaitingSubtask = awaitingReader.next(); + final Optional<FileSourceSplit> nextSplit = splitAssigner.getNext(null); Review comment: Good catch. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/ContinuousFileSplitEnumerator.java ########## @@ -0,0 +1,156 @@ +/* + * 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.flink.connector.file.src.impl; + +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.base.source.event.RequestSplitEvent; +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.PendingSplitsCheckpoint; +import org.apache.flink.connector.file.src.assigners.FileSplitAssigner; +import org.apache.flink.connector.file.src.enumerate.FileEnumerator; +import org.apache.flink.core.fs.Path; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A continuously monitoring enumerator. + */ +public class ContinuousFileSplitEnumerator implements SplitEnumerator<FileSourceSplit, PendingSplitsCheckpoint> { + + private static final Logger LOG = LoggerFactory.getLogger(ContinuousFileSplitEnumerator.class); + + private final SplitEnumeratorContext<FileSourceSplit> context; + + private final FileSplitAssigner splitAssigner; + + private final FileEnumerator enumerator; + + private final HashSet<Path> pathsAlreadyProcessed; Review comment: Indeed. This implementation is not optimal, yet. This needs to eventually track the processed files by a type of low-watermark for the file creation timestamp, and only have that low-watermark in the state and file paths after that low-watermark. I would consider that simply a better enumerator implementation, and would do this in a follow-up patch. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/reader/BulkFormat.java ########## @@ -0,0 +1,158 @@ +/* + * 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.flink.connector.file.src.reader; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.util.MutableRecordAndPosition; +import org.apache.flink.connector.file.src.util.RecordAndPosition; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nullable; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; + +/** + * The {@code BulkFormat} reads and decodes batches of records at a time. Examples of bulk formats + * are formats like ORC or Parquet. + * + * <p>The actual reading is done by the {@link BulkFormat.Reader}, which is created in the + * {@link BulkFormat#createReader(Configuration, Path, long, long)} or + * {@link BulkFormat#createReader(Configuration, Path, long, long, long)} methods. + * The outer class acts mainly as a configuration holder and factory for the reader. + * + * <h2>Checkpointing</h2> + * + * <p>The bulk reader returns an iterator structure per batch. The iterator produces records together + * with a position. That position is the point from where the reading can be resumed AFTER + * the records was emitted. So that position points effectively to the record AFTER the current record. + * + * <p>The simplest way to return this position information is to always assume a zero offset in the file + * and simply increment the record count. Note that in this case the fist record would be returned with + * a record count of one, the second one with a record count of two, etc. + * + * <p>Formats that have the ability to efficiently seek to a record (or to every n-th record) by offset + * in the file can work with the position field to avoid having to read and discard many records on recovery. + * + * <h2>Serializable</h2> + * + * <p>Like many other API classes in Flink, the outer class is serializable to support sending instances + * to distributed workers for parallel execution. This is purely short-term serialization for RPC and + * no instance of this will be long-term persisted in a serialized form. + * + * <h2>Record Batching</h2> + * + * <p>Internally in the file source, the readers pass batches of records from the reading + * threads (that perform the typically blocking I/O operations) to the async mailbox threads that + * do the streaming and batch data processing. Passing records in batches (rather than one-at-a-time) + * much reduce the thread-to-thread handover overhead. + * + * <p>For the {@code BulkFormat}, one batch (as returned by {@link BulkFormat.Reader#readBatch()}) is + * handed over as one. + */ +public interface BulkFormat<T> extends Serializable { + + /** + * Creates a new reader that reads from {@code filePath} starting at {@code offset} and reads + * until {@code length} bytes after the offset. + */ + default BulkFormat.Reader<T> createReader(Configuration config, Path filePath, long offset, long length) throws IOException { + return createReader(config, filePath, offset, length, 0L); + } + + /** + * Creates a new reader that reads from {@code filePath} starting at {@code offset} and reads + * until {@code length} bytes after the offset. A number of {@code recordsToSkip} records should be + * read and discarded after the offset. This is typically part of restoring a reader to a checkpointed + * position. + */ + BulkFormat.Reader<T> createReader(Configuration config, Path filePath, long offset, long length, long recordsToSkip) throws IOException; + + /** + * Gets the type produced by this format. This type will be the type produced by the file + * source as a whole. + */ + TypeInformation<T> getProducedType(); Review comment: I am torn on this one. The `ResultTypeQueryable` is mainly used by the `StreamExecutionEnviroment` and `DataStream` classes. The formats here are not used directly with those. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/util/IteratorResultIterator.java ########## @@ -0,0 +1,64 @@ +/* + * 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.flink.connector.file.src.util; + +import org.apache.flink.connector.file.src.reader.BulkFormat; + +import javax.annotation.Nullable; + +import java.util.Iterator; + +/** + * A simple {@link BulkFormat.RecordIterator} that returns the elements of an iterator, one after the other. + * + * @param <E> The type of the record returned by the iterator. + */ +public final class IteratorResultIterator<E> implements BulkFormat.RecordIterator<E> { + + private final Iterator<E> records; + + private final MutableRecordAndPosition<E> recordAndPosition; + + public IteratorResultIterator( + final Iterator<E> records, + final long offset, + final long startingSkipCount) { + this.records = records; + this.recordAndPosition = new MutableRecordAndPosition<>(); + this.recordAndPosition.setPosition(offset, startingSkipCount); + } + + // ------------------------------------------------------------------------- + // Result Iterator Methods + // ------------------------------------------------------------------------- + + @Nullable + @Override + public RecordAndPosition<E> next() { + if (records.hasNext()) { + recordAndPosition.setNext(records.next()); + return recordAndPosition; + } else { + return null; + } + } + + @Override + public void close() {} Review comment: Sounds good, will do that. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/FileSourceSplitReader.java ########## @@ -0,0 +1,110 @@ +/* + * 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.flink.connector.file.src.impl; + +import org.apache.flink.configuration.Configuration; +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.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connector.file.src.util.RecordAndPosition; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Queue; + +/** + * The {@link SplitReader} implementation for the file source. + */ +final class FileSourceSplitReader<T> implements SplitReader<RecordAndPosition<T>, FileSourceSplit> { + + private static final Logger LOG = LoggerFactory.getLogger(FileSourceSplitReader.class); + + private final Configuration config; + private final BulkFormat<T> readerFactory; + + private final Queue<FileSourceSplit> splits; + + @Nullable + private BulkFormat.Reader<T> currentReader; + @Nullable + private String currentSplitId; + + public FileSourceSplitReader(Configuration config, BulkFormat<T> readerFactory) { + this.config = config; + this.readerFactory = readerFactory; + this.splits = new ArrayDeque<>(); + } + + @Override + public RecordsWithSplitIds<RecordAndPosition<T>> fetch() throws IOException { + checkSplitOrStartNext(); + + final BulkFormat.RecordIterator<T> nextBatch = currentReader.readBatch(); + return nextBatch == null ? finishSplit() : FileRecords.forRecords(currentSplitId, nextBatch); + } + + @Override + public void handleSplitsChanges(final SplitsChange<FileSourceSplit> splitChange) { + if (!(splitChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException(String.format( + "The SplitChange type of %s is not supported.", splitChange.getClass())); + } + + LOG.debug("Handling split change {}", splitChange); + splits.addAll(splitChange.splits()); + } + + @Override + public void wakeUp() {} + + private void checkSplitOrStartNext() throws IOException { + if (currentReader != null) { + return; + } + + final FileSourceSplit nextSplit = splits.poll(); + if (nextSplit == null) { + throw new IOException("Cannot fetch from another split - no split remaining"); + } + + currentSplitId = nextSplit.splitId(); + currentReader = nextSplit.skippedRecordCount() == 0 ? Review comment: This actually changed a bit in the latest fix, so I cannot apply the here suggested pattern any ore. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/FileSourceSplitReader.java ########## @@ -0,0 +1,110 @@ +/* + * 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.flink.connector.file.src.impl; + +import org.apache.flink.configuration.Configuration; +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.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connector.file.src.util.RecordAndPosition; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Queue; + +/** + * The {@link SplitReader} implementation for the file source. + */ +final class FileSourceSplitReader<T> implements SplitReader<RecordAndPosition<T>, FileSourceSplit> { + + private static final Logger LOG = LoggerFactory.getLogger(FileSourceSplitReader.class); + + private final Configuration config; + private final BulkFormat<T> readerFactory; + + private final Queue<FileSourceSplit> splits; + + @Nullable + private BulkFormat.Reader<T> currentReader; + @Nullable + private String currentSplitId; + + public FileSourceSplitReader(Configuration config, BulkFormat<T> readerFactory) { + this.config = config; + this.readerFactory = readerFactory; + this.splits = new ArrayDeque<>(); + } + + @Override + public RecordsWithSplitIds<RecordAndPosition<T>> fetch() throws IOException { + checkSplitOrStartNext(); + + final BulkFormat.RecordIterator<T> nextBatch = currentReader.readBatch(); + return nextBatch == null ? finishSplit() : FileRecords.forRecords(currentSplitId, nextBatch); + } + + @Override + public void handleSplitsChanges(final SplitsChange<FileSourceSplit> splitChange) { + if (!(splitChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException(String.format( + "The SplitChange type of %s is not supported.", splitChange.getClass())); + } + + LOG.debug("Handling split change {}", splitChange); + splits.addAll(splitChange.splits()); + } + + @Override + public void wakeUp() {} + + private void checkSplitOrStartNext() throws IOException { + if (currentReader != null) { + return; + } + + final FileSourceSplit nextSplit = splits.poll(); + if (nextSplit == null) { + throw new IOException("Cannot fetch from another split - no split remaining"); + } + + currentSplitId = nextSplit.splitId(); + currentReader = nextSplit.skippedRecordCount() == 0 ? Review comment: This actually changed a bit in the latest fix, so I cannot apply the here suggested pattern any more. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/FileSourceSplit.java ########## @@ -0,0 +1,190 @@ +/* + * 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.flink.connector.file.src; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Arrays; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link SourceSplit} that represents a file, or a region of a file. + */ +@PublicEvolving +public class FileSourceSplit implements SourceSplit, Serializable { + + private static final long serialVersionUID = 1L; + + private static final String[] NO_HOSTS = new String[0]; + + /** The unique ID of the split. Unique within the scope of this source. */ + private final String id; + + /** The path of the file referenced by this split. */ + private final Path filePath; + + /** The position of the first byte in the file to process. */ + private final long offset; + + /** The number of bytes in the file to process. */ + private final long length; + + /** The number of records to be skipped from the beginning of the split. + * This is for file formats that cannot pinpoint every exact record position via an offset, + * due to read buffers or bulk encoding or compression. */ + private final long skippedRecordCount; + + /** The names of the hosts storing this range of the file. Empty, if no host information is available. */ + private final String[] hostnames; + + /** The splits are frequently serialized into checkpoints. + * Caching the byte representation makes repeated serialization cheap. + * This field is used by {@link FileSourceSplitSerializer}. */ + @Nullable + transient byte[] serializedFormCache; Review comment: This optimization is mainly for the enumerator side. There we checkpoint all un-assigned splits frequently with every checkpoint. There can be millions and I want to avoid re-serializing them every few seconds. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/FileSourceSplitReader.java ########## @@ -0,0 +1,110 @@ +/* + * 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.flink.connector.file.src.impl; + +import org.apache.flink.configuration.Configuration; +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.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connector.file.src.util.RecordAndPosition; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Queue; + +/** + * The {@link SplitReader} implementation for the file source. + */ +final class FileSourceSplitReader<T> implements SplitReader<RecordAndPosition<T>, FileSourceSplit> { + + private static final Logger LOG = LoggerFactory.getLogger(FileSourceSplitReader.class); + + private final Configuration config; + private final BulkFormat<T> readerFactory; + + private final Queue<FileSourceSplit> splits; + + @Nullable + private BulkFormat.Reader<T> currentReader; + @Nullable + private String currentSplitId; + + public FileSourceSplitReader(Configuration config, BulkFormat<T> readerFactory) { + this.config = config; + this.readerFactory = readerFactory; + this.splits = new ArrayDeque<>(); + } + + @Override + public RecordsWithSplitIds<RecordAndPosition<T>> fetch() throws IOException { + checkSplitOrStartNext(); + + final BulkFormat.RecordIterator<T> nextBatch = currentReader.readBatch(); + return nextBatch == null ? finishSplit() : FileRecords.forRecords(currentSplitId, nextBatch); + } + + @Override + public void handleSplitsChanges(final SplitsChange<FileSourceSplit> splitChange) { + if (!(splitChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException(String.format( + "The SplitChange type of %s is not supported.", splitChange.getClass())); + } + + LOG.debug("Handling split change {}", splitChange); + splits.addAll(splitChange.splits()); + } + + @Override + public void wakeUp() {} Review comment: I am a bit hesitant to add more complexity when we don't know a single case where it is applicable and efficient it is also not really needed. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/reader/BulkFormat.java ########## @@ -0,0 +1,158 @@ +/* + * 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.flink.connector.file.src.reader; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.util.MutableRecordAndPosition; +import org.apache.flink.connector.file.src.util.RecordAndPosition; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nullable; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; + +/** + * The {@code BulkFormat} reads and decodes batches of records at a time. Examples of bulk formats + * are formats like ORC or Parquet. + * + * <p>The actual reading is done by the {@link BulkFormat.Reader}, which is created in the + * {@link BulkFormat#createReader(Configuration, Path, long, long)} or + * {@link BulkFormat#createReader(Configuration, Path, long, long, long)} methods. + * The outer class acts mainly as a configuration holder and factory for the reader. + * + * <h2>Checkpointing</h2> + * + * <p>The bulk reader returns an iterator structure per batch. The iterator produces records together + * with a position. That position is the point from where the reading can be resumed AFTER + * the records was emitted. So that position points effectively to the record AFTER the current record. + * + * <p>The simplest way to return this position information is to always assume a zero offset in the file + * and simply increment the record count. Note that in this case the fist record would be returned with + * a record count of one, the second one with a record count of two, etc. + * + * <p>Formats that have the ability to efficiently seek to a record (or to every n-th record) by offset + * in the file can work with the position field to avoid having to read and discard many records on recovery. + * + * <h2>Serializable</h2> + * + * <p>Like many other API classes in Flink, the outer class is serializable to support sending instances + * to distributed workers for parallel execution. This is purely short-term serialization for RPC and + * no instance of this will be long-term persisted in a serialized form. + * + * <h2>Record Batching</h2> + * + * <p>Internally in the file source, the readers pass batches of records from the reading + * threads (that perform the typically blocking I/O operations) to the async mailbox threads that + * do the streaming and batch data processing. Passing records in batches (rather than one-at-a-time) + * much reduce the thread-to-thread handover overhead. + * + * <p>For the {@code BulkFormat}, one batch (as returned by {@link BulkFormat.Reader#readBatch()}) is + * handed over as one. + */ +public interface BulkFormat<T> extends Serializable { + + /** + * Creates a new reader that reads from {@code filePath} starting at {@code offset} and reads + * until {@code length} bytes after the offset. + */ + default BulkFormat.Reader<T> createReader(Configuration config, Path filePath, long offset, long length) throws IOException { + return createReader(config, filePath, offset, length, 0L); + } + + /** + * Creates a new reader that reads from {@code filePath} starting at {@code offset} and reads + * until {@code length} bytes after the offset. A number of {@code recordsToSkip} records should be + * read and discarded after the offset. This is typically part of restoring a reader to a checkpointed + * position. + */ + BulkFormat.Reader<T> createReader(Configuration config, Path filePath, long offset, long length, long recordsToSkip) throws IOException; + + /** + * Gets the type produced by this format. This type will be the type produced by the file + * source as a whole. + */ + TypeInformation<T> getProducedType(); Review comment: I can add it, if you feel strongly about this. I am usually a bit cautious adding more interfaces, because especially in a public API one can never undo this again. ########## File path: flink-formats/flink-orc/src/main/java/org/apache/flink/orc/util/SerializableHadoopConfigWrapper.java ########## @@ -0,0 +1,78 @@ +/* + * 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.flink.orc.util; + +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Utility class to make a {@link Configuration Hadoop Configuration} serializable. + */ +public final class SerializableHadoopConfigWrapper implements Serializable { + + private static final long serialVersionUID = 1L; + + private transient Configuration hadoopConfig; + + public SerializableHadoopConfigWrapper(Configuration hadoopConfig) { + this.hadoopConfig = checkNotNull(hadoopConfig); + } + + public Configuration getHadoopConfig() { + return hadoopConfig; + } + + // ------------------------------------------------------------------------ + + private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); + + // we write the Hadoop config through a separate serializer to avoid cryptic exceptions when it + // corrupts the serialization stream + final DataOutputSerializer ser = new DataOutputSerializer(256); + hadoopConfig.write(out); Review comment: right, good catch! ########## File path: flink-formats/flink-orc/src/main/java/org/apache/flink/orc/util/SerializableHadoopConfigWrapper.java ########## @@ -0,0 +1,78 @@ +/* + * 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.flink.orc.util; + +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Utility class to make a {@link Configuration Hadoop Configuration} serializable. + */ +public final class SerializableHadoopConfigWrapper implements Serializable { + + private static final long serialVersionUID = 1L; + + private transient Configuration hadoopConfig; + + public SerializableHadoopConfigWrapper(Configuration hadoopConfig) { + this.hadoopConfig = checkNotNull(hadoopConfig); + } + + public Configuration getHadoopConfig() { + return hadoopConfig; + } + + // ------------------------------------------------------------------------ + + private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); + + // we write the Hadoop config through a separate serializer to avoid cryptic exceptions when it + // corrupts the serialization stream + final DataOutputSerializer ser = new DataOutputSerializer(256); + hadoopConfig.write(out); Review comment: I'll add a test for that. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/FileSourceSplit.java ########## @@ -0,0 +1,190 @@ +/* + * 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.flink.connector.file.src; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Arrays; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link SourceSplit} that represents a file, or a region of a file. + */ +@PublicEvolving +public class FileSourceSplit implements SourceSplit, Serializable { + + private static final long serialVersionUID = 1L; + + private static final String[] NO_HOSTS = new String[0]; + + /** The unique ID of the split. Unique within the scope of this source. */ + private final String id; + + /** The path of the file referenced by this split. */ + private final Path filePath; + + /** The position of the first byte in the file to process. */ + private final long offset; + + /** The number of bytes in the file to process. */ + private final long length; + + /** The number of records to be skipped from the beginning of the split. + * This is for file formats that cannot pinpoint every exact record position via an offset, + * due to read buffers or bulk encoding or compression. */ + private final long skippedRecordCount; + + /** The names of the hosts storing this range of the file. Empty, if no host information is available. */ + private final String[] hostnames; + + /** The splits are frequently serialized into checkpoints. + * Caching the byte representation makes repeated serialization cheap. + * This field is used by {@link FileSourceSplitSerializer}. */ + @Nullable + transient byte[] serializedFormCache; Review comment: The current implementation does not exploit this, true. We could change it such that it only creates a new `PendingSplitsCheckpoint` instance if there was a change. I added the caching symmetrically to the `FileSourceSplit` because it is "hidden" (adds no complexity for the user) and comes for free (no extra work done). ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/FileSourceSplitReader.java ########## @@ -0,0 +1,110 @@ +/* + * 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.flink.connector.file.src.impl; + +import org.apache.flink.configuration.Configuration; +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.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connector.file.src.util.RecordAndPosition; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Queue; + +/** + * The {@link SplitReader} implementation for the file source. + */ +final class FileSourceSplitReader<T> implements SplitReader<RecordAndPosition<T>, FileSourceSplit> { + + private static final Logger LOG = LoggerFactory.getLogger(FileSourceSplitReader.class); + + private final Configuration config; + private final BulkFormat<T> readerFactory; + + private final Queue<FileSourceSplit> splits; + + @Nullable + private BulkFormat.Reader<T> currentReader; + @Nullable + private String currentSplitId; + + public FileSourceSplitReader(Configuration config, BulkFormat<T> readerFactory) { + this.config = config; + this.readerFactory = readerFactory; + this.splits = new ArrayDeque<>(); + } + + @Override + public RecordsWithSplitIds<RecordAndPosition<T>> fetch() throws IOException { + checkSplitOrStartNext(); + + final BulkFormat.RecordIterator<T> nextBatch = currentReader.readBatch(); + return nextBatch == null ? finishSplit() : FileRecords.forRecords(currentSplitId, nextBatch); + } + + @Override + public void handleSplitsChanges(final SplitsChange<FileSourceSplit> splitChange) { + if (!(splitChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException(String.format( + "The SplitChange type of %s is not supported.", splitChange.getClass())); + } + + LOG.debug("Handling split change {}", splitChange); + splits.addAll(splitChange.splits()); + } + + @Override + public void wakeUp() {} Review comment: Java async IO with a selector would probably not be a file format but a different Source with a different Split Reader. There we have the wakeup() method already. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/reader/BulkFormat.java ########## @@ -0,0 +1,158 @@ +/* + * 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.flink.connector.file.src.reader; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.util.MutableRecordAndPosition; +import org.apache.flink.connector.file.src.util.RecordAndPosition; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nullable; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; + +/** + * The {@code BulkFormat} reads and decodes batches of records at a time. Examples of bulk formats + * are formats like ORC or Parquet. + * + * <p>The actual reading is done by the {@link BulkFormat.Reader}, which is created in the + * {@link BulkFormat#createReader(Configuration, Path, long, long)} or + * {@link BulkFormat#createReader(Configuration, Path, long, long, long)} methods. + * The outer class acts mainly as a configuration holder and factory for the reader. + * + * <h2>Checkpointing</h2> + * + * <p>The bulk reader returns an iterator structure per batch. The iterator produces records together + * with a position. That position is the point from where the reading can be resumed AFTER + * the records was emitted. So that position points effectively to the record AFTER the current record. + * + * <p>The simplest way to return this position information is to always assume a zero offset in the file + * and simply increment the record count. Note that in this case the fist record would be returned with + * a record count of one, the second one with a record count of two, etc. + * + * <p>Formats that have the ability to efficiently seek to a record (or to every n-th record) by offset + * in the file can work with the position field to avoid having to read and discard many records on recovery. + * + * <h2>Serializable</h2> + * + * <p>Like many other API classes in Flink, the outer class is serializable to support sending instances + * to distributed workers for parallel execution. This is purely short-term serialization for RPC and + * no instance of this will be long-term persisted in a serialized form. + * + * <h2>Record Batching</h2> + * + * <p>Internally in the file source, the readers pass batches of records from the reading + * threads (that perform the typically blocking I/O operations) to the async mailbox threads that + * do the streaming and batch data processing. Passing records in batches (rather than one-at-a-time) + * much reduce the thread-to-thread handover overhead. + * + * <p>For the {@code BulkFormat}, one batch (as returned by {@link BulkFormat.Reader#readBatch()}) is + * handed over as one. + */ +public interface BulkFormat<T> extends Serializable { + + /** + * Creates a new reader that reads from {@code filePath} starting at {@code offset} and reads + * until {@code length} bytes after the offset. + */ + default BulkFormat.Reader<T> createReader(Configuration config, Path filePath, long offset, long length) throws IOException { + return createReader(config, filePath, offset, length, 0L); + } + + /** + * Creates a new reader that reads from {@code filePath} starting at {@code offset} and reads + * until {@code length} bytes after the offset. A number of {@code recordsToSkip} records should be + * read and discarded after the offset. This is typically part of restoring a reader to a checkpointed + * position. + */ + BulkFormat.Reader<T> createReader(Configuration config, Path filePath, long offset, long length, long recordsToSkip) throws IOException; + + /** + * Gets the type produced by this format. This type will be the type produced by the file + * source as a whole. + */ + TypeInformation<T> getProducedType(); Review comment: I think you are right. It makes sense to add the interface. I guess my main reservation was that the name `ResultTypeQueryable` is not a great name. But that shouldn't stop anything here. ########## File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/reader/BulkFormat.java ########## @@ -0,0 +1,158 @@ +/* + * 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.flink.connector.file.src.reader; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.util.MutableRecordAndPosition; +import org.apache.flink.connector.file.src.util.RecordAndPosition; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nullable; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; + +/** + * The {@code BulkFormat} reads and decodes batches of records at a time. Examples of bulk formats + * are formats like ORC or Parquet. + * + * <p>The actual reading is done by the {@link BulkFormat.Reader}, which is created in the + * {@link BulkFormat#createReader(Configuration, Path, long, long)} or + * {@link BulkFormat#createReader(Configuration, Path, long, long, long)} methods. + * The outer class acts mainly as a configuration holder and factory for the reader. + * + * <h2>Checkpointing</h2> + * + * <p>The bulk reader returns an iterator structure per batch. The iterator produces records together + * with a position. That position is the point from where the reading can be resumed AFTER + * the records was emitted. So that position points effectively to the record AFTER the current record. + * + * <p>The simplest way to return this position information is to always assume a zero offset in the file + * and simply increment the record count. Note that in this case the fist record would be returned with + * a record count of one, the second one with a record count of two, etc. + * + * <p>Formats that have the ability to efficiently seek to a record (or to every n-th record) by offset + * in the file can work with the position field to avoid having to read and discard many records on recovery. + * + * <h2>Serializable</h2> + * + * <p>Like many other API classes in Flink, the outer class is serializable to support sending instances + * to distributed workers for parallel execution. This is purely short-term serialization for RPC and + * no instance of this will be long-term persisted in a serialized form. + * + * <h2>Record Batching</h2> + * + * <p>Internally in the file source, the readers pass batches of records from the reading + * threads (that perform the typically blocking I/O operations) to the async mailbox threads that + * do the streaming and batch data processing. Passing records in batches (rather than one-at-a-time) + * much reduce the thread-to-thread handover overhead. + * + * <p>For the {@code BulkFormat}, one batch (as returned by {@link BulkFormat.Reader#readBatch()}) is + * handed over as one. + */ +public interface BulkFormat<T> extends Serializable { + + /** + * Creates a new reader that reads from {@code filePath} starting at {@code offset} and reads + * until {@code length} bytes after the offset. + */ + default BulkFormat.Reader<T> createReader(Configuration config, Path filePath, long offset, long length) throws IOException { + return createReader(config, filePath, offset, length, 0L); + } + + /** + * Creates a new reader that reads from {@code filePath} starting at {@code offset} and reads + * until {@code length} bytes after the offset. A number of {@code recordsToSkip} records should be + * read and discarded after the offset. This is typically part of restoring a reader to a checkpointed + * position. + */ + BulkFormat.Reader<T> createReader(Configuration config, Path filePath, long offset, long length, long recordsToSkip) throws IOException; + + /** + * Gets the type produced by this format. This type will be the type produced by the file + * source as a whole. + */ + TypeInformation<T> getProducedType(); Review comment: I added it. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org