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
    TsFile data typeHadoop writable
    BOOLEANBooleanWritable
    INT32IntWritable
    INT64LongWritable
    FLOATFloatWritable
    DOUBLEDoubleWritable
    TEXTText

    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.

    The MapWritable struct will be like:

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

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

    Read Example: calculate the sum

    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

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

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