You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ha...@apache.org on 2022/11/10 00:33:26 UTC

[iotdb] branch master updated: [IOTDB-4894] Fix bug that TsFileSketchTool prints only the first page info when there are multiple pages in a chunk (#7955)

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

haonan 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 05633d6e8c [IOTDB-4894] Fix bug that TsFileSketchTool prints only the first page info when there are multiple pages in a chunk  (#7955)
05633d6e8c is described below

commit 05633d6e8c59e67a4fd12a8771cb6f4a4edc268f
Author: Rui,Lei <33...@users.noreply.github.com>
AuthorDate: Thu Nov 10 08:33:19 2022 +0800

    [IOTDB-4894] Fix bug that TsFileSketchTool prints only the first page info when there are multiple pages in a chunk  (#7955)
---
 .../apache/iotdb/db/tools/TsFileSketchTool.java    | 193 ++++++++++++++-------
 1 file changed, 128 insertions(+), 65 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java b/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java
index 9c8ecef480..593d3353aa 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java
@@ -45,8 +45,10 @@ import java.io.PrintWriter;
 import java.nio.BufferOverflowException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.TreeMap;
 
 public class TsFileSketchTool {
@@ -116,20 +118,33 @@ public class TsFileSketchTool {
       printlnBoth(
           pw, String.format("%20s", reader.readFileMetadata().getMetaOffset()) + "|\t[marker] 2");
     }
-    // get all timeseries index
+
+    // get all TimerseriesIndex
     Map<Long, Pair<Path, TimeseriesMetadata>> timeseriesMetadataMap =
         reader.getAllTimeseriesMetadataWithOffset();
 
-    // print timeseries index
-    printTimeseriesIndex(timeseriesMetadataMap);
-
+    // get all IndexOfTimerseriesIndex (excluding the root node in TsFileMetadata)
     MetadataIndexNode metadataIndexNode = tsFileMetaData.getMetadataIndex();
     TreeMap<Long, MetadataIndexNode> metadataIndexNodeMap = new TreeMap<>();
     List<String> treeOutputStringBuffer = new ArrayList<>();
     loadIndexTree(metadataIndexNode, metadataIndexNodeMap, treeOutputStringBuffer, 0);
 
-    // print IndexOfTimerseriesIndex
-    printIndexOfTimerseriesIndex(metadataIndexNodeMap);
+    // iterate timeseriesMetadataMap and metadataIndexNodeMap to print info in increasing order of
+    // position
+    Iterator<Entry<Long, Pair<Path, TimeseriesMetadata>>> ite1 =
+        timeseriesMetadataMap.entrySet().iterator();
+    Iterator<Entry<Long, MetadataIndexNode>> ite2 = metadataIndexNodeMap.entrySet().iterator();
+    Entry<Long, Pair<Path, TimeseriesMetadata>> value1 = (ite1.hasNext() ? ite1.next() : null);
+    Entry<Long, MetadataIndexNode> value2 = (ite2.hasNext() ? ite2.next() : null);
+    while (value1 != null || value2 != null) {
+      if (value2 == null || (value1 != null && value1.getKey().compareTo(value2.getKey()) <= 0)) {
+        printTimeseriesIndex(value1.getKey(), value1.getValue());
+        value1 = (ite1.hasNext() ? ite1.next() : null);
+      } else {
+        printIndexOfTimerseriesIndex(value2.getKey(), value2.getValue());
+        value2 = (ite2.hasNext() ? ite2.next() : null);
+      }
+    }
 
     // print TsFile Metadata
     printTsFileMetadata(tsFileMetaData);
@@ -151,38 +166,29 @@ public class TsFileSketchTool {
 
   private void printTsFileMetadata(TsFileMetadata tsFileMetaData) {
     try {
-      printlnBoth(pw, String.format("%20s", reader.getFileMetadataPos()) + "|\t[TsFileMetadata]");
-      printlnBoth(
-          pw, String.format("%20s", "") + "|\t\t[meta offset] " + tsFileMetaData.getMetaOffset());
-      printlnBoth(
-          pw,
-          String.format("%20s", "")
-              + "|\t\t[num of devices] "
-              + tsFileMetaData.getMetadataIndex().getChildren().size());
+      printlnBoth(pw, splitStr + " [TsFileMetadata] begins");
+
+      // metadataIndex
+      MetadataIndexNode rootNode = tsFileMetaData.getMetadataIndex();
+      printIndexOfTimerseriesIndex(reader.getFileMetadataPos(), rootNode);
+
+      // metaOffset
       printlnBoth(
-          pw,
-          String.format("%20s", "")
-              + "|\t\t"
-              + tsFileMetaData.getMetadataIndex().getChildren().size()
-              + " key&TsMetadataIndex");
+          pw, String.format("%20s", "") + "|\t[meta offset] " + tsFileMetaData.getMetaOffset());
+
       // bloom filter
       BloomFilter bloomFilter = tsFileMetaData.getBloomFilter();
       printlnBoth(
           pw,
           String.format("%20s", "")
-              + "|\t\t[bloom filter bit vector byte array length] "
-              + bloomFilter.serialize().length);
-      printlnBoth(pw, String.format("%20s", "") + "|\t\t[bloom filter bit vector byte array] ");
-      printlnBoth(
-          pw,
-          String.format("%20s", "")
-              + "|\t\t[bloom filter number of bits] "
-              + bloomFilter.getSize());
-      printlnBoth(
-          pw,
-          String.format("%20s", "")
-              + "|\t\t[bloom filter number of hash functions] "
+              + "|\t[bloom filter] "
+              + "bit vector byte array length="
+              + bloomFilter.serialize().length
+              + ", filterSize="
+              + bloomFilter.getSize()
+              + ", hashFunctionSize="
               + bloomFilter.getHashFunctionSize());
+      printlnBoth(pw, splitStr + " [TsFileMetadata] ends");
 
       printlnBoth(
           pw,
@@ -223,6 +229,27 @@ public class TsFileSketchTool {
     }
   }
 
+  private void printIndexOfTimerseriesIndex(long pos, MetadataIndexNode metadataIndexNode) {
+    printlnBoth(
+        pw,
+        String.format("%20s", pos)
+            + "|\t[IndexOfTimerseriesIndex Node] type="
+            + metadataIndexNode.getNodeType());
+    for (MetadataIndexEntry metadataIndexEntry : metadataIndexNode.getChildren()) {
+      printlnBoth(
+          pw,
+          String.format("%20s", "")
+              + "|\t\t<"
+              + metadataIndexEntry.getName()
+              + ", "
+              + metadataIndexEntry.getOffset()
+              + ">");
+    }
+    printlnBoth(
+        pw,
+        String.format("%20s", "") + "|\t\t<endOffset, " + metadataIndexNode.getEndOffset() + ">");
+  }
+
   private void printFileInfo() {
     try {
       printlnBoth(pw, "");
@@ -248,7 +275,7 @@ public class TsFileSketchTool {
         printlnBoth(
             pw,
             splitStr
-                + "\t[Chunk Group] of "
+                + " [Chunk Group] of "
                 + chunkGroupMetadata.getDevice()
                 + ", num of Chunks:"
                 + chunkGroupMetadata.getChunkMetadataList().size());
@@ -266,18 +293,9 @@ public class TsFileSketchTool {
               pw,
               String.format("%20d", chunkMetadata.getOffsetOfChunkHeader())
                   + "|\t[Chunk] of "
-                  + chunkMetadata.getMeasurementUid()
-                  + ", numOfPoints:"
-                  + chunkMetadata.getNumOfPoints()
-                  + ", time range:["
-                  + chunkMetadata.getStartTime()
-                  + ","
-                  + chunkMetadata.getEndTime()
-                  + "], tsDataType:"
-                  + chunkMetadata.getDataType()
-                  + ", \n"
-                  + String.format("%20s", "")
-                  + " \t"
+                  + new Path(
+                      chunkGroupHeader.getDeviceID(), chunkMetadata.getMeasurementUid(), false)
+                  + ", "
                   + chunkMetadata.getStatistics());
           printlnBoth(
               pw,
@@ -285,30 +303,51 @@ public class TsFileSketchTool {
                   + "|\t\t[chunk header] "
                   + "marker="
                   + chunk.getHeader().getChunkType()
-                  + ", measurementId="
+                  + ", measurementID="
                   + chunk.getHeader().getMeasurementID()
                   + ", dataSize="
                   + chunk.getHeader().getDataSize()
-                  + ", serializedSize="
-                  + chunk.getHeader().getSerializedSize());
-
-          printlnBoth(pw, String.format("%20s", "") + "|\t\t[chunk] " + chunk.getData());
+                  + ", dataType="
+                  + chunk.getHeader().getDataType()
+                  + ", compressionType="
+                  + chunk.getHeader().getCompressionType()
+                  + ", encodingType="
+                  + chunk.getHeader().getEncodingType());
           PageHeader pageHeader;
           if (((byte) (chunk.getHeader().getChunkType() & 0x3F))
               == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
             pageHeader = PageHeader.deserializeFrom(chunk.getData(), chunkMetadata.getStatistics());
-          } else {
-            pageHeader =
-                PageHeader.deserializeFrom(chunk.getData(), chunk.getHeader().getDataType());
+            printlnBoth(
+                pw,
+                String.format("%20s", "")
+                    + "|\t\t[page] "
+                    + " UncompressedSize:"
+                    + pageHeader.getUncompressedSize()
+                    + ", CompressedSize:"
+                    + pageHeader.getCompressedSize());
+          } else { // more than one page in this chunk
+            ByteBuffer chunkDataBuffer = chunk.getData();
+            int pageID = 0;
+            while (chunkDataBuffer.remaining() > 0) {
+              pageID++;
+              // deserialize a PageHeader from chunkDataBuffer
+              pageHeader =
+                  PageHeader.deserializeFrom(chunkDataBuffer, chunk.getHeader().getDataType());
+              // skip the compressed bytes
+              chunkDataBuffer.position(chunkDataBuffer.position() + pageHeader.getCompressedSize());
+              // print page info
+              printlnBoth(
+                  pw,
+                  String.format("%20s", "")
+                      + String.format("|\t\t[page-%s] ", pageID)
+                      + " UncompressedSize:"
+                      + pageHeader.getUncompressedSize()
+                      + ", CompressedSize:"
+                      + pageHeader.getCompressedSize()
+                      + ", "
+                      + pageHeader.getStatistics());
+            }
           }
-          printlnBoth(
-              pw,
-              String.format("%20s", "")
-                  + "|\t\t[page] "
-                  + " CompressedSize:"
-                  + pageHeader.getCompressedSize()
-                  + ", UncompressedSize:"
-                  + pageHeader.getUncompressedSize());
           nextChunkGroupHeaderPos =
               chunkMetadata.getOffsetOfChunkHeader()
                   + chunk.getHeader().getSerializedSize()
@@ -326,8 +365,7 @@ public class TsFileSketchTool {
             break;
         }
 
-        printlnBoth(
-            pw, splitStr + "\t[Chunk Group] of " + chunkGroupMetadata.getDevice() + " ends");
+        printlnBoth(pw, splitStr + " [Chunk Group] of " + chunkGroupMetadata.getDevice() + " ends");
       }
     } catch (IOException e) {
       e.printStackTrace();
@@ -345,7 +383,9 @@ public class TsFileSketchTool {
                 + "|\t[TimeseriesIndex] of "
                 + entry.getValue().left
                 + ", tsDataType:"
-                + entry.getValue().right.getTSDataType());
+                + entry.getValue().right.getTSDataType()
+                + ", "
+                + entry.getValue().right.getStatistics());
         for (IChunkMetadata chunkMetadata : reader.getChunkMetadataList(entry.getValue().left)) {
           printlnBoth(
               pw,
@@ -355,11 +395,32 @@ public class TsFileSketchTool {
                   + ", offset="
                   + chunkMetadata.getOffsetOfChunkHeader());
         }
+      }
+      printlnBoth(pw, splitStr);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private void printTimeseriesIndex(long pos, Pair<Path, TimeseriesMetadata> timeseriesMetadata) {
+    try {
+      printlnBoth(
+          pw,
+          String.format("%20s", pos)
+              + "|\t[TimeseriesIndex] of "
+              + timeseriesMetadata.left
+              + ", tsDataType:"
+              + timeseriesMetadata.right.getTSDataType()
+              + ", "
+              + timeseriesMetadata.right.getStatistics());
+      for (IChunkMetadata chunkMetadata : reader.getChunkMetadataList(timeseriesMetadata.left)) {
         printlnBoth(
             pw,
-            String.format("%20s", "") + "|\t\t[" + entry.getValue().right.getStatistics() + "] ");
+            String.format("%20s", "")
+                + "|\t\t[ChunkIndex] "
+                + "offset="
+                + chunkMetadata.getOffsetOfChunkHeader());
       }
-      printlnBoth(pw, splitStr);
     } catch (IOException e) {
       e.printStackTrace();
     }
@@ -426,9 +487,11 @@ public class TsFileSketchTool {
   }
 
   private class TsFileSketchToolReader extends TsFileSequenceReader {
+
     public TsFileSketchToolReader(String file) throws IOException {
       super(file);
     }
+
     /**
      * Traverse the metadata index from MetadataIndexEntry to get TimeseriesMetadatas
      *