Java tutorial
/*********************************************************************************************************************** * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) * * Licensed 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 eu.stratosphere.api.common.io; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import eu.stratosphere.api.common.io.statistics.BaseStatistics; import eu.stratosphere.api.common.operators.base.GenericDataSourceBase; import eu.stratosphere.configuration.ConfigConstants; import eu.stratosphere.configuration.Configuration; import eu.stratosphere.configuration.GlobalConfiguration; import eu.stratosphere.core.fs.BlockLocation; import eu.stratosphere.core.fs.FSDataInputStream; import eu.stratosphere.core.fs.FileInputSplit; import eu.stratosphere.core.fs.FileStatus; import eu.stratosphere.core.fs.FileSystem; import eu.stratosphere.core.fs.Path; /** * Describes the base interface that is used for reading from a file input. For specific input types the * <tt>nextRecord()</tt> and <tt>reachedEnd()</tt> methods need to be implemented. Additionally, one may override * <tt>open(FileInputSplit)</tt> and <tt>close()</tt> to * * * * While reading the runtime checks whether the end was reached using reachedEnd() * and if not the next pair is read using the nextPair() method. * * Describes the base interface that is used describe an input that produces records that are processed * by Stratosphere. * <p> * The input format handles the following: * <ul> * <li>It describes how the input is split into splits that can be processed in parallel.</li> * <li>It describes how to read records from the input split.</li> * <li>It describes how to gather basic statistics from the input.</li> * </ul> * <p> * The life cycle of an input format is the following: * <ol> * <li>After being instantiated (parameterless), it is configured with a {@link Configuration} object. * Basic fields are read from the configuration, such as for example a file path, if the format describes * files as input.</li> * <li>It is called to create the input splits.</li> * <li>Optionally: It is called by the compiler to produce basic statistics about the input.</li> * <li>Each parallel input task creates an instance, configures it and opens it for a specific split.</li> * <li>All records are read from the input</li> * <li>The input format is closed</li> * </ol> */ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSplit> { // -------------------------------------- Constants ------------------------------------------- private static final Log LOG = LogFactory.getLog(FileInputFormat.class); private static final long serialVersionUID = 1L; /** * The fraction that the last split may be larger than the others. */ private static final float MAX_SPLIT_SIZE_DISCREPANCY = 1.1f; /** * The timeout (in milliseconds) to wait for a filesystem stream to respond. */ private static long DEFAULT_OPENING_TIMEOUT; /** * Files with that suffix are unsplittable at a file level * and compressed. */ protected static final String DEFLATE_SUFFIX = ".deflate"; /** * The splitLength is set to -1L for reading the whole split. */ protected static final long READ_WHOLE_SPLIT_FLAG = -1L; static { initDefaultsFromConfiguration(); } private static final void initDefaultsFromConfiguration() { final long to = GlobalConfiguration.getLong(ConfigConstants.FS_STREAM_OPENING_TIMEOUT_KEY, ConfigConstants.DEFAULT_FS_STREAM_OPENING_TIMEOUT); if (to < 0) { LOG.error("Invalid timeout value for filesystem stream opening: " + to + ". Using default value of " + ConfigConstants.DEFAULT_FS_STREAM_OPENING_TIMEOUT); DEFAULT_OPENING_TIMEOUT = ConfigConstants.DEFAULT_FS_STREAM_OPENING_TIMEOUT; } else if (to == 0) { DEFAULT_OPENING_TIMEOUT = 300000; // 5 minutes } else { DEFAULT_OPENING_TIMEOUT = to; } } static final long getDefaultOpeningTimeout() { return DEFAULT_OPENING_TIMEOUT; } // -------------------------------------------------------------------------------------------- // Variables for internal operation. // They are all transient, because we do not want them so be serialized // -------------------------------------------------------------------------------------------- /** * The input stream reading from the input file. */ protected transient FSDataInputStream stream; /** * The start of the split that this parallel instance must consume. */ protected transient long splitStart; /** * The length of the split that this parallel instance must consume. */ protected transient long splitLength; // -------------------------------------------------------------------------------------------- // The configuration parameters. Configured on the instance and serialized to be shipped. // -------------------------------------------------------------------------------------------- /** * The path to the file that contains the input. */ protected Path filePath; /** * The the minimal split size, set by the configure() method. */ protected long minSplitSize = 0; /** * The desired number of splits, as set by the configure() method. */ protected int numSplits = -1; /** * Stream opening timeout. */ protected long openTimeout = DEFAULT_OPENING_TIMEOUT; /** * Some file input formats are not splittable on a block level (avro, deflate) * Therefore, the FileInputFormat can only read whole files. */ protected boolean unsplittable = false; // -------------------------------------------------------------------------------------------- // Constructors // -------------------------------------------------------------------------------------------- public FileInputFormat() { } protected FileInputFormat(Path filePath) { if (filePath == null) { throw new IllegalArgumentException("The file path must not be null."); } this.filePath = filePath; } // -------------------------------------------------------------------------------------------- // Getters/setters for the configurable parameters // -------------------------------------------------------------------------------------------- public Path getFilePath() { return filePath; } public void setFilePath(String filePath) { if (filePath == null) { throw new IllegalArgumentException("File path may not be null."); } // TODO The job-submission web interface passes empty args (and thus empty // paths) to compute the preview graph. The following is a workaround for // this situation and we should fix this. if (filePath.isEmpty()) { setFilePath(new Path()); return; } setFilePath(new Path(filePath)); } public void setFilePath(Path filePath) { if (filePath == null) { throw new IllegalArgumentException("File path may not be null."); } this.filePath = filePath; } public long getMinSplitSize() { return minSplitSize; } public void setMinSplitSize(long minSplitSize) { if (minSplitSize < 0) { throw new IllegalArgumentException("The minimum split size cannot be negative."); } this.minSplitSize = minSplitSize; } public int getNumSplits() { return numSplits; } public void setNumSplits(int numSplits) { if (numSplits < -1 || numSplits == 0) { throw new IllegalArgumentException( "The desired number of splits must be positive or -1 (= don't care)."); } this.numSplits = numSplits; } public long getOpenTimeout() { return openTimeout; } public void setOpenTimeout(long openTimeout) { if (openTimeout < 0) { throw new IllegalArgumentException( "The timeout for opening the input splits must be positive or zero (= infinite)."); } this.openTimeout = openTimeout; } // -------------------------------------------------------------------------------------------- // Getting information about the split that is currently open // -------------------------------------------------------------------------------------------- /** * Gets the start of the current split. * * @return The start of the split. */ public long getSplitStart() { return splitStart; } /** * Gets the length or remaining length of the current split. * * @return The length or remaining length of the current split. */ public long getSplitLength() { return splitLength; } // -------------------------------------------------------------------------------------------- // Pre-flight: Configuration, Splits, Sampling // -------------------------------------------------------------------------------------------- /** * Configures the file input format by reading the file path from the configuration. * * @see eu.stratosphere.api.common.io.InputFormat#configure(eu.stratosphere.configuration.Configuration) */ @Override public void configure(Configuration parameters) { // get the file path String filePath = parameters.getString(FILE_PARAMETER_KEY, null); if (filePath != null) { try { this.filePath = new Path(filePath); } catch (RuntimeException rex) { throw new RuntimeException( "Could not create a valid URI from the given file path name: " + rex.getMessage()); } } else if (this.filePath == null) { throw new IllegalArgumentException("File path was not specified in input format, or configuration."); } } /** * Obtains basic file statistics containing only file size. If the input is a directory, then the size is the sum of all contained files. * * @see eu.stratosphere.api.common.io.InputFormat#getStatistics(eu.stratosphere.api.common.io.statistics.BaseStatistics) */ @Override public FileBaseStatistics getStatistics(BaseStatistics cachedStats) throws IOException { final FileBaseStatistics cachedFileStats = (cachedStats != null && cachedStats instanceof FileBaseStatistics) ? (FileBaseStatistics) cachedStats : null; try { final Path path = this.filePath; final FileSystem fs = FileSystem.get(path.toUri()); return getFileStats(cachedFileStats, path, fs, new ArrayList<FileStatus>(1)); } catch (IOException ioex) { if (LOG.isWarnEnabled()) { LOG.warn("Could not determine statistics for file '" + this.filePath + "' due to an io error: " + ioex.getMessage()); } } catch (Throwable t) { if (LOG.isErrorEnabled()) { LOG.error("Unexpected problen while getting the file statistics for file '" + this.filePath + "': " + t.getMessage(), t); } } // no statistics available return null; } protected FileBaseStatistics getFileStats(FileBaseStatistics cachedStats, Path filePath, FileSystem fs, ArrayList<FileStatus> files) throws IOException { // get the file info and check whether the cached statistics are still valid. final FileStatus file = fs.getFileStatus(filePath); long latestModTime = file.getModificationTime(); // enumerate all files and check their modification time stamp. if (file.isDir()) { FileStatus[] fss = fs.listStatus(filePath); files.ensureCapacity(fss.length); for (FileStatus s : fss) { if (!s.isDir()) { files.add(s); latestModTime = Math.max(s.getModificationTime(), latestModTime); testForUnsplittable(s); } } } else { files.add(file); testForUnsplittable(file); } // check whether the cached statistics are still valid, if we have any if (cachedStats != null && latestModTime <= cachedStats.getLastModificationTime()) { return cachedStats; } // calculate the whole length long len = 0; for (FileStatus s : files) { len += s.getLen(); } // sanity check if (len <= 0) { len = BaseStatistics.SIZE_UNKNOWN; } return new FileBaseStatistics(latestModTime, len, BaseStatistics.AVG_RECORD_BYTES_UNKNOWN); } @Override public Class<FileInputSplit> getInputSplitType() { return FileInputSplit.class; } /** * Computes the input splits for the file. By default, one file block is one split. If more splits * are requested than blocks are available, then a split may by a fraction of a block and splits may cross * block boundaries. * * @param minNumSplits The minimum desired number of file splits. * @return The computed file splits. * * @see eu.stratosphere.api.common.io.InputFormat#createInputSplits(int) */ @Override public FileInputSplit[] createInputSplits(int minNumSplits) throws IOException { if (minNumSplits < 1) { throw new IllegalArgumentException("Number of input splits has to be at least 1."); } // take the desired number of splits into account minNumSplits = Math.max(minNumSplits, this.numSplits); final Path path = this.filePath; final List<FileInputSplit> inputSplits = new ArrayList<FileInputSplit>(minNumSplits); // get all the files that are involved in the splits List<FileStatus> files = new ArrayList<FileStatus>(); long totalLength = 0; final FileSystem fs = path.getFileSystem(); final FileStatus pathFile = fs.getFileStatus(path); if (!acceptFile(pathFile)) { throw new IOException("The given file does not pass the file-filter"); } if (pathFile.isDir()) { // input is directory. list all contained files final FileStatus[] dir = fs.listStatus(path); for (int i = 0; i < dir.length; i++) { if (!dir[i].isDir() && acceptFile(dir[i])) { files.add(dir[i]); totalLength += dir[i].getLen(); // as soon as there is one deflate file in a directory, we can not split it testForUnsplittable(dir[i]); } } } else { testForUnsplittable(pathFile); files.add(pathFile); totalLength += pathFile.getLen(); } // returns if unsplittable if (unsplittable) { int splitNum = 0; for (final FileStatus file : files) { final BlockLocation[] blocks = fs.getFileBlockLocations(file, 0, file.getLen()); Set<String> hosts = new HashSet<String>(); for (BlockLocation block : blocks) { hosts.addAll(Arrays.asList(block.getHosts())); } long len = file.getLen(); if (testForUnsplittable(file)) { len = READ_WHOLE_SPLIT_FLAG; } FileInputSplit fis = new FileInputSplit(splitNum++, file.getPath(), 0, len, hosts.toArray(new String[hosts.size()])); inputSplits.add(fis); } return inputSplits.toArray(new FileInputSplit[inputSplits.size()]); } final long maxSplitSize = (minNumSplits < 1) ? Long.MAX_VALUE : (totalLength / minNumSplits + (totalLength % minNumSplits == 0 ? 0 : 1)); // now that we have the files, generate the splits int splitNum = 0; for (final FileStatus file : files) { final long len = file.getLen(); final long blockSize = file.getBlockSize(); final long minSplitSize; if (this.minSplitSize <= blockSize) { minSplitSize = this.minSplitSize; } else { if (LOG.isWarnEnabled()) { LOG.warn("Minimal split size of " + this.minSplitSize + " is larger than the block size of " + blockSize + ". Decreasing minimal split size to block size."); } minSplitSize = blockSize; } final long splitSize = Math.max(minSplitSize, Math.min(maxSplitSize, blockSize)); final long halfSplit = splitSize >>> 1; final long maxBytesForLastSplit = (long) (splitSize * MAX_SPLIT_SIZE_DISCREPANCY); if (len > 0) { // get the block locations and make sure they are in order with respect to their offset final BlockLocation[] blocks = fs.getFileBlockLocations(file, 0, len); Arrays.sort(blocks); long bytesUnassigned = len; long position = 0; int blockIndex = 0; while (bytesUnassigned > maxBytesForLastSplit) { // get the block containing the majority of the data blockIndex = getBlockIndexForPosition(blocks, position, halfSplit, blockIndex); // create a new split FileInputSplit fis = new FileInputSplit(splitNum++, file.getPath(), position, splitSize, blocks[blockIndex].getHosts()); inputSplits.add(fis); // adjust the positions position += splitSize; bytesUnassigned -= splitSize; } // assign the last split if (bytesUnassigned > 0) { blockIndex = getBlockIndexForPosition(blocks, position, halfSplit, blockIndex); final FileInputSplit fis = new FileInputSplit(splitNum++, file.getPath(), position, bytesUnassigned, blocks[blockIndex].getHosts()); inputSplits.add(fis); } } else { // special case with a file of zero bytes size final BlockLocation[] blocks = fs.getFileBlockLocations(file, 0, 0); String[] hosts; if (blocks.length > 0) { hosts = blocks[0].getHosts(); } else { hosts = new String[0]; } final FileInputSplit fis = new FileInputSplit(splitNum++, file.getPath(), 0, 0, hosts); inputSplits.add(fis); } } return inputSplits.toArray(new FileInputSplit[inputSplits.size()]); } private boolean testForUnsplittable(FileStatus pathFile) { if (pathFile.getPath().getName().endsWith(DEFLATE_SUFFIX)) { unsplittable = true; return true; } return false; } /** * A simple hook to filter files and directories from the input. * The method may be overridden. Hadoop's FileInputFormat has a similar mechanism and applies the * same filters by default. * * @param fileStatus * @return true, if the given file or directory is accepted */ protected boolean acceptFile(FileStatus fileStatus) { final String name = fileStatus.getPath().getName(); return !name.startsWith("_") && !name.startsWith("."); } /** * Retrieves the index of the <tt>BlockLocation</tt> that contains the part of the file described by the given * offset. * * @param blocks The different blocks of the file. Must be ordered by their offset. * @param offset The offset of the position in the file. * @param startIndex The earliest index to look at. * @return The index of the block containing the given position. */ private int getBlockIndexForPosition(BlockLocation[] blocks, long offset, long halfSplitSize, int startIndex) { // go over all indexes after the startIndex for (int i = startIndex; i < blocks.length; i++) { long blockStart = blocks[i].getOffset(); long blockEnd = blockStart + blocks[i].getLength(); if (offset >= blockStart && offset < blockEnd) { // got the block where the split starts // check if the next block contains more than this one does if (i < blocks.length - 1 && blockEnd - offset < halfSplitSize) { return i + 1; } else { return i; } } } throw new IllegalArgumentException("The given offset is not contained in the any block."); } // -------------------------------------------------------------------------------------------- /** * Opens an input stream to the file defined in the input format. * The stream is positioned at the beginning of the given split. * <p> * The stream is actually opened in an asynchronous thread to make sure any interruptions to the thread * working on the input format do not reach the file system. */ @Override public void open(FileInputSplit split) throws IOException { if (!(split instanceof FileInputSplit)) { throw new IllegalArgumentException("File Input Formats can only be used with FileInputSplits."); } final FileInputSplit fileSplit = (FileInputSplit) split; this.splitStart = fileSplit.getStart(); this.splitLength = fileSplit.getLength(); if (LOG.isDebugEnabled()) { LOG.debug("Opening input split " + fileSplit.getPath() + " [" + this.splitStart + "," + this.splitLength + "]"); } // open the split in an asynchronous thread final InputSplitOpenThread isot = new InputSplitOpenThread(fileSplit, this.openTimeout); isot.start(); try { this.stream = isot.waitForCompletion(); // Wrap stream in a extracting (decompressing) stream if file ends with .deflate. if (fileSplit.getPath().getName().endsWith(DEFLATE_SUFFIX)) { this.stream = new InflaterInputStreamFSInputWrapper(stream); } } catch (Throwable t) { throw new IOException("Error opening the Input Split " + fileSplit.getPath() + " [" + splitStart + "," + splitLength + "]: " + t.getMessage(), t); } // get FSDataInputStream if (this.splitStart != 0) { this.stream.seek(this.splitStart); } } /** * Closes the file input stream of the input format. */ @Override public void close() throws IOException { if (this.stream != null) { // close input stream this.stream.close(); stream = null; } } public String toString() { return this.filePath == null ? "File Input (unknown file)" : "File Input (" + this.filePath.toString() + ')'; } // ============================================================================================ /** * Encapsulation of the basic statistics the optimizer obtains about a file. Contained are the size of the file * and the average bytes of a single record. The statistics also have a time-stamp that records the modification * time of the file and indicates as such for which time the statistics were valid. */ public static class FileBaseStatistics implements BaseStatistics { protected final long fileModTime; // timestamp of the last modification protected final long fileSize; // size of the file(s) in bytes protected final float avgBytesPerRecord; // the average number of bytes for a record /** * Creates a new statistics object. * * @param fileModTime * The timestamp of the latest modification of any of the involved files. * @param fileSize * The size of the file, in bytes. <code>-1</code>, if unknown. * @param avgBytesPerRecord * The average number of byte in a record, or <code>-1.0f</code>, if unknown. */ public FileBaseStatistics(long fileModTime, long fileSize, float avgBytesPerRecord) { this.fileModTime = fileModTime; this.fileSize = fileSize; this.avgBytesPerRecord = avgBytesPerRecord; } /** * Gets the timestamp of the last modification. * * @return The timestamp of the last modification. */ public long getLastModificationTime() { return fileModTime; } /** * Gets the file size. * * @return The fileSize. * @see eu.stratosphere.api.common.io.statistics.BaseStatistics#getTotalInputSize() */ @Override public long getTotalInputSize() { return this.fileSize; } /** * Gets the estimates number of records in the file, computed as the file size divided by the * average record width, rounded up. * * @return The estimated number of records in the file. * @see eu.stratosphere.api.common.io.statistics.BaseStatistics#getNumberOfRecords() */ @Override public long getNumberOfRecords() { return (this.fileSize == SIZE_UNKNOWN || this.avgBytesPerRecord == AVG_RECORD_BYTES_UNKNOWN) ? NUM_RECORDS_UNKNOWN : (long) Math.ceil(this.fileSize / this.avgBytesPerRecord); } /** * Gets the estimated average number of bytes per record. * * @return The average number of bytes per record. * @see eu.stratosphere.api.common.io.statistics.BaseStatistics#getAverageRecordWidth() */ @Override public float getAverageRecordWidth() { return this.avgBytesPerRecord; } @Override public String toString() { return "size=" + this.fileSize + ", recWidth=" + this.avgBytesPerRecord + ", modAt=" + this.fileModTime; } } // ============================================================================================ /** * Obtains a DataInputStream in an thread that is not interrupted. * This is a necessary hack around the problem that the HDFS client is very sensitive to InterruptedExceptions. */ public static class InputSplitOpenThread extends Thread { private final FileInputSplit split; private final long timeout; private volatile FSDataInputStream fdis; private volatile Throwable error; private volatile boolean aborted; public InputSplitOpenThread(FileInputSplit split, long timeout) { super("Transient InputSplit Opener"); setDaemon(true); this.split = split; this.timeout = timeout; } @Override public void run() { try { final FileSystem fs = FileSystem.get(this.split.getPath().toUri()); this.fdis = fs.open(this.split.getPath()); // check for canceling and close the stream in that case, because no one will obtain it if (this.aborted) { final FSDataInputStream f = this.fdis; this.fdis = null; f.close(); } } catch (Throwable t) { this.error = t; } } public FSDataInputStream waitForCompletion() throws Throwable { final long start = System.currentTimeMillis(); long remaining = this.timeout; do { try { // wait for the task completion this.join(remaining); } catch (InterruptedException iex) { // we were canceled, so abort the procedure abortWait(); throw iex; } } while (this.error == null && this.fdis == null && (remaining = this.timeout + start - System.currentTimeMillis()) > 0); if (this.error != null) { throw this.error; } if (this.fdis != null) { return this.fdis; } else { // double-check that the stream has not been set by now. we don't know here whether // a) the opener thread recognized the canceling and closed the stream // b) the flag was set such that the stream did not see it and we have a valid stream // In any case, close the stream and throw an exception. abortWait(); final boolean stillAlive = this.isAlive(); final StringBuilder bld = new StringBuilder(256); for (StackTraceElement e : this.getStackTrace()) { bld.append("\tat ").append(e.toString()).append('\n'); } throw new IOException("Input opening request timed out. Opener was " + (stillAlive ? "" : "NOT ") + " alive. Stack of split open thread:\n" + bld.toString()); } } /** * Double checked procedure setting the abort flag and closing the stream. */ private final void abortWait() { this.aborted = true; final FSDataInputStream inStream = this.fdis; this.fdis = null; if (inStream != null) { try { inStream.close(); } catch (Throwable t) { } } } } // ============================================================================================ // Parameterization via configuration // ============================================================================================ // ------------------------------------- Config Keys ------------------------------------------ /** * The config parameter which defines the input file path. */ private static final String FILE_PARAMETER_KEY = "input.file.path"; // ----------------------------------- Config Builder ----------------------------------------- /** * Creates a configuration builder that can be used to set the input format's parameters to the config in a fluent * fashion. * * @return A config builder for setting parameters. */ public static ConfigBuilder configureFileFormat(GenericDataSourceBase<?, ?> target) { return new ConfigBuilder(target.getParameters()); } /** * Abstract builder used to set parameters to the input format's configuration in a fluent way. */ protected static abstract class AbstractConfigBuilder<T> { /** * The configuration into which the parameters will be written. */ protected final Configuration config; // -------------------------------------------------------------------- /** * Creates a new builder for the given configuration. * * @param targetConfig The configuration into which the parameters will be written. */ protected AbstractConfigBuilder(Configuration targetConfig) { this.config = targetConfig; } // -------------------------------------------------------------------- /** * Sets the path to the file or directory to be read by this file input format. * * @param filePath The path to the file or directory. * @return The builder itself. */ public T filePath(String filePath) { this.config.setString(FILE_PARAMETER_KEY, filePath); @SuppressWarnings("unchecked") T ret = (T) this; return ret; } } /** * A builder used to set parameters to the input format's configuration in a fluent way. */ public static class ConfigBuilder extends AbstractConfigBuilder<ConfigBuilder> { /** * Creates a new builder for the given configuration. * * @param targetConfig The configuration into which the parameters will be written. */ protected ConfigBuilder(Configuration targetConfig) { super(targetConfig); } } }