Hadoop formats

Project Configuration

Support for Hadoop is contained in the flink-hadoop-compatibility Maven module.

Add the following dependency to your pom.xml to use hadoop

  1. <dependency>
  2. <groupId>org.apache.flink</groupId>
  3. <artifactId>flink-hadoop-compatibility_2.12</artifactId>
  4. <version>1.17.2</version>
  5. </dependency>

If you want to run your Flink application locally (e.g. from your IDE), you also need to add a hadoop-client dependency such as:

  1. <dependency>
  2. <groupId>org.apache.hadoop</groupId>
  3. <artifactId>hadoop-client</artifactId>
  4. <version>2.10.2</version>
  5. <scope>provided</scope>
  6. </dependency>

Using Hadoop InputFormats

To use Hadoop InputFormats with Flink the format must first be wrapped using either readHadoopFile or createHadoopInput of the HadoopInputs utility class. The former is used for input formats derived from FileInputFormat while the latter has to be used for general purpose input formats. The resulting InputFormat can be used to create a data source by using ExecutionEnvironment#createInput.

The resulting DataSet contains 2-tuples where the first field is the key and the second field is the value retrieved from the Hadoop InputFormat.

The following example shows how to use Hadoop’s TextInputFormat.

Java

  1. ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
  2. DataSet<Tuple2<LongWritable, Text>> input =
  3. env.createInput(HadoopInputs.readHadoopFile(new TextInputFormat(),
  4. LongWritable.class, Text.class, textPath));
  5. // Do something with the data.
  6. [...]

Scala

  1. val env = ExecutionEnvironment.getExecutionEnvironment
  2. val input: DataSet[(LongWritable, Text)] =
  3. env.createInput(HadoopInputs.readHadoopFile(
  4. new TextInputFormat, classOf[LongWritable], classOf[Text], textPath))
  5. // Do something with the data.
  6. [...]

Using Hadoop OutputFormats

Flink provides a compatibility wrapper for Hadoop OutputFormats. Any class that implements org.apache.hadoop.mapred.OutputFormat or extends org.apache.hadoop.mapreduce.OutputFormat is supported. The OutputFormat wrapper expects its input data to be a DataSet containing 2-tuples of key and value. These are to be processed by the Hadoop OutputFormat.

The following example shows how to use Hadoop’s TextOutputFormat.

Java

  1. // Obtain the result we want to emit
  2. DataSet<Tuple2<Text, IntWritable>> hadoopResult = [...];
  3. // Set up the Hadoop TextOutputFormat.
  4. HadoopOutputFormat<Text, IntWritable> hadoopOF =
  5. // create the Flink wrapper.
  6. new HadoopOutputFormat<Text, IntWritable>(
  7. // set the Hadoop OutputFormat and specify the job.
  8. new TextOutputFormat<Text, IntWritable>(), job
  9. );
  10. hadoopOF.getConfiguration().set("mapreduce.output.textoutputformat.separator", " ");
  11. TextOutputFormat.setOutputPath(job, new Path(outputPath));
  12. // Emit data using the Hadoop TextOutputFormat.
  13. hadoopResult.output(hadoopOF);

Scala

  1. // Obtain your result to emit.
  2. val hadoopResult: DataSet[(Text, IntWritable)] = [...]
  3. val hadoopOF = new HadoopOutputFormat[Text,IntWritable](
  4. new TextOutputFormat[Text, IntWritable],
  5. new JobConf)
  6. hadoopOF.getJobConf.set("mapred.textoutputformat.separator", " ")
  7. FileOutputFormat.setOutputPath(hadoopOF.getJobConf, new Path(resultPath))
  8. hadoopResult.output(hadoopOF)