inputFormat.setFilePath(tempFile.toURI().toString()); inputFormat.configure(config); inputFormat.openInputFormat(); FileInputSplit[] inputSplits = inputFormat.createInputSplits(3); inputFormat.open(inputSplit); while (!inputFormat.reachedEnd()) { if ((bytes = inputFormat.nextRecord(bytes)) != null) { Assert.assertArrayEquals(new byte[]{--prev}, bytes); inputFormat.closeInputFormat();
files.put(filePath, status); } else if (format.getNestedFileEnumeration() && format.acceptFile(status)){ files.putAll(listEligibleFiles(fileSystem, status.getPath()));
/** * Configures the file input format by reading the file path from the configuration. * * @see org.apache.flink.api.common.io.InputFormat#configure(org.apache.flink.configuration.Configuration) */ @Override public void configure(Configuration parameters) { if (getFilePaths().length == 0) { // file path was not specified yet. Try to set it from the parameters. String filePath = parameters.getString(FILE_PARAMETER_KEY, null); if (filePath == null) { throw new IllegalArgumentException("File path was not specified in input format or configuration."); } else { setFilePath(filePath); } } if (!this.enumerateNestedFiles) { this.enumerateNestedFiles = parameters.getBoolean(ENUMERATE_NESTED_FILES_FLAG, false); } }
/** * 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 org.apache.flink.api.common.io.InputFormat#getStatistics(org.apache.flink.api.common.io.statistics.BaseStatistics) */ @Override public FileBaseStatistics getStatistics(BaseStatistics cachedStats) throws IOException { final FileBaseStatistics cachedFileStats = cachedStats instanceof FileBaseStatistics ? (FileBaseStatistics) cachedStats : null; try { return getFileStats(cachedFileStats, getFilePaths(), new ArrayList<>(getFilePaths().length)); } catch (IOException ioex) { if (LOG.isWarnEnabled()) { LOG.warn("Could not determine statistics for paths '" + Arrays.toString(getFilePaths()) + "' due to an io error: " + ioex.getMessage()); } } catch (Throwable t) { if (LOG.isErrorEnabled()) { LOG.error("Unexpected problem while getting the file statistics for paths '" + Arrays.toString(getFilePaths()) + "': " + t.getMessage(), t); } } // no statistics available return null; }
if (acceptFile(dir) && enumerateNestedFiles) { length += addFilesInDir(dir.getPath(), files, logExcludedFiles); } else { if (logExcludedFiles && LOG.isDebugEnabled()) { if(acceptFile(dir)) { files.add(dir); length += dir.getLen(); testForUnsplittable(dir); } else { if (logExcludedFiles && LOG.isDebugEnabled()) {
protected FileInputFormat(Path filePath) { if (filePath != null) { setFilePath(filePath); } }
@Override public void open(FileInputSplit split) throws IOException { super.open(split); dataFileReader = initReader(split); dataFileReader.sync(split.getStart()); lastSync = dataFileReader.previousSync(); }
@Test public void testCreateInputSplitSingleFile() throws IOException { String tempFile = TestFileUtils.createTempFile("Hello World"); FileInputFormat fif = new DummyFileInputFormat(); fif.setFilePath(tempFile); fif.configure(new Configuration()); FileInputSplit[] splits = fif.createInputSplits(2); Assert.assertEquals(2, splits.length); Assert.assertEquals(tempFile, splits[0].getPath().toString()); Assert.assertEquals(tempFile, splits[1].getPath().toString()); }
@Override public void open(Configuration parameters) throws Exception { super.open(parameters); format.configure(parameters); if (LOG.isDebugEnabled()) { LOG.debug("Opened {} (taskIdx= {}) for path: {}", getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), path); } }
@Override public void open() throws Exception { super.open(); checkState(this.reader == null, "The reader is already initialized."); checkState(this.serializer != null, "The serializer has not been set. " + "Probably the setOutputType() was not called. Please report it."); this.format.setRuntimeContext(getRuntimeContext()); this.format.configure(new Configuration()); this.checkpointLock = getContainingTask().getCheckpointLock(); // set the reader context based on the time characteristic final TimeCharacteristic timeCharacteristic = getOperatorConfig().getTimeCharacteristic(); final long watermarkInterval = getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval(); this.readerContext = StreamSourceContexts.getSourceContext( timeCharacteristic, getProcessingTimeService(), checkpointLock, getContainingTask().getStreamStatusMaintainer(), output, watermarkInterval, -1); // and initialize the split reading thread this.reader = new SplitReader<>(format, serializer, readerContext, checkpointLock, restoredReaderState); this.restoredReaderState = null; this.reader.start(); }
String tempFile3 = TestFileUtils.createTempFile(23); FileInputFormat fif = new MultiDummyFileInputFormat(); fif.setFilePaths(tempFile1, tempFile2, tempFile3); fif.configure(new Configuration()); FileInputSplit[] splits = fif.createInputSplits(3);
long totalLength = 0; for (Path path : getFilePaths()) { final FileSystem fs = path.getFileSystem(); final FileStatus pathFile = fs.getFileStatus(path); totalLength += addFilesInDir(path, files, true); } else { testForUnsplittable(pathFile); if(testForUnsplittable(file)) { len = READ_WHOLE_SPLIT_FLAG; blockIndex = getBlockIndexForPosition(blocks, position, halfSplit, blockIndex); blockIndex = getBlockIndexForPosition(blocks, position, halfSplit, blockIndex); final FileInputSplit fis = new FileInputSplit(splitNum++, file.getPath(), position, bytesUnassigned, blocks[blockIndex].getHosts());
public String toString() { return getFilePaths() == null || getFilePaths().length == 0 ? "File Input (unknown file)" : "File Input (" + Arrays.toString(this.getFilePaths()) + ')'; }
long interval, FilePathFilter filter) { inputFormat.setFilesFilter(filter);
/** * Creates the input splits to be forwarded to the downstream tasks of the * {@link ContinuousFileReaderOperator}. Splits are sorted <b>by modification time</b> before * being forwarded and only splits belonging to files in the {@code eligibleFiles} * list will be processed. * @param eligibleFiles The files to process. */ private Map<Long, List<TimestampedFileInputSplit>> getInputSplitsSortedByModTime( Map<Path, FileStatus> eligibleFiles) throws IOException { Map<Long, List<TimestampedFileInputSplit>> splitsByModTime = new TreeMap<>(); if (eligibleFiles.isEmpty()) { return splitsByModTime; } for (FileInputSplit split: format.createInputSplits(readerParallelism)) { FileStatus fileStatus = eligibleFiles.get(split.getPath()); if (fileStatus != null) { Long modTime = fileStatus.getModificationTime(); List<TimestampedFileInputSplit> splitsToForward = splitsByModTime.get(modTime); if (splitsToForward == null) { splitsToForward = new ArrayList<>(); splitsByModTime.put(modTime, splitsToForward); } splitsToForward.add(new TimestampedFileInputSplit( modTime, split.getSplitNumber(), split.getPath(), split.getStart(), split.getLength(), split.getHostnames())); } } return splitsByModTime; }
if (enumerateNestedFiles && acceptFile(fileStatus)) { tasks.add(executorService.submit(() -> { try { getFileStatusInternal(fs, fileStatus.getPath(), files, tasks, executorService, logExcludedFiles); } catch (IOException e) { LOG.error("Fail to get file status.", e); if (acceptFile(fileStatus)) { files.add(fileStatus); testForUnsplittable(fileStatus); } else { if (logExcludedFiles && LOG.isDebugEnabled()) {
LOG.info("Start to get all files"); final Path[] paths = getFilePaths(); getFileStatus(paths, files, executorService); final FileInputSplit[] inputSplits = createFileInputSplits(minNumSplits, files, executorService);
this.stream = decorateInputStream(this.stream, fileSplit);
public void setFilePath(String filePath) { if (filePath == null) { throw new IllegalArgumentException("File path cannot 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. // comment (Stephan Ewen) this should be no longer relevant with the current Java/Scala APIs. if (filePath.isEmpty()) { setFilePath(new Path()); return; } try { this.setFilePath(new Path(filePath)); } catch (RuntimeException rex) { throw new RuntimeException("Could not create a valid URI from the given file path name: " + rex.getMessage()); } }
/** * Opens the given input split. This method opens the input stream to the specified file, allocates read buffers * and positions the stream at the correct position, making sure that any partial record at the beginning is skipped. * * @param split The input split to open. * * @see org.apache.flink.api.common.io.FileInputFormat#open(org.apache.flink.core.fs.FileInputSplit) */ @Override public void open(FileInputSplit split) throws IOException { super.open(split); initBuffers(); this.offset = splitStart; if (this.splitStart != 0) { this.stream.seek(offset); readLine(); // if the first partial record already pushes the stream over // the limit of our split, then no record starts within this split if (this.overLimit) { this.end = true; } } else { fillBuffer(0); } }