You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by le...@apache.org on 2024/02/05 08:26:43 UTC

(iotdb) 23/30: add convex hull metadata

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

leirui pushed a commit to branch research/LTS-visualization
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit b1188bd2f4a6b29cd942cac5e0d1c74fc95be165
Author: Lei Rui <10...@qq.com>
AuthorDate: Sat Feb 3 01:52:29 2024 +0800

    add convex hull metadata
---
 .../groupby/GroupByWithoutValueFilterDataSet.java  |  89 ++++++-----
 .../apache/iotdb/db/integration/tri/MyTest_ch.java | 116 ++++++++++++++
 .../tsfile/file/metadata/statistics/QuickHull.java | 137 +++++++++++++++++
 .../file/metadata/statistics/QuickHullPoint.java   |  16 ++
 .../file/metadata/statistics/Statistics.java       | 169 +++++++++++++++++----
 5 files changed, 453 insertions(+), 74 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java
index a51924bb9d8..e71bc91f6c6 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java
@@ -31,11 +31,9 @@ import org.apache.iotdb.db.query.aggregation.AggregateResult;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.factory.AggregateResultFactory;
 import org.apache.iotdb.db.query.filter.TsFileFilter;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.MinMaxInfo;
 import org.apache.iotdb.tsfile.read.common.IOMonitor2;
-import org.apache.iotdb.tsfile.read.common.IOMonitor2.DataSetType;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.common.RowRecord;
 import org.apache.iotdb.tsfile.read.expression.IExpression;
