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/15 13:20:47 UTC

[GitHub] [iotdb] bigreybear opened a new pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

bigreybear opened a new pull request #4391:
URL: https://github.com/apache/iotdb/pull/4391


   This PR has been merged with new_vector branch in  apache repository.
   All CI in metadata package passed, while some other packages remaining to be fixed as new_vector branch.


-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -137,20 +146,20 @@ protected abstract boolean processFullMatchedMNode(IMNode node, int idx, int lev
   protected void processMultiLevelWildcard(IMNode node, int idx, int level)
       throws MetadataException {
     for (IMNode child : node.getChildren().values()) {
+      traverseContext.push(node);
       traverse(child, idx + 1, level + 1);
+      traverseContext.pop();
     }

Review comment:
       all push and pop are out of loop now




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1186,7 +1192,11 @@ protected void transferToResult(IMNode node) {
         new MNodeCollector<List<PartialPath>>(root, pathPattern) {
           @Override
           protected void transferToResult(IMNode node) {
-            resultSet.add(node.getPartialPath());
+            try {
+              resultSet.add(getCurrentPartialPath(node));
+            } catch (MetadataException e) {
+              logger.error(e.getMessage());

Review comment:
       I'm not sure how to deal with this. It is a exception will not happen actually, so I did not throw it again here.
   The exception is caused by a construct of ParitialPath, the content is nodes during a traverse which should assure to avoid illegal path. But this method override an abstract method which does not throw exception, so this catch did not throw again.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44428812/badge)](https://coveralls.io/builds/44428812)
   
   Coverage increased (+0.01%) to 66.979% when pulling **f8b9dbf4e45034ea0fad0791936eb9fb7dd5b431 on bigreybear:new_vector_with_treeTemplate** into **d9d8733f17878cd07e4510a7f9b3af92049863c3 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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -221,27 +235,77 @@ protected void processNameMatch(IMNode node, int idx, int level) throws Metadata
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    IMeasurementSchema targetSchema = upperTemplate.getSchemaMap().get(targetName);
-    if (targetSchema != null) {
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), targetSchema.getMeasurementId(), targetSchema, null),
-          idx + 1,
-          level + 1);
+
+    IMNode targetNode = upperTemplate.getDirectNode(targetName);
+    if (targetNode != null) {
+      traverseContext.push(node);
+      traverse(targetNode, idx + 1, level + 1);
+      traverseContext.pop();
     }
 
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverseContext.push(node);
+        traverse(child, idx, level + 1);
+        traverseContext.pop();
       }
     }
   }
 
   public void setPrefixMatch(boolean isPrefixMatch) {
     this.isPrefixMatch = isPrefixMatch;
   }
