You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by lt...@apache.org on 2019/07/10 07:06:39 UTC

[incubator-iotdb] branch fix_statstic_bug updated: add ut test and fix a bug of Statistics when restoring

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

lta pushed a commit to branch fix_statstic_bug
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git


The following commit(s) were added to refs/heads/fix_statstic_bug by this push:
     new 2024d70  add ut test and fix a bug of Statistics when restoring
2024d70 is described below

commit 2024d70e2316b789862871ee3fdd2eb81a5d94d8
Author: lta <li...@163.com>
AuthorDate: Wed Jul 10 15:06:26 2019 +0800

    add ut test and fix a bug of Statistics when restoring
---
 .../db/engine/storagegroup/TsFileProcessor.java    |  4 ++
 .../engine/storagegroup/TsFileProcessorTest.java   | 80 ++++++++++++++++++++--
 .../iotdb/tsfile/file/metadata/ChunkMetaData.java  | 26 ++++++-
 .../iotdb/tsfile/file/metadata/TsDigest.java       | 24 +++++++
 .../file/metadata/statistics/Statistics.java       | 10 ++-
 5 files changed, 133 insertions(+), 11 deletions(-)

diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index d918e75..e176ddf 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -475,6 +475,10 @@ public class TsFileProcessor {
     return workMemTable.memSize();
   }
 
+  RestorableTsFileIOWriter getWriter() {
+    return writer;
+  }
+
   String getStorageGroupName() {
     return storageGroupName;
   }
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
index 6d9dc13..11ef1d9 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
@@ -39,11 +39,13 @@ import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.db.utils.FileSchemaUtils;
 import org.apache.iotdb.db.utils.TimeValuePair;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetaData;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.record.TSRecord;
 import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
+import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -75,8 +77,9 @@ public class TsFileProcessorTest {
   public void testWriteAndFlush()
       throws WriteProcessException, IOException, TsFileProcessorException {
     processor = new TsFileProcessor(storageGroup, new File(filePath),
-        FileSchemaUtils.constructFileSchema(deviceId), SysTimeVersionController.INSTANCE, x->{},
-        ()-> true, true);
+        FileSchemaUtils.constructFileSchema(deviceId), SysTimeVersionController.INSTANCE, x -> {
+    },
+        () -> true, true);
 
     Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = processor
         .query(deviceId, measurementId, dataType, props, context);
@@ -117,13 +120,80 @@ public class TsFileProcessorTest {
     processor.syncClose();
   }
 
+  @Test
+  public void testWriteAndRestoreMetadata()
+      throws IOException {
+    processor = new TsFileProcessor(storageGroup, new File(filePath),
+        FileSchemaUtils.constructFileSchema(deviceId), SysTimeVersionController.INSTANCE, x -> {
+    },
+        () -> true, true);
+
+    Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = processor
+        .query(deviceId, measurementId, dataType, props, context);
+    ReadOnlyMemChunk left = pair.left;
+    List<ChunkMetaData> right = pair.right;
+    assertTrue(left.isEmpty());
+    assertEquals(0, right.size());
+
+    for (int i = 1; i <= 100; i++) {
+      TSRecord record = new TSRecord(i, deviceId);
+      record.addTuple(DataPoint.getDataPoint(dataType, measurementId, String.valueOf(i)));
+      processor.insert(new InsertPlan(record));
+    }
+
+    // query data in memory
+    pair = processor.query(deviceId, measurementId, dataType, props, context);
+    left = pair.left;
+    assertFalse(left.isEmpty());
+    int num = 1;
+    Iterator<TimeValuePair> iterator = left.getIterator();
+    for (; num <= 100; num++) {
+      iterator.hasNext();
+      TimeValuePair timeValuePair = iterator.next();
+      assertEquals(num, timeValuePair.getTimestamp());
+      assertEquals(num, timeValuePair.getValue().getInt());
+    }
+
+    // flush asynchronously
+    processor.syncFlush();
+
+    pair = processor.query(deviceId, measurementId, dataType, props, context);
+    left = pair.left;
+    right = pair.right;
+    assertTrue(left.isEmpty());
+    assertEquals(1, right.size());
+    assertEquals(measurementId, right.get(0).getMeasurementUid());
+    assertEquals(dataType, right.get(0).getTsDataType());
+
+    RestorableTsFileIOWriter tsFileIOWriter = processor.getWriter();
+    List<ChunkGroupMetaData> chunkGroupMetaDatas = tsFileIOWriter.getChunkGroupMetaDatas();
+    RestorableTsFileIOWriter tsFileIOWriterRestore = new RestorableTsFileIOWriter(
+        new File(filePath));
+    List<ChunkGroupMetaData> chunkGroupMetaDatasRestore = tsFileIOWriterRestore
+        .getChunkGroupMetaDatas();
+    assertEquals(chunkGroupMetaDatas.size(), chunkGroupMetaDatasRestore.size());
+    for (int i = 0; i < chunkGroupMetaDatas.size(); i++) {
+      ChunkGroupMetaData chunkGroupMetaData = chunkGroupMetaDatas.get(i);
+      ChunkGroupMetaData chunkGroupMetaDataRestore = chunkGroupMetaDatasRestore.get(i);
+      for (int j = 0; j < chunkGroupMetaData.getChunkMetaDataList().size(); j++) {
+        ChunkMetaData chunkMetaData = chunkGroupMetaData.getChunkMetaDataList().get(j);
+        ChunkMetaData chunkMetaDataRestore = chunkGroupMetaDataRestore.getChunkMetaDataList()
+            .get(j);
+        assertEquals(chunkMetaData, chunkMetaDataRestore);
+      }
+    }
+
+    processor.syncClose();
+  }
+
 
   @Test
   public void testMultiFlush()
       throws WriteProcessException, IOException, TsFileProcessorException {
     processor = new TsFileProcessor(storageGroup, new File(filePath),
-        FileSchemaUtils.constructFileSchema(deviceId), SysTimeVersionController.INSTANCE, x->{},
-        ()->true, true);
+        FileSchemaUtils.constructFileSchema(deviceId), SysTimeVersionController.INSTANCE, x -> {
+    },
+        () -> true, true);
 
     Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = processor
         .query(deviceId, measurementId, dataType, props, context);
@@ -171,7 +241,7 @@ public class TsFileProcessorTest {
               throw new TsFileProcessorException(e);
             }
           }
