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/09/29 12:21:34 UTC

[GitHub] [iotdb] zyk990424 opened a new pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and VectorMeasurementMNode

zyk990424 opened a new pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056


   ## Description
   
   ### Motivition
   1. Introduce UnaryMeasurementMNode and VectorMeasurementMNode for better readability and maintainability. The logical process of such two scenarios will be more clear.
   2. Check the MNode usage inside and outside metadata module and eliminate outside usage as much as possible. This obeys OOD principle and make metadata serialization work more easier.
   
   ### Modification
   
   #### Introduce UnaryMeasurementMNode and VectorMeasurementMNode
   1. Make origin MeasurementMNode abstract and introduce two subclass, UnaryMeasurementMNode and VectorMeasurementMNode.
   2. Use factory method to get and init MeasurementMNode.
   3. Encapsulate MNode type transform/cast and eliminate force type cast code as much as possible
   4. Introduce MNodeTypeMismatchException
   
   #### Improve MNode usage
   1. Replace getNodeByPath method in MManager with type specified MNode access method
   2. Eliminate deviceMNode usage in MergeTask.class 
   3. Transfer some schema collection code from MManager to MTree
   4. Refactor LastCache interface and usage using New MeasurementMNode. Fix potential bug
   
   ### Coming work
   Renaming vector to multi, subMeasurement to component, including MeasurementMNode and MeasurementSchema.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43158532/badge)](https://coveralls.io/builds/43158532)
   
   Coverage increased (+0.01%) to 67.755% when pulling **cb0026f19ab53f63b4f35ad23355149bab0a23d7 on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
zyk990424 commented on a change in pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#discussion_r720304318



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -1203,9 +1205,21 @@ private void createSchemaAutomatically(
     Set<PartialPath> registeredSeries = new HashSet<>();
     for (ChunkGroupMetadata chunkGroupMetadata : chunkGroupMetadataList) {
       String device = chunkGroupMetadata.getDevice();
-      IMNode node =
-          IoTDB.metaManager.getDeviceNodeWithAutoCreate(
-              new PartialPath(device), true, true, sgLevel);
+      Set<String> existSeriesSet = new HashSet<>();
+      PartialPath devicePath = new PartialPath(device);
+      PartialPath storageGroupPath = MetaUtils.getStorageGroupPathByLevel(devicePath, sgLevel);
+      try {
+        IoTDB.metaManager.setStorageGroup(storageGroupPath);

Review comment:
       The only one exception needed to process is StorageGroupAlreadySetException and the process code is implemented. The other exceptions should be throw to inform user what's wrong with there operation.




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43214554/badge)](https://coveralls.io/builds/43214554)
   
   Coverage decreased (-0.006%) to 67.739% when pulling **3ab802c779c6f11b634a22775f7586fc8b66be15 on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] qiaojialin commented on a change in pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#discussion_r720062060



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/UnaryMeasurementMNode.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.db.metadata.mnode;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+/** Represents an MNode which has a Measurement or Sensor attached to it. */
+public class UnaryMeasurementMNode extends MeasurementMNode {
+
+  /** measurement's Schema for one timeseries represented by current leaf node */
+  private MeasurementSchema schema;

Review comment:
       ```suggestion
     private UnaryMeasurementSchema schema;
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -1203,9 +1205,21 @@ private void createSchemaAutomatically(
     Set<PartialPath> registeredSeries = new HashSet<>();
     for (ChunkGroupMetadata chunkGroupMetadata : chunkGroupMetadataList) {
       String device = chunkGroupMetadata.getDevice();
-      IMNode node =
-          IoTDB.metaManager.getDeviceNodeWithAutoCreate(
-              new PartialPath(device), true, true, sgLevel);
+      Set<String> existSeriesSet = new HashSet<>();
+      PartialPath devicePath = new PartialPath(device);
+      PartialPath storageGroupPath = MetaUtils.getStorageGroupPathByLevel(devicePath, sgLevel);
+      try {
+        IoTDB.metaManager.setStorageGroup(storageGroupPath);

Review comment:
       This may throw many exceptions... try to check first or getOrCreateStorageGroup 

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementFlatCollector.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.db.metadata.mtree.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.MultiMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.UnaryMeasurementMNode;
+
+import java.util.List;
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+
+// This class defines MeasurementMNode as target node and

Review comment:
       ```suggestion
   // This class defines UnaryMeasurementMNode as target node and
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1011,39 +1078,51 @@ protected void transferToResult(IMNode node) {
   /**
    * Get the count of timeseries matching the given path.
    *
-   * @param path a path pattern or a full path, may contain wildcard
+   * @param pathPattern a path pattern or a full path, may contain wildcard
    */
-  public int getAllTimeseriesCount(PartialPath path) throws MetadataException {
-    CounterTraverser counter = new MeasurementCounter(root, path);
+  public int getAllTimeseriesCount(PartialPath pathPattern) throws MetadataException {
+    CounterTraverser counter = new MeasurementCounter(root, pathPattern);
+    counter.traverse();
+    return counter.getCount();
+  }
+
+  /**
+   * Get the count of timeseries component matching the given path.
+   *
+   * @param pathPattern a path pattern or a full path, may contain wildcard
+   */
+  public int getAllTimeseriesFlatCount(PartialPath pathPattern) throws MetadataException {
+    CounterTraverser counter = new MeasurementFlatCounter(root, pathPattern);
     counter.traverse();

Review comment:
       put the traverse to getCount() methods looks more compact;
   
   getCount() {
   if(!hasTraverse) {
   traverse;
   hasTraverse=true;
   }
   return xxx;
   }

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementSchemaFlatCollector.java
##########
@@ -34,20 +36,21 @@
 import static org.apache.iotdb.db.metadata.lastCache.LastCacheManager.getLastTimeStamp;
 
 // This class implements the measurement collection function.
-public class MeasurementSchemaCollector
-    extends MeasurementCollector<List<Pair<PartialPath, String[]>>> {
+public class MeasurementSchemaFlatCollector

Review comment:
       ```suggestion
   public class FlatMeasurementSchemaCollector
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1097,6 +1080,37 @@ public String getDeviceId(PartialPath devicePath) {
   // endregion
 
   // region Interfaces for timeseries, measurement and schema info Query
+
+  /**
+   * PartialPath of aligned time series will be organized to one VectorPartialPath. BEFORE this
+   * method, all the aligned time series is NOT united. For example,
+   * VectorMeasurementSchema(root.sg.d1.vector1 [s1,s2]) will be * root.sg.d1.vector1[s1],
+   * root.sg.d1.vector1[s2]
+   *
+   * @param fullPaths full path list without uniting the sub measurement under the same aligned time
+   *     series.
+   * @return Size of partial path list could NOT equal to the input list size. For example, the
+   *     VectorMeasurementSchema (s1,s2) would be returned once.
+   */
+  public List<PartialPath> groupVectorPaths(List<PartialPath> fullPaths) throws MetadataException {

Review comment:
       javadoc
   
   E.g.,
   Input:
   Output:

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementFlatCollector.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.db.metadata.mtree.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.MultiMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.UnaryMeasurementMNode;
+
+import java.util.List;
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+
+// This class defines MeasurementMNode as target node and
+// defines the measurement component process framework.
+// Each component of MultiMeasurement will be processed.
+public abstract class MeasurementFlatCollector<T> extends CollectorTraverser<T> {

Review comment:
       ```suggestion
   public abstract class FlatMeasurementCollector<T> extends CollectorTraverser<T> {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/MeasurementCounter.java
##########
@@ -21,19 +21,9 @@
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
-
-import java.util.List;
-import java.util.regex.Pattern;
-
-import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
 
 // This method implements the measurement count function.
-// TODO distinguish timeseries count and measurement count, an aligned timeseries stands for one
-// timeseries but several measurement
+// One MultiMeasurement represent one aligned timeseries will only be count once.

Review comment:
       ```suggestion
   // One MultiMeasurement will only be count once.
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementPathFlatCollector.java
##########
@@ -22,26 +22,27 @@
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.VectorPartialPath;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.MultiMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.UnaryMeasurementMNode;
 
 import java.util.LinkedList;
 import java.util.List;
 
 // This class implements the measurement path collection function.
-public class MeasurementPathCollector extends MeasurementCollector<List<PartialPath>> {
+public class MeasurementPathFlatCollector extends MeasurementFlatCollector<List<PartialPath>> {

Review comment:
       ```suggestion
   public class FlatMeasurementPathCollector extends FlatMeasurementCollector<List<PartialPath>> {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/MeasurementFlatCounter.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.db.metadata.mtree.traverser.counter;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+
+import java.util.List;
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+
+// This method implements the measurement component count function.
+// Each component of MultiMeasurement will be count once.
+public class MeasurementFlatCounter extends CounterTraverser {

Review comment:
       ```suggestion
   public class FlatMeasurementCounter extends CounterTraverser {
   ```
   Add javadoc example
   
   MTree: root.sg.d.vector[s1,s2], root.sg.d.s3
   FlatMeasurementCounter.count(root) = 3

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1097,6 +1080,37 @@ public String getDeviceId(PartialPath devicePath) {
   // endregion
 
   // region Interfaces for timeseries, measurement and schema info Query
+
+  /**
+   * PartialPath of aligned time series will be organized to one VectorPartialPath. BEFORE this
+   * method, all the aligned time series is NOT united. For example,
+   * VectorMeasurementSchema(root.sg.d1.vector1 [s1,s2]) will be * root.sg.d1.vector1[s1],

Review comment:
       ```suggestion
      * VectorMeasurementSchema(root.sg.d1.vector1 [s1,s2]) will be 
      * root.sg.d1.vector1[s1],
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/TSEntityPathCollector.java
##########
@@ -67,10 +65,10 @@ protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
     if (!node.isMeasurement()) {
       return false;
     }

Review comment:
       TSEntityPathCollector -> EntityPathCollector

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MetadataConstant.java
##########
@@ -40,7 +40,28 @@ private MetadataConstant() {
   public static final String MTREE_SNAPSHOT_TMP =
       MTREE_PREFIX + IoTDBConstant.FILE_NAME_SEPARATOR + MTREE_VERSION + ".snapshot.bin.tmp";
 
-  public static final short MNODE_TYPE = 0;
-  public static final short STORAGE_GROUP_MNODE_TYPE = 1;
-  public static final short MEASUREMENT_MNODE_TYPE = 2;
+  public static final byte INTERNAL_MNODE_TYPE = 0;
+  public static final byte STORAGE_GROUP_MNODE_TYPE = 1;
+  public static final byte MEASUREMENT_MNODE_TYPE = 2;
+  public static final byte ENTITY_MNODE_TYPE = 3;

Review comment:
       How to upgrade from 0.12?




-- 
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



[GitHub] [iotdb] coveralls commented on pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43150361/badge)](https://coveralls.io/builds/43150361)
   
   Coverage increased (+0.02%) to 67.761% when pulling **467c718b3b2a149523c834e0b1db93f3af6d1cff on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43213548/badge)](https://coveralls.io/builds/43213548)
   
   Coverage increased (+0.02%) to 67.764% when pulling **a28fb7251bb5d6a219552c5af49a67cf775eeaf5 on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43151105/badge)](https://coveralls.io/builds/43151105)
   
   Coverage decreased (-0.001%) to 67.744% when pulling **467c718b3b2a149523c834e0b1db93f3af6d1cff on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
zyk990424 commented on a change in pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#discussion_r720300004



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementSchemaFlatCollector.java
##########
@@ -34,20 +36,21 @@
 import static org.apache.iotdb.db.metadata.lastCache.LastCacheManager.getLastTimeStamp;
 
 // This class implements the measurement collection function.
-public class MeasurementSchemaCollector
-    extends MeasurementCollector<List<Pair<PartialPath, String[]>>> {
+public class MeasurementSchemaFlatCollector

Review comment:
       Fixed.




-- 
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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
zyk990424 commented on a change in pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#discussion_r720301774



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/TSEntityPathCollector.java
##########
@@ -67,10 +65,10 @@ protected boolean processFullMatchedMNode(IMNode node, int idx, int level) {
     if (!node.isMeasurement()) {
       return false;
     }

Review comment:
       I renamed this class to BelongedEntityPathCollector. This will describe the task more clearly and this class do have difference with EntityPathCollector. The input pattern is to match measurement rather than entity.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/MeasurementCounter.java
##########
@@ -21,19 +21,9 @@
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
-
-import java.util.List;
-import java.util.regex.Pattern;
-
-import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
 
 // This method implements the measurement count function.
-// TODO distinguish timeseries count and measurement count, an aligned timeseries stands for one
-// timeseries but several measurement
+// One MultiMeasurement represent one aligned timeseries will only be count once.

Review comment:
       Fixed.




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43153291/badge)](https://coveralls.io/builds/43153291)
   
   Coverage decreased (-0.01%) to 67.73% when pulling **1e8a9edf9468366692a2c4c210d7ec91607c8204 on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
zyk990424 commented on a change in pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#discussion_r720298779



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1011,39 +1078,51 @@ protected void transferToResult(IMNode node) {
   /**
    * Get the count of timeseries matching the given path.
    *
-   * @param path a path pattern or a full path, may contain wildcard
+   * @param pathPattern a path pattern or a full path, may contain wildcard
    */
-  public int getAllTimeseriesCount(PartialPath path) throws MetadataException {
-    CounterTraverser counter = new MeasurementCounter(root, path);
+  public int getAllTimeseriesCount(PartialPath pathPattern) throws MetadataException {
+    CounterTraverser counter = new MeasurementCounter(root, pathPattern);
+    counter.traverse();
+    return counter.getCount();
+  }
+
+  /**
+   * Get the count of timeseries component matching the given path.
+   *
+   * @param pathPattern a path pattern or a full path, may contain wildcard
+   */
+  public int getAllTimeseriesFlatCount(PartialPath pathPattern) throws MetadataException {
+    CounterTraverser counter = new MeasurementFlatCounter(root, pathPattern);
     counter.traverse();

Review comment:
       I think keep two these two steps is clear and flexible. It helps remind user to invoke traverse() before getCount() while the MTree is changed.




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43216411/badge)](https://coveralls.io/builds/43216411)
   
   Coverage increased (+0.004%) to 67.749% when pulling **16a39f44c7e348d6232d95b53b546b76a3487e1a on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43159048/badge)](https://coveralls.io/builds/43159048)
   
   Coverage decreased (-0.009%) to 67.737% when pulling **617424edddab67872acaaac376d4259dfb9c0377 on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177






-- 
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



[GitHub] [iotdb] qiaojialin merged pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
qiaojialin merged pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056


   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43150597/badge)](https://coveralls.io/builds/43150597)
   
   Coverage increased (+0.005%) to 67.75% when pulling **467c718b3b2a149523c834e0b1db93f3af6d1cff on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls commented on pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43150361/badge)](https://coveralls.io/builds/43150361)
   
   Coverage increased (+0.02%) to 67.761% when pulling **467c718b3b2a149523c834e0b1db93f3af6d1cff on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43179557/badge)](https://coveralls.io/builds/43179557)
   
   Coverage increased (+0.02%) to 67.769% when pulling **faa66eadc743d5096c84740360ddd89e75a71dc9 on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
zyk990424 commented on a change in pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#discussion_r720295314



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1097,6 +1080,37 @@ public String getDeviceId(PartialPath devicePath) {
   // endregion
 
   // region Interfaces for timeseries, measurement and schema info Query
+
+  /**
+   * PartialPath of aligned time series will be organized to one VectorPartialPath. BEFORE this
+   * method, all the aligned time series is NOT united. For example,
+   * VectorMeasurementSchema(root.sg.d1.vector1 [s1,s2]) will be * root.sg.d1.vector1[s1],

Review comment:
       Fixed.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1097,6 +1080,37 @@ public String getDeviceId(PartialPath devicePath) {
   // endregion
 
   // region Interfaces for timeseries, measurement and schema info Query
+
+  /**
+   * PartialPath of aligned time series will be organized to one VectorPartialPath. BEFORE this
+   * method, all the aligned time series is NOT united. For example,
+   * VectorMeasurementSchema(root.sg.d1.vector1 [s1,s2]) will be * root.sg.d1.vector1[s1],
+   * root.sg.d1.vector1[s2]
+   *
+   * @param fullPaths full path list without uniting the sub measurement under the same aligned time
+   *     series.
+   * @return Size of partial path list could NOT equal to the input list size. For example, the
+   *     VectorMeasurementSchema (s1,s2) would be returned once.
+   */
+  public List<PartialPath> groupVectorPaths(List<PartialPath> fullPaths) throws MetadataException {

Review comment:
       Fixed.




-- 
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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
zyk990424 commented on a change in pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#discussion_r720302101



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/counter/MeasurementFlatCounter.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.db.metadata.mtree.traverser.counter;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+
+import java.util.List;
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+
+// This method implements the measurement component count function.
+// Each component of MultiMeasurement will be count once.
+public class MeasurementFlatCounter extends CounterTraverser {

Review comment:
       Fixed.




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43159945/badge)](https://coveralls.io/builds/43159945)
   
   Coverage increased (+0.007%) to 67.753% when pulling **617424edddab67872acaaac376d4259dfb9c0377 on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4056: [IOTDB-1675] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#issuecomment-930153177


   
   [![Coverage Status](https://coveralls.io/builds/43153335/badge)](https://coveralls.io/builds/43153335)
   
   Coverage increased (+0.007%) to 67.753% when pulling **1e8a9edf9468366692a2c4c210d7ec91607c8204 on zyk990424:MultiMeasurement** into **0163d3aa76d452fe2c1d864790451dcfd3d5cd5e on apache:master**.
   


-- 
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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
zyk990424 commented on a change in pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#discussion_r720299683



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementFlatCollector.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.db.metadata.mtree.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.MultiMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.UnaryMeasurementMNode;
+
+import java.util.List;
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+
+// This class defines MeasurementMNode as target node and

Review comment:
       This class do concerns UnaryMeasurementMNode and MultiMeasurementMNode both as target MNode. I think this doc is fine.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementFlatCollector.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.db.metadata.mtree.traverser.collector;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.MultiMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.UnaryMeasurementMNode;
+
+import java.util.List;
+import java.util.regex.Pattern;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
+
+// This class defines MeasurementMNode as target node and
+// defines the measurement component process framework.
+// Each component of MultiMeasurement will be processed.
+public abstract class MeasurementFlatCollector<T> extends CollectorTraverser<T> {

Review comment:
       Fixed.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementPathFlatCollector.java
##########
@@ -22,26 +22,27 @@
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.VectorPartialPath;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.MultiMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.UnaryMeasurementMNode;
 
 import java.util.LinkedList;
 import java.util.List;
 
 // This class implements the measurement path collection function.
-public class MeasurementPathCollector extends MeasurementCollector<List<PartialPath>> {
+public class MeasurementPathFlatCollector extends MeasurementFlatCollector<List<PartialPath>> {

Review comment:
       Fixed.




-- 
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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
zyk990424 commented on a change in pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#discussion_r720297394



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/UnaryMeasurementMNode.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.db.metadata.mnode;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+/** Represents an MNode which has a Measurement or Sensor attached to it. */
+public class UnaryMeasurementMNode extends MeasurementMNode {
+
+  /** measurement's Schema for one timeseries represented by current leaf node */
+  private MeasurementSchema schema;

Review comment:
       Fixed. There will be a large quantity of codes for replacing.




-- 
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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #4056: [IOTDB-1675][IOTDB-1785] Introduce UnaryMeasurementMNode and MultiMeasurementMNode

Posted by GitBox <gi...@apache.org>.
zyk990424 commented on a change in pull request #4056:
URL: https://github.com/apache/iotdb/pull/4056#discussion_r720296631



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MetadataConstant.java
##########
@@ -40,7 +40,28 @@ private MetadataConstant() {
   public static final String MTREE_SNAPSHOT_TMP =
       MTREE_PREFIX + IoTDBConstant.FILE_NAME_SEPARATOR + MTREE_VERSION + ".snapshot.bin.tmp";
 
-  public static final short MNODE_TYPE = 0;
-  public static final short STORAGE_GROUP_MNODE_TYPE = 1;
-  public static final short MEASUREMENT_MNODE_TYPE = 2;
+  public static final byte INTERNAL_MNODE_TYPE = 0;
+  public static final byte STORAGE_GROUP_MNODE_TYPE = 1;
+  public static final byte MEASUREMENT_MNODE_TYPE = 2;
+  public static final byte ENTITY_MNODE_TYPE = 3;

Review comment:
       Currently, they are only used in MNodeTypeMismatchException and there are no problem with upgrade. 




-- 
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