+
+  /**
+   * @param currentNode the node need to get the full path of
+   * @return full path from traverse start node to the current node
+   */
+  protected PartialPath getPivotPartialPath(IMNode currentNode) throws IllegalPathException {
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    return new PartialPath(builder.toString());
+  }
+
+  protected MeasurementPath getPivotMeasurementPathInTraverse(IMeasurementMNode currentNode)
+      throws MetadataException {
+    IMNode par = traverseContext.peek();
+    if (!(par instanceof IEntityMNode)) {
+      throw new MetadataException("Measurement not under entity: " + currentNode.getName());
+    }
+
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    MeasurementPath retPath =
+        new MeasurementPath(new PartialPath(builder.toString()), currentNode.getSchema());
+    retPath.setUnderAlignedEntity(((IEntityMNode) par).isAligned());
+    return retPath;

Review comment:
       fixed now




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -2051,6 +2074,12 @@ IEntityMNode setUsingSchemaTemplate(IMNode node) throws MetadataException {
     // this operation may change mtree structure and node type
     // invoke mnode.setUseTemplate is invalid
     IEntityMNode entityMNode = mtree.setToEntity(node);
+
+    // to ensure alignment adapt with former node or template
+    entityMNode.setAligned(
+        node.isEntity()
+            ? node.getAsEntityMNode().isAligned()
+            : node.getUpperTemplate().isDirectAligned());
     entityMNode.setUseTemplate(true);

Review comment:
       In this method (setUsingTemplate), it will check alignment and replace it if direct measurement exists now.
   To implement this, I add setUseTemplate into Interface IMNode.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
##########
@@ -80,9 +81,11 @@ public Template(CreateTemplatePlan plan) throws IllegalPathException {
       IMeasurementSchema curSchema;
       int size = plan.getMeasurements().get(i).size();
       if (size > 1) {
+        //

Review comment:
       remove




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1721,11 +1721,31 @@ public void deleteLastCacheByDevice(
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
       throws MetadataException, IOException {
+    // devicePath is a logical path which is parent of measurement, whether in template or not
     PartialPath devicePath = plan.getDeviceId();
     String[] measurementList = plan.getMeasurements();
     IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
 
-    // 1. get device node
+    // 1. get device node, set using template if accessed.
+

Review comment:
       Thanks! It has been removed now.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44321853/badge)](https://coveralls.io/builds/44321853)
   
   Coverage increased (+0.03%) to 66.602% when pulling **248d247579c359a2d26a7e2fb950c88ff959bbc8 on bigreybear:new_vector_with_treeTemplate** into **44fccddf553e80c29e055cb9a1d59630762917bf 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] SilverNarcissus commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1426,6 +1435,53 @@ public void checkTemplateOnPath(PartialPath path) throws MetadataException {
     checkTemplateOnSubtree(cur);
   }
 
+  /**
+   * Check route 1: If template has no direct measurement, just pass the check.
+   *
+   * <p>Check route 2: If template has direct measurement and mounted node is Internal, it should be
+   * set to Entity.
+   *
+   * <p>Check route 3: If template has direct measurement and mounted node is Entity,
+   *
+   * <p>route 3.1: mounted node has no measurement child, then its alignment will be set as the
+   * template.
+   *
+   * <p>route 3.2: mounted node has measurement child, then alignment of it and template should be
+   * identical, otherwise cast a exception.
+   *
+   * @return return the node competent to be mounted.
+   */
+  public IMNode checkTemplateAlignmentWithMountedNode(IMNode mountedNode, Template template)
+      throws MetadataException {
+    boolean hasDirectMeasurement = false;
+    for (IMNode child : template.getDirectNodes()) {
+      if (child.isMeasurement()) {
+        hasDirectMeasurement = true;
+      }
+    }
+    if (hasDirectMeasurement) {
+      if (!mountedNode.isEntity()) {
+        return setToEntity(mountedNode);
+      } else {
+        for (IMNode child : mountedNode.getChildren().values()) {
+          if (child.isMeasurement()) {
+            if (template.getAlignedPrefixSet().contains("")

Review comment:
       What's the meaning of empty string here? Maybe we need some comment




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44321874/badge)](https://coveralls.io/builds/44321874)
   
   Coverage decreased (-0.01%) to 66.562% when pulling **248d247579c359a2d26a7e2fb950c88ff959bbc8 on bigreybear:new_vector_with_treeTemplate** into **44fccddf553e80c29e055cb9a1d59630762917bf 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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
##########
@@ -80,9 +81,11 @@ public Template(CreateTemplatePlan plan) throws IllegalPathException {
       IMeasurementSchema curSchema;
       int size = plan.getMeasurements().get(i).size();
       if (size > 1) {
+        //

Review comment:
       thanks! will be remove at next commit

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
##########
@@ -80,9 +81,11 @@ public Template(CreateTemplatePlan plan) throws IllegalPathException {
       IMeasurementSchema curSchema;
       int size = plan.getMeasurements().get(i).size();
       if (size > 1) {
+        //

Review comment:
       thanks! will be removed at next commit




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1917,6 +1919,26 @@ private void internalAlignedCreateTimeseries(
         encodings,
         TSFileDescriptor.getInstance().getConfig().getCompressor());
   }
+
+  /**
+   * Before to insert, set using template if template accessed. Better performance if refactored
+   * with getDeviceNodeWithAutoCreate
+   */
+  private void activateTemplateBeforeAccess(PartialPath deviceId, String[] measurements)
+      throws MetadataException {
+    StringBuilder builder = new StringBuilder(deviceId.getFullPath());
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    for (String measurement : measurements) {
+      builder.append(measurement);
+      IMNode mountedNode = mtree.getTemplateMountedNode(new PartialPath(builder.toString()));
+      if (mountedNode != null && !mountedNode.isUseTemplate()) {
+        setUsingSchemaTemplate(mountedNode);
+      }
+      builder.delete(builder.length() - measurement.length(), builder.length());
+    }
+  }

Review comment:
       Thanks for you insight!
   Now, I will get the template mounted node index, rather than IMNode, so that auto create process methods are decoupled from mounted node detecting.




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1725,7 +1726,8 @@ public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
     String[] measurementList = plan.getMeasurements();
     IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
 
-    // 1. get device node
+    // 1. get device node, set using template if accessed.
+    activateTemplateBeforeAccess(devicePath, measurementList);
     IMNode deviceMNode = getDeviceNodeWithAutoCreate(devicePath);

Review comment:
       As fixed above, this methods no longer exists.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   


-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44377359/badge)](https://coveralls.io/builds/44377359)
   
   Coverage increased (+0.3%) to 66.765% when pulling **11b86e4b83b51a6dbaa113ccc27e2a95cfd222da on bigreybear:new_vector_with_treeTemplate** into **96df1861643aa7f8f72d1bb276569d0eb68ee901 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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44321023/badge)](https://coveralls.io/builds/44321023)
   
   Coverage decreased (-0.03%) to 66.575% when pulling **834bb3a99831339d0d8849a51dee6fb9b2b8bc0c on bigreybear:new_vector_with_treeTemplate** into **88f19d705ae07e5ecc3edc16d2e3cf8fe8256399 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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1186,7 +1192,11 @@ protected void transferToResult(IMNode node) {
         new MNodeCollector<List<PartialPath>>(root, pathPattern) {
           @Override
           protected void transferToResult(IMNode node) {
-            resultSet.add(node.getPartialPath());
+            try {
+              resultSet.add(getCurrentPartialPath(node));
+            } catch (MetadataException e) {
+              logger.error(e.getMessage());

Review comment:
       I'm not sure how to deal with this. It is a exception will not happen actually, so I did not throw it again here.
   The exception is caused by a construct of ParitialPath of which content are nodes during a traverse which should assure to avoid illegal path. But this method override an abstract method which does not throw exception, so this catch did not throw again.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1186,7 +1192,11 @@ protected void transferToResult(IMNode node) {
         new MNodeCollector<List<PartialPath>>(root, pathPattern) {
           @Override
           protected void transferToResult(IMNode node) {
-            resultSet.add(node.getPartialPath());
+            try {
+              resultSet.add(getCurrentPartialPath(node));
+            } catch (MetadataException e) {
+              logger.error(e.getMessage());

Review comment:
       I'm not sure how to deal with this. It is a exception will not happen actually, so I did not throw it again here.
   The exception is caused by a construct of ParitialPath where content are nodes during a traverse which should assure to avoid illegal path. But this method override an abstract method which does not throw exception, so this catch did not throw again.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -221,27 +235,77 @@ protected void processNameMatch(IMNode node, int idx, int level) throws Metadata
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    IMeasurementSchema targetSchema = upperTemplate.getSchemaMap().get(targetName);
-    if (targetSchema != null) {
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), targetSchema.getMeasurementId(), targetSchema, null),
-          idx + 1,
-          level + 1);
+
+    IMNode targetNode = upperTemplate.getDirectNode(targetName);
+    if (targetNode != null) {
+      traverseContext.push(node);
+      traverse(targetNode, idx + 1, level + 1);
+      traverseContext.pop();
     }
 
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverseContext.push(node);
+        traverse(child, idx, level + 1);
+        traverseContext.pop();
       }
     }
   }
 
   public void setPrefixMatch(boolean isPrefixMatch) {
     this.isPrefixMatch = isPrefixMatch;
   }
+
+  /**
+   * @param currentNode the node need to get the full path of
+   * @return full path from traverse start node to the current node
+   */
+  protected PartialPath getPivotPartialPath(IMNode currentNode) throws IllegalPathException {
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    return new PartialPath(builder.toString());
+  }
+
+  protected MeasurementPath getPivotMeasurementPathInTraverse(IMeasurementMNode currentNode)
+      throws MetadataException {
+    IMNode par = traverseContext.peek();
+    if (!(par instanceof IEntityMNode)) {
+      throw new MetadataException("Measurement not under entity: " + currentNode.getName());
+    }

Review comment:
       Is this necessary? How could the parent MNode of current MeasurementMNode not be Entity?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -221,27 +235,77 @@ protected void processNameMatch(IMNode node, int idx, int level) throws Metadata
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    IMeasurementSchema targetSchema = upperTemplate.getSchemaMap().get(targetName);
-    if (targetSchema != null) {
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), targetSchema.getMeasurementId(), targetSchema, null),
-          idx + 1,
-          level + 1);
+
+    IMNode targetNode = upperTemplate.getDirectNode(targetName);
+    if (targetNode != null) {
+      traverseContext.push(node);
+      traverse(targetNode, idx + 1, level + 1);
+      traverseContext.pop();
     }
 
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverseContext.push(node);
+        traverse(child, idx, level + 1);
+        traverseContext.pop();
       }
     }
   }
 
   public void setPrefixMatch(boolean isPrefixMatch) {
     this.isPrefixMatch = isPrefixMatch;
   }
+
+  /**
+   * @param currentNode the node need to get the full path of
+   * @return full path from traverse start node to the current node
+   */
+  protected PartialPath getPivotPartialPath(IMNode currentNode) throws IllegalPathException {
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    return new PartialPath(builder.toString());
+  }
+
+  protected MeasurementPath getPivotMeasurementPathInTraverse(IMeasurementMNode currentNode)
+      throws MetadataException {
+    IMNode par = traverseContext.peek();
+    if (!(par instanceof IEntityMNode)) {
+      throw new MetadataException("Measurement not under entity: " + currentNode.getName());
+    }

Review comment:
       Avoid instanceof. Use IMNode.isEntity instead.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -221,27 +235,77 @@ protected void processNameMatch(IMNode node, int idx, int level) throws Metadata
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    IMeasurementSchema targetSchema = upperTemplate.getSchemaMap().get(targetName);
-    if (targetSchema != null) {
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), targetSchema.getMeasurementId(), targetSchema, null),
-          idx + 1,
-          level + 1);
+
+    IMNode targetNode = upperTemplate.getDirectNode(targetName);
+    if (targetNode != null) {
+      traverseContext.push(node);
+      traverse(targetNode, idx + 1, level + 1);
+      traverseContext.pop();
     }
 
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverseContext.push(node);
+        traverse(child, idx, level + 1);
+        traverseContext.pop();
       }
     }
   }
 
   public void setPrefixMatch(boolean isPrefixMatch) {
     this.isPrefixMatch = isPrefixMatch;
   }
+
+  /**
+   * @param currentNode the node need to get the full path of
+   * @return full path from traverse start node to the current node
+   */
+  protected PartialPath getPivotPartialPath(IMNode currentNode) throws IllegalPathException {
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    return new PartialPath(builder.toString());
+  }
+
+  protected MeasurementPath getPivotMeasurementPathInTraverse(IMeasurementMNode currentNode)
+      throws MetadataException {
+    IMNode par = traverseContext.peek();
+    if (!(par instanceof IEntityMNode)) {
+      throw new MetadataException("Measurement not under entity: " + currentNode.getName());
+    }
+
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    MeasurementPath retPath =
+        new MeasurementPath(new PartialPath(builder.toString()), currentNode.getSchema());
+    retPath.setUnderAlignedEntity(((IEntityMNode) par).isAligned());
+    return retPath;

Review comment:
       Replace (IEntityMNode) par with par.getAsEntity

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -137,20 +146,20 @@ protected abstract boolean processFullMatchedMNode(IMNode node, int idx, int lev
   protected void processMultiLevelWildcard(IMNode node, int idx, int level)
       throws MetadataException {
     for (IMNode child : node.getChildren().values()) {
+      traverseContext.push(node);
       traverse(child, idx + 1, level + 1);
+      traverseContext.pop();
     }

Review comment:
       If  you want to make the context contains node from root to current node's parent, maybe put this push and pop out of the for loop is better?




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44410042/badge)](https://coveralls.io/builds/44410042)
   
   Coverage increased (+0.5%) to 66.979% when pulling **5a38e3fe60c7d199470ea9739d92a561ace0b5a6 on bigreybear:new_vector_with_treeTemplate** into **96df1861643aa7f8f72d1bb276569d0eb68ee901 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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -221,27 +235,77 @@ protected void processNameMatch(IMNode node, int idx, int level) throws Metadata
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    IMeasurementSchema targetSchema = upperTemplate.getSchemaMap().get(targetName);
-    if (targetSchema != null) {
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), targetSchema.getMeasurementId(), targetSchema, null),
-          idx + 1,
-          level + 1);
+
+    IMNode targetNode = upperTemplate.getDirectNode(targetName);
+    if (targetNode != null) {
+      traverseContext.push(node);
+      traverse(targetNode, idx + 1, level + 1);
+      traverseContext.pop();
     }
 
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverseContext.push(node);
+        traverse(child, idx, level + 1);
+        traverseContext.pop();
       }
     }
   }
 
   public void setPrefixMatch(boolean isPrefixMatch) {
     this.isPrefixMatch = isPrefixMatch;
   }
+
+  /**
+   * @param currentNode the node need to get the full path of
+   * @return full path from traverse start node to the current node
+   */
+  protected PartialPath getPivotPartialPath(IMNode currentNode) throws IllegalPathException {
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    return new PartialPath(builder.toString());
+  }
+
+  protected MeasurementPath getPivotMeasurementPathInTraverse(IMeasurementMNode currentNode)
+      throws MetadataException {
+    IMNode par = traverseContext.peek();
+    if (!(par instanceof IEntityMNode)) {
+      throw new MetadataException("Measurement not under entity: " + currentNode.getName());
+    }

Review comment:
       actually it is not necessary, will be removed then




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -221,27 +235,77 @@ protected void processNameMatch(IMNode node, int idx, int level) throws Metadata
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    IMeasurementSchema targetSchema = upperTemplate.getSchemaMap().get(targetName);
-    if (targetSchema != null) {
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), targetSchema.getMeasurementId(), targetSchema, null),
-          idx + 1,
-          level + 1);
+
+    IMNode targetNode = upperTemplate.getDirectNode(targetName);
+    if (targetNode != null) {
+      traverseContext.push(node);
+      traverse(targetNode, idx + 1, level + 1);
+      traverseContext.pop();
     }
 
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverseContext.push(node);
+        traverse(child, idx, level + 1);
+        traverseContext.pop();
       }
     }
   }
 
   public void setPrefixMatch(boolean isPrefixMatch) {
     this.isPrefixMatch = isPrefixMatch;
   }
