You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2021/12/30 07:06:56 UTC

[iotdb] branch master updated: [IOTDB-2217] Provide an iterator to return timeseries Path in dictionary order (#4645)

This is an automated email from the ASF dual-hosted git repository.

jackietien pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new a2a8ac8  [IOTDB-2217] Provide an iterator to return timeseries Path in dictionary order (#4645)
a2a8ac8 is described below

commit a2a8ac8f389fcd04c17d0b1cc80bde2b7013ad45
Author: Zesong Sun <sz...@mails.tsinghua.edu.cn>
AuthorDate: Thu Dec 30 15:06:26 2021 +0800

    [IOTDB-2217] Provide an iterator to return timeseries Path in dictionary order (#4645)
---
 .../iotdb/tsfile/read/TsFileSequenceReader.java    | 90 +++++++++++++++++++++-
 .../tsfile/write/MetadataIndexConstructorTest.java | 34 +++++++-
 2 files changed, 120 insertions(+), 4 deletions(-)

diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
index bfb8e31..a8fd34a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
@@ -649,7 +649,7 @@ public class TsFileSequenceReader implements AutoCloseable {
   }
 
   /**
-   * this function return all timeseries names in this file
+   * this function return all timeseries names
    *
    * @return list of Paths
    * @throws IOException io error
@@ -665,6 +665,94 @@ public class TsFileSequenceReader implements AutoCloseable {
     return paths;
   }
 
+  /**
+   * @return an iterator of timeseries list, in which names of timeseries are ordered in dictionary
+   *     order
+   * @throws IOException io error
+   */
+  public Iterator<List<Path>> getPathsIterator() throws IOException {
+    readFileMetadata();
+
+    MetadataIndexNode metadataIndexNode = tsFileMetaData.getMetadataIndex();
+    List<MetadataIndexEntry> metadataIndexEntryList = metadataIndexNode.getChildren();
+    Queue<Pair<String, Pair<Long, Long>>> queue = new LinkedList<>();
+    for (int i = 0; i < metadataIndexEntryList.size(); i++) {
+      MetadataIndexEntry metadataIndexEntry = metadataIndexEntryList.get(i);
+      long endOffset = tsFileMetaData.getMetadataIndex().getEndOffset();
+      if (i != metadataIndexEntryList.size() - 1) {
+        endOffset = metadataIndexEntryList.get(i + 1).getOffset();
+      }
+      ByteBuffer buffer = readData(metadataIndexEntry.getOffset(), endOffset);
+      getAllPaths(metadataIndexEntry, buffer, null, metadataIndexNode.getNodeType(), queue);
+    }
+    return new Iterator<List<Path>>() {
+      @Override
+      public boolean hasNext() {
+        return !queue.isEmpty();
+      }
+
+      @Override
+      public List<Path> next() {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        Pair<String, Pair<Long, Long>> startEndPair = queue.remove();
+        List<Path> paths = new ArrayList<>();
+        try {
+          ByteBuffer nextBuffer = readData(startEndPair.right.left, startEndPair.right.right);
+          while (nextBuffer.hasRemaining()) {
+            paths.add(
+                new Path(
+                    startEndPair.left,
+                    TimeseriesMetadata.deserializeFrom(nextBuffer, false).getMeasurementId()));
+          }
+          return paths;
+        } catch (IOException e) {
+          throw new TsFileRuntimeException(
+              "Error occurred while reading a time series metadata block.");
+        }
+      }
+    };
+  }
+
+  private void getAllPaths(
+      MetadataIndexEntry metadataIndex,
+      ByteBuffer buffer,
+      String deviceId,
+      MetadataIndexNodeType type,
+      Queue<Pair<String, Pair<Long, Long>>> queue)
+      throws IOException {
+    try {
+      if (type.equals(MetadataIndexNodeType.LEAF_DEVICE)) {
+        deviceId = metadataIndex.getName();
+      }
+      MetadataIndexNode metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+      int metadataIndexListSize = metadataIndexNode.getChildren().size();
+      for (int i = 0; i < metadataIndexListSize; i++) {
+        long startOffset = metadataIndexNode.getChildren().get(i).getOffset();
+        long endOffset = metadataIndexNode.getEndOffset();
+        if (i != metadataIndexListSize - 1) {
+          endOffset = metadataIndexNode.getChildren().get(i + 1).getOffset();
+        }
+        if (metadataIndexNode.getNodeType().equals(MetadataIndexNodeType.LEAF_MEASUREMENT)) {
+          queue.add(new Pair<>(deviceId, new Pair<>(startOffset, endOffset)));
+          continue;
+        }
+        ByteBuffer nextBuffer =
+            readData(metadataIndexNode.getChildren().get(i).getOffset(), endOffset);
+        getAllPaths(
+            metadataIndexNode.getChildren().get(i),
+            nextBuffer,
+            deviceId,
+            metadataIndexNode.getNodeType(),
+            queue);
+      }
+    } catch (BufferOverflowException e) {
+      logger.error("Something error happened while getting all paths of file {}", file);
+      throw e;
+    }
+  }
+
   private TimeseriesMetadata tryToGetFirstTimeseriesMetadata(MetadataIndexNode measurementNode)
       throws IOException {
     // Not aligned timeseries
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/MetadataIndexConstructorTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/MetadataIndexConstructorTest.java
index ee73f8f..cebf730 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/MetadataIndexConstructorTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/MetadataIndexConstructorTest.java
@@ -52,6 +52,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -151,7 +152,7 @@ public class MetadataIndexConstructorTest {
     int[][] vectorMeasurement = new int[deviceNum][];
     String[][] singleMeasurement = new String[deviceNum][];
     for (int i = 0; i < deviceNum; i++) {
-      devices[i] = "d" + i;
+      devices[i] = "d" + generateIndexString(i, deviceNum);
       vectorMeasurement[i] = new int[0];
       singleMeasurement[i] = new String[measurementNum];
       for (int j = 0; j < measurementNum; j++) {
@@ -204,8 +205,14 @@ public class MetadataIndexConstructorTest {
     List<String> correctDevices = new ArrayList<>(); // contains all device by sequence
     List<List<String>> correctFirstMeasurements =
         new ArrayList<>(); // contains first measurements of every leaf, group by device
+    List<String> correctPaths = new ArrayList<>(); // contains all paths by sequence
     generateCorrectResult(
-        correctDevices, correctFirstMeasurements, devices, vectorMeasurement, singleMeasurement);
+        correctDevices,
+        correctFirstMeasurements,
+        correctPaths,
+        devices,
+        vectorMeasurement,
+        singleMeasurement);
     // 4. compare correct result with TsFile's metadata
     Arrays.sort(devices);
     // 4.1 make sure device in order
@@ -229,6 +236,7 @@ public class MetadataIndexConstructorTest {
       e.printStackTrace();
       fail(e.getMessage());
     }
+
     // 4.3 make sure split leaf correctly
     for (int j = 0; j < actualDevices.size(); j++) {
       for (int i = 0; i < actualMeasurements.get(j).size(); i++) {
@@ -237,6 +245,20 @@ public class MetadataIndexConstructorTest {
             correctFirstMeasurements.get(j).get(i * conf.getMaxDegreeOfIndexNode()));
       }
     }
+    try (TsFileSequenceReader reader = new TsFileSequenceReader(FILE_PATH)) {
+      Iterator<List<Path>> iterator = reader.getPathsIterator();
+      int idx = 0;
+      while (iterator.hasNext()) {
+        for (Path actualPath : iterator.next()) {
+          assertEquals(actualPath.getFullPath(), correctPaths.get(idx));
+          idx++;
+        }
+      }
+      assertEquals(correctPaths.size(), idx);
+    } catch (IOException e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
   }
 
   /**
@@ -331,6 +353,7 @@ public class MetadataIndexConstructorTest {
   private void generateCorrectResult(
       List<String> correctDevices,
       List<List<String>> correctMeasurements,
+      List<String> correctPaths,
       String[] devices,
       int[][] vectorMeasurement,
       String[][] singleMeasurement) {
@@ -341,16 +364,21 @@ public class MetadataIndexConstructorTest {
       List<String> measurements = new ArrayList<>();
       // single-variable measurement
       if (singleMeasurement != null) {
-        measurements.addAll(Arrays.asList(singleMeasurement[i]));
+        for (String measurement : singleMeasurement[i]) {
+          measurements.add(measurement);
+          correctPaths.add(new Path(device, measurement).getFullPath());
+        }
       }
       // multi-variable measurement
       for (int vectorIndex = 0; vectorIndex < vectorMeasurement[i].length; vectorIndex++) {
         measurements.add("");
+        correctPaths.add(new Path(device, "").getFullPath());
         int measurementNum = vectorMeasurement[i][vectorIndex];
         for (int measurementIndex = 0; measurementIndex < measurementNum; measurementIndex++) {
           String measurementName =
               measurementPrefix + generateIndexString(measurementIndex, measurementNum);
           measurements.add(TsFileConstant.PATH_SEPARATOR + measurementName);
+          correctPaths.add(new Path(device, measurementName).getFullPath());
         }
       }
       Collections.sort(measurements);