提交 c56be665 编写于 作者: V vasia 提交者: Fabian Hueske

[FLINK-1307] Allow file input from nested directory structure

This closes #260
上级 13968cd4
......@@ -1584,7 +1584,7 @@ Data Sources
<div data-lang="java" markdown="1">
Data sources create the initial data sets, such as from files or from Java collections. The general
mechanism of of creating data sets is abstracted behind an
mechanism of creating data sets is abstracted behind an
{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java "InputFormat"%}.
Flink comes
with several built-in formats to create data sets from common file formats. Many of them have
......@@ -1662,11 +1662,30 @@ DataSet<Tuple2<String, Integer> dbData =
// manually provide the type information as shown in the examples above.
{% endhighlight %}
#### Recursive Traversal of the Input Path Directory
For file-based inputs, when the input path is a directory, nested files are not enumerated by default. Instead, only the files inside the base directory are read, while nested files are ignored. Recursive enumeration of nested files can be enabled through the `recursive.file.enumeration` configuration parameter, like in the following example.
{% highlight java %}
// enable recursive enumeration of nested input files
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
// create a configuration object
Configuration parameters = new Configuration();
// set the recursive enumeration parameter
parameters.setBoolean("recursive.file.enumeration", true);
// pass the configuration to the data source
DataSet<String> logs = env.readTextFile("file:///path/with.nested/files")
.withParameters(parameters);
{% endhighlight %}
</div>
<div data-lang="scala" markdown="1">
Data sources create the initial data sets, such as from files or from Java collections. The general
mechanism of of creating data sets is abstracted behind an
mechanism of creating data sets is abstracted behind an
{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java "InputFormat"%}.
Flink comes
with several built-in formats to create data sets from common file formats. Many of them have
......@@ -1730,9 +1749,27 @@ val values = env.fromElements("Foo", "bar", "foobar", "fubar")
// generate a number sequence
val numbers = env.generateSequence(1, 10000000);
{% endhighlight %}
#### Recursive Traversal of the Input Path Directory
For file-based inputs, when the input path is a directory, nested files are not enumerated by default. Instead, only the files inside the base directory are read, while nested files are ignored. Recursive enumeration of nested files can be enabled through the `recursive.file.enumeration` configuration parameter, like in the following example.
{% highlight scala %}
// enable recursive enumeration of nested input files
val env = ExecutionEnvironment.getExecutionEnvironment
// create a configuration object
val parameters = new Configuration
// set the recursive enumeration parameter
parameters.setBoolean("recursive.file.enumeration", true)
// pass the configuration to the data source
env.readTextFile("file:///path/with.nested/files").withParameters(parameters)
{% endhighlight %}
</div>
</div>
[Back to top](#top)
Data Sinks
......
......@@ -66,7 +66,7 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
* 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.
......@@ -151,6 +151,12 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
* Therefore, the FileInputFormat can only read whole files.
*/
protected boolean unsplittable = false;
/**
* The flag to specify whether recursive traversal of the input directory
* structure is enabled.
*/
protected boolean enumerateNestedFiles = false;
// --------------------------------------------------------------------------------------------
// Constructors
......@@ -231,7 +237,7 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
}
this.openTimeout = openTimeout;
}
// --------------------------------------------------------------------------------------------
// Getting information about the split that is currently open
// --------------------------------------------------------------------------------------------
......@@ -278,6 +284,9 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
else if (this.filePath == null) {
throw new IllegalArgumentException("File path was not specified in input format, or configuration.");
}
Boolean nestedFilesFlag = parameters.getBoolean(ENUMERATE_NESTED_FILES_FLAG, false);
this.enumerateNestedFiles = nestedFilesFlag;
}
/**
......@@ -319,7 +328,11 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
// get the file info and check whether the cached statistics are still valid.
final FileStatus file = fs.getFileStatus(filePath);
long latestModTime = file.getModificationTime();
long totalLength = 0;
if(!acceptFile(file)) {
throw new IOException("The given file does not pass the file-filter");
}
// enumerate all files and check their modification time stamp.
if (file.isDir()) {
FileStatus[] fss = fs.listStatus(filePath);
......@@ -327,33 +340,35 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
for (FileStatus s : fss) {
if (!s.isDir()) {
files.add(s);
latestModTime = Math.max(s.getModificationTime(), latestModTime);
testForUnsplittable(s);
if (acceptFile(s)) {
files.add(s);
totalLength += s.getLen();
latestModTime = Math.max(s.getModificationTime(), latestModTime);
testForUnsplittable(s);
}
}
else {
if (enumerateNestedFiles) {
totalLength += addNestedFiles(s.getPath(), files, 0);
}
}
}
} else {
files.add(file);
testForUnsplittable(file);
totalLength += file.getLen();
}
// 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;
if (totalLength <= 0) {
totalLength = BaseStatistics.SIZE_UNKNOWN;
}
return new FileBaseStatistics(latestModTime, len, BaseStatistics.AVG_RECORD_BYTES_UNKNOWN);
return new FileBaseStatistics(latestModTime, totalLength, BaseStatistics.AVG_RECORD_BYTES_UNKNOWN);
}
@Override
......@@ -363,7 +378,7 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
/**
* 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
* are requested than blocks are available, then a split may be a fraction of a block and splits may cross
* block boundaries.
*
* @param minNumSplits The minimum desired number of file splits.
......@@ -397,16 +412,23 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
// 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]);
if (dir[i].isDir()) {
if (enumerateNestedFiles) {
totalLength += addNestedFiles(dir[i].getPath(), files, 0);
}
}
else {
if (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();
}
......@@ -506,6 +528,30 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
return inputSplits.toArray(new FileInputSplit[inputSplits.size()]);
}
/**
* Recursively traverse the input directory structure
* and enumerate all accepted nested files.
* @return the total length of accepted files.
*/
private long addNestedFiles(Path path, List<FileStatus> files, long length)
throws IOException {
final FileSystem fs = path.getFileSystem();
for(FileStatus dir: fs.listStatus(path)) {
if (dir.isDir()) {
addNestedFiles(dir.getPath(), files, length);
}
else {
if(acceptFile(dir)) {
files.add(dir);
length += dir.getLen();
testForUnsplittable(dir);
}
}
}
return length;
}
private boolean testForUnsplittable(FileStatus pathFile) {
if(pathFile.getPath().getName().endsWith(DEFLATE_SUFFIX)) {
unsplittable = true;
......@@ -807,6 +853,11 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
* The config parameter which defines the input file path.
*/
private static final String FILE_PARAMETER_KEY = "input.file.path";
/**
* The config parameter which defines whether input directories are recursively traversed.
*/
private static final String ENUMERATE_NESTED_FILES_FLAG = "recursive.file.enumeration";
// ----------------------------------- Config Builder -----------------------------------------
......
/*
* 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.api.common.io;
import java.io.File;
import java.io.IOException;
import org.apache.flink.api.common.io.statistics.BaseStatistics;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.FileInputSplit;
import org.apache.flink.core.fs.Path;
import org.apache.flink.testutils.TestFileUtils;
import org.apache.flink.types.IntValue;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class EnumerateNestedFilesTest {
protected Configuration config;
final String tempPath = System.getProperty("java.io.tmpdir");
private final DummyFileInputFormat format = new DummyFileInputFormat();
@Before
public void setup() {
this.config = new Configuration();
}
@After
public void setdown() throws Exception {
if (this.format != null) {
this.format.close();
}
}
/**
* Test without nested directory and recursive.file.enumeration = true
*/
@Test
public void testNoNestedDirectoryTrue() {
try {
String filePath = TestFileUtils.createTempFile("foo");
this.format.setFilePath(new Path(filePath));
this.config.setBoolean("recursive.file.enumeration", true);
format.configure(this.config);
FileInputSplit[] splits = format.createInputSplits(1);
Assert.assertEquals(1, splits.length);
} catch (Exception ex) {
ex.printStackTrace();
Assert.fail(ex.getMessage());
}
}
/**
* Test with one nested directory and recursive.file.enumeration = true
*/
@Test
public void testOneNestedDirectoryTrue() {
try {
String firstLevelDir = TestFileUtils.randomFileName();
String secondLevelDir = TestFileUtils.randomFileName();
File nestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir);
nestedDir.mkdirs();
nestedDir.deleteOnExit();
File insideNestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir + System.getProperty("file.separator") + secondLevelDir);
insideNestedDir.mkdirs();
insideNestedDir.deleteOnExit();
// create a file in the first-level and two files in the nested dir
TestFileUtils.createTempFileInDirectory(nestedDir.getAbsolutePath(), "paella");
TestFileUtils.createTempFileInDirectory(insideNestedDir.getAbsolutePath(), "kalamari");
TestFileUtils.createTempFileInDirectory(insideNestedDir.getAbsolutePath(), "fideua");
this.format.setFilePath(new Path(nestedDir.toURI().toString()));
this.config.setBoolean("recursive.file.enumeration", true);
format.configure(this.config);
FileInputSplit[] splits = format.createInputSplits(1);
Assert.assertEquals(3, splits.length);
} catch (Exception ex) {
ex.printStackTrace();
Assert.fail(ex.getMessage());
}
}
/**
* Test with one nested directory and recursive.file.enumeration = false
*/
@Test
public void testOneNestedDirectoryFalse() {
try {
String firstLevelDir = TestFileUtils.randomFileName();
String secondLevelDir = TestFileUtils.randomFileName();
File nestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir);
nestedDir.mkdirs();
nestedDir.deleteOnExit();
File insideNestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir + System.getProperty("file.separator") + secondLevelDir);
insideNestedDir.mkdirs();
insideNestedDir.deleteOnExit();
// create a file in the first-level and two files in the nested dir
TestFileUtils.createTempFileInDirectory(nestedDir.getAbsolutePath(), "paella");
TestFileUtils.createTempFileInDirectory(insideNestedDir.getAbsolutePath(), "kalamari");
TestFileUtils.createTempFileInDirectory(insideNestedDir.getAbsolutePath(), "fideua");
this.format.setFilePath(new Path(nestedDir.toURI().toString()));
this.config.setBoolean("recursive.file.enumeration", false);
format.configure(this.config);
FileInputSplit[] splits = format.createInputSplits(1);
Assert.assertEquals(1, splits.length);
} catch (Exception ex) {
ex.printStackTrace();
Assert.fail(ex.getMessage());
}
}
/**
* Test with two nested directories and recursive.file.enumeration = true
*/
@Test
public void testTwoNestedDirectoriesTrue() {
try {
String firstLevelDir = TestFileUtils.randomFileName();
String secondLevelDir = TestFileUtils.randomFileName();
String thirdLevelDir = TestFileUtils.randomFileName();
File nestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir);
nestedDir.mkdirs();
nestedDir.deleteOnExit();
File insideNestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir + System.getProperty("file.separator") + secondLevelDir);
insideNestedDir.mkdirs();
insideNestedDir.deleteOnExit();
File nestedNestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir + System.getProperty("file.separator") + secondLevelDir
+ System.getProperty("file.separator") + thirdLevelDir);
nestedNestedDir.mkdirs();
nestedNestedDir.deleteOnExit();
// create a file in the first-level, two files in the second level and one in the third level
TestFileUtils.createTempFileInDirectory(nestedDir.getAbsolutePath(), "paella");
TestFileUtils.createTempFileInDirectory(insideNestedDir.getAbsolutePath(), "kalamari");
TestFileUtils.createTempFileInDirectory(insideNestedDir.getAbsolutePath(), "fideua");
TestFileUtils.createTempFileInDirectory(nestedNestedDir.getAbsolutePath(), "bravas");
this.format.setFilePath(new Path(nestedDir.toURI().toString()));
this.config.setBoolean("recursive.file.enumeration", true);
format.configure(this.config);
FileInputSplit[] splits = format.createInputSplits(1);
Assert.assertEquals(4, splits.length);
} catch (Exception ex) {
ex.printStackTrace();
Assert.fail(ex.getMessage());
}
}
@Test
public void testGetStatisticsOneFileInNestedDir() {
try {
final long SIZE = 1024 * 500;
String firstLevelDir = TestFileUtils.randomFileName();
String secondLevelDir = TestFileUtils.randomFileName();
File nestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir);
nestedDir.mkdirs();
nestedDir.deleteOnExit();
File insideNestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir + System.getProperty("file.separator") + secondLevelDir);
insideNestedDir.mkdirs();
insideNestedDir.deleteOnExit();
// create a file in the nested dir
TestFileUtils.createTempFileInDirectory(insideNestedDir.getAbsolutePath(), SIZE);
this.format.setFilePath(new Path(nestedDir.toURI().toString()));
this.config.setBoolean("recursive.file.enumeration", true);
format.configure(this.config);
BaseStatistics stats = format.getStatistics(null);
Assert.assertEquals("The file size from the statistics is wrong.", SIZE, stats.getTotalInputSize());
} catch (Exception ex) {
ex.printStackTrace();
Assert.fail(ex.getMessage());
}
}
@Test
public void testGetStatisticsMultipleNestedFiles() {
try {
final long SIZE1 = 2077;
final long SIZE2 = 31909;
final long SIZE3 = 10;
final long TOTAL = SIZE1 + SIZE2 + SIZE3;
String firstLevelDir = TestFileUtils.randomFileName();
String secondLevelDir = TestFileUtils.randomFileName();
File nestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir);
nestedDir.mkdirs();
nestedDir.deleteOnExit();
File insideNestedDir = new File(tempPath + System.getProperty("file.separator")
+ firstLevelDir + System.getProperty("file.separator") + secondLevelDir);
insideNestedDir.mkdirs();
insideNestedDir.deleteOnExit();
// create a file in the first-level and two files in the nested dir
TestFileUtils.createTempFileInDirectory(nestedDir.getAbsolutePath(), SIZE1);
TestFileUtils.createTempFileInDirectory(insideNestedDir.getAbsolutePath(), SIZE2);
TestFileUtils.createTempFileInDirectory(insideNestedDir.getAbsolutePath(), SIZE3);
this.format.setFilePath(new Path(nestedDir.toURI().toString()));
this.config.setBoolean("recursive.file.enumeration", true);
format.configure(this.config);
BaseStatistics stats = format.getStatistics(null);
Assert.assertEquals("The file size from the statistics is wrong.", TOTAL, stats.getTotalInputSize());
} catch (Exception ex) {
ex.printStackTrace();
Assert.fail(ex.getMessage());
}
}
// ------------------------------------------------------------------------
private class DummyFileInputFormat extends FileInputFormat<IntValue> {
private static final long serialVersionUID = 1L;
@Override
public boolean reachedEnd() throws IOException {
return true;
}
@Override
public IntValue nextRecord(IntValue reuse) throws IOException {
return null;
}
}
}
\ No newline at end of file
......@@ -18,8 +18,10 @@
package org.apache.flink.api.common.io;
import java.io.BufferedOutputStream;
import java.io.BufferedWriter;
import java.io.File;
import java.io.FileOutputStream;
import java.io.FileWriter;
import java.io.IOException;
import java.net.URI;
......@@ -288,6 +290,55 @@ public class FileInputFormatTest {
Assert.fail(e.getMessage());
}
}
@Test
public void testGetStatsIgnoredUnderscoreFiles() {
try {
final long SIZE = 2048;
final long TOTAL = 2*SIZE;
// create two accepted and two ignored files
File tempDir = new File(System.getProperty("java.io.tmpdir"));
File f = null;
do {
f = new File(tempDir, TestFileUtils.randomFileName(""));
} while (f.exists());
f.mkdirs();
f.deleteOnExit();
File child1 = new File(f, "dataFile1.txt");
File child2 = new File(f, "another_file.bin");
File luigiFile = new File(f, "_luigi");
File success = new File(f, "_SUCCESS");
File[] files = { child1, child2, luigiFile, success };
for (File child : files) {
child.deleteOnExit();
BufferedOutputStream out = new BufferedOutputStream(new FileOutputStream(child));
try {
for (long bytes = SIZE; bytes > 0; bytes--) {
out.write(0);
}
} finally {
out.close();
}
}
final DummyFileInputFormat format = new DummyFileInputFormat();
format.setFilePath(f.toURI().toString());
format.configure(new Configuration());
// check that only valid files are used for statistics computation
BaseStatistics stats = format.getStatistics(null);
Assert.assertEquals(TOTAL, stats.getTotalInputSize());
}
catch (Exception e) {
System.err.println(e.getMessage());
e.printStackTrace();
Assert.fail(e.getMessage());
}
}
// ------------------------------------------------------------------------
......
......@@ -63,7 +63,27 @@ public class TestFileUtils {
}
return f.toURI().toString();
}
public static String createTempFileInDirectory(String dir, long bytes) throws IOException {
File f;
do {
f = new File(dir + "/" + randomFileName());
} while (f.exists());
f.getParentFile().mkdirs();
f.createNewFile();
f.deleteOnExit();
BufferedOutputStream out = new BufferedOutputStream(new FileOutputStream(f));
try {
for (; bytes > 0; bytes--) {
out.write(0);
}
} finally {
out.close();
}
return f.toURI().toString();
}
public static String createTempFile(String contents) throws IOException {
File f = File.createTempFile(FILE_PREFIX, FILE_SUFFIX);
f.deleteOnExit();
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册