MapReduce TsFile

Outline

  • TsFile-Hadoop-Connector User Guide
    • About TsFile-Hadoop-Connector
    • System Requirements
    • Data Type Correspondence
    • TSFInputFormat Explanation
    • Examples
      • Read Example: calculate the sum
      • Write Example: write the average into Tsfile

TsFile-Hadoop-Connector User Guide

About TsFile-Hadoop-Connector

TsFile-Hadoop-Connector implements the support of Hadoop for external data sources of Tsfile type. This enables users to read, write and query Tsfile by Hadoop.

With this connector, you can

  • load a single TsFile, from either the local file system or hdfs, into Hadoop
  • load all files in a specific directory, from either the local file system or hdfs, into hadoop
  • write data from Hadoop into TsFile

System Requirements

Hadoop VersionJava VersionTsFile Version
2.7.31.80.11.1

Note: For more information about how to download and use TsFile, please see the following link: https://github.com/apache/iotdb/tree/master/tsfile.

Data Type Correspondence

TsFile data typeHadoop writable
BOOLEANBooleanWritable
INT32IntWritable
INT64LongWritable
FLOATFloatWritable
DOUBLEDoubleWritable
TEXTText

TSFInputFormat Explanation

TSFInputFormat extract data from tsfile and format them into records of MapWritable.

Suppose that we want to extract data of the device named d1 which has three sensors named s1, s2, s3.

s1‘s type is BOOLEAN, s2‘s type is DOUBLE, s3‘s type is TEXT.

The MapWritable struct will be like:

  1. {
  2. "time_stamp": 10000000,
  3. "device_id": d1,
  4. "s1": true,
  5. "s2": 3.14,
  6. "s3": "middle"
  7. }

In the Map job of Hadoop, you can get any value you want by key as following:

mapwritable.get(new Text("s1"))

Note: All keys in MapWritable are Text type.

Examples

Read Example: calculate the sum

First of all, we should tell InputFormat what kind of data we want from tsfile.

  1. // configure reading time enable
  2. TSFInputFormat.setReadTime(job, true);
  3. // configure reading deviceId enable
  4. TSFInputFormat.setReadDeviceId(job, true);
  5. // configure reading which deltaObjectIds
  6. String[] deviceIds = {"device_1"};
  7. TSFInputFormat.setReadDeviceIds(job, deltaObjectIds);
  8. // configure reading which measurementIds
  9. String[] measurementIds = {"sensor_1", "sensor_2", "sensor_3"};
  10. TSFInputFormat.setReadMeasurementIds(job, measurementIds);

And then,the output key and value of mapper and reducer should be specified

  1. // set inputformat and outputformat
  2. job.setInputFormatClass(TSFInputFormat.class);
  3. // set mapper output key and value
  4. job.setMapOutputKeyClass(Text.class);
  5. job.setMapOutputValueClass(DoubleWritable.class);
  6. // set reducer output key and value
  7. job.setOutputKeyClass(Text.class);
  8. job.setOutputValueClass(DoubleWritable.class);

Then, the mapper and reducer class is how you deal with the MapWritable produced by TSFInputFormat class.

  1. public static class TSMapper extends Mapper<NullWritable, MapWritable, Text, DoubleWritable> {
  2. @Override
  3. protected void map(NullWritable key, MapWritable value,
  4. Mapper<NullWritable, MapWritable, Text, DoubleWritable>.Context context)
  5. throws IOException, InterruptedException {
  6. Text deltaObjectId = (Text) value.get(new Text("device_id"));
  7. context.write(deltaObjectId, (DoubleWritable) value.get(new Text("sensor_3")));
  8. }
  9. }
  10. public static class TSReducer extends Reducer<Text, DoubleWritable, Text, DoubleWritable> {
  11. @Override
  12. protected void reduce(Text key, Iterable<DoubleWritable> values,
  13. Reducer<Text, DoubleWritable, Text, DoubleWritable>.Context context)
  14. throws IOException, InterruptedException {
  15. double sum = 0;
  16. for (DoubleWritable value : values) {
  17. sum = sum + value.get();
  18. }
  19. context.write(key, new DoubleWritable(sum));
  20. }
  21. }

Note: For the complete code, please see the following link: https://github.com/apache/iotdb/blob/master/example/hadoop/src/main/java/org/apache/iotdb//hadoop/tsfile/TSFMRReadExample.java

Write Example: write the average into Tsfile

Except for the OutputFormatClass, the rest of configuration code for hadoop map-reduce job is almost same as above.

  1. job.setOutputFormatClass(TSFOutputFormat.class);
  2. // set reducer output key and value
  3. job.setOutputKeyClass(NullWritable.class);
  4. job.setOutputValueClass(HDFSTSRecord.class);

Then, the mapper and reducer class is how you deal with the MapWritable produced by TSFInputFormat class.

  1. public static class TSMapper extends Mapper<NullWritable, MapWritable, Text, MapWritable> {
  2. @Override
  3. protected void map(NullWritable key, MapWritable value,
  4. Mapper<NullWritable, MapWritable, Text, MapWritable>.Context context)
  5. throws IOException, InterruptedException {
  6. Text deltaObjectId = (Text) value.get(new Text("device_id"));
  7. long timestamp = ((LongWritable)value.get(new Text("timestamp"))).get();
  8. if (timestamp % 100000 == 0) {
  9. context.write(deltaObjectId, new MapWritable(value));
  10. }
  11. }
  12. }
  13. /**
  14. * This reducer calculate the average value.
  15. */
  16. public static class TSReducer extends Reducer<Text, MapWritable, NullWritable, HDFSTSRecord> {
  17. @Override
  18. protected void reduce(Text key, Iterable<MapWritable> values,
  19. Reducer<Text, MapWritable, NullWritable, HDFSTSRecord>.Context context) throws IOException, InterruptedException {
  20. long sensor1_value_sum = 0;
  21. long sensor2_value_sum = 0;
  22. double sensor3_value_sum = 0;
  23. long num = 0;
  24. for (MapWritable value : values) {
  25. num++;
  26. sensor1_value_sum += ((LongWritable)value.get(new Text("sensor_1"))).get();
  27. sensor2_value_sum += ((LongWritable)value.get(new Text("sensor_2"))).get();
  28. sensor3_value_sum += ((DoubleWritable)value.get(new Text("sensor_3"))).get();
  29. }
  30. HDFSTSRecord tsRecord = new HDFSTSRecord(1L, key.toString());
  31. DataPoint dPoint1 = new LongDataPoint("sensor_1", sensor1_value_sum / num);
  32. DataPoint dPoint2 = new LongDataPoint("sensor_2", sensor2_value_sum / num);
  33. DataPoint dPoint3 = new DoubleDataPoint("sensor_3", sensor3_value_sum / num);
  34. tsRecord.addTuple(dPoint1);
  35. tsRecord.addTuple(dPoint2);
  36. tsRecord.addTuple(dPoint3);
  37. context.write(NullWritable.get(), tsRecord);
  38. }
  39. }

Note: For the complete code, please see the following link: https://github.com/apache/iotdb/blob/master/example/hadoop/src/main/java/org/apache/iotdb//hadoop/tsfile/TSMRWriteExample.java