|
||||||||||
| PREV CLASS NEXT CLASS | FRAMES NO FRAMES | |||||||||
| SUMMARY: NESTED | FIELD | CONSTR | METHOD | DETAIL: FIELD | CONSTR | METHOD | |||||||||
java.lang.Objectorg.apache.hadoop.mapreduce.InputFormat<K,V>
org.apache.hadoop.mapreduce.lib.input.FileInputFormat<LongWritable,Text>
org.apache.hadoop.mapreduce.lib.input.NLineInputFormat
@InterfaceAudience.Public @InterfaceStability.Stable public class NLineInputFormat
NLineInputFormat which splits N lines of input as one split. In many "pleasantly" parallel applications, each process/mapper processes the same input file (s), but with computations are controlled by different parameters.(Referred to as "parameter sweeps"). One way to achieve this, is to specify a set of parameters (one set per line) as input in a control file (which is the input path to the map-reduce application, where as the input dataset is specified via a config variable in JobConf.). The NLineInputFormat can be used in such applications, that splits the input file such that by default, one line is fed as a value to one map task, and key is the offset. i.e. (k,v) is (LongWritable, Text). The location hints will span the whole mapred cluster.
| Field Summary | |
|---|---|
static String |
LINES_PER_MAP
|
| Fields inherited from class org.apache.hadoop.mapreduce.lib.input.FileInputFormat |
|---|
DEFAULT_LIST_STATUS_NUM_THREADS, INPUT_DIR, INPUT_DIR_RECURSIVE, LIST_STATUS_NUM_THREADS, NUM_INPUT_FILES, PATHFILTER_CLASS, SPLIT_MAXSIZE, SPLIT_MINSIZE |
| Constructor Summary | |
|---|---|
NLineInputFormat()
|
|
| Method Summary | |
|---|---|
protected static FileSplit |
createFileSplit(Path fileName,
long begin,
long length)
NLineInputFormat uses LineRecordReader, which always reads (and consumes) at least one character out of its upper split boundary. |
RecordReader<LongWritable,Text> |
createRecordReader(InputSplit genericSplit,
TaskAttemptContext context)
Create a record reader for a given split. |
static int |
getNumLinesPerSplit(JobContext job)
Get the number of lines per split |
List<InputSplit> |
getSplits(JobContext job)
Logically splits the set of input files for the job, splits N lines of the input as one split. |
static List<FileSplit> |
getSplitsForFile(FileStatus status,
Configuration conf,
int numLinesPerSplit)
|
static void |
setNumLinesPerSplit(Job job,
int numLines)
Set the number of lines per split |
| Methods inherited from class org.apache.hadoop.mapreduce.lib.input.FileInputFormat |
|---|
addInputPath, addInputPathRecursively, addInputPaths, computeSplitSize, getBlockIndex, getFormatMinSplitSize, getInputDirRecursive, getInputPathFilter, getInputPaths, getMaxSplitSize, getMinSplitSize, isSplitable, listStatus, makeSplit, makeSplit, setInputDirRecursive, setInputPathFilter, setInputPaths, setInputPaths, setMaxInputSplitSize, setMinInputSplitSize |
| Methods inherited from class java.lang.Object |
|---|
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait |
| Field Detail |
|---|
public static final String LINES_PER_MAP
| Constructor Detail |
|---|
public NLineInputFormat()
| Method Detail |
|---|
public RecordReader<LongWritable,Text> createRecordReader(InputSplit genericSplit,
TaskAttemptContext context)
throws IOException
InputFormatRecordReader.initialize(InputSplit, TaskAttemptContext) before
the split is used.
createRecordReader in class InputFormat<LongWritable,Text>genericSplit - the split to be readcontext - the information about the task
IOException
public List<InputSplit> getSplits(JobContext job)
throws IOException
getSplits in class FileInputFormat<LongWritable,Text>job - the job context
InputSplits for the job.
IOExceptionFileInputFormat.getSplits(JobContext)
public static List<FileSplit> getSplitsForFile(FileStatus status,
Configuration conf,
int numLinesPerSplit)
throws IOException
IOException
protected static FileSplit createFileSplit(Path fileName,
long begin,
long length)
fileName - Path of filebegin - the position of the first byte in the file to processlength - number of bytes in InputSplit
public static void setNumLinesPerSplit(Job job,
int numLines)
job - the job to modifynumLines - the number of lines per splitpublic static int getNumLinesPerSplit(JobContext job)
job - the job
|
||||||||||
| PREV CLASS NEXT CLASS | FRAMES NO FRAMES | |||||||||
| SUMMARY: NESTED | FIELD | CONSTR | METHOD | DETAIL: FIELD | CONSTR | METHOD | |||||||||