Avoid exceptions for dataSource spec when using s3 (#3544)

This commit is contained in:
Navis Ryu 2016-10-15 10:24:19 +09:00 committed by Gian Merlino
parent 5dc95389f7
commit 4554c1214b
1 changed files with 13 additions and 0 deletions

View File

@ -34,6 +34,7 @@ import io.druid.data.input.InputRow;
import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.indexer.JobHelper;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.NullWritable;
@ -159,6 +160,18 @@ public class DatasourceInputFormat extends InputFormat<NullWritable, InputRow>
protected boolean isSplitable(FileSystem fs, Path file) {
return false;
}
@Override
protected FileStatus[] listStatus(JobConf job) throws IOException
{
// to avoid globbing which needs input path should be hadoop-compatible (':' is not acceptable in path, etc.)
List<FileStatus> statusList = Lists.newArrayList();
for (Path path : FileInputFormat.getInputPaths(job)) {
// load spec in segment points specifically zip file itself
statusList.add(path.getFileSystem(job).getFileStatus(path));
}
return statusList.toArray(new FileStatus[statusList.size()]);
}
};
}
};