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/07/19 09:03:45 UTC

[GitHub] [iotdb] zyk990424 opened a new pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   ## MNode Abstraction and Structure Improvement
   
   
   ### Extract IMNode Interface
   
   Extracting IMNode Interface for further function extension.
   
   ### Extract Abstract MNode
   
   Since Measurement doesn't support child management from jira 1495, attributes in origin MNode like children and template is unnecessary. Therefore, the MNode structure is changed referring to rel/0.10 version.
   
   ### Developed based on jira 1495 and 1507
   
   Jira 1495 eliminate the nested measurement and Jira 1507 extract TemplateManager and TagManager from MManger for better maintaining and extending.
   
   <!--
   In each section, please describe design decisions made, including:
    - Choice of algorithms
    - Behavioral aspects. What configuration values are acceptable? How are corner cases and error 
       conditions handled, such as when there are insufficient resources?
    - Class organization and design (how the logic is split between classes, inheritance, composition, 
       design patterns)
    - Method organization and design (how the logic is split between methods, parameters and return types)
    - Naming (class, method, API, configuration, HTTP endpoint, names of emitted metrics)
   -->
   
   
   <!-- It's good to describe an alternative design (or mention an alternative name) for every design 
   (or naming) decision point and compare the alternatives with the designs that you've implemented 
   (or the names you've chosen) to highlight the advantages of the chosen designs and names. -->
   
   <!-- If there was a discussion of the design of the feature implemented in this PR elsewhere 
   (e. g. a "Proposal" issue, any other issue, or a thread in the development mailing list), 
   link to that discussion from this PR description and explain what have changed in your final design 
   compared to your original proposal or the consensus version in the end of the discussion. 
   If something hasn't changed since the original discussion, you can omit a detailed discussion of 
   those aspects of the design here, perhaps apart from brief mentioning for the sake of readability 
   of this PR description. -->
   
   <!-- Some of the aspects mentioned above may be omitted for simple and small changes. -->
   
   <hr>
   
   This PR has:
   - [ ] been self-reviewed.
       - [ ] concurrent read
       - [ ] concurrent write
       - [ ] concurrent read and write 
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] added Javadocs for most classes and all non-trivial methods. 
   - [ ] added or updated version, __license__, or notice information
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious 
     for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold 
     for code coverage.
   - [ ] added integration tests.
   - [ ] been tested in a test IoTDB cluster.
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items 
   apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items 
   from the checklist above are strictly necessary, but it would be very helpful if you at least 
   self-review the PR. -->
   
   <hr>
   
   ##### Key changed/added classes (or packages if there are too many classes) in this PR
   