+
+  /**
+   * @param currentNode the node need to get the full path of
+   * @return full path from traverse start node to the current node
+   */
+  protected PartialPath getPivotPartialPath(IMNode currentNode) throws IllegalPathException {
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    return new PartialPath(builder.toString());
+  }
+
+  protected MeasurementPath getPivotMeasurementPathInTraverse(IMeasurementMNode currentNode)
+      throws MetadataException {
+    IMNode par = traverseContext.peek();
+    if (!(par instanceof IEntityMNode)) {
+      throw new MetadataException("Measurement not under entity: " + currentNode.getName());
+    }

Review comment:
       removed now




-- 
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] SilverNarcissus commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1721,11 +1721,31 @@ public void deleteLastCacheByDevice(
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
       throws MetadataException, IOException {
+    // devicePath is a logical path which is parent of measurement, whether in template or not
     PartialPath devicePath = plan.getDeviceId();
     String[] measurementList = plan.getMeasurements();
     IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
 
-    // 1. get device node
+    // 1. get device node, set using template if accessed.
+

Review comment:
       remove this empty line

##########
File path: server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
##########
@@ -869,11 +871,13 @@ public void testTemplate() throws MetadataException {
         new UnaryMeasurementSchema("s11", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
     assertNotNull(node.getSchemaTemplate());
 
-    Set<IMeasurementSchema> allSchema =
-        new HashSet<>(node.getSchemaTemplate().getSchemaMap().values());
+    Set<String> allSchema = new HashSet<>();
+    for (IMeasurementSchema schema : node.getSchemaTemplate().getSchemaMap().values()) {
+      allSchema.add("root.sg1.d1" + TsFileConstant.PATH_SEPARATOR + schema.getMeasurementId());
+    }
     for (MeasurementPath measurementPath :
-        manager.getMeasurementPaths(new PartialPath("root.sg1.d1.*"))) {
-      allSchema.remove(measurementPath.getMeasurementSchema());
+        manager.getMeasurementPaths(new PartialPath("root.sg1.d1.**"))) {

Review comment:
       What's the ** means currently? Maybe I miss something in the design part.




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1426,6 +1435,53 @@ public void checkTemplateOnPath(PartialPath path) throws MetadataException {
     checkTemplateOnSubtree(cur);
   }
 
+  /**
+   * Check route 1: If template has no direct measurement, just pass the check.
+   *
+   * <p>Check route 2: If template has direct measurement and mounted node is Internal, it should be
+   * set to Entity.
+   *
+   * <p>Check route 3: If template has direct measurement and mounted node is Entity,
+   *
+   * <p>route 3.1: mounted node has no measurement child, then its alignment will be set as the
+   * template.
+   *
+   * <p>route 3.2: mounted node has measurement child, then alignment of it and template should be
+   * identical, otherwise cast a exception.
+   *
+   * @return return the node competent to be mounted.
+   */
+  public IMNode checkTemplateAlignmentWithMountedNode(IMNode mountedNode, Template template)
+      throws MetadataException {
+    boolean hasDirectMeasurement = false;
+    for (IMNode child : template.getDirectNodes()) {
+      if (child.isMeasurement()) {
+        hasDirectMeasurement = true;
+      }
+    }
+    if (hasDirectMeasurement) {
+      if (!mountedNode.isEntity()) {
+        return setToEntity(mountedNode);
+      } else {
+        for (IMNode child : mountedNode.getChildren().values()) {
+          if (child.isMeasurement()) {
+            if (template.getAlignedPrefixSet().contains("")

Review comment:
       Thanks! This string tries to represent alignment of direct measurement of a template. I will add more comment.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44428812/badge)](https://coveralls.io/builds/44428812)
   
   Coverage increased (+0.01%) to 66.979% when pulling **f8b9dbf4e45034ea0fad0791936eb9fb7dd5b431 on bigreybear:new_vector_with_treeTemplate** into **d9d8733f17878cd07e4510a7f9b3af92049863c3 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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
##########
@@ -869,11 +871,13 @@ public void testTemplate() throws MetadataException {
         new UnaryMeasurementSchema("s11", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
     assertNotNull(node.getSchemaTemplate());
 
-    Set<IMeasurementSchema> allSchema =
-        new HashSet<>(node.getSchemaTemplate().getSchemaMap().values());
+    Set<String> allSchema = new HashSet<>();
+    for (IMeasurementSchema schema : node.getSchemaTemplate().getSchemaMap().values()) {
+      allSchema.add("root.sg1.d1" + TsFileConstant.PATH_SEPARATOR + schema.getMeasurementId());
+    }
     for (MeasurementPath measurementPath :
-        manager.getMeasurementPaths(new PartialPath("root.sg1.d1.*"))) {
-      allSchema.remove(measurementPath.getMeasurementSchema());
+        manager.getMeasurementPaths(new PartialPath("root.sg1.d1.**"))) {

Review comment:
       Sorry for the confusion, the usage of '**' has not change ever, it is still a multil-level wildcard.
   
   The reason to change here, is that template converted to a tree structured one now. As before, this template has 2 schemasName which won't present in measurement path, 's11' and 'vector'. Now measurements of the template contain 2 sublist, which is 's11' and 'vector.s1....s10'. 
   
   So, in order to traverse this template, I modify it to '**' as above.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -171,77 +180,131 @@ protected void processOneLevelWildcard(IMNode node, int idx, int level) throws M
       }
       traverse(child, idx + 1, level + 1);
     }
+    traverseContext.pop();
+
     if (multiLevelWildcard) {
+      traverseContext.push(node);
       for (IMNode child : node.getChildren().values()) {
         traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
 
-    if (!isMeasurementTraverser || !node.isUseTemplate()) {
+    if (!node.isUseTemplate()) {
       return;
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-      if (!Pattern.matches(targetNameRegex, schema.getMeasurementId())) {
+
+    traverseContext.push(node);
+    for (IMNode child : upperTemplate.getDirectNodes()) {
+      if (!Pattern.matches(targetNameRegex, child.getName())) {
         continue;
       }
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-          idx + 1,
-          level + 1);
+      traverse(child, idx + 1, level + 1);
     }
+    traverseContext.pop();
+
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      traverseContext.push(node);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
   }
 
+  @SuppressWarnings("Duplicates")
   protected void processNameMatch(IMNode node, int idx, int level) throws MetadataException {
     boolean multiLevelWildcard = nodes[idx].equals(MULTI_LEVEL_PATH_WILDCARD);
     String targetName = nodes[idx + 1];
     IMNode next = node.getChild(targetName);
     if (next != null) {
+      traverseContext.push(node);
       traverse(next, idx + 1, level + 1);
+      traverseContext.pop();
     }
     if (multiLevelWildcard) {
+      traverseContext.push(node);
       for (IMNode child : node.getChildren().values()) {
         traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
 
-    if (!isMeasurementTraverser || !node.isUseTemplate()) {
+    if (!node.isUseTemplate()) {
       return;
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    IMeasurementSchema targetSchema = upperTemplate.getSchemaMap().get(targetName);
-    if (targetSchema != null) {
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), targetSchema.getMeasurementId(), targetSchema, null),
-          idx + 1,
-          level + 1);
+
+    IMNode targetNode = upperTemplate.getDirectNode(targetName);
+    if (targetNode != null) {
+      traverseContext.push(node);
+      traverse(targetNode, idx + 1, level + 1);
+      traverseContext.pop();
     }
 
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      traverseContext.push(node);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
   }
 
   public void setPrefixMatch(boolean isPrefixMatch) {
     this.isPrefixMatch = isPrefixMatch;
   }
+
+  /**
+   * @param currentNode the node need to get the full path of
+   * @return full path from traverse start node to the current node
+   */
+  protected PartialPath getCurrentPartialPath(IMNode currentNode) throws IllegalPathException {
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    return new PartialPath(builder.toString());
+  }
+
+  protected MeasurementPath getCurrentMeasurementPathInTraverse(IMeasurementMNode currentNode)
+      throws MetadataException {
+    IMNode par = traverseContext.peek();
+
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    MeasurementPath retPath =
+        new MeasurementPath(new PartialPath(builder.toString()), currentNode.getSchema());
+    retPath.setUnderAlignedEntity(par.getAsEntityMNode().isAligned());
+    return retPath;
+  }

Review comment:
       Maybe move this method to MeasurementCollector is better, if this method is only used in that kind of tasks.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1998,6 +2018,9 @@ public synchronized void setSchemaTemplate(SetTemplatePlan plan) throws Metadata
 
       templateManager.checkIsTemplateAndMNodeCompatible(template, node);
 
+      // node might be replaced when check with alignment
+      node = mtree.checkTemplateAlignmentWithMountedNode(node, template);
+
       node.setSchemaTemplate(template);

Review comment:
       If this node is not using template, why should it be replaced by entityMNode?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -2051,6 +2074,12 @@ IEntityMNode setUsingSchemaTemplate(IMNode node) throws MetadataException {
     // this operation may change mtree structure and node type
     // invoke mnode.setUseTemplate is invalid
     IEntityMNode entityMNode = mtree.setToEntity(node);
+
+    // to ensure alignment adapt with former node or template
+    entityMNode.setAligned(
+        node.isEntity()
+            ? node.getAsEntityMNode().isAligned()
+            : node.getUpperTemplate().isDirectAligned());
     entityMNode.setUseTemplate(true);

Review comment:
       Move the field "isUsingTemplate" from EntityMNode to InternalMNode. This may be better to adapt to Tree template.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1456,6 +1520,95 @@ public void checkTemplateInUseOnLowerNode(IMNode node) throws TemplateIsInUseExc
       checkTemplateInUseOnLowerNode(child);
     }
   }
+
+  /**
+   * Note that template and MTree cannot have overlap paths.
+   *
+   * @return true iff path corresponding to a measurement inside a template, whether using or not.
+   */
+  public boolean isPathExistsWithinTemplate(PartialPath path) {
+    if (path.getNodes().length < 2) {
+      return false;
+    }
+    String[] pathNodes = path.getNodes();
+    IMNode cur = root;
+    Template upperTemplate = cur.getUpperTemplate();
+    for (int i = 1; i < pathNodes.length; i++) {
+      if (cur.hasChild(pathNodes[i])) {
+        cur = cur.getChild(pathNodes[i]);
+        if (cur.isMeasurement()) {
+          return false;
+        }
+        upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      } else if (upperTemplate != null) {
+        String suffixPath =
+            new PartialPath(Arrays.copyOfRange(pathNodes, i, pathNodes.length)).toString();
+        if (upperTemplate.hasSchema(suffixPath)) {
+          return true;
+        } else {
+          // has template, but not match
+          return false;
+        }
+      } else {
+        // no child and no template
+        return false;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check measurement path and return the mounted node index on path. The node could have not
+   * created yet. The result is used for getDeviceNodeWithAutoCreate, which return corresponding
+   * IMNode on MTree.
+   *
+   * @return index on full path of the node which matches all measurements path with its
+   *     upperTemplate.
+   */
+  public int getMountedNodeIndexOnMeasurementPath(PartialPath measurementPath)
+      throws MetadataException {
+    String[] fullPathNodes = measurementPath.getNodes();
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+
+    if (!cur.getName().equals(fullPathNodes[0])) {
+      throw new IllegalPathException(measurementPath.toString());
+    }
+
+    for (int index = 1; index < fullPathNodes.length; index++) {
+      upperTemplate = cur.getSchemaTemplate() != null ? cur.getSchemaTemplate() : upperTemplate;
+      if (!cur.hasChild(fullPathNodes[index])) {
+        if (upperTemplate != null) {
+          String suffixPath =
+              new PartialPath(Arrays.copyOfRange(fullPathNodes, index, fullPathNodes.length))
+                  .toString();
+
+          // if suffix matches template, then fullPathNodes[index-1] should be the node to use
+          // template on MTree
+          if (upperTemplate.hasSchema(suffixPath)) {
+            return index - 1;
+          }
+
+          // overlap with template, cast exception for now
+          if (upperTemplate.getDirectNode(fullPathNodes[index]) != null) {
+            throw new MetadataException(
+                String.format(
+                    "Path [%s] overlaps but not matches template [%s] under node [%s]",
+                    measurementPath.getFullPath(), upperTemplate.getName(), fullPathNodes[index]));

Review comment:
       Shall we make this kind exception message as a specific Exception class?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -2051,6 +2074,12 @@ IEntityMNode setUsingSchemaTemplate(IMNode node) throws MetadataException {
     // this operation may change mtree structure and node type
     // invoke mnode.setUseTemplate is invalid
     IEntityMNode entityMNode = mtree.setToEntity(node);
+
+    // to ensure alignment adapt with former node or template
+    entityMNode.setAligned(
+        node.isEntity()
+            ? node.getAsEntityMNode().isAligned()
+            : node.getUpperTemplate().isDirectAligned());
     entityMNode.setUseTemplate(true);

Review comment:
       Currently, a MNode using template is possibly not an entityMNode since the template may not has direct measurement. Thus the node type transform need check first.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1725,11 +1725,30 @@ public void deleteLastCacheByDevice(
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
       throws MetadataException, IOException {
+    // devicePath is a logical path which is parent of measurement, whether in template or not
     PartialPath devicePath = plan.getDeviceId();
     String[] measurementList = plan.getMeasurements();
     IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
 
-    // 1. get device node
+    // 1. get device node, set using template if accessed.
+    int indexRecord = -1;
+    // check every measurement path
+    for (String measurementId : measurementList) {
+      PartialPath fullPath = devicePath.concatNode(measurementId);
+      int index = mtree.getMountedNodeIndexOnMeasurementPath(fullPath);
+      if (index != fullPath.getNodeLength() - 1) {
+        // this measurement is in template, need to assure mounted node exists and set using
+        // template.
+        if (index != indexRecord) {
+          // Without allowing overlap of template and MTree, this block run only once
+          String[] mountedPathNodes = Arrays.copyOfRange(fullPath.getNodes(), 0, index + 1);
+          IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
+          setUsingSchemaTemplate(mountedNode);
+          indexRecord = index;
+        }
+      }
+    }

Review comment:
       Since insertPlan only process one device, it is unnecessary to finish the whole loop. Break as early as the device has been checked.




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -2051,6 +2074,12 @@ IEntityMNode setUsingSchemaTemplate(IMNode node) throws MetadataException {
     // this operation may change mtree structure and node type
     // invoke mnode.setUseTemplate is invalid
     IEntityMNode entityMNode = mtree.setToEntity(node);
+
+    // to ensure alignment adapt with former node or template
+    entityMNode.setAligned(
+        node.isEntity()
+            ? node.getAsEntityMNode().isAligned()
+            : node.getUpperTemplate().isDirectAligned());
     entityMNode.setUseTemplate(true);

Review comment:
       In this method (setUsingTemplate), it will check alignment and replace it if direct measurement exists now.
   To implement this, I add setUseTemplate into Interface IMNode. And this method will return an IMNode rather than IEntityNode now.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -2051,6 +2074,12 @@ IEntityMNode setUsingSchemaTemplate(IMNode node) throws MetadataException {
     // this operation may change mtree structure and node type
     // invoke mnode.setUseTemplate is invalid
     IEntityMNode entityMNode = mtree.setToEntity(node);
+
+    // to ensure alignment adapt with former node or template
+    entityMNode.setAligned(
+        node.isEntity()
+            ? node.getAsEntityMNode().isAligned()
+            : node.getUpperTemplate().isDirectAligned());
     entityMNode.setUseTemplate(true);

Review comment:
       In this method (setUsingTemplate), it will check alignment and replace it if direct measurement exists now.
   To implement this, I add setUseTemplate into Interface IMNode. 
   And this method will return an IMNode rather than IEntityNode now.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44399249/badge)](https://coveralls.io/builds/44399249)
   
   Coverage increased (+0.4%) to 66.894% when pulling **6d5d384d2635b50febe2216135175eecbebc00b1 on bigreybear:new_vector_with_treeTemplate** into **96df1861643aa7f8f72d1bb276569d0eb68ee901 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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44421879/badge)](https://coveralls.io/builds/44421879)
   
   Coverage remained the same at 66.967% when pulling **6d5d384d2635b50febe2216135175eecbebc00b1 on bigreybear:new_vector_with_treeTemplate** into **d9d8733f17878cd07e4510a7f9b3af92049863c3 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] SilverNarcissus commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1186,7 +1192,11 @@ protected void transferToResult(IMNode node) {
         new MNodeCollector<List<PartialPath>>(root, pathPattern) {
           @Override
           protected void transferToResult(IMNode node) {
-            resultSet.add(node.getPartialPath());
+            try {
+              resultSet.add(getCurrentPartialPath(node));
+            } catch (MetadataException e) {
+              logger.error(e.getMessage());

Review comment:
       Should we throw exception again here?




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1456,6 +1520,95 @@ public void checkTemplateInUseOnLowerNode(IMNode node) throws TemplateIsInUseExc
       checkTemplateInUseOnLowerNode(child);
     }
   }
+
+  /**
+   * Note that template and MTree cannot have overlap paths.
+   *
+   * @return true iff path corresponding to a measurement inside a template, whether using or not.
+   */
+  public boolean isPathExistsWithinTemplate(PartialPath path) {
+    if (path.getNodes().length < 2) {
+      return false;
+    }
+    String[] pathNodes = path.getNodes();
+    IMNode cur = root;
+    Template upperTemplate = cur.getUpperTemplate();
+    for (int i = 1; i < pathNodes.length; i++) {
+      if (cur.hasChild(pathNodes[i])) {
+        cur = cur.getChild(pathNodes[i]);
+        if (cur.isMeasurement()) {
+          return false;
+        }
+        upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      } else if (upperTemplate != null) {
+        String suffixPath =
+            new PartialPath(Arrays.copyOfRange(pathNodes, i, pathNodes.length)).toString();
+        if (upperTemplate.hasSchema(suffixPath)) {
+          return true;
+        } else {
+          // has template, but not match
+          return false;
+        }
+      } else {
+        // no child and no template
+        return false;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check measurement path and return the mounted node index on path. The node could have not
+   * created yet. The result is used for getDeviceNodeWithAutoCreate, which return corresponding
+   * IMNode on MTree.
+   *
+   * @return index on full path of the node which matches all measurements path with its
+   *     upperTemplate.
+   */
+  public int getMountedNodeIndexOnMeasurementPath(PartialPath measurementPath)
+      throws MetadataException {
+    String[] fullPathNodes = measurementPath.getNodes();
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+
+    if (!cur.getName().equals(fullPathNodes[0])) {
+      throw new IllegalPathException(measurementPath.toString());
+    }
+
+    for (int index = 1; index < fullPathNodes.length; index++) {
+      upperTemplate = cur.getSchemaTemplate() != null ? cur.getSchemaTemplate() : upperTemplate;
+      if (!cur.hasChild(fullPathNodes[index])) {
+        if (upperTemplate != null) {
+          String suffixPath =
+              new PartialPath(Arrays.copyOfRange(fullPathNodes, index, fullPathNodes.length))
+                  .toString();
+
+          // if suffix matches template, then fullPathNodes[index-1] should be the node to use
+          // template on MTree
+          if (upperTemplate.hasSchema(suffixPath)) {
+            return index - 1;
+          }
+
+          // overlap with template, cast exception for now
+          if (upperTemplate.getDirectNode(fullPathNodes[index]) != null) {
+            throw new MetadataException(
+                String.format(
+                    "Path [%s] overlaps but not matches template [%s] under node [%s]",
+                    measurementPath.getFullPath(), upperTemplate.getName(), fullPathNodes[index]));

Review comment:
       A new exception TemplateImcompatibleException replaces all these usages.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1917,6 +1919,26 @@ private void internalAlignedCreateTimeseries(
         encodings,
         TSFileDescriptor.getInstance().getConfig().getCompressor());
   }
+
+  /**
+   * Before to insert, set using template if template accessed. Better performance if refactored
+   * with getDeviceNodeWithAutoCreate
+   */
+  private void activateTemplateBeforeAccess(PartialPath deviceId, String[] measurements)
+      throws MetadataException {
+    StringBuilder builder = new StringBuilder(deviceId.getFullPath());
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    for (String measurement : measurements) {
+      builder.append(measurement);
+      IMNode mountedNode = mtree.getTemplateMountedNode(new PartialPath(builder.toString()));
+      if (mountedNode != null && !mountedNode.isUseTemplate()) {
+        setUsingSchemaTemplate(mountedNode);
+      }
+      builder.delete(builder.length() - measurement.length(), builder.length());
+    }
+  }

Review comment:
       If the mountedNode has not been created on MTree, how could you activate it?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1725,7 +1726,8 @@ public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
     String[] measurementList = plan.getMeasurements();
     IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
 
-    // 1. get device node
+    // 1. get device node, set using template if accessed.
+    activateTemplateBeforeAccess(devicePath, measurementList);
     IMNode deviceMNode = getDeviceNodeWithAutoCreate(devicePath);

Review comment:
       If the device now is represented by template, this method maybe invalid.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44322670/badge)](https://coveralls.io/builds/44322670)
   
   Coverage increased (+0.04%) to 66.539% when pulling **4e92f3003959e467b5245c74acfe48f195bdb629 on bigreybear:new_vector_with_treeTemplate** into **96df1861643aa7f8f72d1bb276569d0eb68ee901 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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1186,7 +1192,11 @@ protected void transferToResult(IMNode node) {
         new MNodeCollector<List<PartialPath>>(root, pathPattern) {
           @Override
           protected void transferToResult(IMNode node) {
-            resultSet.add(node.getPartialPath());
+            try {
+              resultSet.add(getCurrentPartialPath(node));
+            } catch (MetadataException e) {
+              logger.error(e.getMessage());

Review comment:
       I'm not sure how to deal with this. It is a exception will not happen actually, so I did not throw it again here.
   The exception is caused by a construct of ParitialPath, of which content are nodes during a traverse which should assure to avoid illegal path. But this method override an abstract method which does not throw exception, so this catch did not throw again.




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: session/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
##########
@@ -1127,6 +1133,199 @@ public void testConversionFunction()
     session.close();
   }
 
+  @Test
+  public void testInsertDeleteWithTemplate()
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+    session = new Session("127.0.0.1", 6667, "root", "root");
+    session.open();
+
+    initTreeTemplate("root.sg.loc1");
+    List<String> measurements = new ArrayList<>();
+    List<String> values = new ArrayList<>();
+    String deviceId = "root.sg.loc1.sector.GPS";
+    Set<String> checkSet = new HashSet<>();
+    SessionDataSet dataSet;
+
+    // insert record set using template
+
+    measurements.add("x");
+    measurements.add("y");
+    values.add("1.0");
+    values.add("2.0");
+
+    checkSet.add("root.sg.loc1.sector.GPS.x");
+    checkSet.add("root.sg.loc1.sector.GPS.y");
+    checkSet.add("root.sg.loc1.sector.y");
+    checkSet.add("root.sg.loc1.sector.x");
+    checkSet.add("root.sg.loc1.sector.vehicle.x");
+    checkSet.add("root.sg.loc1.sector.vehicle.y");
+    checkSet.add("root.sg.loc1.sector.vehicle.GPS.x");
+    checkSet.add("root.sg.loc1.sector.vehicle.GPS.y");
+
+    session.insertRecord(deviceId, 1L, measurements, values);
+    dataSet = session.executeQueryStatement("show timeseries");
+    while (dataSet.hasNext()) {
+      checkSet.remove(dataSet.next().getFields().get(0).toString());
+    }
+    assertTrue(checkSet.isEmpty());
+
+    // insert aligned under unaligned node
+    try {
+      session.insertAlignedRecord("root.sg.loc1.sector.GPS", 3L, measurements, values);
+    } catch (StatementExecutionException e) {
+      assertEquals(
+          "303: Timeseries under path [root.sg.loc1.sector.GPS] is not aligned , please set InsertPlan.isAligned() = false",
+          e.getMessage());
+    }
+
+    // insert overlap unmatched series
+    measurements.set(1, "speed");
+    try {
+      session.insertRecord(deviceId, 5L, measurements, values);
+      fail();
+    } catch (StatementExecutionException e) {
+      assertEquals(
+          "303: Path [root.sg.loc1.sector.GPS.speed] overlaps but not matches template [treeTemplate] under node [GPS]",
+          e.getMessage());
+    }
+
+    // insert tablets
+    List<IMeasurementSchema> schemaList = new ArrayList<>();
+    schemaList.add(new UnaryMeasurementSchema("x", TSDataType.FLOAT));
+    schemaList.add(new UnaryMeasurementSchema("y", TSDataType.FLOAT));
+    Tablet tablet = new Tablet("root.sg.loc1.sector", schemaList);
+    tablet.setAligned(true);
+
+    long timestamp = System.currentTimeMillis();
+
+    for (long row = 0; row < 10; row++) {
+      int rowIndex = tablet.rowSize++;
+      tablet.addTimestamp(rowIndex, timestamp);
+      tablet.addValue(
+          schemaList.get(0).getMeasurementId(), rowIndex, new SecureRandom().nextFloat());
+      tablet.addValue(
+          schemaList.get(1).getMeasurementId(), rowIndex, new SecureRandom().nextFloat());
+      timestamp++;
+    }
+
+    if (tablet.rowSize != 0) {
+      session.insertAlignedTablet(tablet);
+      tablet.reset();
+    }
+
+    dataSet = session.executeQueryStatement("select count(*) from root");
+
+    while (dataSet.hasNext()) {
+      RowRecord rowRecord = dataSet.next();
+      Assert.assertEquals(10L, rowRecord.getFields().get(0).getLongV());
+      Assert.assertEquals(10L, rowRecord.getFields().get(1).getLongV());
+    }
+
+    // delete series inside template
+    try {
+      session.deleteTimeseries("root.sg.loc1.sector.x");

Review comment:
       Thanks! And it has been added now.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44364777/badge)](https://coveralls.io/builds/44364777)
   
   Coverage increased (+0.3%) to 66.761% when pulling **b7930cdb1bc4efeea0e13205529526f8b28f032c on bigreybear:new_vector_with_treeTemplate** into **96df1861643aa7f8f72d1bb276569d0eb68ee901 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] SilverNarcissus commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
##########
@@ -1426,6 +1435,53 @@ public void checkTemplateOnPath(PartialPath path) throws MetadataException {
     checkTemplateOnSubtree(cur);
   }
 
+  /**
+   * Check route 1: If template has no direct measurement, just pass the check.
+   *
+   * <p>Check route 2: If template has direct measurement and mounted node is Internal, it should be
+   * set to Entity.
+   *
+   * <p>Check route 3: If template has direct measurement and mounted node is Entity,
+   *
+   * <p>route 3.1: mounted node has no measurement child, then its alignment will be set as the
+   * template.
+   *
+   * <p>route 3.2: mounted node has measurement child, then alignment of it and template should be
+   * identical, otherwise cast a exception.
+   *
+   * @return return the node competent to be mounted.
+   */
+  public IMNode checkTemplateAlignmentWithMountedNode(IMNode mountedNode, Template template)
+      throws MetadataException {
+    boolean hasDirectMeasurement = false;
+    for (IMNode child : template.getDirectNodes()) {
+      if (child.isMeasurement()) {
+        hasDirectMeasurement = true;
+      }
+    }
+    if (hasDirectMeasurement) {
+      if (!mountedNode.isEntity()) {
+        return setToEntity(mountedNode);
+      } else {
+        for (IMNode child : mountedNode.getChildren().values()) {
+          if (child.isMeasurement()) {
+            if (template.getAlignedPrefixSet().contains("")

Review comment:
       What's the meaning of empty string here?




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44358533/badge)](https://coveralls.io/builds/44358533)
   
   Coverage increased (+0.1%) to 66.631% when pulling **0fac30140f35ae3e2750773044edd81b111a53e4 on bigreybear:new_vector_with_treeTemplate** into **96df1861643aa7f8f72d1bb276569d0eb68ee901 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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/Traverser.java
##########
@@ -171,77 +180,131 @@ protected void processOneLevelWildcard(IMNode node, int idx, int level) throws M
       }
       traverse(child, idx + 1, level + 1);
     }
+    traverseContext.pop();
+
     if (multiLevelWildcard) {
+      traverseContext.push(node);
       for (IMNode child : node.getChildren().values()) {
         traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
 
-    if (!isMeasurementTraverser || !node.isUseTemplate()) {
+    if (!node.isUseTemplate()) {
       return;
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-      if (!Pattern.matches(targetNameRegex, schema.getMeasurementId())) {
+
+    traverseContext.push(node);
+    for (IMNode child : upperTemplate.getDirectNodes()) {
+      if (!Pattern.matches(targetNameRegex, child.getName())) {
         continue;
       }
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-          idx + 1,
-          level + 1);
+      traverse(child, idx + 1, level + 1);
     }
+    traverseContext.pop();
+
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      traverseContext.push(node);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
   }
 
+  @SuppressWarnings("Duplicates")
   protected void processNameMatch(IMNode node, int idx, int level) throws MetadataException {
     boolean multiLevelWildcard = nodes[idx].equals(MULTI_LEVEL_PATH_WILDCARD);
     String targetName = nodes[idx + 1];
     IMNode next = node.getChild(targetName);
     if (next != null) {
+      traverseContext.push(node);
       traverse(next, idx + 1, level + 1);
+      traverseContext.pop();
     }
     if (multiLevelWildcard) {
+      traverseContext.push(node);
       for (IMNode child : node.getChildren().values()) {
         traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
 
-    if (!isMeasurementTraverser || !node.isUseTemplate()) {
+    if (!node.isUseTemplate()) {
       return;
     }
 
     Template upperTemplate = node.getUpperTemplate();
-    IMeasurementSchema targetSchema = upperTemplate.getSchemaMap().get(targetName);
-    if (targetSchema != null) {
-      traverse(
-          MeasurementMNode.getMeasurementMNode(
-              node.getAsEntityMNode(), targetSchema.getMeasurementId(), targetSchema, null),
-          idx + 1,
-          level + 1);
+
+    IMNode targetNode = upperTemplate.getDirectNode(targetName);
+    if (targetNode != null) {
+      traverseContext.push(node);
+      traverse(targetNode, idx + 1, level + 1);
+      traverseContext.pop();
     }
 
     if (multiLevelWildcard) {
-      for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
-        traverse(
-            MeasurementMNode.getMeasurementMNode(
-                node.getAsEntityMNode(), schema.getMeasurementId(), schema, null),
-            idx,
-            level + 1);
+      traverseContext.push(node);
+      for (IMNode child : upperTemplate.getDirectNodes()) {
+        traverse(child, idx, level + 1);
       }
+      traverseContext.pop();
     }
   }
 
   public void setPrefixMatch(boolean isPrefixMatch) {
     this.isPrefixMatch = isPrefixMatch;
   }
+
+  /**
+   * @param currentNode the node need to get the full path of
+   * @return full path from traverse start node to the current node
+   */
+  protected PartialPath getCurrentPartialPath(IMNode currentNode) throws IllegalPathException {
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    return new PartialPath(builder.toString());
+  }
+
+  protected MeasurementPath getCurrentMeasurementPathInTraverse(IMeasurementMNode currentNode)
+      throws MetadataException {
+    IMNode par = traverseContext.peek();
+
+    Iterator<IMNode> nodes = traverseContext.descendingIterator();
+    StringBuilder builder = new StringBuilder(nodes.next().getName());
+    while (nodes.hasNext()) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+      builder.append(nodes.next().getName());
+    }
+    if (builder.length() != 0) {
+      builder.append(TsFileConstant.PATH_SEPARATOR);
+    }
+    builder.append(currentNode.getName());
+    MeasurementPath retPath =
+        new MeasurementPath(new PartialPath(builder.toString()), currentNode.getSchema());
+    retPath.setUnderAlignedEntity(par.getAsEntityMNode().isAligned());
+    return retPath;
+  }

Review comment:
       It has been moved now




-- 
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] SilverNarcissus commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: session/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
##########
@@ -1127,6 +1133,199 @@ public void testConversionFunction()
     session.close();
   }
 
+  @Test
+  public void testInsertDeleteWithTemplate()
+      throws IoTDBConnectionException, StatementExecutionException, IOException {
+    session = new Session("127.0.0.1", 6667, "root", "root");
+    session.open();
+
+    initTreeTemplate("root.sg.loc1");
+    List<String> measurements = new ArrayList<>();
+    List<String> values = new ArrayList<>();
+    String deviceId = "root.sg.loc1.sector.GPS";
+    Set<String> checkSet = new HashSet<>();
+    SessionDataSet dataSet;
+
+    // insert record set using template
+
+    measurements.add("x");
+    measurements.add("y");
+    values.add("1.0");
+    values.add("2.0");
+
+    checkSet.add("root.sg.loc1.sector.GPS.x");
+    checkSet.add("root.sg.loc1.sector.GPS.y");
+    checkSet.add("root.sg.loc1.sector.y");
+    checkSet.add("root.sg.loc1.sector.x");
+    checkSet.add("root.sg.loc1.sector.vehicle.x");
+    checkSet.add("root.sg.loc1.sector.vehicle.y");
+    checkSet.add("root.sg.loc1.sector.vehicle.GPS.x");
+    checkSet.add("root.sg.loc1.sector.vehicle.GPS.y");
+
+    session.insertRecord(deviceId, 1L, measurements, values);
+    dataSet = session.executeQueryStatement("show timeseries");
+    while (dataSet.hasNext()) {
+      checkSet.remove(dataSet.next().getFields().get(0).toString());
+    }
+    assertTrue(checkSet.isEmpty());
+
+    // insert aligned under unaligned node
+    try {
+      session.insertAlignedRecord("root.sg.loc1.sector.GPS", 3L, measurements, values);
+    } catch (StatementExecutionException e) {
+      assertEquals(
+          "303: Timeseries under path [root.sg.loc1.sector.GPS] is not aligned , please set InsertPlan.isAligned() = false",
+          e.getMessage());
+    }
+
+    // insert overlap unmatched series
+    measurements.set(1, "speed");
+    try {
+      session.insertRecord(deviceId, 5L, measurements, values);
+      fail();
+    } catch (StatementExecutionException e) {
+      assertEquals(
+          "303: Path [root.sg.loc1.sector.GPS.speed] overlaps but not matches template [treeTemplate] under node [GPS]",
+          e.getMessage());
+    }
+
+    // insert tablets
+    List<IMeasurementSchema> schemaList = new ArrayList<>();
+    schemaList.add(new UnaryMeasurementSchema("x", TSDataType.FLOAT));
+    schemaList.add(new UnaryMeasurementSchema("y", TSDataType.FLOAT));
+    Tablet tablet = new Tablet("root.sg.loc1.sector", schemaList);
+    tablet.setAligned(true);
+
+    long timestamp = System.currentTimeMillis();
+
+    for (long row = 0; row < 10; row++) {
+      int rowIndex = tablet.rowSize++;
+      tablet.addTimestamp(rowIndex, timestamp);
+      tablet.addValue(
+          schemaList.get(0).getMeasurementId(), rowIndex, new SecureRandom().nextFloat());
+      tablet.addValue(
+          schemaList.get(1).getMeasurementId(), rowIndex, new SecureRandom().nextFloat());
+      timestamp++;
+    }
+
+    if (tablet.rowSize != 0) {
+      session.insertAlignedTablet(tablet);
+      tablet.reset();
+    }
+
+    dataSet = session.executeQueryStatement("select count(*) from root");
+
+    while (dataSet.hasNext()) {
+      RowRecord rowRecord = dataSet.next();
+      Assert.assertEquals(10L, rowRecord.getFields().get(0).getLongV());
+      Assert.assertEquals(10L, rowRecord.getFields().get(1).getLongV());
+    }
+
+    // delete series inside template
+    try {
+      session.deleteTimeseries("root.sg.loc1.sector.x");

Review comment:
       add a fail() after this line




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1998,6 +2018,9 @@ public synchronized void setSchemaTemplate(SetTemplatePlan plan) throws Metadata
 
       templateManager.checkIsTemplateAndMNodeCompatible(template, node);
 
+      // node might be replaced when check with alignment
+      node = mtree.checkTemplateAlignmentWithMountedNode(node, template);
+
       node.setSchemaTemplate(template);

Review comment:
       That's true. This check should be a part of setUsingTemplate methods rather than this one.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1998,6 +2018,9 @@ public synchronized void setSchemaTemplate(SetTemplatePlan plan) throws Metadata
 
       templateManager.checkIsTemplateAndMNodeCompatible(template, node);
 
+      // node might be replaced when check with alignment
+      node = mtree.checkTemplateAlignmentWithMountedNode(node, template);
+
       node.setSchemaTemplate(template);

Review comment:
       That's true. This check should be a part of setUsingTemplate methods rather than this one. Fixed now.




-- 
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] bigreybear commented on a change in pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1725,11 +1725,30 @@ public void deleteLastCacheByDevice(
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
       throws MetadataException, IOException {
+    // devicePath is a logical path which is parent of measurement, whether in template or not
     PartialPath devicePath = plan.getDeviceId();
     String[] measurementList = plan.getMeasurements();
     IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
 
-    // 1. get device node
+    // 1. get device node, set using template if accessed.
+    int indexRecord = -1;
+    // check every measurement path
+    for (String measurementId : measurementList) {
+      PartialPath fullPath = devicePath.concatNode(measurementId);
+      int index = mtree.getMountedNodeIndexOnMeasurementPath(fullPath);
+      if (index != fullPath.getNodeLength() - 1) {
+        // this measurement is in template, need to assure mounted node exists and set using
+        // template.
+        if (index != indexRecord) {
+          // Without allowing overlap of template and MTree, this block run only once
+          String[] mountedPathNodes = Arrays.copyOfRange(fullPath.getNodes(), 0, index + 1);
+          IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
+          setUsingSchemaTemplate(mountedNode);
+          indexRecord = index;
+        }
+      }
+    }

Review comment:
       Since it needs to check each measurementPath with template whether compatible,  I use a boolean to avoid reduntant process.




-- 
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 #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   
   [![Coverage Status](https://coveralls.io/builds/44410715/badge)](https://coveralls.io/builds/44410715)
   
   Coverage increased (+0.4%) to 66.903% when pulling **5a38e3fe60c7d199470ea9739d92a561ace0b5a6 on bigreybear:new_vector_with_treeTemplate** into **96df1861643aa7f8f72d1bb276569d0eb68ee901 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 merged pull request #4391: [IOTDB-1931] Adapt tree structrued Template with MManager

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


   


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