You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/11/07 16:22:06 UTC

[GitHub] [iotdb] qiaojialin commented on a change in pull request #4331: [IOTDB-1626]TsFile API supports write and register on aligned timeseries

qiaojialin commented on a change in pull request #4331:
URL: https://github.com/apache/iotdb/pull/4331#discussion_r744278030



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/Schema.java
##########
@@ -33,77 +33,84 @@
 public class Schema implements Serializable {
 
   /**
-   * Path (device + measurement) -> measurementSchema By default, use the LinkedHashMap to store the
-   * order of insertion
+   * Path (devicePath + DeviceInfo) -> measurementSchema By default, use the LinkedHashMap to store

Review comment:
       what is DeviceInfo?

##########
File path: example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteAlignedWithTablet.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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;
+
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.write.TsFileWriter;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.UnaryMeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TsFileWriteAlignedWithTablet {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileWriteAlignedWithTablet.class);
+  private static String deviceId = "root.sg.d1";
+
+  public static void main(String[] args) throws IOException {
+    File f = FSFactoryProducer.getFSFactory().getFile("alignedTablet.tsfile");
+    if (f.exists() && !f.delete()) {
+      throw new RuntimeException("can not delete " + f.getAbsolutePath());
+    }
+    try (TsFileWriter tsFileWriter = new TsFileWriter(f)) {
+      List<UnaryMeasurementSchema> measurementSchemas = new ArrayList<>();
+      measurementSchemas.add(new UnaryMeasurementSchema("s1", TSDataType.TEXT, TSEncoding.PLAIN));
+      measurementSchemas.add(new UnaryMeasurementSchema("s2", TSDataType.TEXT, TSEncoding.PLAIN));
+      measurementSchemas.add(new UnaryMeasurementSchema("s3", TSDataType.TEXT, TSEncoding.PLAIN));
+      measurementSchemas.add(new UnaryMeasurementSchema("s4", TSDataType.INT64, TSEncoding.RLE));
+
+      // register align timeseries
+      tsFileWriter.registerAlignedTimeseries(new Path(deviceId), measurementSchemas);
+
+      List<IMeasurementSchema> writeMeasurementScheams = new ArrayList<>();
+      // example 1
+      writeMeasurementScheams.add(measurementSchemas.get(0));
+      writeAlignedWithTablet(tsFileWriter, deviceId, writeMeasurementScheams, 10, 0, 0);
+
+      // example 2
+      writeMeasurementScheams.clear();
+      writeMeasurementScheams.add(measurementSchemas.get(0));
+      writeMeasurementScheams.add(measurementSchemas.get(1));
+      writeAlignedWithTablet(tsFileWriter, deviceId, writeMeasurementScheams, 200000, 10, 0);
+
+      // example 3
+      writeMeasurementScheams.clear();
+      writeMeasurementScheams.add(measurementSchemas.get(2));
+      writeAlignedWithTablet(tsFileWriter, deviceId, writeMeasurementScheams, 10, 0, 0);
+
+      writeWithTablet(tsFileWriter); // write nonAligned timeseries
+    } catch (WriteProcessException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private static void writeAlignedWithTablet(
+      TsFileWriter tsFileWriter,
+      String deviceId,
+      List<IMeasurementSchema> schemas,
+      long rowNum,
+      long startTime,
+      long startValue)
+      throws IOException, WriteProcessException {
+    Tablet tablet = new Tablet(deviceId, schemas);
+    long[] timestamps = tablet.timestamps;
+    Object[] values = tablet.values;
+    long sensorNum = schemas.size();
+
+    for (long r = 0; r < rowNum; r++, startValue++) {
+      int row = tablet.rowSize++;
+      timestamps[row] = startTime++;
+      for (int i = 0; i < sensorNum; i++) {
+        Binary[] textSensor = (Binary[]) values[i];
+        textSensor[row] = new Binary("testString.........");
+      }
+      // write
+      if (tablet.rowSize == tablet.getMaxRowNumber()) {
+        tsFileWriter.writeAligned(tablet);
+        tablet.reset();
+      }
+    }
+    // write
+    if (tablet.rowSize != 0) {
+      tsFileWriter.writeAligned(tablet);
+      tablet.reset();
+    }
+  }
+
+  private static void writeWithTablet(TsFileWriter tsFileWriter)
+      throws WriteProcessException, IOException {
+    // register nonAlign timeseries
+    tsFileWriter.registerTimeseries(
+        new Path("root.sg.d2"), new UnaryMeasurementSchema("s1", TSDataType.INT64, TSEncoding.RLE));
+    tsFileWriter.registerTimeseries(
+        new Path("root.sg.d2"), new UnaryMeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE));
+    // construct Tablet
+    List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
+    measurementSchemas.add(new UnaryMeasurementSchema("s1", TSDataType.INT64, TSEncoding.RLE));
+    measurementSchemas.add(new UnaryMeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE));
+    Tablet tablet = new Tablet("root.sg.d2", measurementSchemas);
+    long[] timestamps = tablet.timestamps;
+    Object[] values = tablet.values;
+    int rowNum = 100;
+    int sensorNum = measurementSchemas.size();
+    long timestamp = 1;
+    long value = 1000000L;
+    for (int r = 0; r < rowNum; r++, value++) {
+      int row = tablet.rowSize++;
+      timestamps[row] = timestamp++;
+      for (int i = 0; i < sensorNum; i++) {
+        long[] sensor = (long[]) values[i];
+        sensor[row] = value;
+      }
+      // write
+      if (tablet.rowSize == tablet.getMaxRowNumber()) {
+        tsFileWriter.write(tablet);

Review comment:
       ```suggestion
           tsFileWriter.writeNonAligned(tablet);
   ```
   Make some distinguish

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/PageHeader.java
##########
@@ -54,6 +54,9 @@ public static int estimateMaxPageHeaderSizeWithoutStatistics() {
   public static PageHeader deserializeFrom(
       InputStream inputStream, TSDataType dataType, boolean hasStatistic) throws IOException {
     int uncompressedSize = ReadWriteForEncodingUtils.readUnsignedVarInt(inputStream);
+    if (uncompressedSize == 0) { // Empty Page

Review comment:
       when we get an empty page?

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/Tablet.java
##########
@@ -288,7 +287,7 @@ public int getTotalValueOccupation() {
     int columnIndex = 0;
     for (int i = 0; i < schemas.size(); i++) {
       IMeasurementSchema schema = schemas.get(i);
-      if (schema instanceof UnaryMeasurementSchema) {
+      if (schema instanceof IMeasurementSchema) {

Review comment:
       why change this?

##########
File path: example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteAlignedWithTSRecord.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.write.TsFileWriter;
+import org.apache.iotdb.tsfile.write.record.TSRecord;
+import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
+import org.apache.iotdb.tsfile.write.record.datapoint.LongDataPoint;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.UnaryMeasurementSchema;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TsFileWriteAlignedWithTSRecord {
+
+  public static void main(String[] args) throws IOException {
+    File f = FSFactoryProducer.getFSFactory().getFile("alignedRecord.tsfile");
+    if (f.exists() && !f.delete()) {
+      throw new RuntimeException("can not delete " + f.getAbsolutePath());
+    }
+    try (TsFileWriter tsFileWriter = new TsFileWriter(f)) {
+      List<UnaryMeasurementSchema> measurementSchemas = new ArrayList<>();
+      measurementSchemas.add(new UnaryMeasurementSchema("s1", TSDataType.INT64, TSEncoding.RLE));
+      measurementSchemas.add(new UnaryMeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE));
+      measurementSchemas.add(new UnaryMeasurementSchema("s3", TSDataType.INT64, TSEncoding.RLE));

Review comment:
       As User could only see UnaryMeasurementSchema, we could rename this to MeasurementSchema

##########
File path: example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteAlignedWithTablet.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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;
+
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.write.TsFileWriter;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.UnaryMeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TsFileWriteAlignedWithTablet {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileWriteAlignedWithTablet.class);
+  private static String deviceId = "root.sg.d1";
+
+  public static void main(String[] args) throws IOException {
+    File f = FSFactoryProducer.getFSFactory().getFile("alignedTablet.tsfile");
+    if (f.exists() && !f.delete()) {
+      throw new RuntimeException("can not delete " + f.getAbsolutePath());
+    }
+    try (TsFileWriter tsFileWriter = new TsFileWriter(f)) {
+      List<UnaryMeasurementSchema> measurementSchemas = new ArrayList<>();
+      measurementSchemas.add(new UnaryMeasurementSchema("s1", TSDataType.TEXT, TSEncoding.PLAIN));
+      measurementSchemas.add(new UnaryMeasurementSchema("s2", TSDataType.TEXT, TSEncoding.PLAIN));
+      measurementSchemas.add(new UnaryMeasurementSchema("s3", TSDataType.TEXT, TSEncoding.PLAIN));
+      measurementSchemas.add(new UnaryMeasurementSchema("s4", TSDataType.INT64, TSEncoding.RLE));
+
+      // register align timeseries
+      tsFileWriter.registerAlignedTimeseries(new Path(deviceId), measurementSchemas);
+
+      List<IMeasurementSchema> writeMeasurementScheams = new ArrayList<>();

Review comment:
       Registering N and write M<N is a particular case, more cases is M=N.
   So better to give an example that just use the measurementSchemas to write data.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkGroupWriterImpl.java
##########
@@ -45,150 +41,98 @@
 

Review comment:
       Rename to NonAlignedChunkGroupWriterImpl

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/TimePageReader.java
##########
@@ -53,12 +54,15 @@ public TimePageReader(PageHeader pageHeader, ByteBuffer pageData, Decoder timeDe
   }
 
   public long[] nexTimeBatch() throws IOException {
-    long[] timeBatch = new long[(int) pageHeader.getStatistics().getCount()];
-    int index = 0;
+    List<Long> timeList = new ArrayList<>();
+    // long[] timeBatch = new long[(int) pageHeader.getStatistics().getCount()]; //
+    // Todo:bug.当TimePage有多页,statistics就会为null

Review comment:
       need to remove these lines

##########
File path: example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteWithTSRecord.java
##########
@@ -39,43 +44,58 @@
 
   public static void main(String[] args) {
     try {
-      String path = "test.tsfile";
+      String path = "Record.tsfile";
       File f = FSFactoryProducer.getFSFactory().getFile(path);
       if (f.exists()) {
         f.delete();
       }
 
       try (TsFileWriter tsFileWriter = new TsFileWriter(f)) {
-        // add measurements into file schema
-        for (int i = 0; i < 4; i++) {
-          // add measurements into file schema
-          tsFileWriter.registerTimeseries(
-              new Path(Constant.DEVICE_PREFIX + i, Constant.SENSOR_1),
-              new UnaryMeasurementSchema(Constant.SENSOR_1, TSDataType.INT64, TSEncoding.RLE));
-          tsFileWriter.registerTimeseries(
-              new Path(Constant.DEVICE_PREFIX + i, Constant.SENSOR_2),
-              new UnaryMeasurementSchema(Constant.SENSOR_2, TSDataType.INT64, TSEncoding.RLE));
-          tsFileWriter.registerTimeseries(
-              new Path(Constant.DEVICE_PREFIX + i, Constant.SENSOR_3),
-              new UnaryMeasurementSchema(Constant.SENSOR_3, TSDataType.INT64, TSEncoding.RLE));
-        }
+        List<UnaryMeasurementSchema> schemas = new ArrayList<>();
+        schemas.add(new UnaryMeasurementSchema("s1", TSDataType.INT64, TSEncoding.RLE));
+        schemas.add(new UnaryMeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE));
+        schemas.add(new UnaryMeasurementSchema("s3", TSDataType.INT64, TSEncoding.RLE));
 
-        // construct TSRecord
-        for (int i = 0; i < 100; i++) {
-          TSRecord tsRecord = new TSRecord(i, Constant.DEVICE_PREFIX + (i % 4));
-          DataPoint dPoint1 = new LongDataPoint(Constant.SENSOR_1, i);
-          DataPoint dPoint2 = new LongDataPoint(Constant.SENSOR_2, i);
-          DataPoint dPoint3 = new LongDataPoint(Constant.SENSOR_3, i);
-          tsRecord.addTuple(dPoint1);
-          tsRecord.addTuple(dPoint2);
-          tsRecord.addTuple(dPoint3);
-          // write TSRecord
-          tsFileWriter.write(tsRecord);
-        }
+        // register timeseries
+        tsFileWriter.registerTimeseries(new Path("root.sg.d1"), schemas.get(0));
+        tsFileWriter.registerTimeseries(new Path("root.sg.d1"), schemas.get(1));
+        tsFileWriter.registerTimeseries(new Path("root.sg.d1"), schemas.get(2));
+
+        List<IMeasurementSchema> writeMeasurementScheams = new ArrayList<>();

Review comment:
       the same, give a most popular example.

##########
File path: example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteAlignedWithTablet.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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;
+
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.write.TsFileWriter;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.UnaryMeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TsFileWriteAlignedWithTablet {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileWriteAlignedWithTablet.class);
+  private static String deviceId = "root.sg.d1";
+
+  public static void main(String[] args) throws IOException {
+    File f = FSFactoryProducer.getFSFactory().getFile("alignedTablet.tsfile");
+    if (f.exists() && !f.delete()) {
+      throw new RuntimeException("can not delete " + f.getAbsolutePath());
+    }
+    try (TsFileWriter tsFileWriter = new TsFileWriter(f)) {
+      List<UnaryMeasurementSchema> measurementSchemas = new ArrayList<>();
+      measurementSchemas.add(new UnaryMeasurementSchema("s1", TSDataType.TEXT, TSEncoding.PLAIN));
+      measurementSchemas.add(new UnaryMeasurementSchema("s2", TSDataType.TEXT, TSEncoding.PLAIN));
+      measurementSchemas.add(new UnaryMeasurementSchema("s3", TSDataType.TEXT, TSEncoding.PLAIN));
+      measurementSchemas.add(new UnaryMeasurementSchema("s4", TSDataType.INT64, TSEncoding.RLE));
+
+      // register align timeseries
+      tsFileWriter.registerAlignedTimeseries(new Path(deviceId), measurementSchemas);
+
+      List<IMeasurementSchema> writeMeasurementScheams = new ArrayList<>();
+      // example 1
+      writeMeasurementScheams.add(measurementSchemas.get(0));
+      writeAlignedWithTablet(tsFileWriter, deviceId, writeMeasurementScheams, 10, 0, 0);
+
+      // example 2
+      writeMeasurementScheams.clear();
+      writeMeasurementScheams.add(measurementSchemas.get(0));
+      writeMeasurementScheams.add(measurementSchemas.get(1));
+      writeAlignedWithTablet(tsFileWriter, deviceId, writeMeasurementScheams, 200000, 10, 0);
+
+      // example 3
+      writeMeasurementScheams.clear();
+      writeMeasurementScheams.add(measurementSchemas.get(2));
+      writeAlignedWithTablet(tsFileWriter, deviceId, writeMeasurementScheams, 10, 0, 0);
+
+      writeWithTablet(tsFileWriter); // write nonAligned timeseries
+    } catch (WriteProcessException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private static void writeAlignedWithTablet(
+      TsFileWriter tsFileWriter,
+      String deviceId,
+      List<IMeasurementSchema> schemas,
+      long rowNum,
+      long startTime,
+      long startValue)
+      throws IOException, WriteProcessException {
+    Tablet tablet = new Tablet(deviceId, schemas);
+    long[] timestamps = tablet.timestamps;
+    Object[] values = tablet.values;
+    long sensorNum = schemas.size();
+
+    for (long r = 0; r < rowNum; r++, startValue++) {
+      int row = tablet.rowSize++;
+      timestamps[row] = startTime++;
+      for (int i = 0; i < sensorNum; i++) {
+        Binary[] textSensor = (Binary[]) values[i];
+        textSensor[row] = new Binary("testString.........");
+      }
+      // write
+      if (tablet.rowSize == tablet.getMaxRowNumber()) {
+        tsFileWriter.writeAligned(tablet);
+        tablet.reset();
+      }
+    }
+    // write
+    if (tablet.rowSize != 0) {
+      tsFileWriter.writeAligned(tablet);
+      tablet.reset();
+    }
+  }
+
+  private static void writeWithTablet(TsFileWriter tsFileWriter)

Review comment:
       ```suggestion
     private static void writeNonalignedWithTablet(TsFileWriter tsFileWriter)
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org