-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. New mlog line number: {}.", logWriter.getLogNum());
+      }
+    } else {
+      logger.info(
+          "New mlog line number: {}, time from last modification: {} ms",
+          logWriter.getLogNum(),
+          System.currentTimeMillis() - logFile.lastModified());
+      createMTreeSnapshot();
+    }
+  }
+
+  public void createMTreeSnapshot() {
+    try {
+      mtree.createSnapshot();
+      logWriter.clear();
+    } catch (IOException e) {
+      logger.warn("Failed to create MTree snapshot", e);
+    }
+  }
+
   /** function for clearing MTree */
   public void clear() {
     try {
-      templateMap.clear();
-      this.mtree = new MTree();
+      this.mtree.clear();

Review comment:
       Fixed




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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41684406/badge)](https://coveralls.io/builds/41684406)
   
   Coverage decreased (-0.02%) to 68.185% when pulling **770758204ef5ed297f6ef443458cf52614e192b4 on zyk990424:jira_1498** into **95e3b69402cd40555d8a9654833b552d7c7e3d9d 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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41466413/badge)](https://coveralls.io/builds/41466413)
   
   Coverage increased (+0.01%) to 68.15% when pulling **1f665f38843ef42e44974cb04472151bc100e959 on zyk990424:jira_1498** into **2bff46a40d2bf2a69ccd9bdd749d4aa0d1a7de66 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] wangchao316 commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {

Review comment:
       Good Job. I have a little question.
   this bransh only for print log? 
   I fell if only for print log , That's not necessary.

##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBAutoCreateSchemaIT.java
##########
@@ -24,10 +24,7 @@
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.jdbc.IoTDBSQLException;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.*;

Review comment:
       generate , we should not use import *

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {

Review comment:
       the same above.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {
+    if (upper == null) {
+      return true;

Review comment:
       upper == null, return true?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetaUtils;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class InternalMNode extends MNode implements IMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  /**
+   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
+   * synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  protected transient volatile Map<String, IMNode> children = null;
+
+  /**
+   * suppress warnings reason: volatile for double synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  private transient volatile Map<String, IMNode> aliasChildren = null;
+
+  // device template
+  protected Template deviceTemplate = null;
+
+  private volatile boolean useTemplate = false;
+
+  /** Constructor of MNode. */
+  public InternalMNode(IMNode parent, String name) {
+    super(parent, name);
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return (children != null && children.containsKey(name))
+        || (aliasChildren != null && aliasChildren.containsKey(name));
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   */
+  @Override
+  public void addChild(String name, IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+    child.setParent(this);
+    children.putIfAbsent(name, child);
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  public IMNode addChild(IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    child.setParent(this);
+    children.putIfAbsent(child.getName(), child);
+    return child;
+  }
+
+  /** delete a child */
+  @Override
+  public void deleteChild(String name) {
+    if (children != null) {
+      children.remove(name);
+    }
+  }
+
+  /** delete the alias of a child */
+  @Override
+  public void deleteAliasChild(String alias) {
+    if (aliasChildren != null) {
+      aliasChildren.remove(alias);
+    }
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return deviceTemplate;
+  }
+
+  @Override
+  public void setDeviceTemplate(Template deviceTemplate) {
+    this.deviceTemplate = deviceTemplate;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IMNode getChild(String name) {
+    IMNode child = null;
+    if (children != null) {
+      child = children.get(name);
+    }
+    if (child != null) {
+      return child;
+    }
+    return aliasChildren == null ? null : aliasChildren.get(name);
+  }
+
+  @Override
+  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
+    IMNode node = null;
+    // for aligned timeseries
+    List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(new PartialPath(name));
+    for (String measurement : measurementList) {
+      IMNode nodeOfMeasurement = getChild(measurement);
+      if (node == null) {
+        node = nodeOfMeasurement;
+      } else {
+        if (node != nodeOfMeasurement) {
+          throw new AlignedTimeseriesException(
+              "Cannot get node of children in different aligned timeseries", name);
+        }
+      }
+    }
+    return node;
+  }
+
+  /** get the count of all MeasurementMNode whose ancestor is current node */
+  @Override
+  public int getMeasurementMNodeCount() {
+    if (children == null) {
+      return 0;
+    }
+    int measurementMNodeCount = 0;
+    for (IMNode child : children.values()) {
+      measurementMNodeCount += child.getMeasurementMNodeCount();
+    }
+    return measurementMNodeCount;
+  }
+
+  /** add an alias */
+  @Override
+  public boolean addAlias(String alias, IMNode child) {
+    if (aliasChildren == null) {
+      // double check, alias children volatile
+      synchronized (this) {
+        if (aliasChildren == null) {
+          aliasChildren = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    return aliasChildren.computeIfAbsent(alias, aliasName -> child) == child;
+  }
+
+  @Override
+  public Map<String, IMNode> getChildren() {
+    if (children == null) {
+      return Collections.emptyMap();
+    }
+    return children;
+  }
+
+  @Override
+  public Map<String, IMNode> getAliasChildren() {
+    if (aliasChildren == null) {
+      return Collections.emptyMap();
+    }
+    return aliasChildren;
+  }
+
+  @Override
+  public void setChildren(Map<String, IMNode> children) {
+    this.children = children;
+  }
+
+  public void setAliasChildren(Map<String, IMNode> aliasChildren) {
+    this.aliasChildren = aliasChildren;
+  }
+
+  @Override
+  public void serializeTo(MLogWriter logWriter) throws IOException {

Review comment:
       serializeTo and deserializeFrom is relative,  if  you implement serializeTo, you will need add a deserializeFrom.

##########
File path: server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
##########
@@ -1408,13 +1461,13 @@ public void testTotalSeriesNumber() throws Exception {
           CompressionType.GZIP,
           null);
       manager.createTimeseries(
-          new PartialPath("root.laptop.d1.s1.t1"),
+          new PartialPath("root.laptop.d1.s2.t1"),
           TSDataType.INT32,

Review comment:
       why does need alter test value ? 

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. New mlog line number: {}.", logWriter.getLogNum());
+      }
+    } else {
+      logger.info(
+          "New mlog line number: {}, time from last modification: {} ms",
+          logWriter.getLogNum(),
+          System.currentTimeMillis() - logFile.lastModified());
+      createMTreeSnapshot();
+    }
+  }
+
+  public void createMTreeSnapshot() {
+    try {
+      mtree.createSnapshot();
+      logWriter.clear();
+    } catch (IOException e) {
+      logger.warn("Failed to create MTree snapshot", e);
+    }
+  }
+
   /** function for clearing MTree */
   public void clear() {
     try {
-      templateMap.clear();
-      this.mtree = new MTree();
+      this.mtree.clear();

Review comment:
       if this.mtree is null, will throw nullpointexception.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {

Review comment:
       public method need add some comment

##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateTimeseriesIT.java
##########
@@ -65,25 +62,30 @@ public void tearDown() throws Exception {
   }
 
   /** Test creating a time series that is a prefix path of an existing time series */
+  @Ignore // nested measurement has been forbidden
   @Test
   public void testCreateTimeseries1() throws Exception {
     String[] timeSeriesArray = {"root.sg1.aa.bb", "root.sg1.aa.bb.cc", "root.sg1.aa"};
 
-    for (String timeSeries : timeSeriesArray) {
-      statement.execute(
-          String.format(
-              "create timeseries %s with datatype=INT64, encoding=PLAIN, compression=SNAPPY",
-              timeSeries));
-    }
+    try {
+      for (String timeSeries : timeSeriesArray) {
+        statement.execute(
+            String.format(
+                "create timeseries %s with datatype=INT64, encoding=PLAIN, compression=SNAPPY",
+                timeSeries));
+      }
 
-    // ensure that current timeseries in cache is right.
-    createTimeSeries1Tool(timeSeriesArray);
+      // ensure that current timeseries in cache is right.
+      createTimeSeries1Tool(timeSeriesArray);

Review comment:
       ? space




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   


-- 
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] wangchao316 commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {

Review comment:
       Good Job. I have a little question.
   this bransh only for print log? 
   I fell if only for print log , That's not necessary.

##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBAutoCreateSchemaIT.java
##########
@@ -24,10 +24,7 @@
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.jdbc.IoTDBSQLException;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.*;

Review comment:
       generate , we should not use import *

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {

Review comment:
       the same above.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {
+    if (upper == null) {
+      return true;

Review comment:
       upper == null, return true?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetaUtils;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class InternalMNode extends MNode implements IMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  /**
+   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
+   * synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  protected transient volatile Map<String, IMNode> children = null;
+
+  /**
+   * suppress warnings reason: volatile for double synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  private transient volatile Map<String, IMNode> aliasChildren = null;
+
+  // device template
+  protected Template deviceTemplate = null;
+
+  private volatile boolean useTemplate = false;
+
+  /** Constructor of MNode. */
+  public InternalMNode(IMNode parent, String name) {
+    super(parent, name);
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return (children != null && children.containsKey(name))
+        || (aliasChildren != null && aliasChildren.containsKey(name));
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   */
+  @Override
+  public void addChild(String name, IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+    child.setParent(this);
+    children.putIfAbsent(name, child);
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  public IMNode addChild(IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    child.setParent(this);
+    children.putIfAbsent(child.getName(), child);
+    return child;
+  }
+
+  /** delete a child */
+  @Override
+  public void deleteChild(String name) {
+    if (children != null) {
+      children.remove(name);
+    }
+  }
+
+  /** delete the alias of a child */
+  @Override
+  public void deleteAliasChild(String alias) {
+    if (aliasChildren != null) {
+      aliasChildren.remove(alias);
+    }
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return deviceTemplate;
+  }
+
+  @Override
+  public void setDeviceTemplate(Template deviceTemplate) {
+    this.deviceTemplate = deviceTemplate;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IMNode getChild(String name) {
+    IMNode child = null;
+    if (children != null) {
+      child = children.get(name);
+    }
+    if (child != null) {
+      return child;
+    }
+    return aliasChildren == null ? null : aliasChildren.get(name);
+  }
+
+  @Override
+  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
+    IMNode node = null;
+    // for aligned timeseries
+    List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(new PartialPath(name));
+    for (String measurement : measurementList) {
+      IMNode nodeOfMeasurement = getChild(measurement);
+      if (node == null) {
+        node = nodeOfMeasurement;
+      } else {
+        if (node != nodeOfMeasurement) {
+          throw new AlignedTimeseriesException(
+              "Cannot get node of children in different aligned timeseries", name);
+        }
+      }
+    }
+    return node;
+  }
+
+  /** get the count of all MeasurementMNode whose ancestor is current node */
+  @Override
+  public int getMeasurementMNodeCount() {
+    if (children == null) {
+      return 0;
+    }
+    int measurementMNodeCount = 0;
+    for (IMNode child : children.values()) {
+      measurementMNodeCount += child.getMeasurementMNodeCount();
+    }
+    return measurementMNodeCount;
+  }
+
+  /** add an alias */
+  @Override
+  public boolean addAlias(String alias, IMNode child) {
+    if (aliasChildren == null) {
+      // double check, alias children volatile
+      synchronized (this) {
+        if (aliasChildren == null) {
+          aliasChildren = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    return aliasChildren.computeIfAbsent(alias, aliasName -> child) == child;
+  }
+
+  @Override
+  public Map<String, IMNode> getChildren() {
+    if (children == null) {
+      return Collections.emptyMap();
+    }
+    return children;
+  }
+
+  @Override
+  public Map<String, IMNode> getAliasChildren() {
+    if (aliasChildren == null) {
+      return Collections.emptyMap();
+    }
+    return aliasChildren;
+  }
+
+  @Override
+  public void setChildren(Map<String, IMNode> children) {
+    this.children = children;
+  }
+
+  public void setAliasChildren(Map<String, IMNode> aliasChildren) {
+    this.aliasChildren = aliasChildren;
+  }
+
+  @Override
+  public void serializeTo(MLogWriter logWriter) throws IOException {

Review comment:
       serializeTo and deserializeFrom is relative,  if  you implement serializeTo, you will need add a deserializeFrom.

##########
File path: server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
##########
@@ -1408,13 +1461,13 @@ public void testTotalSeriesNumber() throws Exception {
           CompressionType.GZIP,
           null);
       manager.createTimeseries(
-          new PartialPath("root.laptop.d1.s1.t1"),
+          new PartialPath("root.laptop.d1.s2.t1"),
           TSDataType.INT32,

Review comment:
       why does need alter test value ? 

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. New mlog line number: {}.", logWriter.getLogNum());
+      }
+    } else {
+      logger.info(
+          "New mlog line number: {}, time from last modification: {} ms",
+          logWriter.getLogNum(),
+          System.currentTimeMillis() - logFile.lastModified());
+      createMTreeSnapshot();
+    }
+  }
+
+  public void createMTreeSnapshot() {
+    try {
+      mtree.createSnapshot();
+      logWriter.clear();
+    } catch (IOException e) {
+      logger.warn("Failed to create MTree snapshot", e);
+    }
+  }
+
   /** function for clearing MTree */
   public void clear() {
     try {
-      templateMap.clear();
-      this.mtree = new MTree();
+      this.mtree.clear();

Review comment:
       if this.mtree is null, will throw nullpointexception.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {

Review comment:
       public method need add some comment

##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateTimeseriesIT.java
##########
@@ -65,25 +62,30 @@ public void tearDown() throws Exception {
   }
 
   /** Test creating a time series that is a prefix path of an existing time series */
+  @Ignore // nested measurement has been forbidden
   @Test
   public void testCreateTimeseries1() throws Exception {
     String[] timeSeriesArray = {"root.sg1.aa.bb", "root.sg1.aa.bb.cc", "root.sg1.aa"};
 
-    for (String timeSeries : timeSeriesArray) {
-      statement.execute(
-          String.format(
-              "create timeseries %s with datatype=INT64, encoding=PLAIN, compression=SNAPPY",
-              timeSeries));
-    }
+    try {
+      for (String timeSeries : timeSeriesArray) {
+        statement.execute(
+            String.format(
+                "create timeseries %s with datatype=INT64, encoding=PLAIN, compression=SNAPPY",
+                timeSeries));
+      }
 
-    // ensure that current timeseries in cache is right.
-    createTimeSeries1Tool(timeSeriesArray);
+      // ensure that current timeseries in cache is right.
+      createTimeSeries1Tool(timeSeriesArray);

Review comment:
       ? space




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41710275/badge)](https://coveralls.io/builds/41710275)
   
   Coverage increased (+0.003%) to 68.215% when pulling **9c764e601773c5cae120cb6df41f58ce28742450 on zyk990424:jira_1498** into **d1729eafc6d9ba7833f728439c401d9af3409414 on apache:master**.
   


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

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

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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -1744,9 +1856,9 @@ public static MTree deserializeFrom(File mtreeSnapshot) {
   }
 
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private static MTree deserializeFromReader(MLogReader mLogReader) {
-    Deque<MNode> nodeStack = new ArrayDeque<>();
-    MNode node = null;
+  private static InternalMNode deserializeFromReader(MLogReader mLogReader) {

Review comment:
       This method with MNode return could be used in MTree internal recover by replacing root node.




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {

Review comment:
       The mtreeModifiedCheck has already been simplified in the commit "[IOTDB-1498] Simplify checkMTreeModified in MManager".




-- 
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] wangchao316 commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {

Review comment:
       Good Job. I have a little question.
   this bransh only for print log? 
   I fell if only for print log , That's not necessary.

##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBAutoCreateSchemaIT.java
##########
@@ -24,10 +24,7 @@
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.jdbc.IoTDBSQLException;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.*;

Review comment:
       generate , we should not use import *

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {

Review comment:
       the same above.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {
+    if (upper == null) {
+      return true;

Review comment:
       upper == null, return true?

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetaUtils;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class InternalMNode extends MNode implements IMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  /**
+   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
+   * synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  protected transient volatile Map<String, IMNode> children = null;
+
+  /**
+   * suppress warnings reason: volatile for double synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  private transient volatile Map<String, IMNode> aliasChildren = null;
+
+  // device template
+  protected Template deviceTemplate = null;
+
+  private volatile boolean useTemplate = false;
+
+  /** Constructor of MNode. */
+  public InternalMNode(IMNode parent, String name) {
+    super(parent, name);
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return (children != null && children.containsKey(name))
+        || (aliasChildren != null && aliasChildren.containsKey(name));
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   */
+  @Override
+  public void addChild(String name, IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+    child.setParent(this);
+    children.putIfAbsent(name, child);
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  public IMNode addChild(IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    child.setParent(this);
+    children.putIfAbsent(child.getName(), child);
+    return child;
+  }
+
+  /** delete a child */
+  @Override
+  public void deleteChild(String name) {
+    if (children != null) {
+      children.remove(name);
+    }
+  }
+
+  /** delete the alias of a child */
+  @Override
+  public void deleteAliasChild(String alias) {
+    if (aliasChildren != null) {
+      aliasChildren.remove(alias);
+    }
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return deviceTemplate;
+  }
+
+  @Override
+  public void setDeviceTemplate(Template deviceTemplate) {
+    this.deviceTemplate = deviceTemplate;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IMNode getChild(String name) {
+    IMNode child = null;
+    if (children != null) {
+      child = children.get(name);
+    }
+    if (child != null) {
+      return child;
+    }
+    return aliasChildren == null ? null : aliasChildren.get(name);
+  }
+
+  @Override
+  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
+    IMNode node = null;
+    // for aligned timeseries
+    List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(new PartialPath(name));
+    for (String measurement : measurementList) {
+      IMNode nodeOfMeasurement = getChild(measurement);
+      if (node == null) {
+        node = nodeOfMeasurement;
+      } else {
+        if (node != nodeOfMeasurement) {
+          throw new AlignedTimeseriesException(
+              "Cannot get node of children in different aligned timeseries", name);
+        }
+      }
+    }
+    return node;
+  }
+
+  /** get the count of all MeasurementMNode whose ancestor is current node */
+  @Override
+  public int getMeasurementMNodeCount() {
+    if (children == null) {
+      return 0;
+    }
+    int measurementMNodeCount = 0;
+    for (IMNode child : children.values()) {
+      measurementMNodeCount += child.getMeasurementMNodeCount();
+    }
+    return measurementMNodeCount;
+  }
+
+  /** add an alias */
+  @Override
+  public boolean addAlias(String alias, IMNode child) {
+    if (aliasChildren == null) {
+      // double check, alias children volatile
+      synchronized (this) {
+        if (aliasChildren == null) {
+          aliasChildren = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    return aliasChildren.computeIfAbsent(alias, aliasName -> child) == child;
+  }
+
+  @Override
+  public Map<String, IMNode> getChildren() {
+    if (children == null) {
+      return Collections.emptyMap();
+    }
+    return children;
+  }
+
+  @Override
+  public Map<String, IMNode> getAliasChildren() {
+    if (aliasChildren == null) {
+      return Collections.emptyMap();
+    }
+    return aliasChildren;
+  }
+
+  @Override
+  public void setChildren(Map<String, IMNode> children) {
+    this.children = children;
+  }
+
+  public void setAliasChildren(Map<String, IMNode> aliasChildren) {
+    this.aliasChildren = aliasChildren;
+  }
+
+  @Override
+  public void serializeTo(MLogWriter logWriter) throws IOException {

Review comment:
       serializeTo and deserializeFrom is relative,  if  you implement serializeTo, you will need add a deserializeFrom.

##########
File path: server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
##########
@@ -1408,13 +1461,13 @@ public void testTotalSeriesNumber() throws Exception {
           CompressionType.GZIP,
           null);
       manager.createTimeseries(
-          new PartialPath("root.laptop.d1.s1.t1"),
+          new PartialPath("root.laptop.d1.s2.t1"),
           TSDataType.INT32,

Review comment:
       why does need alter test value ? 

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. New mlog line number: {}.", logWriter.getLogNum());
+      }
+    } else {
+      logger.info(
+          "New mlog line number: {}, time from last modification: {} ms",
+          logWriter.getLogNum(),
+          System.currentTimeMillis() - logFile.lastModified());
+      createMTreeSnapshot();
+    }
+  }
+
+  public void createMTreeSnapshot() {
+    try {
+      mtree.createSnapshot();
+      logWriter.clear();
+    } catch (IOException e) {
+      logger.warn("Failed to create MTree snapshot", e);
+    }
+  }
+
   /** function for clearing MTree */
   public void clear() {
     try {
-      templateMap.clear();
-      this.mtree = new MTree();
+      this.mtree.clear();

Review comment:
       if this.mtree is null, will throw nullpointexception.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {

Review comment:
       public method need add some comment

##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateTimeseriesIT.java
##########
@@ -65,25 +62,30 @@ public void tearDown() throws Exception {
   }
 
   /** Test creating a time series that is a prefix path of an existing time series */
+  @Ignore // nested measurement has been forbidden
   @Test
   public void testCreateTimeseries1() throws Exception {
     String[] timeSeriesArray = {"root.sg1.aa.bb", "root.sg1.aa.bb.cc", "root.sg1.aa"};
 
-    for (String timeSeries : timeSeriesArray) {
-      statement.execute(
-          String.format(
-              "create timeseries %s with datatype=INT64, encoding=PLAIN, compression=SNAPPY",
-              timeSeries));
-    }
+    try {
+      for (String timeSeries : timeSeriesArray) {
+        statement.execute(
+            String.format(
+                "create timeseries %s with datatype=INT64, encoding=PLAIN, compression=SNAPPY",
+                timeSeries));
+      }
 
-    // ensure that current timeseries in cache is right.
-    createTimeSeries1Tool(timeSeriesArray);
+      // ensure that current timeseries in cache is right.
+      createTimeSeries1Tool(timeSeriesArray);

Review comment:
       ? space




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
##########
@@ -67,12 +75,21 @@ public MeasurementMNode(
   }
 
   public MeasurementMNode(
-      MNode parent, String measurementName, IMeasurementSchema schema, String alias) {
+      IMNode parent, String measurementName, IMeasurementSchema schema, String alias) {
     super(parent, measurementName);
     this.schema = schema;
     this.alias = alias;
   }
 
+  @Override
+  public int getMeasurementMNodeCount() {
+    return 1;

Review comment:
       Maybe unnecessary. It's understandable and there may be no changes on this value.




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41466416/badge)](https://coveralls.io/builds/41466416)
   
   Coverage decreased (-0.02%) to 68.115% when pulling **1f665f38843ef42e44974cb04472151bc100e959 on zyk990424:jira_1498** into **2bff46a40d2bf2a69ccd9bdd749d4aa0d1a7de66 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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
##########
@@ -67,12 +75,21 @@ public MeasurementMNode(
   }
 
   public MeasurementMNode(
-      MNode parent, String measurementName, IMeasurementSchema schema, String alias) {
+      IMNode parent, String measurementName, IMeasurementSchema schema, String alias) {
     super(parent, measurementName);
     this.schema = schema;
     this.alias = alias;
   }
 
+  @Override
+  public int getMeasurementMNodeCount() {
+    return 1;

Review comment:
       You can use a constant to replace it

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
##########
@@ -205,4 +220,88 @@ public TSDataType getDataType(String measurementId) {
       return schema.getValueTSDataTypeList().get(index);
     }
   }
+
+  @Override
+  public boolean hasChild(String name) {
+    return false;
+  }
+
+  @Override
+  public void addChild(String name, IMNode child) {
+    // Do nothing
+  }
+
+  @Override
+  public IMNode addChild(IMNode child) {
+    return null;
+  }
+
+  @Override
+  public void deleteChild(String name) {
+    // Do nothing
+  }
+
+  @Override
+  public void deleteAliasChild(String alias) {
+    // Do nothing
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return null;

Review comment:
       May be throw a exception here and explain we can't get template from a measurement node

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Map;
+
+public interface IMNode extends Serializable {

Review comment:
       Maybe you can add some doc to this critical interface




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. New mlog line number: {}.", logWriter.getLogNum());
+      }
+    } else {
+      logger.info(
+          "New mlog line number: {}, time from last modification: {} ms",
+          logWriter.getLogNum(),
+          System.currentTimeMillis() - logFile.lastModified());
+      createMTreeSnapshot();
+    }
+  }
+
+  public void createMTreeSnapshot() {
+    try {
+      mtree.createSnapshot();
+      logWriter.clear();
+    } catch (IOException e) {
+      logger.warn("Failed to create MTree snapshot", e);
+    }
+  }
+
   /** function for clearing MTree */
   public void clear() {
     try {
-      templateMap.clear();
-      this.mtree = new MTree();
+      this.mtree.clear();

Review comment:
       Thanks for reminding.

##########
File path: server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
##########
@@ -1408,13 +1461,13 @@ public void testTotalSeriesNumber() throws Exception {
           CompressionType.GZIP,
           null);
       manager.createTimeseries(
-          new PartialPath("root.laptop.d1.s1.t1"),
+          new PartialPath("root.laptop.d1.s2.t1"),
           TSDataType.INT32,

Review comment:
       The MeasurementMNode doesn't support addChild anymore, which is declared in jira 1495. The  related test case should be modified or deleted.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {
+    if (upper == null) {
+      return true;

Review comment:
       If the upperTemplate is null, then the current template to be set  will have no conflict with current mtree. The rule is that the lower level template should contains all schema defined in upper template.




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41530680/badge)](https://coveralls.io/builds/41530680)
   
   Coverage increased (+0.001%) to 68.165% when pulling **9edd70dedc0943500166a3b11533a79250753d5f on zyk990424:jira_1498** into **79e50d651a2b488ac211009947b2a01c5244c37b on apache:master**.
   


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

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

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



[GitHub] [iotdb] qiaojialin commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
##########
@@ -1744,9 +1856,9 @@ public static MTree deserializeFrom(File mtreeSnapshot) {
   }
 
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private static MTree deserializeFromReader(MLogReader mLogReader) {
-    Deque<MNode> nodeStack = new ArrayDeque<>();
-    MNode node = null;
+  private static InternalMNode deserializeFromReader(MLogReader mLogReader) {

Review comment:
       I'm not sure about this.. please add some javadoc

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {

Review comment:
       This check is not needed

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {

Review comment:
       +1




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Map;
+
+public interface IMNode extends Serializable {

Review comment:
       Fixed




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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41594636/badge)](https://coveralls.io/builds/41594636)
   
   Coverage decreased (-0.08%) to 68.084% when pulling **d44e29293221a64e3cc0f7ce191afdbc03a29a24 on zyk990424:jira_1498** into **79e50d651a2b488ac211009947b2a01c5244c37b 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] wangchao316 commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetaUtils;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class InternalMNode extends MNode implements IMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  /**
+   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
+   * synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  protected transient volatile Map<String, IMNode> children = null;
+
+  /**
+   * suppress warnings reason: volatile for double synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  private transient volatile Map<String, IMNode> aliasChildren = null;
+
+  // device template
+  protected Template deviceTemplate = null;
+
+  private volatile boolean useTemplate = false;
+
+  /** Constructor of MNode. */
+  public InternalMNode(IMNode parent, String name) {
+    super(parent, name);
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return (children != null && children.containsKey(name))
+        || (aliasChildren != null && aliasChildren.containsKey(name));
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   */
+  @Override
+  public void addChild(String name, IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+    child.setParent(this);
+    children.putIfAbsent(name, child);
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  public IMNode addChild(IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    child.setParent(this);
+    children.putIfAbsent(child.getName(), child);
+    return child;
+  }
+
+  /** delete a child */
+  @Override
+  public void deleteChild(String name) {
+    if (children != null) {
+      children.remove(name);
+    }
+  }
+
+  /** delete the alias of a child */
+  @Override
+  public void deleteAliasChild(String alias) {
+    if (aliasChildren != null) {
+      aliasChildren.remove(alias);
+    }
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return deviceTemplate;
+  }
+
+  @Override
+  public void setDeviceTemplate(Template deviceTemplate) {
+    this.deviceTemplate = deviceTemplate;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IMNode getChild(String name) {
+    IMNode child = null;
+    if (children != null) {
+      child = children.get(name);
+    }
+    if (child != null) {
+      return child;
+    }
+    return aliasChildren == null ? null : aliasChildren.get(name);
+  }
+
+  @Override
+  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
+    IMNode node = null;
+    // for aligned timeseries
+    List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(new PartialPath(name));
+    for (String measurement : measurementList) {
+      IMNode nodeOfMeasurement = getChild(measurement);
+      if (node == null) {
+        node = nodeOfMeasurement;
+      } else {
+        if (node != nodeOfMeasurement) {
+          throw new AlignedTimeseriesException(
+              "Cannot get node of children in different aligned timeseries", name);
+        }
+      }
+    }
+    return node;
+  }
+
+  /** get the count of all MeasurementMNode whose ancestor is current node */
+  @Override
+  public int getMeasurementMNodeCount() {
+    if (children == null) {
+      return 0;
+    }
+    int measurementMNodeCount = 0;
+    for (IMNode child : children.values()) {
+      measurementMNodeCount += child.getMeasurementMNodeCount();
+    }
+    return measurementMNodeCount;
+  }
+
+  /** add an alias */
+  @Override
+  public boolean addAlias(String alias, IMNode child) {
+    if (aliasChildren == null) {
+      // double check, alias children volatile
+      synchronized (this) {
+        if (aliasChildren == null) {
+          aliasChildren = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    return aliasChildren.computeIfAbsent(alias, aliasName -> child) == child;
+  }
+
+  @Override
+  public Map<String, IMNode> getChildren() {
+    if (children == null) {
+      return Collections.emptyMap();
+    }
+    return children;
+  }
+
+  @Override
+  public Map<String, IMNode> getAliasChildren() {
+    if (aliasChildren == null) {
+      return Collections.emptyMap();
+    }
+    return aliasChildren;
+  }
+
+  @Override
+  public void setChildren(Map<String, IMNode> children) {
+    this.children = children;
+  }
+
+  public void setAliasChildren(Map<String, IMNode> aliasChildren) {
+    this.aliasChildren = aliasChildren;
+  }
+
+  @Override
+  public void serializeTo(MLogWriter logWriter) throws IOException {

Review comment:
       where is deserializeFrom ?  I do not find this.




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {

Review comment:
       This method is part of existing code and I just separate it  from MManager, but it will be deleted in further feature development referring to jira 1526.




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {

Review comment:
       The template check will be modified in further feature development. 
   The mtreeModifiedCheck has already been simplified in the commit "[IOTDB-1498] Simplify checkMTreeModified in MManager".




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41703976/badge)](https://coveralls.io/builds/41703976)
   
   Coverage decreased (-0.02%) to 68.188% when pulling **440e9447b944b744a82eb6d14d493ec9b326f703 on zyk990424:jira_1498** into **95e3b69402cd40555d8a9654833b552d7c7e3d9d 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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41466413/badge)](https://coveralls.io/builds/41466413)
   
   Coverage increased (+0.01%) to 68.15% when pulling **1f665f38843ef42e44974cb04472151bc100e959 on zyk990424:jira_1498** into **2bff46a40d2bf2a69ccd9bdd749d4aa0d1a7de66 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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41705449/badge)](https://coveralls.io/builds/41705449)
   
   Coverage decreased (-0.001%) to 68.211% when pulling **a25e2db8ef9d48f90e2fe734b200b01e082105d5 on zyk990424:jira_1498** into **d1729eafc6d9ba7833f728439c401d9af3409414 on apache:master**.
   


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

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

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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
##########
@@ -205,4 +220,88 @@ public TSDataType getDataType(String measurementId) {
       return schema.getValueTSDataTypeList().get(index);
     }
   }
+
+  @Override
+  public boolean hasChild(String name) {
+    return false;
+  }
+
+  @Override
+  public void addChild(String name, IMNode child) {
+    // Do nothing
+  }
+
+  @Override
+  public IMNode addChild(IMNode child) {
+    return null;
+  }
+
+  @Override
+  public void deleteChild(String name) {
+    // Do nothing
+  }
+
+  @Override
+  public void deleteAliasChild(String alias) {
+    // Do nothing
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return null;

Review comment:
       Fixed




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

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

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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
##########
@@ -1408,13 +1461,13 @@ public void testTotalSeriesNumber() throws Exception {
           CompressionType.GZIP,
           null);
       manager.createTimeseries(
-          new PartialPath("root.laptop.d1.s1.t1"),
+          new PartialPath("root.laptop.d1.s2.t1"),
           TSDataType.INT32,

Review comment:
       The MeasurementMNode doesn't support addChild anymore, which is declared in jira 1495. The  related test case should be modified or deleted.




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {

Review comment:
       Fixed

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {

Review comment:
       Fixed




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

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

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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetaUtils;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class InternalMNode extends MNode implements IMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  /**
+   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
+   * synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  protected transient volatile Map<String, IMNode> children = null;
+
+  /**
+   * suppress warnings reason: volatile for double synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  private transient volatile Map<String, IMNode> aliasChildren = null;
+
+  // device template
+  protected Template deviceTemplate = null;
+
+  private volatile boolean useTemplate = false;
+
+  /** Constructor of MNode. */
+  public InternalMNode(IMNode parent, String name) {
+    super(parent, name);
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return (children != null && children.containsKey(name))
+        || (aliasChildren != null && aliasChildren.containsKey(name));
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   */
+  @Override
+  public void addChild(String name, IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+    child.setParent(this);
+    children.putIfAbsent(name, child);
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  public IMNode addChild(IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    child.setParent(this);
+    children.putIfAbsent(child.getName(), child);
+    return child;
+  }
+
+  /** delete a child */
+  @Override
+  public void deleteChild(String name) {
+    if (children != null) {
+      children.remove(name);
+    }
+  }
+
+  /** delete the alias of a child */
+  @Override
+  public void deleteAliasChild(String alias) {
+    if (aliasChildren != null) {
+      aliasChildren.remove(alias);
+    }
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return deviceTemplate;
+  }
+
+  @Override
+  public void setDeviceTemplate(Template deviceTemplate) {
+    this.deviceTemplate = deviceTemplate;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IMNode getChild(String name) {
+    IMNode child = null;
+    if (children != null) {
+      child = children.get(name);
+    }
+    if (child != null) {
+      return child;
+    }
+    return aliasChildren == null ? null : aliasChildren.get(name);
+  }
+
+  @Override
+  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
+    IMNode node = null;
+    // for aligned timeseries
+    List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(new PartialPath(name));
+    for (String measurement : measurementList) {
+      IMNode nodeOfMeasurement = getChild(measurement);
+      if (node == null) {
+        node = nodeOfMeasurement;
+      } else {
+        if (node != nodeOfMeasurement) {
+          throw new AlignedTimeseriesException(
+              "Cannot get node of children in different aligned timeseries", name);
+        }
+      }
+    }
+    return node;
+  }
+
+  /** get the count of all MeasurementMNode whose ancestor is current node */
+  @Override
+  public int getMeasurementMNodeCount() {
+    if (children == null) {
+      return 0;
+    }
+    int measurementMNodeCount = 0;
+    for (IMNode child : children.values()) {
+      measurementMNodeCount += child.getMeasurementMNodeCount();
+    }
+    return measurementMNodeCount;
+  }
+
+  /** add an alias */
+  @Override
+  public boolean addAlias(String alias, IMNode child) {
+    if (aliasChildren == null) {
+      // double check, alias children volatile
+      synchronized (this) {
+        if (aliasChildren == null) {
+          aliasChildren = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    return aliasChildren.computeIfAbsent(alias, aliasName -> child) == child;
+  }
+
+  @Override
+  public Map<String, IMNode> getChildren() {
+    if (children == null) {
+      return Collections.emptyMap();
+    }
+    return children;
+  }
+
+  @Override
+  public Map<String, IMNode> getAliasChildren() {
+    if (aliasChildren == null) {
+      return Collections.emptyMap();
+    }
+    return aliasChildren;
+  }
+
+  @Override
+  public void setChildren(Map<String, IMNode> children) {
+    this.children = children;
+  }
+
+  public void setAliasChildren(Map<String, IMNode> aliasChildren) {
+    this.aliasChildren = aliasChildren;
+  }
+
+  @Override
+  public void serializeTo(MLogWriter logWriter) throws IOException {

Review comment:
       It's just under the serializeChildren method.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetaUtils;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class InternalMNode extends MNode implements IMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  /**
+   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
+   * synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  protected transient volatile Map<String, IMNode> children = null;
+
+  /**
+   * suppress warnings reason: volatile for double synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  private transient volatile Map<String, IMNode> aliasChildren = null;
+
+  // device template
+  protected Template deviceTemplate = null;
+
+  private volatile boolean useTemplate = false;
+
+  /** Constructor of MNode. */
+  public InternalMNode(IMNode parent, String name) {
+    super(parent, name);
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return (children != null && children.containsKey(name))
+        || (aliasChildren != null && aliasChildren.containsKey(name));
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   */
+  @Override
+  public void addChild(String name, IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+    child.setParent(this);
+    children.putIfAbsent(name, child);
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  public IMNode addChild(IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    child.setParent(this);
+    children.putIfAbsent(child.getName(), child);
+    return child;
+  }
+
+  /** delete a child */
+  @Override
+  public void deleteChild(String name) {
+    if (children != null) {
+      children.remove(name);
+    }
+  }
+
+  /** delete the alias of a child */
+  @Override
+  public void deleteAliasChild(String alias) {
+    if (aliasChildren != null) {
+      aliasChildren.remove(alias);
+    }
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return deviceTemplate;
+  }
+
+  @Override
+  public void setDeviceTemplate(Template deviceTemplate) {
+    this.deviceTemplate = deviceTemplate;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IMNode getChild(String name) {
+    IMNode child = null;
+    if (children != null) {
+      child = children.get(name);
+    }
+    if (child != null) {
+      return child;
+    }
+    return aliasChildren == null ? null : aliasChildren.get(name);
+  }
+
+  @Override
+  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
+    IMNode node = null;
+    // for aligned timeseries
+    List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(new PartialPath(name));
+    for (String measurement : measurementList) {
+      IMNode nodeOfMeasurement = getChild(measurement);
+      if (node == null) {
+        node = nodeOfMeasurement;
+      } else {
+        if (node != nodeOfMeasurement) {
+          throw new AlignedTimeseriesException(
+              "Cannot get node of children in different aligned timeseries", name);
+        }
+      }
+    }
+    return node;
+  }
+
+  /** get the count of all MeasurementMNode whose ancestor is current node */
+  @Override
+  public int getMeasurementMNodeCount() {
+    if (children == null) {
+      return 0;
+    }
+    int measurementMNodeCount = 0;
+    for (IMNode child : children.values()) {
+      measurementMNodeCount += child.getMeasurementMNodeCount();
+    }
+    return measurementMNodeCount;
+  }
+
+  /** add an alias */
+  @Override
+  public boolean addAlias(String alias, IMNode child) {
+    if (aliasChildren == null) {
+      // double check, alias children volatile
+      synchronized (this) {
+        if (aliasChildren == null) {
+          aliasChildren = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    return aliasChildren.computeIfAbsent(alias, aliasName -> child) == child;
+  }
+
+  @Override
+  public Map<String, IMNode> getChildren() {
+    if (children == null) {
+      return Collections.emptyMap();
+    }
+    return children;
+  }
+
+  @Override
+  public Map<String, IMNode> getAliasChildren() {
+    if (aliasChildren == null) {
+      return Collections.emptyMap();
+    }
+    return aliasChildren;
+  }
+
+  @Override
+  public void setChildren(Map<String, IMNode> children) {
+    this.children = children;
+  }
+
+  public void setAliasChildren(Map<String, IMNode> aliasChildren) {
+    this.aliasChildren = aliasChildren;
+  }
+
+  @Override
+  public void serializeTo(MLogWriter logWriter) throws IOException {

Review comment:
       At line 259 in InternalMNode.java

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetaUtils;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class InternalMNode extends MNode implements IMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  /**
+   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
+   * synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  protected transient volatile Map<String, IMNode> children = null;
+
+  /**
+   * suppress warnings reason: volatile for double synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  private transient volatile Map<String, IMNode> aliasChildren = null;
+
+  // device template
+  protected Template deviceTemplate = null;
+
+  private volatile boolean useTemplate = false;
+
+  /** Constructor of MNode. */
+  public InternalMNode(IMNode parent, String name) {
+    super(parent, name);
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return (children != null && children.containsKey(name))
+        || (aliasChildren != null && aliasChildren.containsKey(name));
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   */
+  @Override
+  public void addChild(String name, IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+    child.setParent(this);
+    children.putIfAbsent(name, child);
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  public IMNode addChild(IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    child.setParent(this);
+    children.putIfAbsent(child.getName(), child);
+    return child;
+  }
+
+  /** delete a child */
+  @Override
+  public void deleteChild(String name) {
+    if (children != null) {
+      children.remove(name);
+    }
+  }
+
+  /** delete the alias of a child */
+  @Override
+  public void deleteAliasChild(String alias) {
+    if (aliasChildren != null) {
+      aliasChildren.remove(alias);
+    }
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return deviceTemplate;
+  }
+
+  @Override
+  public void setDeviceTemplate(Template deviceTemplate) {
+    this.deviceTemplate = deviceTemplate;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IMNode getChild(String name) {
+    IMNode child = null;
+    if (children != null) {
+      child = children.get(name);
+    }
+    if (child != null) {
+      return child;
+    }
+    return aliasChildren == null ? null : aliasChildren.get(name);
+  }
+
+  @Override
+  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
+    IMNode node = null;
+    // for aligned timeseries
+    List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(new PartialPath(name));
+    for (String measurement : measurementList) {
+      IMNode nodeOfMeasurement = getChild(measurement);
+      if (node == null) {
+        node = nodeOfMeasurement;
+      } else {
+        if (node != nodeOfMeasurement) {
+          throw new AlignedTimeseriesException(
+              "Cannot get node of children in different aligned timeseries", name);
+        }
+      }
+    }
+    return node;
+  }
+
+  /** get the count of all MeasurementMNode whose ancestor is current node */
+  @Override
+  public int getMeasurementMNodeCount() {
+    if (children == null) {
+      return 0;
+    }
+    int measurementMNodeCount = 0;
+    for (IMNode child : children.values()) {
+      measurementMNodeCount += child.getMeasurementMNodeCount();
+    }
+    return measurementMNodeCount;
+  }
+
+  /** add an alias */
+  @Override
+  public boolean addAlias(String alias, IMNode child) {
+    if (aliasChildren == null) {
+      // double check, alias children volatile
+      synchronized (this) {
+        if (aliasChildren == null) {
+          aliasChildren = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    return aliasChildren.computeIfAbsent(alias, aliasName -> child) == child;
+  }
+
+  @Override
+  public Map<String, IMNode> getChildren() {
+    if (children == null) {
+      return Collections.emptyMap();
+    }
+    return children;
+  }
+
+  @Override
+  public Map<String, IMNode> getAliasChildren() {
+    if (aliasChildren == null) {
+      return Collections.emptyMap();
+    }
+    return aliasChildren;
+  }
+
+  @Override
+  public void setChildren(Map<String, IMNode> children) {
+    this.children = children;
+  }
+
+  public void setAliasChildren(Map<String, IMNode> aliasChildren) {
+    this.aliasChildren = aliasChildren;
+  }
+
+  @Override
+  public void serializeTo(MLogWriter logWriter) throws IOException {

Review comment:
       public static InternalMNode deserializeFrom(MNodePlan plan) {
       return new InternalMNode(null, plan.getName());
     }

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetaUtils;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class InternalMNode extends MNode implements IMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  /**
+   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
+   * synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  protected transient volatile Map<String, IMNode> children = null;
+
+  /**
+   * suppress warnings reason: volatile for double synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  private transient volatile Map<String, IMNode> aliasChildren = null;
+
+  // device template
+  protected Template deviceTemplate = null;
+
+  private volatile boolean useTemplate = false;
+
+  /** Constructor of MNode. */
+  public InternalMNode(IMNode parent, String name) {
+    super(parent, name);
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return (children != null && children.containsKey(name))
+        || (aliasChildren != null && aliasChildren.containsKey(name));
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   */
+  @Override
+  public void addChild(String name, IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+    child.setParent(this);
+    children.putIfAbsent(name, child);
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  public IMNode addChild(IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    child.setParent(this);
+    children.putIfAbsent(child.getName(), child);
+    return child;
+  }
+
+  /** delete a child */
+  @Override
+  public void deleteChild(String name) {
+    if (children != null) {
+      children.remove(name);
+    }
+  }
+
+  /** delete the alias of a child */
+  @Override
+  public void deleteAliasChild(String alias) {
+    if (aliasChildren != null) {
+      aliasChildren.remove(alias);
+    }
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return deviceTemplate;
+  }
+
+  @Override
+  public void setDeviceTemplate(Template deviceTemplate) {
+    this.deviceTemplate = deviceTemplate;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IMNode getChild(String name) {
+    IMNode child = null;
+    if (children != null) {
+      child = children.get(name);
+    }
+    if (child != null) {
+      return child;
+    }
+    return aliasChildren == null ? null : aliasChildren.get(name);
+  }
+
+  @Override
+  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
+    IMNode node = null;
+    // for aligned timeseries
+    List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(new PartialPath(name));
+    for (String measurement : measurementList) {
+      IMNode nodeOfMeasurement = getChild(measurement);
+      if (node == null) {
+        node = nodeOfMeasurement;
+      } else {
+        if (node != nodeOfMeasurement) {
+          throw new AlignedTimeseriesException(
+              "Cannot get node of children in different aligned timeseries", name);
+        }
+      }
+    }
+    return node;
+  }
+
+  /** get the count of all MeasurementMNode whose ancestor is current node */
+  @Override
+  public int getMeasurementMNodeCount() {
+    if (children == null) {
+      return 0;
+    }
+    int measurementMNodeCount = 0;
+    for (IMNode child : children.values()) {
+      measurementMNodeCount += child.getMeasurementMNodeCount();
+    }
+    return measurementMNodeCount;
+  }
+
+  /** add an alias */
+  @Override
+  public boolean addAlias(String alias, IMNode child) {
+    if (aliasChildren == null) {
+      // double check, alias children volatile
+      synchronized (this) {
+        if (aliasChildren == null) {
+          aliasChildren = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    return aliasChildren.computeIfAbsent(alias, aliasName -> child) == child;
+  }
+
+  @Override
+  public Map<String, IMNode> getChildren() {
+    if (children == null) {
+      return Collections.emptyMap();
+    }
+    return children;
+  }
+
+  @Override
+  public Map<String, IMNode> getAliasChildren() {
+    if (aliasChildren == null) {
+      return Collections.emptyMap();
+    }
+    return aliasChildren;
+  }
+
+  @Override
+  public void setChildren(Map<String, IMNode> children) {
+    this.children = children;
+  }
+
+  public void setAliasChildren(Map<String, IMNode> aliasChildren) {
+    this.aliasChildren = aliasChildren;
+  }
+
+  @Override
+  public void serializeTo(MLogWriter logWriter) throws IOException {

Review comment:
       Only MLogReader uses MNode's deserializeFrom method, and the param is MNodePlan.




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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






-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetaUtils;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is the implementation of Metadata Node. One MNode instance represents one node in the
+ * Metadata Tree
+ */
+public class InternalMNode extends MNode implements IMNode {
+
+  private static final long serialVersionUID = -770028375899514063L;
+
+  /**
+   * use in Measurement Node so it's protected suppress warnings reason: volatile for double
+   * synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  protected transient volatile Map<String, IMNode> children = null;
+
+  /**
+   * suppress warnings reason: volatile for double synchronized check
+   *
+   * <p>This will be a ConcurrentHashMap instance
+   */
+  @SuppressWarnings("squid:S3077")
+  private transient volatile Map<String, IMNode> aliasChildren = null;
+
+  // device template
+  protected Template deviceTemplate = null;
+
+  private volatile boolean useTemplate = false;
+
+  /** Constructor of MNode. */
+  public InternalMNode(IMNode parent, String name) {
+    super(parent, name);
+  }
+
+  /** check whether the MNode has a child with the name */
+  @Override
+  public boolean hasChild(String name) {
+    return (children != null && children.containsKey(name))
+        || (aliasChildren != null && aliasChildren.containsKey(name));
+  }
+
+  /**
+   * add a child to current mnode
+   *
+   * @param name child's name
+   * @param child child's node
+   */
+  @Override
+  public void addChild(String name, IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+    child.setParent(this);
+    children.putIfAbsent(name, child);
+  }
+
+  /**
+   * Add a child to the current mnode.
+   *
+   * <p>This method will not take the child's name as one of the inputs and will also make this
+   * Mnode be child node's parent. All is to reduce the probability of mistaken by users and be more
+   * convenient for users to use. And the return of this method is used to conveniently construct a
+   * chain of time series for users.
+   *
+   * @param child child's node
+   * @return return the MNode already added
+   */
+  public IMNode addChild(IMNode child) {
+    /* use cpu time to exchange memory
+     * measurementNode's children should be null to save memory
+     * add child method will only be called when writing MTree, which is not a frequent operation
+     */
+    if (children == null) {
+      // double check, children is volatile
+      synchronized (this) {
+        if (children == null) {
+          children = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    child.setParent(this);
+    children.putIfAbsent(child.getName(), child);
+    return child;
+  }
+
+  /** delete a child */
+  @Override
+  public void deleteChild(String name) {
+    if (children != null) {
+      children.remove(name);
+    }
+  }
+
+  /** delete the alias of a child */
+  @Override
+  public void deleteAliasChild(String alias) {
+    if (aliasChildren != null) {
+      aliasChildren.remove(alias);
+    }
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return deviceTemplate;
+  }
+
+  @Override
+  public void setDeviceTemplate(Template deviceTemplate) {
+    this.deviceTemplate = deviceTemplate;
+  }
+
+  /** get the child with the name */
+  @Override
+  public IMNode getChild(String name) {
+    IMNode child = null;
+    if (children != null) {
+      child = children.get(name);
+    }
+    if (child != null) {
+      return child;
+    }
+    return aliasChildren == null ? null : aliasChildren.get(name);
+  }
+
+  @Override
+  public IMNode getChildOfAlignedTimeseries(String name) throws MetadataException {
+    IMNode node = null;
+    // for aligned timeseries
+    List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(new PartialPath(name));
+    for (String measurement : measurementList) {
+      IMNode nodeOfMeasurement = getChild(measurement);
+      if (node == null) {
+        node = nodeOfMeasurement;
+      } else {
+        if (node != nodeOfMeasurement) {
+          throw new AlignedTimeseriesException(
+              "Cannot get node of children in different aligned timeseries", name);
+        }
+      }
+    }
+    return node;
+  }
+
+  /** get the count of all MeasurementMNode whose ancestor is current node */
+  @Override
+  public int getMeasurementMNodeCount() {
+    if (children == null) {
+      return 0;
+    }
+    int measurementMNodeCount = 0;
+    for (IMNode child : children.values()) {
+      measurementMNodeCount += child.getMeasurementMNodeCount();
+    }
+    return measurementMNodeCount;
+  }
+
+  /** add an alias */
+  @Override
+  public boolean addAlias(String alias, IMNode child) {
+    if (aliasChildren == null) {
+      // double check, alias children volatile
+      synchronized (this) {
+        if (aliasChildren == null) {
+          aliasChildren = new ConcurrentHashMap<>();
+        }
+      }
+    }
+
+    return aliasChildren.computeIfAbsent(alias, aliasName -> child) == child;
+  }
+
+  @Override
+  public Map<String, IMNode> getChildren() {
+    if (children == null) {
+      return Collections.emptyMap();
+    }
+    return children;
+  }
+
+  @Override
+  public Map<String, IMNode> getAliasChildren() {
+    if (aliasChildren == null) {
+      return Collections.emptyMap();
+    }
+    return aliasChildren;
+  }
+
+  @Override
+  public void setChildren(Map<String, IMNode> children) {
+    this.children = children;
+  }
+
+  public void setAliasChildren(Map<String, IMNode> aliasChildren) {
+    this.aliasChildren = aliasChildren;
+  }
+
+  @Override
+  public void serializeTo(MLogWriter logWriter) throws IOException {

Review comment:
       Fixed




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

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

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



[GitHub] [iotdb] SilverNarcissus commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
##########
@@ -67,12 +75,21 @@ public MeasurementMNode(
   }
 
   public MeasurementMNode(
-      MNode parent, String measurementName, IMeasurementSchema schema, String alias) {
+      IMNode parent, String measurementName, IMeasurementSchema schema, String alias) {
     super(parent, measurementName);
     this.schema = schema;
     this.alias = alias;
   }
 
+  @Override
+  public int getMeasurementMNodeCount() {
+    return 1;

Review comment:
       You can use a constant to replace it

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
##########
@@ -205,4 +220,88 @@ public TSDataType getDataType(String measurementId) {
       return schema.getValueTSDataTypeList().get(index);
     }
   }
+
+  @Override
+  public boolean hasChild(String name) {
+    return false;
+  }
+
+  @Override
+  public void addChild(String name, IMNode child) {
+    // Do nothing
+  }
+
+  @Override
+  public IMNode addChild(IMNode child) {
+    return null;
+  }
+
+  @Override
+  public void deleteChild(String name) {
+    // Do nothing
+  }
+
+  @Override
+  public void deleteAliasChild(String alias) {
+    // Do nothing
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return null;

Review comment:
       May be throw a exception here and explain we can't get template from a measurement node

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Map;
+
+public interface IMNode extends Serializable {

Review comment:
       Maybe you can add some doc to this critical interface




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41471742/badge)](https://coveralls.io/builds/41471742)
   
   Coverage remained the same at 68.138% when pulling **ad52b572f7711f68082c6cdf7c332f10d1d3419c on zyk990424:jira_1498** into **2bff46a40d2bf2a69ccd9bdd749d4aa0d1a7de66 on apache:master**.
   


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

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

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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBAutoCreateSchemaIT.java
##########
@@ -24,10 +24,7 @@
 import org.apache.iotdb.jdbc.Config;
 import org.apache.iotdb.jdbc.IoTDBSQLException;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.*;

Review comment:
       Actually, all the unit under this package has been used and IDEA auto simplifies the import. 




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41472481/badge)](https://coveralls.io/builds/41472481)
   
   Coverage decreased (-0.01%) to 68.123% when pulling **ad52b572f7711f68082c6cdf7c332f10d1d3419c on zyk990424:jira_1498** into **2bff46a40d2bf2a69ccd9bdd749d4aa0d1a7de66 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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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






-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41530630/badge)](https://coveralls.io/builds/41530630)
   
   Coverage decreased (-0.008%) to 68.156% when pulling **9edd70dedc0943500166a3b11533a79250753d5f on zyk990424:jira_1498** into **79e50d651a2b488ac211009947b2a01c5244c37b 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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41708326/badge)](https://coveralls.io/builds/41708326)
   
   Coverage increased (+0.005%) to 68.218% when pulling **e8ca6cafc98e9cd15a7152d0b91ef61ba11f0c18 on zyk990424:jira_1498** into **d1729eafc6d9ba7833f728439c401d9af3409414 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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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


   
   [![Coverage Status](https://coveralls.io/builds/41705771/badge)](https://coveralls.io/builds/41705771)
   
   Coverage decreased (-0.001%) to 68.211% when pulling **a25e2db8ef9d48f90e2fe734b200b01e082105d5 on zyk990424:jira_1498** into **d1729eafc6d9ba7833f728439c401d9af3409414 on apache:master**.
   


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

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

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



[GitHub] [iotdb] zyk990424 commented on a change in pull request #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBCreateTimeseriesIT.java
##########
@@ -65,25 +62,30 @@ public void tearDown() throws Exception {
   }
 
   /** Test creating a time series that is a prefix path of an existing time series */
+  @Ignore // nested measurement has been forbidden
   @Test
   public void testCreateTimeseries1() throws Exception {
     String[] timeSeriesArray = {"root.sg1.aa.bb", "root.sg1.aa.bb.cc", "root.sg1.aa"};
 
-    for (String timeSeries : timeSeriesArray) {
-      statement.execute(
-          String.format(
-              "create timeseries %s with datatype=INT64, encoding=PLAIN, compression=SNAPPY",
-              timeSeries));
-    }
+    try {
+      for (String timeSeries : timeSeriesArray) {
+        statement.execute(
+            String.format(
+                "create timeseries %s with datatype=INT64, encoding=PLAIN, compression=SNAPPY",
+                timeSeries));
+      }
 
-    // ensure that current timeseries in cache is right.
-    createTimeSeries1Tool(timeSeriesArray);
+      // ensure that current timeseries in cache is right.
+      createTimeSeries1Tool(timeSeriesArray);

Review comment:
       Maybe, the code is aligned with the context. I think this doesn't matter and could pass spotless.




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {
+    if (upper == null) {
+      return true;

Review comment:
       If the upperTemplate is null, then the current template to be set  will have no conflict with current mtree. The rule is that the lower level template should contains all schema defined in upper template.




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
##########
@@ -67,12 +75,21 @@ public MeasurementMNode(
   }
 
   public MeasurementMNode(
-      MNode parent, String measurementName, IMeasurementSchema schema, String alias) {
+      IMNode parent, String measurementName, IMeasurementSchema schema, String alias) {
     super(parent, measurementName);
     this.schema = schema;
     this.alias = alias;
   }
 
+  @Override
+  public int getMeasurementMNodeCount() {
+    return 1;

Review comment:
       You can use a constant to replace it

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
##########
@@ -205,4 +220,88 @@ public TSDataType getDataType(String measurementId) {
       return schema.getValueTSDataTypeList().get(index);
     }
   }
+
+  @Override
+  public boolean hasChild(String name) {
+    return false;
+  }
+
+  @Override
+  public void addChild(String name, IMNode child) {
+    // Do nothing
+  }
+
+  @Override
+  public IMNode addChild(IMNode child) {
+    return null;
+  }
+
+  @Override
+  public void deleteChild(String name) {
+    // Do nothing
+  }
+
+  @Override
+  public void deleteAliasChild(String alias) {
+    // Do nothing
+  }
+
+  @Override
+  public Template getDeviceTemplate() {
+    return null;

Review comment:
       May be throw a exception here and explain we can't get template from a measurement node

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMNode.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.mnode;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Map;
+
+public interface IMNode extends Serializable {

Review comment:
       Maybe you can add some doc to this critical interface




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. New mlog line number: {}.", logWriter.getLogNum());
+      }
+    } else {
+      logger.info(
+          "New mlog line number: {}, time from last modification: {} ms",
+          logWriter.getLogNum(),
+          System.currentTimeMillis() - logFile.lastModified());
+      createMTreeSnapshot();
+    }
+  }
+
+  public void createMTreeSnapshot() {
+    try {
+      mtree.createSnapshot();
+      logWriter.clear();
+    } catch (IOException e) {
+      logger.warn("Failed to create MTree snapshot", e);
+    }
+  }
+
   /** function for clearing MTree */
   public void clear() {
     try {
-      templateMap.clear();
-      this.mtree = new MTree();
+      this.mtree.clear();

Review comment:
       Thanks for reminding.

##########
File path: server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
##########
@@ -1408,13 +1461,13 @@ public void testTotalSeriesNumber() throws Exception {
           CompressionType.GZIP,
           null);
       manager.createTimeseries(
-          new PartialPath("root.laptop.d1.s1.t1"),
+          new PartialPath("root.laptop.d1.s2.t1"),
           TSDataType.INT32,

Review comment:
       The MeasurementMNode doesn't support addChild anymore, which is declared in jira 1495. The  related test case should be modified or deleted.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.metadata.template;
+
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class TemplateManager {
+
+  // template name -> template
+  private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
+  private static class TemplateManagerHolder {
+
+    private TemplateManagerHolder() {
+      // allowed to do nothing
+    }
+
+    private static final TemplateManager INSTANCE = new TemplateManager();
+  }
+
+  public static TemplateManager getInstance() {
+    return TemplateManagerHolder.INSTANCE;
+  }
+
+  private TemplateManager() {}
+
+  public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+    Template template = new Template(plan);
+    if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+      // already have template
+      throw new MetadataException("Duplicated template name: " + plan.getName());
+    }
+  }
+
+  public Template getTemplate(String templateName) throws UndefinedTemplateException {
+    Template template = templateMap.get(templateName);
+    if (template == null) {
+      throw new UndefinedTemplateException(templateName);
+    }
+    return template;
+  }
+
+  public void setDeviceTemplate(Template template, Pair<IMNode, Template> node)
+      throws MetadataException {
+
+    if (node.left.getDeviceTemplate() != null) {
+      if (node.left.getDeviceTemplate().equals(template)) {
+        throw new DuplicatedTemplateException(template.getName());
+      } else {
+        throw new MetadataException("Specified node already has template");
+      }
+    }
+
+    if (!isTemplateCompatible(node.right, template)) {
+      throw new MetadataException("Incompatible template");
+    }
+
+    checkIsTemplateAndMNodeCompatible(template, node.left);
+
+    node.left.setDeviceTemplate(template);
+  }
+
+  public boolean isTemplateCompatible(Template upper, Template current) {
+    if (upper == null) {
+      return true;

Review comment:
       If the upperTemplate is null, then the current template to be set  will have no conflict with current mtree. The rule is that the lower level template should contains all schema defined in upper template.




-- 
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 #3589: [IOTDB-1498] MNode Abstraction and Structure Improvement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -313,23 +279,52 @@ private int applyMlog(MLogReader mLogReader) {
     return idx;
   }
 
+  private void checkMTreeModified() {
+    if (logWriter == null || logFile == null) {
+      // the logWriter is not initialized now, we skip the check once.
+      return;
+    }
+    if (System.currentTimeMillis() - logFile.lastModified() < mtreeSnapshotThresholdTime) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. Time from last modification: {} ms.",
+            System.currentTimeMillis() - logFile.lastModified());
+      }
+    } else if (logWriter.getLogNum() < mtreeSnapshotInterval) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "MTree snapshot need not be created. New mlog line number: {}.", logWriter.getLogNum());
+      }
+    } else {
+      logger.info(
+          "New mlog line number: {}, time from last modification: {} ms",
+          logWriter.getLogNum(),
+          System.currentTimeMillis() - logFile.lastModified());
+      createMTreeSnapshot();
+    }
+  }
+
+  public void createMTreeSnapshot() {
+    try {
+      mtree.createSnapshot();
+      logWriter.clear();
+    } catch (IOException e) {
+      logger.warn("Failed to create MTree snapshot", e);
+    }
+  }
+
   /** function for clearing MTree */
   public void clear() {
     try {
-      templateMap.clear();
-      this.mtree = new MTree();
+      this.mtree.clear();

Review comment:
       Thanks for reminding.




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