-        }, ()->true, true);
+        }, () -> true, true);
 
     Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = processor
         .query(deviceId, measurementId, dataType, props, context);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java
index 3dfa398..7c89cf2 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.Objects;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
@@ -47,9 +48,8 @@ public class ChunkMetaData {
   private TSDataType tsDataType;
 
   /**
-   * version is used to define the order of operations(insertion, deletion, update).
-   * version is set according to its belonging ChunkGroup only when being queried, so it is not
-   * persisted.
+   * version is used to define the order of operations(insertion, deletion, update). version is set
+   * according to its belonging ChunkGroup only when being queried, so it is not persisted.
    */
   private long version;
 
@@ -264,4 +264,24 @@ public class ChunkMetaData {
   public void setDeletedAt(long deletedAt) {
     this.deletedAt = deletedAt;
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ChunkMetaData that = (ChunkMetaData) o;
+    return offsetOfChunkHeader == that.offsetOfChunkHeader &&
+        numOfPoints == that.numOfPoints &&
+        startTime == that.startTime &&
+        endTime == that.endTime &&
+        version == that.version &&
+        deletedAt == that.deletedAt &&
+        Objects.equals(measurementUid, that.measurementUid) &&
+        tsDataType == that.tsDataType &&
+        Objects.equals(valuesStatistics, that.valuesStatistics);
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsDigest.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsDigest.java
index 869c2f4..1b27efa 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsDigest.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsDigest.java
@@ -26,6 +26,7 @@ import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Map.Entry;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 /**
@@ -218,4 +219,27 @@ public class TsDigest {
     }
     return serializedSize;
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    TsDigest digest = (TsDigest) o;
+    if (serializedSize != digest.serializedSize || sizeOfList != digest.sizeOfList
+        || statistics.size() != digest.statistics.size()) {
+      return false;
+    }
+    for (Entry<String, ByteBuffer> entry : statistics.entrySet()) {
+      String key = entry.getKey();
+      ByteBuffer value = entry.getValue();
+      if (!digest.statistics.containsKey(key) || !value.equals(digest.statistics.get(key))) {
+        return false;
+      }
+    }
+    return true;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
index cdb3179..5758288 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
@@ -78,18 +78,22 @@ public abstract class Statistics<T> {
       throws IOException {
     Statistics statistics = getStatsByType(dataType);
     statistics.fill(inputStream);
+    statistics.isEmpty = false;
     return statistics;
   }
 
   public static Statistics deserialize(ByteBuffer buffer, TSDataType dataType) throws IOException {
     Statistics statistics = getStatsByType(dataType);
     statistics.fill(buffer);
+    statistics.isEmpty = false;
     return statistics;
   }
 
-  public static Statistics deserialize(TsFileInput input, long offset, TSDataType dataType) throws IOException {
+  public static Statistics deserialize(TsFileInput input, long offset, TSDataType dataType)
+      throws IOException {
     Statistics statistics = getStatsByType(dataType);
     statistics.fill(input, offset);
+    statistics.isEmpty = false;
     return statistics;
   }
 
@@ -170,8 +174,8 @@ public abstract class Statistics<T> {
   }
 
   /**
-   * This method with two parameters is only used by {@code unsequence} which updates/inserts/deletes
-   * timestamp.
+   * This method with two parameters is only used by {@code unsequence} which
+   * updates/inserts/deletes timestamp.
    *
    * @param min min timestamp
    * @param max max timestamp