@@ -653,10 +651,10 @@ public class GroupByWithoutValueFilterDataSet extends GroupByEngineDataSet {
       boolean ascending)
       throws StorageEngineException, QueryProcessException {
     if (CONFIG.getEnableTri().equals("MinMax")) {
-      return new LocalGroupByExecutorTri_MinMax( // TODO
+      return new LocalGroupByExecutorTri_MinMax(
           path, allSensors, dataType, context, timeFilter, fileFilter, ascending);
     } else if (CONFIG.getEnableTri().equals("MinMaxLTTB")) {
-      return new LocalGroupByExecutorTri_MinMaxPreselection( // TODO
+      return new LocalGroupByExecutorTri_MinMaxPreselection(
           path, allSensors, dataType, context, timeFilter, fileFilter, ascending);
     } else if (CONFIG.getEnableTri().equals("M4")) {
       return new LocalGroupByExecutorTri_M4(
@@ -667,45 +665,54 @@ public class GroupByWithoutValueFilterDataSet extends GroupByEngineDataSet {
     } else if (CONFIG.getEnableTri().equals("ILTS")) {
       return new LocalGroupByExecutorTri_ILTS(
           path, allSensors, dataType, context, timeFilter, fileFilter, ascending);
-    }
-    // deprecated below
-    else if (CONFIG.isEnableCPV()) {
-      if (TSFileDescriptor.getInstance().getConfig().isEnableMinMaxLSM()) { // MinMax-LSM
-        IOMonitor2.dataSetType =
-            DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor4CPV_EnableMinMaxLSM;
-        return new LocalGroupByExecutor4MinMax(
-            path, allSensors, dataType, context, timeFilter, fileFilter, ascending);
-      } else { // M4-LSM
-        if (TSFileDescriptor.getInstance().getConfig().isUseTimeIndex()
-            && TSFileDescriptor.getInstance().getConfig().isUseValueIndex()) {
-          IOMonitor2.dataSetType =
-              DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor4CPV_UseIndex;
-        } else if (!TSFileDescriptor.getInstance().getConfig().isUseTimeIndex()
-            && TSFileDescriptor.getInstance().getConfig().isUseValueIndex()) {
-          IOMonitor2.dataSetType =
-              DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor4CPV_NoTimeIndex;
-        } else if (TSFileDescriptor.getInstance().getConfig().isUseTimeIndex()
-            && !TSFileDescriptor.getInstance().getConfig().isUseValueIndex()) {
-          IOMonitor2.dataSetType =
-              DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor4CPV_NoValueIndex;
-        } else {
-          IOMonitor2.dataSetType =
-              DataSetType
-                  .GroupByWithoutValueFilterDataSet_LocalGroupByExecutor4CPV_NoTimeValueIndex;
-        }
-        return new LocalGroupByExecutor4CPV(
-            path, allSensors, dataType, context, timeFilter, fileFilter, ascending);
-      }
-    } else { // enableCPV=false
-      if (TSFileDescriptor.getInstance().getConfig().isUseStatistics()) {
-        IOMonitor2.dataSetType =
-            DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor_UseStatistics;
-      } else {
-        IOMonitor2.dataSetType =
-            DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor_NotUseStatistics;
-      }
+    } else {
+      logger.info("No matched enable_tri!");
       return new LocalGroupByExecutor(
           path, allSensors, dataType, context, timeFilter, fileFilter, ascending);
     }
+
+    //    // deprecated below, note that time&value index are also deprecated in Statistics
+    //    else if (CONFIG.isEnableCPV()) {
+    //      if (TSFileDescriptor.getInstance().getConfig().isEnableMinMaxLSM()) { // MinMax-LSM
+    //        IOMonitor2.dataSetType =
+    //
+    // DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor4CPV_EnableMinMaxLSM;
+    //        return new LocalGroupByExecutor4MinMax(
+    //            path, allSensors, dataType, context, timeFilter, fileFilter, ascending);
+    //      } else { // M4-LSM
+    //        if (TSFileDescriptor.getInstance().getConfig().isUseTimeIndex()
+    //            && TSFileDescriptor.getInstance().getConfig().isUseValueIndex()) {
+    //          IOMonitor2.dataSetType =
+    //              DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor4CPV_UseIndex;
+    //        } else if (!TSFileDescriptor.getInstance().getConfig().isUseTimeIndex()
+    //            && TSFileDescriptor.getInstance().getConfig().isUseValueIndex()) {
+    //          IOMonitor2.dataSetType =
+    //
+    // DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor4CPV_NoTimeIndex;
+    //        } else if (TSFileDescriptor.getInstance().getConfig().isUseTimeIndex()
+    //            && !TSFileDescriptor.getInstance().getConfig().isUseValueIndex()) {
+    //          IOMonitor2.dataSetType =
+    //
+    // DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor4CPV_NoValueIndex;
+    //        } else {
+    //          IOMonitor2.dataSetType =
+    //              DataSetType
+    //                  .GroupByWithoutValueFilterDataSet_LocalGroupByExecutor4CPV_NoTimeValueIndex;
+    //        }
+    //        return new LocalGroupByExecutor4CPV(
+    //            path, allSensors, dataType, context, timeFilter, fileFilter, ascending);
+    //      }
+    //    } else { // enableCPV=false
+    //      if (TSFileDescriptor.getInstance().getConfig().isUseStatistics()) {
+    //        IOMonitor2.dataSetType =
+    //            DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor_UseStatistics;
+    //      } else {
+    //        IOMonitor2.dataSetType =
+    //
+    // DataSetType.GroupByWithoutValueFilterDataSet_LocalGroupByExecutor_NotUseStatistics;
+    //      }
+    //      return new LocalGroupByExecutor(
+    //          path, allSensors, dataType, context, timeFilter, fileFilter, ascending);
+    //    }
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/tri/MyTest_ch.java b/server/src/test/java/org/apache/iotdb/db/integration/tri/MyTest_ch.java
new file mode 100644
index 00000000000..f2cd1dd08c0
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/integration/tri/MyTest_ch.java
@@ -0,0 +1,116 @@
+package org.apache.iotdb.db.integration.tri;
+
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.tsfile.file.metadata.statistics.QuickHull;
+import org.apache.iotdb.tsfile.file.metadata.statistics.QuickHullPoint;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.BitSet;
+
+public class MyTest_ch {
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.envSetUp();
+    Class.forName(Config.JDBC_DRIVER_NAME);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void test1() throws Exception {
+    System.out.println("Quick Hull Test");
+    ArrayList<QuickHullPoint> points = new ArrayList<>();
+    points.add(new QuickHullPoint(0, 0, 0));
+    points.add(new QuickHullPoint(1, 3, 1));
+    points.add(new QuickHullPoint(2, 2, 2));
+    points.add(new QuickHullPoint(3, 2, 3));
+    points.add(new QuickHullPoint(4, 0, 4));
+    points.add(new QuickHullPoint(5, 2, 5));
+    points.add(new QuickHullPoint(100, 2, 6));
+
+    BitSet bitSet1 = QuickHull.quickHull(points);
+    System.out.println(bitSet1);
+    //    System.out.println(bitSet1.size());
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    ObjectOutputStream oos = new ObjectOutputStream(baos);
+    oos.writeObject(bitSet1);
+    oos.flush();
+    byte[] bytes = baos.toByteArray();
+    //    byte[] bytes = bitSet1.toByteArray();
+    System.out.println(bytes.length);
+
+    ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+    ObjectInputStream ois = new ObjectInputStream(bais);
+    BitSet bitSet = (BitSet) ois.readObject();
+    System.out.println(bitSet);
+    //    System.out.println(bitSet.size());
+    Assert.assertTrue(bitSet.get(1));
+    Assert.assertFalse(bitSet.get(3));
+  }
+
+  //  public static void main(String args[]) throws Exception {
+  //
+  //    System.out.println("Quick Hull Test");
+  ////    Scanner sc = new Scanner(System.in);
+  ////    System.out.println("Enter the number of points");
+  ////    int N = sc.nextInt();
+  //
+  //    ArrayList<QuickHullPoint> points = new ArrayList<>();
+  ////    System.out.println("Enter the coordinates of each points: <x> <y>");
+  ////    for (int i = 0; i < N; i++) {
+  ////      int x = sc.nextInt();
+  ////      int y = sc.nextInt();
+  ////      Point e = new Point(x, y);
+  ////      points.add(i, new QuickHullPoint(x,y,i));
+  ////    }
+  //    points.add(new QuickHullPoint(0,0,0));
+  //    points.add(new QuickHullPoint(1,3,1));
+  //    points.add(new QuickHullPoint(2,2,2));
+  //    points.add(new QuickHullPoint(3,2,3));
+  //    points.add(new QuickHullPoint(4,0,4));
+  //    points.add(new QuickHullPoint(5,2,5));
+  //
+  ////    QuickHull qh = new QuickHull();
+  //    BitSet bitSet1 = QuickHull.quickHull(points);
+  ////    System.out
+  ////        .println("The points in the Convex hull using Quick Hull are: ");
+  ////    for (int i = 0; i < p.size(); i++) {
+  ////      System.out.println("(" + p.get(i).x + ", " + p.get(i).y + ")");
+  ////    }
+  //    System.out.println(bitSet1.toString());
+  //    System.out.println(bitSet1.size());
+  ////    sc.close();
+  //
+  ////    BitSet bits = new BitSet(100000);
+  ////    System.out.println(bits.size());
+  ////    bits.set(2);
+  //    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+  //    ObjectOutputStream oos = new ObjectOutputStream(baos);
+  //    oos.writeObject(bitSet1);
+  //    oos.close();
+  //    byte[] bytes = baos.toByteArray();
+  //    System.out.println(bytes.length);
+  //    ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+  //    ObjectInputStream ois = new ObjectInputStream(bais);
+  //    BitSet bitSet = (BitSet) ois.readObject();
+  //    System.out.println(bitSet);
+  //    System.out.println(bitSet.get(1));
+  //    System.out.println(bitSet.get(3));
+  //    System.out.println(bitSet.size());
+  //  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/QuickHull.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/QuickHull.java
new file mode 100644
index 00000000000..49425fc346d
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/QuickHull.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.tsfile.file.metadata.statistics;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+public class QuickHull {
+
+  public static BitSet quickHull(
+      List<QuickHullPoint> points) { // TODO check this input points deep clone?
+    BitSet bitSet = new BitSet(points.size());
+
+    //    List<Point> convexHull = new ArrayList<>();
+
+    if (points.size() < 3) {
+      bitSet.set(0);
+      bitSet.set(1);
+      bitSet.set(2);
+      return bitSet;
+    }
+
+    // assume point.x i.e., t is monotonically increasing
+    int minPoint = 0;
+    int maxPoint = points.size() - 1;
+    //    bitSet.set(0);
+    //    bitSet.set(points.size() - 1); // TODO check this has not remove any
+    QuickHullPoint A = points.get(minPoint);
+    QuickHullPoint B = points.get(maxPoint);
+    //    convexHull.add(A);
+    //    convexHull.add(B);
+    //    points.remove(A); // TODO check this input data deep clone?
+    //    points.remove(B);
+    points.remove(A);
+    points.remove(B); // TODO check this
+    bitSet.set(A.idx);
+    bitSet.set(B.idx);
+
+    ArrayList<QuickHullPoint> leftSet = new ArrayList<>();
+    ArrayList<QuickHullPoint> rightSet = new ArrayList<>();
+
+    for (QuickHullPoint p : points) {
+      if (pointLocation(A, B, p) == -1) {
+        leftSet.add(p);
+      } else if (pointLocation(A, B, p) == 1) {
+        rightSet.add(p);
+      }
+    }
+    hullSet(A, B, rightSet, bitSet);
+    hullSet(B, A, leftSet, bitSet);
+
+    return bitSet;
+  }
+
+  public static double distance(QuickHullPoint A, QuickHullPoint B, QuickHullPoint C) {
+    double ABx = B.t - A.t;
+    double ABy = B.v - A.v;
+    double num = ABx * (A.v - C.v) - ABy * (A.t - C.t);
+    if (num < 0) {
+      num = -num;
+    }
+    return num;
+    // 返回的是ABC三角形面积的二倍数
+  }
+
+  public static void hullSet(
+      QuickHullPoint A, QuickHullPoint B, List<QuickHullPoint> set, BitSet bitSet) {
+    //    int insertPosition = hull.indexOf(B);
+    if (set.size() == 0) {
+      return;
+    }
+    if (set.size() == 1) {
+      QuickHullPoint p = set.get(0);
+      set.remove(p);
+      //      hull.add(insertPosition, p);
+      bitSet.set(p.idx);
+      return;
+    }
+    double dist = -Double.MAX_VALUE;
+    //    int furthestPoint = -1;
+    QuickHullPoint furthestPoint = null;
+    for (QuickHullPoint p : set) {
+      double distance = distance(A, B, p);
+      if (distance > dist) {
+        dist = distance;
+        furthestPoint = p;
+      }
+    }
+    //    QuickHullPoint P = set.get(furthestPoint);
+    set.remove(furthestPoint);
+    //    hull.add(insertPosition, P);
+    bitSet.set(furthestPoint.idx);
+
+    // Determine who's to the left of AP
+    ArrayList<QuickHullPoint> leftSetAP = new ArrayList<>();
+    for (QuickHullPoint M : set) {
+      if (pointLocation(A, furthestPoint, M) == 1) {
+        leftSetAP.add(M);
+      }
+    }
+
+    // Determine who's to the left of PB
+    ArrayList<QuickHullPoint> leftSetPB = new ArrayList<>();
+    for (QuickHullPoint M : set) {
+      if (pointLocation(furthestPoint, B, M) == 1) {
+        leftSetPB.add(M);
+      }
+    }
+    hullSet(A, furthestPoint, leftSetAP, bitSet);
+    hullSet(furthestPoint, B, leftSetPB, bitSet);
+  }
+
+  public static int pointLocation(QuickHullPoint A, QuickHullPoint B, QuickHullPoint P) {
+    double cp1 = (B.t - A.t) * (P.v - A.v) - (B.v - A.v) * (P.t - A.t);
+    return Double.compare(cp1, 0);
+    // return 1 means P is above the line connecting A and B
+    // return -1 means P is below the line connecting A and B
+    // return 0 means P is on the line connecting A and B
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/QuickHullPoint.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/QuickHullPoint.java
new file mode 100644
index 00000000000..8709d14ea01
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/QuickHullPoint.java
@@ -0,0 +1,16 @@
+package org.apache.iotdb.tsfile.file.metadata.statistics;
+
+public class QuickHullPoint {
+
+  public int idx; // pos in the input array, start from 0
+
+  public long t;
+
+  public double v;
+
+  public QuickHullPoint(long t, double v, int idx) {
+    this.t = t;
+    this.v = v;
+    this.idx = idx;
+  }
+}
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 da2b27e9233..89a9ffe8c41 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
@@ -32,11 +32,18 @@ import org.eclipse.collections.impl.list.mutable.primitive.DoubleArrayList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.Collections;
+import java.util.List;
 import java.util.Objects;
 
 /**
@@ -64,6 +71,12 @@ public abstract class Statistics<T> {
 
   private StepRegress stepRegress = new StepRegress();
 
+  private List<QuickHullPoint> quickHullPoints = new ArrayList<>();
+  private int quickHullIdx = 0;
+  private boolean quickHullLearned = false;
+
+  private BitSet quickHullBitSet = null;
+
   public ValueIndex valueIndex = new ValueIndex();
 
   /** @author Yuyuan Kang */
@@ -133,10 +146,12 @@ public abstract class Statistics<T> {
     byteLen += ReadWriteIOUtils.write(endTime, outputStream);
     // value statistics of different data type
     byteLen += serializeStats(outputStream);
-    // serialize stepRegress
-    byteLen += serializeStepRegress(outputStream, true);
-    // serialize value index
-    byteLen += serializeValueIndex(outputStream, true);
+    //    // serialize stepRegress
+    //    byteLen += serializeStepRegress(outputStream, true);
+    //    // serialize value index
+    //    byteLen += serializeValueIndex(outputStream, true);
+    // serialize convex hull
+    byteLen += serializeConvexHull(outputStream, true);
     return byteLen;
   }
 
@@ -147,10 +162,12 @@ public abstract class Statistics<T> {
     byteLen += ReadWriteIOUtils.write(endTime, outputStream);
     // value statistics of different data type
     byteLen += serializeStats(outputStream);
-    // serialize stepRegress
-    byteLen += serializeStepRegress(outputStream, log);
-    // serialize value index
-    byteLen += serializeValueIndex(outputStream, log);
+    //    // serialize stepRegress
+    //    byteLen += serializeStepRegress(outputStream, log);
+    //    // serialize value index
+    //    byteLen += serializeValueIndex(outputStream, log);
+    // serialize convex hull
+    byteLen += serializeConvexHull(outputStream, log);
     return byteLen;
   }
 
@@ -200,6 +217,34 @@ public abstract class Statistics<T> {
     return byteLen;
   }
 
+  int serializeConvexHull(OutputStream outputStream, boolean log) throws IOException {
+    if (!quickHullLearned) {
+      // this is necessary, otherwise serialized twice by timeseriesMetadata and chunkMetadata
+      // causing learn() executed more than once!!
+      quickHullBitSet = QuickHull.quickHull(quickHullPoints);
+      quickHullLearned = true;
+      quickHullPoints = null;
+    }
+    //    System.out.println("debug:::::" + quickHullBitSet);
+
+    // write bitset
+    int byteLen = 0;
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    ObjectOutputStream oos = new ObjectOutputStream(baos);
+    oos.writeObject(quickHullBitSet);
+    oos.flush();
+    byte[] bytes = baos.toByteArray();
+    // 参考serializeBloomFilter
+    byteLen += ReadWriteForEncodingUtils.writeUnsignedVarInt(bytes.length, outputStream);
+    outputStream.write(bytes);
+    byteLen += bytes.length;
+
+    if (log) {
+      LOG.info("convex hull,{}", bytes.length);
+    }
+    return byteLen;
+  }
+
   abstract int serializeStats(OutputStream outputStream) throws IOException;
 
   /** read data from the inputStream. */
@@ -273,12 +318,17 @@ public abstract class Statistics<T> {
       // must be sure no overlap between two statistics
       this.count += stats.count;
       mergeStatisticsValue(stats);
-      // TODO M4-LSM assumes that there is always only one page in a chunk
-      // TODO M4-LSM if there are more than one chunk in a time series, then access each
-      // chunkMetadata anyway
-      this.stepRegress = stats.stepRegress;
-      // TODO
-      this.valueIndex = stats.valueIndex;
+      // TODO assumes that there is always only one page in a chunk
+      // TODO if there are more than one chunk in a time series, then access each
+      //    chunkMetadata anyway
+      //      this.stepRegress = stats.stepRegress;
+      //      this.valueIndex = stats.valueIndex;
+
+      // 注意这里要把points也一并赋值,因为当chunk里只有一个page的时候只serialize chunk的metadata
+      this.quickHullPoints = stats.quickHullPoints;
+      this.quickHullBitSet = stats.quickHullBitSet;
+      this.quickHullIdx = stats.quickHullIdx;
+      this.quickHullLearned = stats.quickHullLearned;
 
       isEmpty = false;
     } else {
@@ -311,10 +361,11 @@ public abstract class Statistics<T> {
     if (time > this.endTime) {
       endTime = time;
     }
+    updateStats(value, time); // TP,BP,sum,lastValue,firstValue
     // update time index
-    updateStepRegress(time);
-    updateValueIndex(value);
-    updateStats(value, time);
+    //    updateStepRegress(time);
+    //    updateValueIndex(value);
+    updateConvexHull(time, value);
   }
 
   /** @author Yuyuan Kang */
@@ -326,9 +377,10 @@ public abstract class Statistics<T> {
     if (time > this.endTime) {
       endTime = time;
     }
-    updateStepRegress(time);
-    updateValueIndex(value);
     updateStats(value, time);
+    //    updateStepRegress(time);
+    //    updateValueIndex(value);
+    updateConvexHull(time, value);
   }
 
   /** @author Yuyuan Kang */
@@ -340,9 +392,10 @@ public abstract class Statistics<T> {
     if (time > this.endTime) {
       endTime = time;
     }
-    updateStepRegress(time);
-    updateValueIndex(value);
     updateStats(value, time);
+    //    updateStepRegress(time);
+    //    updateValueIndex(value);
+    updateConvexHull(time, value);
   }
 
   /** @author Yuyuan Kang */
@@ -354,9 +407,10 @@ public abstract class Statistics<T> {
     if (time > this.endTime) {
       endTime = time;
     }
-    updateStepRegress(time);
-    updateValueIndex(value);
     updateStats(value, time);
+    //    updateStepRegress(time);
+    //    updateValueIndex(value);
+    updateConvexHull(time, value);
   }
 
   @Deprecated
@@ -392,9 +446,10 @@ public abstract class Statistics<T> {
     if (time[batchSize - 1] > this.endTime) {
       endTime = time[batchSize - 1];
     }
-    updateStepRegress(time, batchSize);
-    updateValueIndex(values, batchSize);
     updateStats(values, time, batchSize);
+    //    updateStepRegress(time, batchSize);
+    //    updateValueIndex(values, batchSize);
+    updateConvexHull(time, values, batchSize);
   }
 
   /** @author Yuyuan Kang */
@@ -406,9 +461,10 @@ public abstract class Statistics<T> {
     if (time[batchSize - 1] > this.endTime) {
       endTime = time[batchSize - 1];
     }
-    updateStepRegress(time, batchSize);
-    updateValueIndex(values, batchSize);
     updateStats(values, time, batchSize);
+    //    updateStepRegress(time, batchSize);
+    //    updateValueIndex(values, batchSize);
+    updateConvexHull(time, values, batchSize);
   }
 
   /** @author Yuyuan Kang */
@@ -420,9 +476,10 @@ public abstract class Statistics<T> {
     if (time[batchSize - 1] > this.endTime) {
       endTime = time[batchSize - 1];
     }
-    updateStepRegress(time, batchSize);
-    updateValueIndex(values, batchSize);
     updateStats(values, time, batchSize);
+    //    updateStepRegress(time, batchSize);
+    //    updateValueIndex(values, batchSize);
+    updateConvexHull(time, values, batchSize);
   }
 
   /** @author Yuyuan Kang */
@@ -434,9 +491,10 @@ public abstract class Statistics<T> {
     if (time[batchSize - 1] > this.endTime) {
       endTime = time[batchSize - 1];
     }
-    updateStepRegress(time, batchSize);
-    updateValueIndex(values, batchSize);
     updateStats(values, time, batchSize);
+    //    updateStepRegress(time, batchSize);
+    //    updateValueIndex(values, batchSize);
+    updateConvexHull(time, values, batchSize);
   }
 
   @Deprecated
@@ -475,12 +533,40 @@ public abstract class Statistics<T> {
     stepRegress.insert(timestamp);
   }
 
+  void updateConvexHull(long timestamp, Object value) {
+    quickHullPoints.add(new QuickHullPoint(timestamp, (double) value, quickHullIdx++));
+  }
+
   void updateStepRegress(long[] timestamps, int batchSize) {
     for (int i = 0; i < batchSize; i++) {
       updateStepRegress(timestamps[i]);
     }
   }
 
+  void updateConvexHull(long[] timestamps, int[] values, int batchSize) {
+    for (int i = 0; i < batchSize; i++) {
+      updateConvexHull(timestamps[i], values[i]);
+    }
+  }
+
+  void updateConvexHull(long[] timestamps, long[] values, int batchSize) {
+    for (int i = 0; i < batchSize; i++) {
+      updateConvexHull(timestamps[i], values[i]);
+    }
+  }
+
+  void updateConvexHull(long[] timestamps, float[] values, int batchSize) {
+    for (int i = 0; i < batchSize; i++) {
+      updateConvexHull(timestamps[i], values[i]);
+    }
+  }
+
+  void updateConvexHull(long[] timestamps, double[] values, int batchSize) {
+    for (int i = 0; i < batchSize; i++) {
+      updateConvexHull(timestamps[i], values[i]);
+    }
+  }
+
   void updateValueIndex(int value) {
     valueIndex.insert(value);
   }
@@ -601,8 +687,9 @@ public abstract class Statistics<T> {
     statistics.setStartTime(ReadWriteIOUtils.readLong(buffer));
     statistics.setEndTime(ReadWriteIOUtils.readLong(buffer));
     statistics.deserialize(buffer);
-    statistics.deserializeStepRegress(buffer);
-    statistics.deserializeValueIndex(buffer);
+    //    statistics.deserializeStepRegress(buffer);
+    //    statistics.deserializeValueIndex(buffer);
+    statistics.deserializeConvexHull(buffer);
     statistics.isEmpty = false;
     return statistics;
   }
@@ -722,6 +809,18 @@ public abstract class Statistics<T> {
     }
   }
 
+  void deserializeConvexHull(ByteBuffer byteBuffer) throws IOException {
+    try {
+      byte[] bytes = ReadWriteIOUtils.readByteBufferWithSelfDescriptionLength(byteBuffer);
+      ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+      ObjectInputStream ois = new ObjectInputStream(bais);
+      quickHullBitSet = (BitSet) ois.readObject();
+      //      System.out.println("debug:::::" + quickHullBitSet);
+    } catch (ClassNotFoundException e) {
+      System.out.println("deserializeConvexHull: " + e);
+    }
+  }
+
   public long getStartTime() {
     return startTime;
   }
@@ -734,6 +833,10 @@ public abstract class Statistics<T> {
     return stepRegress;
   }
 
+  public BitSet getQuickHullBitSet() {
+    return quickHullBitSet;
+  }
+
   public int getCount() {
     return count;
   }