Github user mxm commented on a diff in the pull request: https://github.com/apache/flink/pull/2618#discussion_r84288776 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichFileInputSplit.java --- @@ -0,0 +1,153 @@ +/* + * 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.streaming.api.functions.source; + +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; + +/** + * An extended {@link FileInputSplit} that also includes information about: + * <ul> + * <li>The modification time of the file this split belongs to.</li> + * <li>When checkpointing, the state of the split at the moment of the checkpoint.</li> + * </ul> + * This class is used by the {@link ContinuousFileMonitoringFunction} and the + * {@link ContinuousFileReaderOperator} to perform continuous file processing. + * */ +public class RichFileInputSplit<S extends Serializable> + extends FileInputSplit implements Comparable<RichFileInputSplit>{ + + /** The modification time of the file this split belongs to. */ + private final long modificationTime; + + /** + * The state of the split. This information is used when + * restoring from a checkpoint and allows to resume reading the + * underlying file from the point we left off. + * */ + private S splitState; + + /** A special {@link RichFileInputSplit} signaling the end of the stream of splits.*/ + public static final RichFileInputSplit EOS = + new RichFileInputSplit<>(Long.MIN_VALUE, -1, null, -1, -1, null); + + /** + * Creates a {@link RichFileInputSplit} based on the file modification time and + * the rest of the information of the {@link FileInputSplit}, as returned by the + * underlying filesystem. + * + * @param modificationTime the modification file of the file this split belongs to + * @param split the rest of the information about the split + */ + public RichFileInputSplit(long modificationTime, FileInputSplit split) { + this(modificationTime, + split.getSplitNumber(), + split.getPath(), + split.getStart(), + split.getLength(), + split.getHostnames()); + } + + /** + * Constructor with the raw split information. + * + * @param modificationTime the modification file of the file this split belongs to + * @param num the number of this input split + * @param file the file name + * @param start the position of the first byte in the file to process + * @param length the number of bytes in the file to process (-1 is flag for "read whole file") + * @param hosts the list of hosts containing the block, possibly <code>null</code> + */ + private RichFileInputSplit(long modificationTime, int num, Path file, long start, long length, String[] hosts) { + super(num, file, start, length, hosts); + + Preconditions.checkArgument(modificationTime >= 0 || modificationTime == Long.MIN_VALUE, + "Invalid File Split Modification Time: "+ modificationTime +"."); + + this.modificationTime = modificationTime; + } + + /** + * Sets the state of the split. This information is used when + * restoring from a checkpoint and allows to resume reading the + * underlying file from the point we left off. + * <p> + * This is applicable to {@link org.apache.flink.api.common.io.FileInputFormat FileInputFormats} + * that implement the {@link org.apache.flink.api.common.io.CheckpointableInputFormat + * CheckpointableInputFormat} interface. + * */ + public void setSplitState(S state) { + this.splitState = state; + } + + /** + * Sets the state of the split to {@code null}. + */ + public void resetSplitState() { + this.setSplitState(null); + } + + /** @return the state of the split. */ + public S getSplitState() { --- End diff -- ```java public Serializable getSplitState() { ```
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---