in DatasourceInputFormat code for determining segment block locations avoid the split calulation by helper TextInputFormat

This commit is contained in:
Himanshu Gupta 2016-03-10 14:24:48 -06:00
parent 68cffe1d91
commit eab8a0b54d
1 changed files with 11 additions and 1 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.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapred.FileInputFormat;
@ -139,7 +140,16 @@ public class DatasourceInputFormat extends InputFormat<NullWritable, InputRow>
@Override
public org.apache.hadoop.mapred.InputFormat get()
{
return new TextInputFormat();
return new TextInputFormat()
{
//Always consider non-splittable as we only want to get location of blocks for the segment
//and not consider the splitting.
//also without this, isSplitable(..) fails with NPE because compressionCodecs is not properly setup.
@Override
protected boolean isSplitable(FileSystem fs, Path file) {
return false;
}
};
}
};