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 11:56:27 UTC

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

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