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 2022/03/11 17:00:28 UTC

[GitHub] [iotdb] MarcosZyk opened a new pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

MarcosZyk opened a new pull request #5205:
URL: https://github.com/apache/iotdb/pull/5205


   ## Description
   
   
   ### Motivation
   1. Prepare for metadata management of new cluster.
   2. Prepare to enhance the performance the coming feature -- persistence schema 
   
   ### Main changes
   1. Set up SGMManager to only manage the metadata of one storage group, including mlog and tagFile
   2. The implementation of MManager interfaces is refactored based on SGMManager
   3. Split MTree to MTreeAboveSG, which is used to manage storage groups, and MTreeBelowSg, which is used to manage devices and measurements of one certain storage group
   4. The template now is persisted by templateLogFile individually
   5. Templates are forbidden to be set above storage group 
   
   None behavior or definition of the interfaces of MManager is changed. 
   
   ### Future work
   1. Rename MManager to SchemaEngine and rename SGMManager to MManager
   2. Implement Upgrade program to  enable user upgrade IoTDB from v0.12.x and v0.13.x
   


-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(nodeNames[i])) {
+        // node b has child sg
+        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+          throw new StorageGroupAlreadySetException(path.getFullPath());
+        } else {
+          throw new StorageGroupAlreadySetException(path.getFullPath(), true);
+        }
+      } else {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        IStorageGroupMNode storageGroupMNode =

Review comment:
       I think the MTree should not mind the creation or initialization of SGMManager, thus I put the related process in ```StorageGroupManager.setStorageGroup(PartialPath path)```




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  private MTreeAboveSG mtree;
+
+  private static class StorageGroupManagerHolder {
+
+    private static final StorageGroupManager INSTANCE = new StorageGroupManager();
+
+    private StorageGroupManagerHolder() {}
+  }
+
+  public static StorageGroupManager getInstance() {
+    return StorageGroupManagerHolder.INSTANCE;
+  }
+
+  private StorageGroupManager() {
+    mtree = new MTreeAboveSG();
+  }
+
+  public synchronized void init() {
+    mtree.init();
+    File dir = new File(IoTDBDescriptor.getInstance().getConfig().getSchemaDir());
+    File[] sgDirs = dir.listFiles((dir1, name) -> name.startsWith(PATH_ROOT + PATH_SEPARATOR));
+    if (sgDirs != null) {
+      for (File sgDir : sgDirs) {
+        try {
+          setStorageGroup(new PartialPath(sgDir.getName()));
+        } catch (MetadataException e) {
+          logger.error("Cannot recover storage group from dir {} because", sgDir.getName(), e);
+        }
+      }
+    }
+  }
+
+  public synchronized void clear() {
+    for (SGMManager sgmManager : getAllSGMManagers()) {
+      sgmManager.clear();
+    }
+    if (mtree != null) {
+      mtree.clear();
+    }
+  }
+
+  @Override
+  public void setStorageGroup(PartialPath path) throws MetadataException {
+    // mtree.setStorageGroup ensures that the storageGroupMNode
+    IStorageGroupMNode storageGroupMNode = mtree.setStorageGroup(path);
+    SGMManager sgmManager = new SGMManager();
+    storageGroupMNode.setSGMManager(sgmManager);
+    sgmManager.init(storageGroupMNode);
+  }
+
+  @Override
+  public SGMManager getBelongedSGMManager(PartialPath path) throws MetadataException {
+    return mtree.getStorageGroupNodeByPath(path).getSGMManager();
+  }
+
+  @Override
+  public SGMManager getSGMManagerByStorageGroupPath(PartialPath path) throws MetadataException {
+    return getStorageGroupNodeByStorageGroupPath(path).getSGMManager();
+  }
+
+  @Override
+  public List<SGMManager> getInvolvedSGMManagers(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    List<SGMManager> result = new ArrayList<>();
+    for (IStorageGroupMNode storageGroupMNode :
+        mtree.getInvolvedStorageGroupNodes(pathPattern, isPrefixMatch)) {
+      result.add(storageGroupMNode.getSGMManager());
+    }
+    return result;
+  }
+
+  @Override
+  public List<SGMManager> getAllSGMManagers() {
+    List<SGMManager> result = new ArrayList<>();
+    for (IStorageGroupMNode storageGroupMNode : mtree.getAllStorageGroupNodes()) {
+      result.add(storageGroupMNode.getSGMManager());
+    }
+    return result;
+  }
+
+  @Override
+  public void deleteStorageGroup(PartialPath storageGroup) throws MetadataException {
+    mtree.deleteStorageGroup(storageGroup);
+  }
+
+  @Override
+  public boolean isStorageGroup(PartialPath path) {
+    return mtree.isStorageGroup(path);
+  }
+
+  @Override
+  public boolean checkStorageGroupByPath(PartialPath path) {
+    return mtree.checkStorageGroupByPath(path);
+  }
+
+  @Override
+  public PartialPath getBelongedStorageGroup(PartialPath path) throws StorageGroupNotSetException {
+    return mtree.getBelongedStorageGroup(path);
+  }
+
+  @Override
+  public List<PartialPath> getBelongedStorageGroups(PartialPath pathPattern)
+      throws MetadataException {
+    return mtree.getBelongedStorageGroups(pathPattern);
+  }
+
+  @Override
+  public List<PartialPath> getMatchedStorageGroups(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    return mtree.getMatchedStorageGroups(pathPattern, isPrefixMatch);
+  }
+
+  @Override
+  public List<PartialPath> getAllStorageGroupPaths() {
+    return mtree.getAllStorageGroupPaths();
+  }
+
+  @Override
+  public Map<String, List<PartialPath>> groupPathByStorageGroup(PartialPath path)
+      throws MetadataException {
+    return mtree.groupPathByStorageGroup(path);
+  }
+
+  @Override
+  public int getStorageGroupNum(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    return mtree.getStorageGroupNum(pathPattern, isPrefixMatch);
+  }
+
+  @Override
+  public IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
+      throws MetadataException {
+    return mtree.getStorageGroupNodeByStorageGroupPath(path);
+  }
+
+  @Override
+  public IStorageGroupMNode getStorageGroupNodeByPath(PartialPath path) throws MetadataException {
+    return mtree.getStorageGroupNodeByPath(path);
+  }
+
+  @Override
+  public List<IStorageGroupMNode> getAllStorageGroupNodes() {
+    return mtree.getAllStorageGroupNodes();
+  }
+
+  @Override
+  public boolean isPathExist(PartialPath path) {
+    return mtree.isPathExist(path);
+  }
+
+  @Override
+  public Pair<Integer, List<SGMManager>> getNodesCountInGivenLevel(
+      PartialPath pathPattern, int level, boolean isPrefixMatch) throws MetadataException {
+    Pair<Integer, Set<IStorageGroupMNode>> resultAboveSG =
+        mtree.getNodesCountInGivenLevel(pathPattern, level, isPrefixMatch);
+    List<SGMManager> sgmManagers = new LinkedList<>();
+    for (IStorageGroupMNode storageGroupMNode : resultAboveSG.right) {
+      sgmManagers.add(storageGroupMNode.getSGMManager());
+    }
+    return new Pair<>(resultAboveSG.left, sgmManagers);
+  }
+
+  @Override
+  public Pair<List<PartialPath>, List<SGMManager>> getNodesListInGivenLevel(
+      PartialPath pathPattern, int nodeLevel, MManager.StorageGroupFilter filter)
+      throws MetadataException {
+    Pair<List<PartialPath>, Set<IStorageGroupMNode>> resultAboveSG =
+        mtree.getNodesListInGivenLevel(pathPattern, nodeLevel, filter);
+    List<SGMManager> sgmManagers = new LinkedList<>();

Review comment:
       Why does the `resultAboveSG.left` is needed ?
   From my understanding, the MTreeAboveSG does not contain the detailed series. 




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -230,67 +178,20 @@ protected MManager() {
         logger.info("create system folder {} failed.", schemaFolder.getAbsolutePath());

Review comment:
       When trigger this condition, IOException will be thrown when init mmanager, since the mlog or tagFile won't be found.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -230,67 +178,20 @@ protected MManager() {
         logger.info("create system folder {} failed.", schemaFolder.getAbsolutePath());

Review comment:
       When trigger this condition, IOException will be thrown when init mmanager, since the folder won't be found.




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -230,67 +178,20 @@ protected MManager() {
         logger.info("create system folder {} failed.", schemaFolder.getAbsolutePath());
       }
     }
-    logFilePath = schemaDir + File.separator + MetadataConstant.METADATA_LOG;
-
-    // do not write log when recover
-    isRecovering = true;
-
-    int cacheSize = config.getmManagerCacheSize();
-    mNodeCache =
-        Caffeine.newBuilder()
-            .maximumSize(cacheSize)
-            .build(
-                new com.github.benmanes.caffeine.cache.CacheLoader<PartialPath, IMNode>() {
-                  @Override
-                  public @Nullable IMNode load(@NonNull PartialPath partialPath)
-                      throws MetadataException {
-
-                    return mtree.getNodeByPathWithStorageGroupCheck(partialPath);
-                  }
-                });
-
-    if (config.isEnableMTreeSnapshot()) {
-      timedCreateMTreeSnapshotThread =
-          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedCreateMTreeSnapshot");
-      timedCreateMTreeSnapshotThread.scheduleAtFixedRate(
-          this::checkMTreeModified,
-          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
-          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
-          TimeUnit.SECONDS);
-    }
-
-    if (config.getSyncMlogPeriodInMs() != 0) {
-      timedForceMLogThread =
-          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedForceMLogThread");
-
-      timedForceMLogThread.scheduleAtFixedRate(
-          this::forceMlog,
-          config.getSyncMlogPeriodInMs(),
-          config.getSyncMlogPeriodInMs(),
-          TimeUnit.MILLISECONDS);
-    }
   }
 
-  // Because the writer will be used later and should not be closed here.
   @SuppressWarnings("squid:S2093")
   public synchronized void init() {
     if (initialized) {
       return;
     }
-    logFile = SystemFileFactory.INSTANCE.getFile(logFilePath);
 
     try {
-      isRecovering = true;
+      timeseriesStatistics.init();
+      templateManager.init();
+      storageGroupManager = new StorageGroupManager();
+      storageGroupManager.init();

Review comment:
       Emm...总感觉这个地方让强迫症看着不太舒服...或许`new StroageGroupManager()`可以放在构造函数里?




-- 
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] xingtanzjr commented on pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   A quick question... Does this version is compatible with metadata files from old version ?


-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(nodeNames[i])) {
+        // node b has child sg
+        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+          throw new StorageGroupAlreadySetException(path.getFullPath());
+        } else {
+          throw new StorageGroupAlreadySetException(path.getFullPath(), true);
+        }
+      } else {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        IStorageGroupMNode storageGroupMNode =

Review comment:
       OK, I miss the following codes...
   
   But on the other hand, maybe the concurrent issue should be handled here ? Because the node may be updated by several threads in parallel.




-- 
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 #5205: [IOTDB-2675][IOTDB-2524] Manage Metadata by Storage Group && Support alias, tag and attributes on aligned timeseries

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


   


-- 
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 #5205: [IOTDB-2675][IOTDB-2524] Manage Metadata by Storage Group && Support alias, tag and attributes on aligned timeseries

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


   
   [![Coverage Status](https://coveralls.io/builds/47421752/badge)](https://coveralls.io/builds/47421752)
   
   Coverage increased (+0.04%) to 65.387% when pulling **621595ee0b0f71fd64c67c3fefebdb11e5246071 on MarcosZyk:schema_engine** into **0426e72c87a5cce9a0fc6a48b33b7531efd49a10 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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -738,23 +439,31 @@ private void ensureStorageGroup(PartialPath path) throws MetadataException {
    */
   public String deleteTimeseries(PartialPath pathPattern, boolean isPrefixMatch)
       throws MetadataException {
-    try {
-      List<MeasurementPath> allTimeseries = mtree.getMeasurementPaths(pathPattern, isPrefixMatch);
-      if (allTimeseries.isEmpty()) {
-        // In the cluster mode, the deletion of a timeseries will be forwarded to all the nodes. For
-        // nodes that do not have the metadata of the timeseries, the coordinator expects a
-        // PathNotExistException.
-        throw new PathNotExistException(pathPattern.getFullPath());
-      }
+    List<SGMManager> sgmManagers =
+        storageGroupManager.getInvolvedSGMManagers(pathPattern, isPrefixMatch);
+    if (sgmManagers.isEmpty()) {
+      // In the cluster mode, the deletion of a timeseries will be forwarded to all the nodes. For
+      // nodes that do not have the metadata of the timeseries, the coordinator expects a
+      // PathNotExistException.
+      throw new PathNotExistException(pathPattern.getFullPath());

Review comment:
       If a exception is threw here, these two scenarios are not distinguished:
    * This node actually doesn't have this series and it is expected.
    * The parameter is not correct for this method and it should return an exception




-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47321201/badge)](https://coveralls.io/builds/47321201)
   
   Coverage decreased (-2.3%) to 65.542% when pulling **1454b29dff0783bb00ee83018662c3e3ff557df8 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47359297/badge)](https://coveralls.io/builds/47359297)
   
   Coverage decreased (-2.3%) to 65.58% when pulling **62221c72ed0d1ea807f0a40e579cc8b48e3cea3a on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/SGMManager.java
##########
@@ -0,0 +1,1765 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
+import org.apache.iotdb.db.exception.metadata.DifferentTemplateException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.NoTemplateOnMNodeException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.TemplateIsInUseException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeBelowSG;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.rescon.TimeseriesStatistics;
+import org.apache.iotdb.db.metadata.tag.TagManager;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.template.TemplateManager;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
+import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.db.utils.TypeInferenceUtils;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.google.gson.JsonObject;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
+
+/**
+ * This class takes the responsibility of serialization of all the metadata info and persistent it
+ * into files. This class contains all the interfaces to modify the metadata for delta system. All
+ * the operations will be insert into the logs temporary in case the downtime of the delta system.
+ *
+ * <p>Since there are too many interfaces and methods in this class, we use code region to help
+ * manage code. The code region starts with //region and ends with //endregion. When using Intellij
+ * Idea to develop, it's easy to fold the code region and see code region overview by collapsing
+ * all.
+ *
+ * <p>The codes are divided into the following code regions:
+ *
+ * <ol>
+ *   <li>Interfaces and Implementation of MManager initialization、snapshot、recover and clear
+ *   <li>Interfaces for CQ
+ *   <li>Interfaces and Implementation for Timeseries operation
+ *   <li>Interfaces and Implementation for StorageGroup and TTL operation
+ *   <li>Interfaces for get and auto create device
+ *   <li>Interfaces for metadata info Query
+ *       <ol>
+ *         <li>Interfaces for metadata count
+ *         <li>Interfaces for level Node info Query
+ *         <li>Interfaces for StorageGroup and TTL info Query
+ *         <li>Interfaces for Entity/Device info Query
+ *         <li>Interfaces for timeseries, measurement and schema info Query
+ *       </ol>
+ *   <li>Interfaces and methods for MNode query
+ *   <li>Interfaces for alias and tag/attribute operations
+ *   <li>Interfaces only for Cluster module usage
+ *   <li>Interfaces for lastCache operations
+ *   <li>Interfaces and Implementation for InsertPlan process
+ *   <li>Interfaces and Implementation for Template operations
+ *   <li>TestOnly Interfaces
+ * </ol>
+ */
+@SuppressWarnings("java:S1135") // ignore todos
+public class SGMManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  /** A thread will check whether the MTree is modified lately each such interval. Unit: second */
+  private static final long MTREE_SNAPSHOT_THREAD_CHECK_TIME = 600L;
+
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private boolean isRecovering;
+  private boolean initialized;
+
+  private final int mtreeSnapshotInterval;
+  private final long mtreeSnapshotThresholdTime;
+  private ScheduledExecutorService timedCreateMTreeSnapshotThread;
+  private ScheduledExecutorService timedForceMLogThread;
+
+  private String sgSchemaDirPath;
+  private String storageGroupFullPath;
+
+  // the log file seriesPath
+  private String logFilePath;
+  private File logFile;
+  private MLogWriter logWriter;
+
+  private TimeseriesStatistics timeseriesStatistics = TimeseriesStatistics.getInstance();
+  private MTreeBelowSG mtree;
+  // device -> DeviceMNode
+  private LoadingCache<PartialPath, IMNode> mNodeCache;
+  private TagManager tagManager;
+
+  // region Interfaces and Implementation of MManager initialization、snapshot、recover and clear
+  public SGMManager() {
+    mtreeSnapshotInterval = config.getMtreeSnapshotInterval();
+    mtreeSnapshotThresholdTime = config.getMtreeSnapshotThresholdTime() * 1000L;
+
+    // do not write log when recover
+    isRecovering = true;
+
+    int cacheSize = config.getmManagerCacheSize();
+    mNodeCache =
+        Caffeine.newBuilder()
+            .maximumSize(cacheSize)
+            .build(
+                new com.github.benmanes.caffeine.cache.CacheLoader<PartialPath, IMNode>() {
+                  @Override
+                  public @Nullable IMNode load(@NonNull PartialPath partialPath)
+                      throws MetadataException {
+
+                    return mtree.getNodeByPath(partialPath);
+                  }
+                });
+
+    if (config.isEnableMTreeSnapshot()) {
+      timedCreateMTreeSnapshotThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedCreateMTreeSnapshot");
+      timedCreateMTreeSnapshotThread.scheduleAtFixedRate(
+          this::checkMTreeModified,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          TimeUnit.SECONDS);
+    }
+
+    if (config.getSyncMlogPeriodInMs() != 0) {
+      timedForceMLogThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedForceMLogThread");
+
+      timedForceMLogThread.scheduleAtFixedRate(
+          this::forceMlog,
+          config.getSyncMlogPeriodInMs(),
+          config.getSyncMlogPeriodInMs(),
+          TimeUnit.MILLISECONDS);
+    }
+  }
+
+  // Because the writer will be used later and should not be closed here.
+  @SuppressWarnings("squid:S2093")
+  public synchronized void init(IStorageGroupMNode storageGroupMNode) {
+    if (initialized) {
+      return;
+    }
+
+    storageGroupFullPath = storageGroupMNode.getFullPath();
+
+    sgSchemaDirPath = config.getSchemaDir() + File.separator + storageGroupFullPath;
+    File sgSchemaFolder = SystemFileFactory.INSTANCE.getFile(sgSchemaDirPath);
+    if (!sgSchemaFolder.exists()) {
+      if (sgSchemaFolder.mkdirs()) {
+        logger.info("create storage group schema folder {}", sgSchemaFolder.getAbsolutePath());
+      } else {
+        logger.info(
+            "create storage group schema folder {} failed.", sgSchemaFolder.getAbsolutePath());
+      }
+    }
+    logFilePath = sgSchemaDirPath + File.separator + MetadataConstant.METADATA_LOG;
+
+    logFile = SystemFileFactory.INSTANCE.getFile(logFilePath);
+
+    try {
+      isRecovering = true;

Review comment:
       Maybe we need to consider combine the `init()` with constructor. `isRecovering = true;` appears both in constructor and `init()`, which is confusing. As general thinking, this line should appear only once in the whole procedure of initialization of SGMManager




-- 
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] xingtanzjr commented on pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   From my understanding, the `MTreeAboveSG` is used to store the tree whose leaf node is storage-group; and the `MTreeBelowSG` is used to represented a MTree whose root is stroage-group. So... we may consider a more suitable name for these two classes because the `Above` and `Below` are not very accurate to describe the usage...


-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(nodeNames[i])) {
+        // node b has child sg
+        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+          throw new StorageGroupAlreadySetException(path.getFullPath());
+        } else {
+          throw new StorageGroupAlreadySetException(path.getFullPath(), true);

Review comment:
       What does `hasChild` mean ?




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(nodeNames[i])) {
+        // node b has child sg
+        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+          throw new StorageGroupAlreadySetException(path.getFullPath());
+        } else {
+          throw new StorageGroupAlreadySetException(path.getFullPath(), true);
+        }
+      } else {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        IStorageGroupMNode storageGroupMNode =
+            new StorageGroupMNode(
+                cur, nodeNames[i], IoTDBDescriptor.getInstance().getConfig().getDefaultTTL());
+        cur.addChild(nodeNames[i], storageGroupMNode);
+        return storageGroupMNode;
+      }
+    }
+  }
+
+  /** Delete a storage group */
+  public void deleteStorageGroup(PartialPath path) throws MetadataException {
+    IStorageGroupMNode cur = getStorageGroupNodeByStorageGroupPath(path);
+    // Suppose current system has root.a.b.sg1, root.a.sg2, and delete root.a.b.sg1
+    // delete the storage group node sg1
+    cur.getParent().deleteChild(cur.getName());
+  }
+
+  /**
+   * Check whether path is storage group or not
+   *
+   * <p>e.g., path = root.a.b.sg. if nor a and b is StorageGroupMNode and sg is a StorageGroupMNode
+   * path is a storage group
+   *
+   * @param path path
+   * @apiNote :for cluster
+   */
+  public boolean isStorageGroup(PartialPath path) {
+    String[] nodeNames = path.getNodes();
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(IoTDBConstant.PATH_ROOT)) {
+      return false;
+    }
+    IMNode cur = root;
+    int i = 1;
+    while (i < nodeNames.length - 1) {
+      cur = cur.getChild(nodeNames[i]);
+      if (cur == null || cur.isStorageGroup()) {
+        return false;
+      }
+      i++;
+    }
+    cur = cur.getChild(nodeNames[i]);
+    return cur != null && cur.isStorageGroup();
+  }
+
+  /** Check whether the given path contains a storage group */
+  public boolean checkStorageGroupByPath(PartialPath path) {
+    String[] nodes = path.getNodes();
+    IMNode cur = root;
+    for (int i = 1; i < nodes.length; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        return false;
+      } else if (cur.isStorageGroup()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Get storage group path by path
+   *
+   * <p>e.g., root.sg1 is storage group, path is root.sg1.d1, return root.sg1
+   *
+   * @return storage group in the given path
+   */
+  public PartialPath getBelongedStorageGroup(PartialPath path) throws StorageGroupNotSetException {
+    String[] nodes = path.getNodes();
+    IMNode cur = root;
+    for (int i = 1; i < nodes.length; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        throw new StorageGroupNotSetException(path.getFullPath());
+      } else if (cur.isStorageGroup()) {
+        return cur.getPartialPath();
+      }
+    }
+    throw new StorageGroupNotSetException(path.getFullPath());
+  }
+
+  /**
+   * Get the storage group that given path pattern matches or belongs to.
+   *
+   * <p>Suppose we have (root.sg1.d1.s1, root.sg2.d2.s2), refer the following cases: 1. given path
+   * "root.sg1", ("root.sg1") will be returned. 2. given path "root.*", ("root.sg1", "root.sg2")
+   * will be returned. 3. given path "root.*.d1.s1", ("root.sg1", "root.sg2") will be returned.
+   *
+   * @param pathPattern a path pattern or a full path
+   * @return a list contains all storage groups related to given path
+   */
+  public List<PartialPath> getBelongedStorageGroups(PartialPath pathPattern)
+      throws MetadataException {
+    return collectStorageGroups(pathPattern, false, true);
+  }
+
+  /**
+   * Get all storage group that the given path pattern matches. If using prefix match, the path
+   * pattern is used to match prefix path. All timeseries start with the matched prefix path will be
+   * collected.
+   *
+   * @param pathPattern a path pattern or a full path
+   * @param isPrefixMatch if true, the path pattern is used to match prefix path
+   * @return a list contains all storage group names under given path pattern
+   */
+  public List<PartialPath> getMatchedStorageGroups(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    return collectStorageGroups(pathPattern, isPrefixMatch, false);
+  }
+
+  private List<PartialPath> collectStorageGroups(
+      PartialPath pathPattern, boolean isPrefixMatch, boolean collectInternal)
+      throws MetadataException {
+    List<PartialPath> result = new LinkedList<>();
+    StorageGroupCollector<List<PartialPath>> collector =
+        new StorageGroupCollector<List<PartialPath>>(root, pathPattern) {
+          @Override
+          protected void collectStorageGroup(IStorageGroupMNode node) {
+            result.add(node.getPartialPath());
+          }
+        };
+    collector.setCollectInternal(collectInternal);
+    collector.setPrefixMatch(isPrefixMatch);
+    collector.traverse();
+    return result;
+  }
+
+  /**
+   * Get all storage group names
+   *
+   * @return a list contains all distinct storage groups
+   */
+  public List<PartialPath> getAllStorageGroupPaths() {
+    List<PartialPath> res = new ArrayList<>();
+    Deque<IMNode> nodeStack = new ArrayDeque<>();
+    nodeStack.add(root);
+    while (!nodeStack.isEmpty()) {
+      IMNode current = nodeStack.pop();
+      if (current.isStorageGroup()) {
+        res.add(current.getPartialPath());
+      } else {
+        nodeStack.addAll(current.getChildren().values());
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Resolve the path or path pattern into StorageGroupName-FullPath pairs. Try determining the
+   * storage group using the children of a mNode. If one child is a storage group node, put a
+   * storageGroupName-fullPath pair into paths.
+   */
+  public Map<String, List<PartialPath>> groupPathByStorageGroup(PartialPath path)
+      throws MetadataException {
+    Map<String, List<PartialPath>> result = new HashMap<>();
+    StorageGroupCollector<Map<String, String>> collector =
+        new StorageGroupCollector<Map<String, String>>(root, path) {
+          @Override
+          protected void collectStorageGroup(IStorageGroupMNode node) {
+            PartialPath sgPath = node.getPartialPath();
+            result.put(sgPath.getFullPath(), path.alterPrefixPath(sgPath));
+          }
+        };
+    collector.setCollectInternal(true);
+    collector.traverse();
+    return result;
+  }
+
+  /**
+   * Get the count of storage group matching the given path. If using prefix match, the path pattern
+   * is used to match prefix path. All timeseries start with the matched prefix path will be
+   * counted.
+   *
+   * @param pathPattern a path pattern or a full path, may contain wildcard.
+   * @param isPrefixMatch if true, the path pattern is used to match prefix path
+   */
+  public int getStorageGroupNum(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    CounterTraverser counter = new StorageGroupCounter(root, pathPattern);
+    counter.setPrefixMatch(isPrefixMatch);
+    counter.traverse();
+    return counter.getCount();
+  }
+
+  /**
+   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg,
+   * device], throw exception Get storage group node, if the give path is not a storage group, throw
+   * exception
+   */
+  public IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
+      throws MetadataException {
+    IStorageGroupMNode node = getStorageGroupNodeByPath(path);
+    if (!node.getPartialPath().equals(path)) {
+      throw new MNodeTypeMismatchException(
+          path.getFullPath(), MetadataConstant.STORAGE_GROUP_MNODE_TYPE);
+    }
+
+    return node;
+  }
+
+  /**
+   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg,
+   * device], return the MNode of root.sg Get storage group node, the give path don't need to be
+   * storage group path.
+   */
+  public IStorageGroupMNode getStorageGroupNodeByPath(PartialPath path) throws MetadataException {
+    String[] nodes = path.getNodes();
+    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    for (int i = 1; i < nodes.length; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        break;
+      }
+      if (cur.isStorageGroup()) {
+        return cur.getAsStorageGroupMNode();
+      }
+    }
+    throw new StorageGroupNotSetException(path.getFullPath());
+  }
+
+  public List<IStorageGroupMNode> getInvolvedStorageGroupNodes(
+      PartialPath pathPattern, boolean isPrefixMatch) throws MetadataException {
+    List<IStorageGroupMNode> result = new ArrayList<>();
+    StorageGroupCollector<List<IStorageGroupMNode>> collector =
+        new StorageGroupCollector<List<IStorageGroupMNode>>(root, pathPattern) {
+          @Override
+          protected void collectStorageGroup(IStorageGroupMNode node) {
+            result.add(node);
+          }
+        };
+    collector.setCollectInternal(true);
+    collector.setPrefixMatch(isPrefixMatch);
+    collector.traverse();
+    return result;
+  }
+
+  /** Get all storage group MNodes */
+  public List<IStorageGroupMNode> getAllStorageGroupNodes() {
+    List<IStorageGroupMNode> ret = new ArrayList<>();
+    Deque<IMNode> nodeStack = new ArrayDeque<>();
+    nodeStack.add(root);
+    while (!nodeStack.isEmpty()) {
+      IMNode current = nodeStack.pop();
+      if (current.isStorageGroup()) {
+        ret.add(current.getAsStorageGroupMNode());
+      } else {
+        nodeStack.addAll(current.getChildren().values());
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Check whether the given path exists.
+   *
+   * @param path a full path or a prefix path
+   */
+  public boolean isPathExist(PartialPath path) {

Review comment:
       Fixed. Already rename this method to ```isStorageGroupAlreadySet``` and comments have been updated.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/IStorageGroupManager.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// This class declares all the interfaces for storage group management.
+public interface IStorageGroupManager {
+
+  void init() throws IOException;
+
+  void clear() throws IOException;
+
+  /**
+   * Set storage group of the given path to MTree.
+   *
+   * @param path storage group path
+   */
+  void setStorageGroup(PartialPath path) throws MetadataException;
+
+  /**
+   * Get the target sgMManager, which the given path belongs to. The path must be a fullPath without
+   * wildcards, * or **. This method is the first step when there's a task on one certain path,
+   * e.g., root.sg1 is a storage group and path = root.sg1.d1, return sgMManager of root.sg1. If
+   * there's no storage group on the given path, StorageGroupNotSetException will be thrown.
+   */
+  SGMManager getBelongedSGMManager(PartialPath path) throws MetadataException;
+
+  /**
+   * Get sgMManager, which the given path represented.
+   *
+   * @param path the path of the target storage group
+   */
+  SGMManager getSGMManagerByStorageGroupPath(PartialPath path) throws MetadataException;
+
+  /**
+   * Get the target sgMManager, which will be involved/covered by the given pathPattern. The path
+   * may contain wildcards, * or **. This method is the first step when there's a task on multiple
+   * paths represented by the given pathPattern. If isPrefixMatch, all storage groups under the
+   * prefixPath that matches the given pathPattern will be collected.
+   */
+  List<SGMManager> getInvolvedSGMManagers(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException;
+
+  List<SGMManager> getAllSGMManagers();
+
+  /**
+   * Delete storage groups of given paths from MTree. Log format: "delete_storage_group,sg1,sg2,sg3"
+   */
+  void deleteStorageGroup(PartialPath storageGroup) throws MetadataException;
+
+  /** Check if the given path is storage group or not. */
+  boolean isStorageGroup(PartialPath path);
+
+  /** Check whether the given path contains a storage group */
+  boolean checkStorageGroupByPath(PartialPath path);
+
+  /**
+   * Get storage group name by path
+   *
+   * <p>e.g., root.sg1 is a storage group and path = root.sg1.d1, return root.sg1
+   *
+   * @param path only full path, cannot be path pattern
+   * @return storage group in the given path
+   */
+  PartialPath getBelongedStorageGroup(PartialPath path) throws StorageGroupNotSetException;
+
+  /**
+   * Get the storage group that given path pattern matches or belongs to.
+   *
+   * <p>Suppose we have (root.sg1.d1.s1, root.sg2.d2.s2), refer the following cases: 1. given path
+   * "root.sg1", ("root.sg1") will be returned. 2. given path "root.*", ("root.sg1", "root.sg2")
+   * will be returned. 3. given path "root.*.d1.s1", ("root.sg1", "root.sg2") will be returned.
+   *
+   * @param pathPattern a path pattern or a full path
+   * @return a list contains all storage groups related to given path pattern
+   */
+  List<PartialPath> getBelongedStorageGroups(PartialPath pathPattern) throws MetadataException;
+
+  /**
+   * Get all storage group matching given path pattern. If using prefix match, the path pattern is
+   * used to match prefix path. All timeseries start with the matched prefix path will be collected.
+   *
+   * @param pathPattern a pattern of a full path
+   * @param isPrefixMatch if true, the path pattern is used to match prefix path
+   * @return A ArrayList instance which stores storage group paths matching given path pattern.
+   */
+  List<PartialPath> getMatchedStorageGroups(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException;
+
+  /** Get all storage group paths */
+  List<PartialPath> getAllStorageGroupPaths();
+
+  /**
+   * For a path, infer all storage groups it may belong to. The path can have wildcards. Resolve the
+   * path or path pattern into StorageGroupName-FullPath pairs that FullPath matches the given path.
+   *
+   * <p>Consider the path into two parts: (1) the sub path which can not contain a storage group
+   * name and (2) the sub path which is substring that begin after the storage group name.
+   *
+   * <p>(1) Suppose the part of the path can not contain a storage group name (e.g.,
+   * "root".contains("root.sg") == false), then: For each one level wildcard *, only one level will
+   * be inferred and the wildcard will be removed. For each multi level wildcard **, then the
+   * inference will go on until the storage groups are found and the wildcard will be kept. (2)
+   * Suppose the part of the path is a substring that begin after the storage group name. (e.g., For
+   * "root.*.sg1.a.*.b.*" and "root.x.sg1" is a storage group, then this part is "a.*.b.*"). For
+   * this part, keep what it is.
+   *
+   * <p>Assuming we have three SGs: root.group1, root.group2, root.area1.group3 Eg1: for input
+   * "root.**", returns ("root.group1", "root.group1.**"), ("root.group2", "root.group2.**")
+   * ("root.area1.group3", "root.area1.group3.**") Eg2: for input "root.*.s1", returns
+   * ("root.group1", "root.group1.s1"), ("root.group2", "root.group2.s1")
+   *
+   * <p>Eg3: for input "root.area1.**", returns ("root.area1.group3", "root.area1.group3.**")
+   *
+   * @param path can be a path pattern or a full path.
+   * @return StorageGroupName-FullPath pairs
+   * @apiNote :for cluster
+   */
+  Map<String, List<PartialPath>> groupPathByStorageGroup(PartialPath path) throws MetadataException;
+
+  /**
+   * To calculate the count of storage group for given path pattern. If using prefix match, the path
+   * pattern is used to match prefix path. All timeseries start with the matched prefix path will be
+   * counted.
+   */
+  int getStorageGroupNum(PartialPath pathPattern, boolean isPrefixMatch) throws MetadataException;
+
+  /**
+   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg],
+   * return the MNode of root.sg Get storage group node by path. Give path like [root, sg, device],
+   * MNodeTypeMismatchException will be thrown. If storage group is not set,
+   * StorageGroupNotSetException will be thrown.
+   */
+  IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
+      throws MetadataException;
+
+  /** Get storage group node by path. the give path don't need to be storage group path. */
+  IStorageGroupMNode getStorageGroupNodeByPath(PartialPath path) throws MetadataException;
+
+  /** Get all storage group MNodes */
+  List<IStorageGroupMNode> getAllStorageGroupNodes();
+
+  /**
+   * Check whether the path exists in MTreeAboveSG. The path may beyond the MTreeAboveSG scope, then
+   * return true if the covered part exists. The rest part will be checked by certain storage group
+   * subTree.
+   *
+   * @param path a full path or a prefix path
+   */
+  boolean isPathExist(PartialPath path);

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] MarcosZyk commented on pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   > A quick question... Does this version is compatible with metadata files from old version ?
   
   Upgrade program has been developed and pushed to this pr. Please take a look at MetadataUpgrader and MetadataUpgradeTest


-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(nodeNames[i])) {
+        // node b has child sg
+        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+          throw new StorageGroupAlreadySetException(path.getFullPath());
+        } else {
+          throw new StorageGroupAlreadySetException(path.getFullPath(), true);

Review comment:
       Got it




-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47325443/badge)](https://coveralls.io/builds/47325443)
   
   Coverage decreased (-2.3%) to 65.541% when pulling **bf2a06afbcf54e82d1aebdb03ddf0f8ff8236de5 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47342082/badge)](https://coveralls.io/builds/47342082)
   
   Coverage decreased (-2.3%) to 65.563% when pulling **8d7feecbeeec0404355af45b739e48048e0cb0ba on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47363515/badge)](https://coveralls.io/builds/47363515)
   
   Coverage decreased (-2.3%) to 65.583% when pulling **4880db021d25c21b678f1435718ab4f1b7c889b5 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47361427/badge)](https://coveralls.io/builds/47361427)
   
   Coverage decreased (-2.3%) to 65.589% when pulling **4880db021d25c21b678f1435718ab4f1b7c889b5 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(nodeNames[i])) {
+        // node b has child sg
+        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+          throw new StorageGroupAlreadySetException(path.getFullPath());
+        } else {
+          throw new StorageGroupAlreadySetException(path.getFullPath(), true);

Review comment:
       If there's already a storage group, named root.a.sg, and users try to set storage group to root.a, then a StorageGroupAlreadySetException with hasChild=true will be thrown. This will forbid MManager from retrying setStorageGroup.




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));

Review comment:
       ```addChild``` invoke concurrentHashMap's ```putIfAbsent```, which ensures that there won't be parent replacement. After line 103, the added child will be re-get. The replacement of existing node only happens when a measurmentMNode is going to be added. If there's no nested device, which means all children of a device is measurement, there won't be subtree loss.




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/SGMManager.java
##########
@@ -0,0 +1,1762 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
+import org.apache.iotdb.db.exception.metadata.DifferentTemplateException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.NoTemplateOnMNodeException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.TemplateIsInUseException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeBelowSG;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.rescon.TimeseriesStatistics;
+import org.apache.iotdb.db.metadata.tag.TagManager;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.template.TemplateManager;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
+import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.db.utils.TypeInferenceUtils;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.google.gson.JsonObject;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
+
+/**
+ * This class takes the responsibility of serialization of all the metadata info of one certain
+ * storage group and persistent it into files. This class contains the interfaces to modify the
+ * metadata in storage group for delta system. All the operations will be inserted into the logs
+ * temporary in case the downtime of the delta system.
+ *
+ * <p>Since there are too many interfaces and methods in this class, we use code region to help
+ * manage code. The code region starts with //region and ends with //endregion. When using Intellij
+ * Idea to develop, it's easy to fold the code region and see code region overview by collapsing
+ * all.
+ *
+ * <p>The codes are divided into the following code regions:
+ *
+ * <ol>
+ *   <li>Interfaces and Implementation of initialization、snapshot、recover and clear
+ *   <li>Interfaces and Implementation for Timeseries operation
+ *   <li>Interfaces and Implementation for StorageGroup and TTL operation
+ *   <li>Interfaces for get and auto create device
+ *   <li>Interfaces for metadata info Query
+ *       <ol>
+ *         <li>Interfaces for metadata count
+ *         <li>Interfaces for level Node info Query
+ *         <li>Interfaces for StorageGroup and TTL info Query
+ *         <li>Interfaces for Entity/Device info Query
+ *         <li>Interfaces for timeseries, measurement and schema info Query
+ *       </ol>
+ *   <li>Interfaces and methods for MNode query
+ *   <li>Interfaces for alias and tag/attribute operations
+ *   <li>Interfaces only for Cluster module usage
+ *   <li>Interfaces and Implementation for InsertPlan process
+ *   <li>Interfaces and Implementation for Template operations
+ *   <li>TestOnly Interfaces
+ * </ol>
+ */
+@SuppressWarnings("java:S1135") // ignore todos
+public class SGMManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  /** A thread will check whether the MTree is modified lately each such interval. Unit: second */
+  private static final long MTREE_SNAPSHOT_THREAD_CHECK_TIME = 600L;
+
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private boolean isRecovering = true;
+  private boolean initialized = false;
+
+  private final int mtreeSnapshotInterval;
+  private final long mtreeSnapshotThresholdTime;
+  private ScheduledExecutorService timedCreateMTreeSnapshotThread;
+  private ScheduledExecutorService timedForceMLogThread;
+
+  private String sgSchemaDirPath;
+  private String storageGroupFullPath;
+
+  // the log file seriesPath
+  private String logFilePath;
+  private File logFile;
+  private MLogWriter logWriter;
+
+  private TimeseriesStatistics timeseriesStatistics = TimeseriesStatistics.getInstance();
+  private MTreeBelowSG mtree;
+  // device -> DeviceMNode
+  private LoadingCache<PartialPath, IMNode> mNodeCache;
+  private TagManager tagManager;
+
+  // region Interfaces and Implementation of initialization、snapshot、recover and clear
+  public SGMManager() {
+    mtreeSnapshotInterval = config.getMtreeSnapshotInterval();
+    mtreeSnapshotThresholdTime = config.getMtreeSnapshotThresholdTime() * 1000L;
+
+    int cacheSize = config.getmManagerCacheSize();
+    mNodeCache =
+        Caffeine.newBuilder()
+            .maximumSize(cacheSize)
+            .build(
+                new com.github.benmanes.caffeine.cache.CacheLoader<PartialPath, IMNode>() {
+                  @Override
+                  public @Nullable IMNode load(@NonNull PartialPath partialPath)
+                      throws MetadataException {
+
+                    return mtree.getNodeByPath(partialPath);
+                  }
+                });
+
+    if (config.isEnableMTreeSnapshot()) {
+      timedCreateMTreeSnapshotThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedCreateMTreeSnapshot");
+      timedCreateMTreeSnapshotThread.scheduleAtFixedRate(
+          this::checkMTreeModified,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          TimeUnit.SECONDS);
+    }
+
+    if (config.getSyncMlogPeriodInMs() != 0) {
+      timedForceMLogThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedForceMLogThread");
+
+      timedForceMLogThread.scheduleAtFixedRate(
+          this::forceMlog,
+          config.getSyncMlogPeriodInMs(),
+          config.getSyncMlogPeriodInMs(),
+          TimeUnit.MILLISECONDS);
+    }
+  }
+
+  // Because the writer will be used later and should not be closed here.
+  @SuppressWarnings("squid:S2093")
+  public synchronized void init(IStorageGroupMNode storageGroupMNode) {
+    if (initialized) {
+      return;
+    }
+
+    storageGroupFullPath = storageGroupMNode.getFullPath();
+
+    sgSchemaDirPath = config.getSchemaDir() + File.separator + storageGroupFullPath;
+    File sgSchemaFolder = SystemFileFactory.INSTANCE.getFile(sgSchemaDirPath);
+    if (!sgSchemaFolder.exists()) {
+      if (sgSchemaFolder.mkdirs()) {
+        logger.info("create storage group schema folder {}", sgSchemaFolder.getAbsolutePath());
+      } else {
+        logger.info(
+            "create storage group schema folder {} failed.", sgSchemaFolder.getAbsolutePath());
+      }
+    }
+    logFilePath = sgSchemaDirPath + File.separator + MetadataConstant.METADATA_LOG;
+
+    logFile = SystemFileFactory.INSTANCE.getFile(logFilePath);
+
+    try {
+      // do not write log when recover
+      isRecovering = true;
+
+      tagManager = new TagManager(sgSchemaDirPath);
+      tagManager.init();
+      mtree = new MTreeBelowSG(storageGroupMNode);
+      mtree.init();

Review comment:
       Maybe we can put the `init()` inside the constructor of MTreeBelowSG ?




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  private MTreeAboveSG mtree;
+
+  public StorageGroupManager() {
+    mtree = new MTreeAboveSG();
+  }
+
+  public synchronized void init() {
+    mtree.init();
+    File dir = new File(IoTDBDescriptor.getInstance().getConfig().getSchemaDir());
+    File[] sgDirs = dir.listFiles((dir1, name) -> name.startsWith(PATH_ROOT + PATH_SEPARATOR));
+    if (sgDirs != null) {
+      for (File sgDir : sgDirs) {
+        try {
+          setStorageGroup(new PartialPath(sgDir.getName()));
+        } catch (MetadataException e) {
+          logger.error("Cannot recover storage group from dir {} because", sgDir.getName(), e);
+        }
+      }
+    }
+  }
+
+  public synchronized void clear() {
+    for (SGMManager sgmManager : getAllSGMManagers()) {
+      sgmManager.clear();
+    }
+    if (mtree != null) {
+      mtree.clear();
+    }
+  }
+
+  @Override
+  public void setStorageGroup(PartialPath path) throws MetadataException {
+    IStorageGroupMNode storageGroupMNode = mtree.setStorageGroup(path);
+    SGMManager sgmManager = new SGMManager();
+    storageGroupMNode.setSGMManager(sgmManager);

Review comment:
       Fixed. Now sgmmanager won't take a storageGroupMNode as an attribute and the setStorageGroupMNode interface is deleted. The init() method should take storageGroupMNode as input to initialize MTreeBelowSG.




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -738,23 +439,31 @@ private void ensureStorageGroup(PartialPath path) throws MetadataException {
    */
   public String deleteTimeseries(PartialPath pathPattern, boolean isPrefixMatch)
       throws MetadataException {
-    try {
-      List<MeasurementPath> allTimeseries = mtree.getMeasurementPaths(pathPattern, isPrefixMatch);
-      if (allTimeseries.isEmpty()) {
-        // In the cluster mode, the deletion of a timeseries will be forwarded to all the nodes. For
-        // nodes that do not have the metadata of the timeseries, the coordinator expects a
-        // PathNotExistException.
-        throw new PathNotExistException(pathPattern.getFullPath());
-      }
+    List<SGMManager> sgmManagers =
+        storageGroupManager.getInvolvedSGMManagers(pathPattern, isPrefixMatch);
+    if (sgmManagers.isEmpty()) {
+      // In the cluster mode, the deletion of a timeseries will be forwarded to all the nodes. For
+      // nodes that do not have the metadata of the timeseries, the coordinator expects a
+      // PathNotExistException.
+      throw new PathNotExistException(pathPattern.getFullPath());

Review comment:
       This is the implementation of old cluster. It's just kept to help pass IT/UT of old cluster. This will be removed in future development of new cluster.




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -18,52 +18,37 @@
  */

Review comment:
       Just a question here, `ensureStorageGroup` use the default level as the storage group level. If the user already set some other storage group whose level is not equal to the default level, what will happen ?




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java
##########
@@ -177,14 +169,28 @@ public void init() throws IOException {
     File mtreeSnapshot = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotPath);
     long time = System.currentTimeMillis();
     if (mtreeSnapshot.exists()) {
-      this.root = deserializeFrom(mtreeSnapshot).root;
-      logger.debug(
-          "spend {} ms to deserialize mtree from snapshot", System.currentTimeMillis() - time);
+      IStorageGroupMNode recoveredTree =
+          deserializeFrom(mtreeSnapshot, storageGroupMNode.getFullPath());
+      if (recoveredTree != null) {
+        this.storageGroupMNode.getParent().replaceChild(storageGroupMNode.getName(), recoveredTree);

Review comment:
       From this line, I cannot infer that some attributes inside recoveredTree has been modified. But it actually did. 
   
   Suggest to use more clear way to implement the changes here...




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

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

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



[GitHub] [iotdb] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java
##########
@@ -536,7 +506,7 @@ public void createAlignedTimeseries(
   public Pair<PartialPath, IMeasurementMNode> deleteTimeseriesAndReturnEmptyStorageGroup(

Review comment:
       Does this method need concurrent control ?




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/SGMManager.java
##########
@@ -0,0 +1,1765 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
+import org.apache.iotdb.db.exception.metadata.DifferentTemplateException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.NoTemplateOnMNodeException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.TemplateIsInUseException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeBelowSG;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.rescon.TimeseriesStatistics;
+import org.apache.iotdb.db.metadata.tag.TagManager;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.template.TemplateManager;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
+import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.db.utils.TypeInferenceUtils;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.google.gson.JsonObject;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
+
+/**
+ * This class takes the responsibility of serialization of all the metadata info and persistent it
+ * into files. This class contains all the interfaces to modify the metadata for delta system. All
+ * the operations will be insert into the logs temporary in case the downtime of the delta system.

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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47359295/badge)](https://coveralls.io/builds/47359295)
   
   Coverage decreased (-2.3%) to 65.601% when pulling **62221c72ed0d1ea807f0a40e579cc8b48e3cea3a on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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 #5205: [IOTDB-2675][IOTDB-2524] Manage Metadata by Storage Group && Support alias, tag and attributes on aligned timeseries

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


   
   [![Coverage Status](https://coveralls.io/builds/47427772/badge)](https://coveralls.io/builds/47427772)
   
   Coverage increased (+0.009%) to 65.353% when pulling **eac5013bcf3e0afccd25bccff67ace8fc781a47a on MarcosZyk:schema_engine** into **d29ef70755bfbb1993e644f4c45b88ef71c0d2c8 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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  private MTreeAboveSG mtree;
+
+  public StorageGroupManager() {
+    mtree = new MTreeAboveSG();
+  }
+
+  public synchronized void init() {
+    mtree.init();
+    File dir = new File(IoTDBDescriptor.getInstance().getConfig().getSchemaDir());
+    File[] sgDirs = dir.listFiles((dir1, name) -> name.startsWith(PATH_ROOT + PATH_SEPARATOR));
+    if (sgDirs != null) {
+      for (File sgDir : sgDirs) {
+        try {
+          setStorageGroup(new PartialPath(sgDir.getName()));
+        } catch (MetadataException e) {
+          logger.error("Cannot recover storage group from dir {} because", sgDir.getName(), e);
+        }
+      }
+    }
+  }
+
+  public synchronized void clear() {
+    for (SGMManager sgmManager : getAllSGMManagers()) {
+      sgmManager.clear();
+    }
+    if (mtree != null) {
+      mtree.clear();
+    }
+  }
+
+  @Override
+  public void setStorageGroup(PartialPath path) throws MetadataException {

Review comment:
       This method is not thread-safe. The lines 82-84 may lead to unexpected result if this method is invoked in parallel.




-- 
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] xingtanzjr edited a comment on pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   From my understanding, the `MTreeAboveSG` is used to store the tree whose leaf node is storage-group; and the `MTreeBelowSG` is used to represented a MTree whose root is stroage-group. So... we may consider a more suitable name for these two classes because the these two words `Above` and `Below` are not very accurate to describe the usage...


-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java
##########
@@ -536,7 +506,7 @@ public void createAlignedTimeseries(
   public Pair<PartialPath, IMeasurementMNode> deleteTimeseriesAndReturnEmptyStorageGroup(

Review comment:
       Fixed. I've synchronized all deletion methods. The ```deleteStorageGroup``` is synchronized in ```StorageGroupSchemaManager``` and ```deleteTimeseries``` is synchronized in belonged SGMManager.




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));

Review comment:
       ConcurrentHashMap helps avoid concurrent issue.




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -547,70 +339,13 @@ public void createTimeseries(CreateTimeSeriesPlan plan) throws MetadataException
 
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public void createTimeseries(CreateTimeSeriesPlan plan, long offset) throws MetadataException {
-    if (!allowToCreateNewSeries) {
+    if (!timeseriesStatistics.isAllowToCreateNewSeries()) {
       throw new MetadataException(
           "IoTDB system load is too large to create timeseries, "
               + "please increase MAX_HEAP_SIZE in iotdb-env.sh/bat and restart");
     }
-    try {
-      PartialPath path = plan.getPath();
-      SchemaUtils.checkDataTypeWithEncoding(plan.getDataType(), plan.getEncoding());
-
-      ensureStorageGroup(path);
-
-      TSDataType type = plan.getDataType();
-      // create time series in MTree
-      IMeasurementMNode leafMNode =
-          mtree.createTimeseries(
-              path,
-              type,
-              plan.getEncoding(),
-              plan.getCompressor(),
-              plan.getProps(),
-              plan.getAlias());
-
-      // the cached mNode may be replaced by new entityMNode in mtree
-      mNodeCache.invalidate(path.getDevicePath());
-
-      // update tag index
-
-      if (offset != -1 && isRecovering) {
-        // the timeseries has already been created and now system is recovering, using the tag info
-        // in tagFile to recover index directly
-        tagManager.recoverIndex(offset, leafMNode);
-      } else if (plan.getTags() != null) {
-        // tag key, tag value
-        tagManager.addIndex(plan.getTags(), leafMNode);
-      }
-
-      // update statistics and schemaDataTypeNumMap
-      totalSeriesNumber.addAndGet(1);
-      if (totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE >= MTREE_SIZE_THRESHOLD) {
-        logger.warn("Current series number {} is too large...", totalSeriesNumber);
-        allowToCreateNewSeries = false;
-      }
-
-      // write log
-      if (!isRecovering) {
-        // either tags or attributes is not empty
-        if ((plan.getTags() != null && !plan.getTags().isEmpty())
-            || (plan.getAttributes() != null && !plan.getAttributes().isEmpty())) {
-          offset = tagManager.writeTagFile(plan.getTags(), plan.getAttributes());
-        }
-        plan.setTagOffset(offset);
-        logWriter.createTimeseries(plan);
-      }
-      leafMNode.setOffset(offset);
-
-    } catch (IOException e) {
-      throw new MetadataException(e);
-    }
-
-    // update id table if not in recovering or disable id table log file
-    if (config.isEnableIDTable() && (!isRecovering || !config.isEnableIDTableLogFile())) {
-      IDTable idTable = IDTableManager.getInstance().getIDTable(plan.getPath().getDevicePath());
-      idTable.createTimeseries(plan);
-    }
+    ensureStorageGroup(plan.getPath());
+    storageGroupManager.getBelongedSGMManager(plan.getPath()).createTimeseries(plan, offset);

Review comment:
       Exception will be thrown in ensureStorageGroup, and the following code won't be executed.




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  private MTreeAboveSG mtree;
+
+  private static class StorageGroupManagerHolder {
+
+    private static final StorageGroupManager INSTANCE = new StorageGroupManager();
+
+    private StorageGroupManagerHolder() {}
+  }
+
+  public static StorageGroupManager getInstance() {
+    return StorageGroupManagerHolder.INSTANCE;
+  }
+
+  private StorageGroupManager() {
+    mtree = new MTreeAboveSG();
+  }
+
+  public synchronized void init() {
+    mtree.init();
+    File dir = new File(IoTDBDescriptor.getInstance().getConfig().getSchemaDir());
+    File[] sgDirs = dir.listFiles((dir1, name) -> name.startsWith(PATH_ROOT + PATH_SEPARATOR));
+    if (sgDirs != null) {
+      for (File sgDir : sgDirs) {
+        try {
+          setStorageGroup(new PartialPath(sgDir.getName()));
+        } catch (MetadataException e) {
+          logger.error("Cannot recover storage group from dir {} because", sgDir.getName(), e);
+        }
+      }
+    }
+  }
+
+  public synchronized void clear() {
+    for (SGMManager sgmManager : getAllSGMManagers()) {
+      sgmManager.clear();
+    }
+    if (mtree != null) {
+      mtree.clear();
+    }
+  }
+
+  @Override
+  public void setStorageGroup(PartialPath path) throws MetadataException {
+    // mtree.setStorageGroup ensures that the storageGroupMNode
+    IStorageGroupMNode storageGroupMNode = mtree.setStorageGroup(path);
+    SGMManager sgmManager = new SGMManager();
+    storageGroupMNode.setSGMManager(sgmManager);
+    sgmManager.init(storageGroupMNode);
+  }
+
+  @Override
+  public SGMManager getBelongedSGMManager(PartialPath path) throws MetadataException {
+    return mtree.getStorageGroupNodeByPath(path).getSGMManager();
+  }
+
+  @Override
+  public SGMManager getSGMManagerByStorageGroupPath(PartialPath path) throws MetadataException {
+    return getStorageGroupNodeByStorageGroupPath(path).getSGMManager();
+  }
+
+  @Override
+  public List<SGMManager> getInvolvedSGMManagers(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    List<SGMManager> result = new ArrayList<>();
+    for (IStorageGroupMNode storageGroupMNode :
+        mtree.getInvolvedStorageGroupNodes(pathPattern, isPrefixMatch)) {
+      result.add(storageGroupMNode.getSGMManager());
+    }
+    return result;
+  }
+
+  @Override
+  public List<SGMManager> getAllSGMManagers() {
+    List<SGMManager> result = new ArrayList<>();
+    for (IStorageGroupMNode storageGroupMNode : mtree.getAllStorageGroupNodes()) {
+      result.add(storageGroupMNode.getSGMManager());
+    }
+    return result;
+  }
+
+  @Override
+  public void deleteStorageGroup(PartialPath storageGroup) throws MetadataException {
+    mtree.deleteStorageGroup(storageGroup);
+  }
+
+  @Override
+  public boolean isStorageGroup(PartialPath path) {
+    return mtree.isStorageGroup(path);
+  }
+
+  @Override
+  public boolean checkStorageGroupByPath(PartialPath path) {
+    return mtree.checkStorageGroupByPath(path);
+  }
+
+  @Override
+  public PartialPath getBelongedStorageGroup(PartialPath path) throws StorageGroupNotSetException {
+    return mtree.getBelongedStorageGroup(path);
+  }
+
+  @Override
+  public List<PartialPath> getBelongedStorageGroups(PartialPath pathPattern)
+      throws MetadataException {
+    return mtree.getBelongedStorageGroups(pathPattern);
+  }
+
+  @Override
+  public List<PartialPath> getMatchedStorageGroups(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    return mtree.getMatchedStorageGroups(pathPattern, isPrefixMatch);
+  }
+
+  @Override
+  public List<PartialPath> getAllStorageGroupPaths() {
+    return mtree.getAllStorageGroupPaths();
+  }
+
+  @Override
+  public Map<String, List<PartialPath>> groupPathByStorageGroup(PartialPath path)
+      throws MetadataException {
+    return mtree.groupPathByStorageGroup(path);
+  }
+
+  @Override
+  public int getStorageGroupNum(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    return mtree.getStorageGroupNum(pathPattern, isPrefixMatch);
+  }
+
+  @Override
+  public IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
+      throws MetadataException {
+    return mtree.getStorageGroupNodeByStorageGroupPath(path);
+  }
+
+  @Override
+  public IStorageGroupMNode getStorageGroupNodeByPath(PartialPath path) throws MetadataException {
+    return mtree.getStorageGroupNodeByPath(path);
+  }
+
+  @Override
+  public List<IStorageGroupMNode> getAllStorageGroupNodes() {
+    return mtree.getAllStorageGroupNodes();
+  }
+
+  @Override
+  public boolean isPathExist(PartialPath path) {
+    return mtree.isPathExist(path);
+  }
+
+  @Override
+  public Pair<Integer, List<SGMManager>> getNodesCountInGivenLevel(
+      PartialPath pathPattern, int level, boolean isPrefixMatch) throws MetadataException {
+    Pair<Integer, Set<IStorageGroupMNode>> resultAboveSG =
+        mtree.getNodesCountInGivenLevel(pathPattern, level, isPrefixMatch);
+    List<SGMManager> sgmManagers = new LinkedList<>();
+    for (IStorageGroupMNode storageGroupMNode : resultAboveSG.right) {
+      sgmManagers.add(storageGroupMNode.getSGMManager());
+    }
+    return new Pair<>(resultAboveSG.left, sgmManagers);
+  }
+
+  @Override
+  public Pair<List<PartialPath>, List<SGMManager>> getNodesListInGivenLevel(
+      PartialPath pathPattern, int nodeLevel, MManager.StorageGroupFilter filter)
+      throws MetadataException {
+    Pair<List<PartialPath>, Set<IStorageGroupMNode>> resultAboveSG =
+        mtree.getNodesListInGivenLevel(pathPattern, nodeLevel, filter);
+    List<SGMManager> sgmManagers = new LinkedList<>();

Review comment:
       This feature manages the mnode in MTree, including measurementMNodes, thus there may be matched results in MTreeAboveSG.




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(nodeNames[i])) {
+        // node b has child sg
+        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+          throw new StorageGroupAlreadySetException(path.getFullPath());
+        } else {
+          throw new StorageGroupAlreadySetException(path.getFullPath(), true);
+        }
+      } else {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        IStorageGroupMNode storageGroupMNode =

Review comment:
       Where is the `SGMManager` set to the newly created `StorageGroupMNode` ?




-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47301723/badge)](https://coveralls.io/builds/47301723)
   
   Coverage increased (+0.01%) to 67.875% when pulling **438048cb897fd11ccf958df0c9d08c34fe770b86 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java
##########
@@ -177,14 +169,28 @@ public void init() throws IOException {
     File mtreeSnapshot = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotPath);
     long time = System.currentTimeMillis();
     if (mtreeSnapshot.exists()) {
-      this.root = deserializeFrom(mtreeSnapshot).root;
-      logger.debug(
-          "spend {} ms to deserialize mtree from snapshot", System.currentTimeMillis() - time);
+      IStorageGroupMNode recoveredTree =
+          deserializeFrom(mtreeSnapshot, storageGroupMNode.getFullPath());
+      if (recoveredTree != null) {
+        this.storageGroupMNode.getParent().replaceChild(storageGroupMNode.getName(), recoveredTree);
+        this.storageGroupMNode = recoveredTree;
+        logger.debug(
+            "spend {} ms to deserialize mtree from snapshot", System.currentTimeMillis() - time);
+      }
     }
   }
 
+  public IStorageGroupMNode getStorageGroupMNode() {
+    return this.storageGroupMNode;
+  }
+
+  public void setStorageGroupMNode(IStorageGroupMNode storageGroupMNode) {
+    this.storageGroupMNode = storageGroupMNode;
+    levelOfSG = storageGroupMNode.getPartialPath().getNodeLength() - 1;
+  }
+
   public void clear() {
-    root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+    storageGroupMNode = null;
   }
 

Review comment:
       Maybe we can add the SGM info in the log of createSnapshot()...




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -169,6 +169,12 @@ public synchronized void replaceChild(String oldChildName, IMNode newChildNode)
       newChildNode.getAsEntityMNode().setUseTemplate(oldChildNode.isUseTemplate());
     }
 
+    if (newChildNode.isStorageGroup() && oldChildNode.isStorageGroup()) {

Review comment:
       It is better to make these logic as part of the implementation of subclass.
   
   Otherwise, we need to add more `if-else` here if we add new type of IMNode, which is easy to be missed...




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/SGMManager.java
##########
@@ -0,0 +1,1762 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
+import org.apache.iotdb.db.exception.metadata.DifferentTemplateException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.NoTemplateOnMNodeException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.TemplateIsInUseException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeBelowSG;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.rescon.TimeseriesStatistics;
+import org.apache.iotdb.db.metadata.tag.TagManager;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.template.TemplateManager;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
+import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.db.utils.TypeInferenceUtils;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.google.gson.JsonObject;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
+
+/**
+ * This class takes the responsibility of serialization of all the metadata info of one certain
+ * storage group and persistent it into files. This class contains the interfaces to modify the
+ * metadata in storage group for delta system. All the operations will be inserted into the logs
+ * temporary in case the downtime of the delta system.
+ *
+ * <p>Since there are too many interfaces and methods in this class, we use code region to help
+ * manage code. The code region starts with //region and ends with //endregion. When using Intellij
+ * Idea to develop, it's easy to fold the code region and see code region overview by collapsing
+ * all.
+ *
+ * <p>The codes are divided into the following code regions:
+ *
+ * <ol>
+ *   <li>Interfaces and Implementation of initialization、snapshot、recover and clear
+ *   <li>Interfaces and Implementation for Timeseries operation
+ *   <li>Interfaces and Implementation for StorageGroup and TTL operation
+ *   <li>Interfaces for get and auto create device
+ *   <li>Interfaces for metadata info Query
+ *       <ol>
+ *         <li>Interfaces for metadata count
+ *         <li>Interfaces for level Node info Query
+ *         <li>Interfaces for StorageGroup and TTL info Query
+ *         <li>Interfaces for Entity/Device info Query
+ *         <li>Interfaces for timeseries, measurement and schema info Query
+ *       </ol>
+ *   <li>Interfaces and methods for MNode query
+ *   <li>Interfaces for alias and tag/attribute operations
+ *   <li>Interfaces only for Cluster module usage
+ *   <li>Interfaces and Implementation for InsertPlan process
+ *   <li>Interfaces and Implementation for Template operations
+ *   <li>TestOnly Interfaces
+ * </ol>
+ */
+@SuppressWarnings("java:S1135") // ignore todos
+public class SGMManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  /** A thread will check whether the MTree is modified lately each such interval. Unit: second */
+  private static final long MTREE_SNAPSHOT_THREAD_CHECK_TIME = 600L;
+
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private boolean isRecovering = true;
+  private boolean initialized = false;
+
+  private final int mtreeSnapshotInterval;
+  private final long mtreeSnapshotThresholdTime;
+  private ScheduledExecutorService timedCreateMTreeSnapshotThread;
+  private ScheduledExecutorService timedForceMLogThread;
+
+  private String sgSchemaDirPath;
+  private String storageGroupFullPath;
+
+  // the log file seriesPath
+  private String logFilePath;
+  private File logFile;
+  private MLogWriter logWriter;
+
+  private TimeseriesStatistics timeseriesStatistics = TimeseriesStatistics.getInstance();
+  private MTreeBelowSG mtree;
+  // device -> DeviceMNode
+  private LoadingCache<PartialPath, IMNode> mNodeCache;
+  private TagManager tagManager;
+
+  // region Interfaces and Implementation of initialization、snapshot、recover and clear
+  public SGMManager() {
+    mtreeSnapshotInterval = config.getMtreeSnapshotInterval();
+    mtreeSnapshotThresholdTime = config.getMtreeSnapshotThresholdTime() * 1000L;
+
+    int cacheSize = config.getmManagerCacheSize();
+    mNodeCache =
+        Caffeine.newBuilder()
+            .maximumSize(cacheSize)
+            .build(
+                new com.github.benmanes.caffeine.cache.CacheLoader<PartialPath, IMNode>() {
+                  @Override
+                  public @Nullable IMNode load(@NonNull PartialPath partialPath)
+                      throws MetadataException {
+
+                    return mtree.getNodeByPath(partialPath);
+                  }
+                });
+
+    if (config.isEnableMTreeSnapshot()) {
+      timedCreateMTreeSnapshotThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedCreateMTreeSnapshot");
+      timedCreateMTreeSnapshotThread.scheduleAtFixedRate(
+          this::checkMTreeModified,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          TimeUnit.SECONDS);
+    }
+
+    if (config.getSyncMlogPeriodInMs() != 0) {
+      timedForceMLogThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedForceMLogThread");
+
+      timedForceMLogThread.scheduleAtFixedRate(
+          this::forceMlog,
+          config.getSyncMlogPeriodInMs(),
+          config.getSyncMlogPeriodInMs(),
+          TimeUnit.MILLISECONDS);
+    }
+  }
+
+  // Because the writer will be used later and should not be closed here.
+  @SuppressWarnings("squid:S2093")
+  public synchronized void init(IStorageGroupMNode storageGroupMNode) {
+    if (initialized) {
+      return;
+    }
+
+    storageGroupFullPath = storageGroupMNode.getFullPath();
+
+    sgSchemaDirPath = config.getSchemaDir() + File.separator + storageGroupFullPath;
+    File sgSchemaFolder = SystemFileFactory.INSTANCE.getFile(sgSchemaDirPath);
+    if (!sgSchemaFolder.exists()) {
+      if (sgSchemaFolder.mkdirs()) {
+        logger.info("create storage group schema folder {}", sgSchemaFolder.getAbsolutePath());
+      } else {
+        logger.info(
+            "create storage group schema folder {} failed.", sgSchemaFolder.getAbsolutePath());

Review comment:
       Fixed. Since SGMManager may be initialized when setStorageGroup, thus the process should be interrupted and now an exception will be thrown.




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));

Review comment:
       Concurrent issue.
   
   If several threads execute this line in parallel with same noneName, there will be some concurrent issues here because the tree is modified by multi threads at the same time




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/IStorageGroupManager.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// This class declares all the interfaces for storage group management.
+public interface IStorageGroupManager {
+
+  void init() throws IOException;
+
+  void clear() throws IOException;
+
+  /**
+   * Set storage group of the given path to MTree.
+   *
+   * @param path storage group path
+   */
+  void setStorageGroup(PartialPath path) throws MetadataException;
+
+  /**
+   * Get the target sgMManager, which the given path belongs to. The path must be a fullPath without
+   * wildcards, * or **. This method is the first step when there's a task on one certain path,
+   * e.g., root.sg1 is a storage group and path = root.sg1.d1, return sgMManager of root.sg1. If
+   * there's no storage group on the given path, StorageGroupNotSetException will be thrown.
+   */
+  SGMManager getBelongedSGMManager(PartialPath path) throws MetadataException;
+
+  /**
+   * Get sgMManager, which the given path represented.
+   *
+   * @param path the path of the target storage group
+   */
+  SGMManager getSGMManagerByStorageGroupPath(PartialPath path) throws MetadataException;
+
+  /**
+   * Get the target sgMManager, which will be involved/covered by the given pathPattern. The path
+   * may contain wildcards, * or **. This method is the first step when there's a task on multiple
+   * paths represented by the given pathPattern. If isPrefixMatch, all storage groups under the
+   * prefixPath that matches the given pathPattern will be collected.
+   */
+  List<SGMManager> getInvolvedSGMManagers(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException;
+
+  List<SGMManager> getAllSGMManagers();
+
+  /**
+   * Delete storage groups of given paths from MTree. Log format: "delete_storage_group,sg1,sg2,sg3"
+   */
+  void deleteStorageGroup(PartialPath storageGroup) throws MetadataException;
+
+  /** Check if the given path is storage group or not. */
+  boolean isStorageGroup(PartialPath path);
+
+  /** Check whether the given path contains a storage group */
+  boolean checkStorageGroupByPath(PartialPath path);
+
+  /**
+   * Get storage group name by path
+   *
+   * <p>e.g., root.sg1 is a storage group and path = root.sg1.d1, return root.sg1
+   *
+   * @param path only full path, cannot be path pattern
+   * @return storage group in the given path
+   */
+  PartialPath getBelongedStorageGroup(PartialPath path) throws StorageGroupNotSetException;
+
+  /**
+   * Get the storage group that given path pattern matches or belongs to.
+   *
+   * <p>Suppose we have (root.sg1.d1.s1, root.sg2.d2.s2), refer the following cases: 1. given path
+   * "root.sg1", ("root.sg1") will be returned. 2. given path "root.*", ("root.sg1", "root.sg2")
+   * will be returned. 3. given path "root.*.d1.s1", ("root.sg1", "root.sg2") will be returned.
+   *
+   * @param pathPattern a path pattern or a full path
+   * @return a list contains all storage groups related to given path pattern
+   */
+  List<PartialPath> getBelongedStorageGroups(PartialPath pathPattern) throws MetadataException;
+
+  /**
+   * Get all storage group matching given path pattern. If using prefix match, the path pattern is
+   * used to match prefix path. All timeseries start with the matched prefix path will be collected.
+   *
+   * @param pathPattern a pattern of a full path
+   * @param isPrefixMatch if true, the path pattern is used to match prefix path
+   * @return A ArrayList instance which stores storage group paths matching given path pattern.
+   */
+  List<PartialPath> getMatchedStorageGroups(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException;
+
+  /** Get all storage group paths */
+  List<PartialPath> getAllStorageGroupPaths();
+
+  /**
+   * For a path, infer all storage groups it may belong to. The path can have wildcards. Resolve the
+   * path or path pattern into StorageGroupName-FullPath pairs that FullPath matches the given path.
+   *
+   * <p>Consider the path into two parts: (1) the sub path which can not contain a storage group
+   * name and (2) the sub path which is substring that begin after the storage group name.
+   *
+   * <p>(1) Suppose the part of the path can not contain a storage group name (e.g.,
+   * "root".contains("root.sg") == false), then: For each one level wildcard *, only one level will
+   * be inferred and the wildcard will be removed. For each multi level wildcard **, then the
+   * inference will go on until the storage groups are found and the wildcard will be kept. (2)
+   * Suppose the part of the path is a substring that begin after the storage group name. (e.g., For
+   * "root.*.sg1.a.*.b.*" and "root.x.sg1" is a storage group, then this part is "a.*.b.*"). For
+   * this part, keep what it is.
+   *
+   * <p>Assuming we have three SGs: root.group1, root.group2, root.area1.group3 Eg1: for input
+   * "root.**", returns ("root.group1", "root.group1.**"), ("root.group2", "root.group2.**")
+   * ("root.area1.group3", "root.area1.group3.**") Eg2: for input "root.*.s1", returns
+   * ("root.group1", "root.group1.s1"), ("root.group2", "root.group2.s1")
+   *
+   * <p>Eg3: for input "root.area1.**", returns ("root.area1.group3", "root.area1.group3.**")
+   *
+   * @param path can be a path pattern or a full path.
+   * @return StorageGroupName-FullPath pairs
+   * @apiNote :for cluster
+   */
+  Map<String, List<PartialPath>> groupPathByStorageGroup(PartialPath path) throws MetadataException;
+
+  /**
+   * To calculate the count of storage group for given path pattern. If using prefix match, the path
+   * pattern is used to match prefix path. All timeseries start with the matched prefix path will be
+   * counted.
+   */
+  int getStorageGroupNum(PartialPath pathPattern, boolean isPrefixMatch) throws MetadataException;
+
+  /**
+   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg],
+   * return the MNode of root.sg Get storage group node by path. Give path like [root, sg, device],
+   * MNodeTypeMismatchException will be thrown. If storage group is not set,
+   * StorageGroupNotSetException will be thrown.
+   */
+  IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
+      throws MetadataException;
+
+  /** Get storage group node by path. the give path don't need to be storage group path. */
+  IStorageGroupMNode getStorageGroupNodeByPath(PartialPath path) throws MetadataException;
+
+  /** Get all storage group MNodes */
+  List<IStorageGroupMNode> getAllStorageGroupNodes();
+
+  /**
+   * Check whether the path exists in MTreeAboveSG. The path may beyond the MTreeAboveSG scope, then
+   * return true if the covered part exists. The rest part will be checked by certain storage group
+   * subTree.
+   *
+   * @param path a full path or a prefix path
+   */
+  boolean isPathExist(PartialPath path);

Review comment:
       Rename the method




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -547,70 +339,13 @@ public void createTimeseries(CreateTimeSeriesPlan plan) throws MetadataException
 
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public void createTimeseries(CreateTimeSeriesPlan plan, long offset) throws MetadataException {
-    if (!allowToCreateNewSeries) {
+    if (!timeseriesStatistics.isAllowToCreateNewSeries()) {
       throw new MetadataException(
           "IoTDB system load is too large to create timeseries, "
               + "please increase MAX_HEAP_SIZE in iotdb-env.sh/bat and restart");
     }
-    try {
-      PartialPath path = plan.getPath();
-      SchemaUtils.checkDataTypeWithEncoding(plan.getDataType(), plan.getEncoding());
-
-      ensureStorageGroup(path);
-
-      TSDataType type = plan.getDataType();
-      // create time series in MTree
-      IMeasurementMNode leafMNode =
-          mtree.createTimeseries(
-              path,
-              type,
-              plan.getEncoding(),
-              plan.getCompressor(),
-              plan.getProps(),
-              plan.getAlias());
-
-      // the cached mNode may be replaced by new entityMNode in mtree
-      mNodeCache.invalidate(path.getDevicePath());
-
-      // update tag index
-
-      if (offset != -1 && isRecovering) {
-        // the timeseries has already been created and now system is recovering, using the tag info
-        // in tagFile to recover index directly
-        tagManager.recoverIndex(offset, leafMNode);
-      } else if (plan.getTags() != null) {
-        // tag key, tag value
-        tagManager.addIndex(plan.getTags(), leafMNode);
-      }
-
-      // update statistics and schemaDataTypeNumMap
-      totalSeriesNumber.addAndGet(1);
-      if (totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE >= MTREE_SIZE_THRESHOLD) {
-        logger.warn("Current series number {} is too large...", totalSeriesNumber);
-        allowToCreateNewSeries = false;
-      }
-
-      // write log
-      if (!isRecovering) {
-        // either tags or attributes is not empty
-        if ((plan.getTags() != null && !plan.getTags().isEmpty())
-            || (plan.getAttributes() != null && !plan.getAttributes().isEmpty())) {
-          offset = tagManager.writeTagFile(plan.getTags(), plan.getAttributes());
-        }
-        plan.setTagOffset(offset);
-        logWriter.createTimeseries(plan);
-      }
-      leafMNode.setOffset(offset);
-
-    } catch (IOException e) {
-      throw new MetadataException(e);
-    }
-
-    // update id table if not in recovering or disable id table log file
-    if (config.isEnableIDTable() && (!isRecovering || !config.isEnableIDTableLogFile())) {
-      IDTable idTable = IDTableManager.getInstance().getIDTable(plan.getPath().getDevicePath());
-      idTable.createTimeseries(plan);
-    }
+    ensureStorageGroup(plan.getPath());
+    storageGroupManager.getBelongedSGMManager(plan.getPath()).createTimeseries(plan, offset);

Review comment:
       Got it




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  private MTreeAboveSG mtree;
+
+  public StorageGroupManager() {
+    mtree = new MTreeAboveSG();
+  }
+
+  public synchronized void init() {
+    mtree.init();
+    File dir = new File(IoTDBDescriptor.getInstance().getConfig().getSchemaDir());
+    File[] sgDirs = dir.listFiles((dir1, name) -> name.startsWith(PATH_ROOT + PATH_SEPARATOR));
+    if (sgDirs != null) {
+      for (File sgDir : sgDirs) {
+        try {
+          setStorageGroup(new PartialPath(sgDir.getName()));
+        } catch (MetadataException e) {
+          logger.error("Cannot recover storage group from dir {} because", sgDir.getName(), e);
+        }
+      }
+    }
+  }
+
+  public synchronized void clear() {
+    for (SGMManager sgmManager : getAllSGMManagers()) {
+      sgmManager.clear();
+    }
+    if (mtree != null) {
+      mtree.clear();
+    }
+  }
+
+  @Override
+  public void setStorageGroup(PartialPath path) throws MetadataException {

Review comment:
       Comments have been added. The synchronized block in MTreeBelowSG and concurrentHashMap in MNode ensure that this is thread-safe.




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {

Review comment:
       Fixed.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/IStorageGroupManager.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// This class declares all the interfaces for storage group management.
+public interface IStorageGroupManager {

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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47322472/badge)](https://coveralls.io/builds/47322472)
   
   Coverage decreased (-2.3%) to 65.546% when pulling **bf2a06afbcf54e82d1aebdb03ddf0f8ff8236de5 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not

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] MarcosZyk removed a comment on pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

Posted by GitBox <gi...@apache.org>.
MarcosZyk removed a comment on pull request #5205:
URL: https://github.com/apache/iotdb/pull/5205#issuecomment-1067508698


   Let's conclude the concurrent issues and related solutions.


-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47368985/badge)](https://coveralls.io/builds/47368985)
   
   Coverage decreased (-2.3%) to 65.586% when pulling **b500dd1248a20198e289fdb7244a3b69b97dca43 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  private MTreeAboveSG mtree;
+
+  public StorageGroupManager() {
+    mtree = new MTreeAboveSG();
+  }
+
+  public synchronized void init() {
+    mtree.init();
+    File dir = new File(IoTDBDescriptor.getInstance().getConfig().getSchemaDir());
+    File[] sgDirs = dir.listFiles((dir1, name) -> name.startsWith(PATH_ROOT + PATH_SEPARATOR));
+    if (sgDirs != null) {
+      for (File sgDir : sgDirs) {
+        try {
+          setStorageGroup(new PartialPath(sgDir.getName()));
+        } catch (MetadataException e) {
+          logger.error("Cannot recover storage group from dir {} because", sgDir.getName(), e);
+        }
+      }
+    }
+  }
+
+  public synchronized void clear() {
+    for (SGMManager sgmManager : getAllSGMManagers()) {
+      sgmManager.clear();
+    }
+    if (mtree != null) {
+      mtree.clear();
+    }
+  }
+
+  @Override
+  public void setStorageGroup(PartialPath path) throws MetadataException {
+    IStorageGroupMNode storageGroupMNode = mtree.setStorageGroup(path);
+    SGMManager sgmManager = new SGMManager();
+    storageGroupMNode.setSGMManager(sgmManager);
+    sgmManager.init(storageGroupMNode);
+  }
+
+  @Override
+  public SGMManager getBelongedSGMManager(PartialPath path) throws MetadataException {

Review comment:
       The node change is just the nodeType change, from storageGroupMNode to storageGroupEntityMNode, the reference of SGMManager will be taken by both the old node and new node, thus there's no concurrent problem. Since all the references points to the old node will re-point to the new node, thus the old node will be collected as a garbage.




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));

Review comment:
       My point is not regarding to the map's write operation.
   
   For example, thread A is setting the SGMManager to the StorageGroupMNode with below codes. If another thread B executes line 103 at the same time, the StorageGroupMNode's parent Node will be replaced in the whole tree




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java
##########
@@ -177,14 +169,28 @@ public void init() throws IOException {
     File mtreeSnapshot = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotPath);
     long time = System.currentTimeMillis();
     if (mtreeSnapshot.exists()) {
-      this.root = deserializeFrom(mtreeSnapshot).root;
-      logger.debug(
-          "spend {} ms to deserialize mtree from snapshot", System.currentTimeMillis() - time);
+      IStorageGroupMNode recoveredTree =
+          deserializeFrom(mtreeSnapshot, storageGroupMNode.getFullPath());
+      if (recoveredTree != null) {
+        this.storageGroupMNode.getParent().replaceChild(storageGroupMNode.getName(), recoveredTree);

Review comment:
       LOL




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/SGMManager.java
##########
@@ -0,0 +1,1762 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
+import org.apache.iotdb.db.exception.metadata.DifferentTemplateException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.NoTemplateOnMNodeException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.TemplateIsInUseException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeBelowSG;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.rescon.TimeseriesStatistics;
+import org.apache.iotdb.db.metadata.tag.TagManager;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.template.TemplateManager;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
+import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.db.utils.TypeInferenceUtils;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.google.gson.JsonObject;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
+
+/**
+ * This class takes the responsibility of serialization of all the metadata info of one certain
+ * storage group and persistent it into files. This class contains the interfaces to modify the
+ * metadata in storage group for delta system. All the operations will be inserted into the logs
+ * temporary in case the downtime of the delta system.
+ *
+ * <p>Since there are too many interfaces and methods in this class, we use code region to help
+ * manage code. The code region starts with //region and ends with //endregion. When using Intellij
+ * Idea to develop, it's easy to fold the code region and see code region overview by collapsing
+ * all.
+ *
+ * <p>The codes are divided into the following code regions:
+ *
+ * <ol>
+ *   <li>Interfaces and Implementation of initialization、snapshot、recover and clear
+ *   <li>Interfaces and Implementation for Timeseries operation
+ *   <li>Interfaces and Implementation for StorageGroup and TTL operation
+ *   <li>Interfaces for get and auto create device
+ *   <li>Interfaces for metadata info Query
+ *       <ol>
+ *         <li>Interfaces for metadata count
+ *         <li>Interfaces for level Node info Query
+ *         <li>Interfaces for StorageGroup and TTL info Query
+ *         <li>Interfaces for Entity/Device info Query
+ *         <li>Interfaces for timeseries, measurement and schema info Query
+ *       </ol>
+ *   <li>Interfaces and methods for MNode query
+ *   <li>Interfaces for alias and tag/attribute operations
+ *   <li>Interfaces only for Cluster module usage
+ *   <li>Interfaces and Implementation for InsertPlan process
+ *   <li>Interfaces and Implementation for Template operations
+ *   <li>TestOnly Interfaces
+ * </ol>
+ */
+@SuppressWarnings("java:S1135") // ignore todos
+public class SGMManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  /** A thread will check whether the MTree is modified lately each such interval. Unit: second */
+  private static final long MTREE_SNAPSHOT_THREAD_CHECK_TIME = 600L;
+
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private boolean isRecovering = true;
+  private boolean initialized = false;
+
+  private final int mtreeSnapshotInterval;
+  private final long mtreeSnapshotThresholdTime;
+  private ScheduledExecutorService timedCreateMTreeSnapshotThread;
+  private ScheduledExecutorService timedForceMLogThread;
+
+  private String sgSchemaDirPath;
+  private String storageGroupFullPath;
+
+  // the log file seriesPath
+  private String logFilePath;
+  private File logFile;
+  private MLogWriter logWriter;
+
+  private TimeseriesStatistics timeseriesStatistics = TimeseriesStatistics.getInstance();
+  private MTreeBelowSG mtree;
+  // device -> DeviceMNode
+  private LoadingCache<PartialPath, IMNode> mNodeCache;
+  private TagManager tagManager;
+
+  // region Interfaces and Implementation of initialization、snapshot、recover and clear
+  public SGMManager() {
+    mtreeSnapshotInterval = config.getMtreeSnapshotInterval();
+    mtreeSnapshotThresholdTime = config.getMtreeSnapshotThresholdTime() * 1000L;
+
+    int cacheSize = config.getmManagerCacheSize();
+    mNodeCache =
+        Caffeine.newBuilder()
+            .maximumSize(cacheSize)
+            .build(
+                new com.github.benmanes.caffeine.cache.CacheLoader<PartialPath, IMNode>() {
+                  @Override
+                  public @Nullable IMNode load(@NonNull PartialPath partialPath)
+                      throws MetadataException {
+
+                    return mtree.getNodeByPath(partialPath);
+                  }
+                });
+
+    if (config.isEnableMTreeSnapshot()) {
+      timedCreateMTreeSnapshotThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedCreateMTreeSnapshot");
+      timedCreateMTreeSnapshotThread.scheduleAtFixedRate(
+          this::checkMTreeModified,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          TimeUnit.SECONDS);
+    }
+
+    if (config.getSyncMlogPeriodInMs() != 0) {
+      timedForceMLogThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedForceMLogThread");
+
+      timedForceMLogThread.scheduleAtFixedRate(
+          this::forceMlog,
+          config.getSyncMlogPeriodInMs(),
+          config.getSyncMlogPeriodInMs(),
+          TimeUnit.MILLISECONDS);
+    }
+  }
+
+  // Because the writer will be used later and should not be closed here.
+  @SuppressWarnings("squid:S2093")
+  public synchronized void init(IStorageGroupMNode storageGroupMNode) {
+    if (initialized) {
+      return;
+    }
+
+    storageGroupFullPath = storageGroupMNode.getFullPath();
+
+    sgSchemaDirPath = config.getSchemaDir() + File.separator + storageGroupFullPath;
+    File sgSchemaFolder = SystemFileFactory.INSTANCE.getFile(sgSchemaDirPath);
+    if (!sgSchemaFolder.exists()) {
+      if (sgSchemaFolder.mkdirs()) {
+        logger.info("create storage group schema folder {}", sgSchemaFolder.getAbsolutePath());
+      } else {
+        logger.info(
+            "create storage group schema folder {} failed.", sgSchemaFolder.getAbsolutePath());

Review comment:
       Shall we interrupt the process if the folder cannot be created ?




-- 
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] xingtanzjr commented on pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   After this change, It seems that every storage group will run two threads pool to execute some fixed rate tasks such as doing snapshot. If the count of thread pool is large, how do we control the number of these threads ?


-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -230,67 +178,20 @@ protected MManager() {
         logger.info("create system folder {} failed.", schemaFolder.getAbsolutePath());
       }
     }
-    logFilePath = schemaDir + File.separator + MetadataConstant.METADATA_LOG;
-
-    // do not write log when recover
-    isRecovering = true;
-
-    int cacheSize = config.getmManagerCacheSize();
-    mNodeCache =
-        Caffeine.newBuilder()
-            .maximumSize(cacheSize)
-            .build(
-                new com.github.benmanes.caffeine.cache.CacheLoader<PartialPath, IMNode>() {
-                  @Override
-                  public @Nullable IMNode load(@NonNull PartialPath partialPath)
-                      throws MetadataException {
-
-                    return mtree.getNodeByPathWithStorageGroupCheck(partialPath);
-                  }
-                });
-
-    if (config.isEnableMTreeSnapshot()) {
-      timedCreateMTreeSnapshotThread =
-          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedCreateMTreeSnapshot");
-      timedCreateMTreeSnapshotThread.scheduleAtFixedRate(
-          this::checkMTreeModified,
-          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
-          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
-          TimeUnit.SECONDS);
-    }
-
-    if (config.getSyncMlogPeriodInMs() != 0) {
-      timedForceMLogThread =
-          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedForceMLogThread");
-
-      timedForceMLogThread.scheduleAtFixedRate(
-          this::forceMlog,
-          config.getSyncMlogPeriodInMs(),
-          config.getSyncMlogPeriodInMs(),
-          TimeUnit.MILLISECONDS);
-    }
   }
 
-  // Because the writer will be used later and should not be closed here.
   @SuppressWarnings("squid:S2093")
   public synchronized void init() {
     if (initialized) {
       return;
     }
-    logFile = SystemFileFactory.INSTANCE.getFile(logFilePath);
 
     try {
-      isRecovering = true;
+      timeseriesStatistics.init();
+      templateManager.init();
+      storageGroupManager = new StorageGroupManager();
+      storageGroupManager.init();

Review comment:
       Fixed. Now singletone is implemented on StorageGroupManager.




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -169,6 +169,12 @@ public synchronized void replaceChild(String oldChildName, IMNode newChildNode)
       newChildNode.getAsEntityMNode().setUseTemplate(oldChildNode.isUseTemplate());
     }
 

Review comment:
       We define the parent node of measurementMNode as Device/Entity, thus there will be a MNode type change when creating timeseries under an existing InternalMNode.




-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47330293/badge)](https://coveralls.io/builds/47330293)
   
   Coverage decreased (-2.3%) to 65.571% when pulling **bf2a06afbcf54e82d1aebdb03ddf0f8ff8236de5 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -230,67 +178,20 @@ protected MManager() {
         logger.info("create system folder {} failed.", schemaFolder.getAbsolutePath());

Review comment:
       What will happen if this condition is triggered? 
   It is a legacy issue but it is better to handle the scenario in this re-factor




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not

Review comment:
       grammar issue




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -18,52 +18,37 @@
  */

Review comment:
       StorageGroupAlreadSetException will be thrown.




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java
##########
@@ -177,14 +169,28 @@ public void init() throws IOException {
     File mtreeSnapshot = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotPath);
     long time = System.currentTimeMillis();
     if (mtreeSnapshot.exists()) {
-      this.root = deserializeFrom(mtreeSnapshot).root;
-      logger.debug(
-          "spend {} ms to deserialize mtree from snapshot", System.currentTimeMillis() - time);
+      IStorageGroupMNode recoveredTree =
+          deserializeFrom(mtreeSnapshot, storageGroupMNode.getFullPath());
+      if (recoveredTree != null) {
+        this.storageGroupMNode.getParent().replaceChild(storageGroupMNode.getName(), recoveredTree);

Review comment:
       Some comments have been added. Hope it will help understand.




-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47308402/badge)](https://coveralls.io/builds/47308402)
   
   Coverage increased (+0.08%) to 67.945% when pulling **7a7ec80c07a95edb4436352fab3504b871f82825 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/SGMManager.java
##########
@@ -0,0 +1,1765 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
+import org.apache.iotdb.db.exception.metadata.DifferentTemplateException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.NoTemplateOnMNodeException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.TemplateIsInUseException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeBelowSG;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.rescon.TimeseriesStatistics;
+import org.apache.iotdb.db.metadata.tag.TagManager;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.template.TemplateManager;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
+import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.db.utils.TypeInferenceUtils;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.google.gson.JsonObject;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
+
+/**
+ * This class takes the responsibility of serialization of all the metadata info and persistent it
+ * into files. This class contains all the interfaces to modify the metadata for delta system. All
+ * the operations will be insert into the logs temporary in case the downtime of the delta system.
+ *
+ * <p>Since there are too many interfaces and methods in this class, we use code region to help
+ * manage code. The code region starts with //region and ends with //endregion. When using Intellij
+ * Idea to develop, it's easy to fold the code region and see code region overview by collapsing
+ * all.
+ *
+ * <p>The codes are divided into the following code regions:
+ *
+ * <ol>
+ *   <li>Interfaces and Implementation of MManager initialization、snapshot、recover and clear
+ *   <li>Interfaces for CQ
+ *   <li>Interfaces and Implementation for Timeseries operation
+ *   <li>Interfaces and Implementation for StorageGroup and TTL operation
+ *   <li>Interfaces for get and auto create device
+ *   <li>Interfaces for metadata info Query
+ *       <ol>
+ *         <li>Interfaces for metadata count
+ *         <li>Interfaces for level Node info Query
+ *         <li>Interfaces for StorageGroup and TTL info Query
+ *         <li>Interfaces for Entity/Device info Query
+ *         <li>Interfaces for timeseries, measurement and schema info Query
+ *       </ol>
+ *   <li>Interfaces and methods for MNode query
+ *   <li>Interfaces for alias and tag/attribute operations
+ *   <li>Interfaces only for Cluster module usage
+ *   <li>Interfaces for lastCache operations
+ *   <li>Interfaces and Implementation for InsertPlan process
+ *   <li>Interfaces and Implementation for Template operations
+ *   <li>TestOnly Interfaces
+ * </ol>
+ */
+@SuppressWarnings("java:S1135") // ignore todos
+public class SGMManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  /** A thread will check whether the MTree is modified lately each such interval. Unit: second */
+  private static final long MTREE_SNAPSHOT_THREAD_CHECK_TIME = 600L;
+
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private boolean isRecovering;
+  private boolean initialized;
+
+  private final int mtreeSnapshotInterval;
+  private final long mtreeSnapshotThresholdTime;
+  private ScheduledExecutorService timedCreateMTreeSnapshotThread;
+  private ScheduledExecutorService timedForceMLogThread;
+
+  private String sgSchemaDirPath;
+  private String storageGroupFullPath;
+
+  // the log file seriesPath
+  private String logFilePath;
+  private File logFile;
+  private MLogWriter logWriter;
+
+  private TimeseriesStatistics timeseriesStatistics = TimeseriesStatistics.getInstance();
+  private MTreeBelowSG mtree;
+  // device -> DeviceMNode
+  private LoadingCache<PartialPath, IMNode> mNodeCache;
+  private TagManager tagManager;
+
+  // region Interfaces and Implementation of MManager initialization、snapshot、recover and clear
+  public SGMManager() {
+    mtreeSnapshotInterval = config.getMtreeSnapshotInterval();
+    mtreeSnapshotThresholdTime = config.getMtreeSnapshotThresholdTime() * 1000L;
+
+    // do not write log when recover
+    isRecovering = true;
+
+    int cacheSize = config.getmManagerCacheSize();
+    mNodeCache =
+        Caffeine.newBuilder()
+            .maximumSize(cacheSize)
+            .build(
+                new com.github.benmanes.caffeine.cache.CacheLoader<PartialPath, IMNode>() {
+                  @Override
+                  public @Nullable IMNode load(@NonNull PartialPath partialPath)
+                      throws MetadataException {
+
+                    return mtree.getNodeByPath(partialPath);
+                  }
+                });
+
+    if (config.isEnableMTreeSnapshot()) {
+      timedCreateMTreeSnapshotThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedCreateMTreeSnapshot");
+      timedCreateMTreeSnapshotThread.scheduleAtFixedRate(
+          this::checkMTreeModified,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          TimeUnit.SECONDS);
+    }
+
+    if (config.getSyncMlogPeriodInMs() != 0) {
+      timedForceMLogThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedForceMLogThread");
+
+      timedForceMLogThread.scheduleAtFixedRate(
+          this::forceMlog,
+          config.getSyncMlogPeriodInMs(),
+          config.getSyncMlogPeriodInMs(),
+          TimeUnit.MILLISECONDS);
+    }
+  }
+
+  // Because the writer will be used later and should not be closed here.
+  @SuppressWarnings("squid:S2093")
+  public synchronized void init(IStorageGroupMNode storageGroupMNode) {
+    if (initialized) {
+      return;
+    }
+
+    storageGroupFullPath = storageGroupMNode.getFullPath();
+
+    sgSchemaDirPath = config.getSchemaDir() + File.separator + storageGroupFullPath;
+    File sgSchemaFolder = SystemFileFactory.INSTANCE.getFile(sgSchemaDirPath);
+    if (!sgSchemaFolder.exists()) {
+      if (sgSchemaFolder.mkdirs()) {
+        logger.info("create storage group schema folder {}", sgSchemaFolder.getAbsolutePath());
+      } else {
+        logger.info(
+            "create storage group schema folder {} failed.", sgSchemaFolder.getAbsolutePath());
+      }
+    }
+    logFilePath = sgSchemaDirPath + File.separator + MetadataConstant.METADATA_LOG;
+
+    logFile = SystemFileFactory.INSTANCE.getFile(logFilePath);
+
+    try {
+      isRecovering = true;

Review comment:
       Thanks for suggestion. Already 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] MarcosZyk commented on pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   Let's conclude the concurrent issues and related solutions.


-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47407563/badge)](https://coveralls.io/builds/47407563)
   
   Coverage increased (+0.001%) to 65.535% when pulling **37e9a97fb1bd1e91af12022af8db0d22895b74fe on MarcosZyk:schema_engine** into **1ff6db96bf688aff6024ef676d5de2bcd03436c0 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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -547,70 +339,13 @@ public void createTimeseries(CreateTimeSeriesPlan plan) throws MetadataException
 
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public void createTimeseries(CreateTimeSeriesPlan plan, long offset) throws MetadataException {
-    if (!allowToCreateNewSeries) {
+    if (!timeseriesStatistics.isAllowToCreateNewSeries()) {
       throw new MetadataException(
           "IoTDB system load is too large to create timeseries, "
               + "please increase MAX_HEAP_SIZE in iotdb-env.sh/bat and restart");
     }
-    try {
-      PartialPath path = plan.getPath();
-      SchemaUtils.checkDataTypeWithEncoding(plan.getDataType(), plan.getEncoding());
-
-      ensureStorageGroup(path);
-
-      TSDataType type = plan.getDataType();
-      // create time series in MTree
-      IMeasurementMNode leafMNode =
-          mtree.createTimeseries(
-              path,
-              type,
-              plan.getEncoding(),
-              plan.getCompressor(),
-              plan.getProps(),
-              plan.getAlias());
-
-      // the cached mNode may be replaced by new entityMNode in mtree
-      mNodeCache.invalidate(path.getDevicePath());
-
-      // update tag index
-
-      if (offset != -1 && isRecovering) {
-        // the timeseries has already been created and now system is recovering, using the tag info
-        // in tagFile to recover index directly
-        tagManager.recoverIndex(offset, leafMNode);
-      } else if (plan.getTags() != null) {
-        // tag key, tag value
-        tagManager.addIndex(plan.getTags(), leafMNode);
-      }
-
-      // update statistics and schemaDataTypeNumMap
-      totalSeriesNumber.addAndGet(1);
-      if (totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE >= MTREE_SIZE_THRESHOLD) {
-        logger.warn("Current series number {} is too large...", totalSeriesNumber);
-        allowToCreateNewSeries = false;
-      }
-
-      // write log
-      if (!isRecovering) {
-        // either tags or attributes is not empty
-        if ((plan.getTags() != null && !plan.getTags().isEmpty())
-            || (plan.getAttributes() != null && !plan.getAttributes().isEmpty())) {
-          offset = tagManager.writeTagFile(plan.getTags(), plan.getAttributes());
-        }
-        plan.setTagOffset(offset);
-        logWriter.createTimeseries(plan);
-      }
-      leafMNode.setOffset(offset);
-
-    } catch (IOException e) {
-      throw new MetadataException(e);
-    }
-
-    // update id table if not in recovering or disable id table log file
-    if (config.isEnableIDTable() && (!isRecovering || !config.isEnableIDTableLogFile())) {
-      IDTable idTable = IDTableManager.getInstance().getIDTable(plan.getPath().getDevicePath());
-      idTable.createTimeseries(plan);
-    }
+    ensureStorageGroup(plan.getPath());
+    storageGroupManager.getBelongedSGMManager(plan.getPath()).createTimeseries(plan, offset);

Review comment:
       If the SGMManager is not set in `ensureStorageGroup(plan.getPath());`, this line will trigger NPE




-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47314685/badge)](https://coveralls.io/builds/47314685)
   
   Coverage decreased (-2.3%) to 65.572% when pulling **7e8a4e88ee0770721bb9b0582e87ba7114e67e89 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {

Review comment:
       This is duplicated with StorageGroupManager in StorageEngine, rename to StorageGroupSchemaManager

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/IStorageGroupManager.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// This class declares all the interfaces for storage group management.
+public interface IStorageGroupManager {

Review comment:
       ```suggestion
   public interface IStorageGroupSchemaManager {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -170,35 +143,12 @@
 
   public static final String TIME_SERIES_TREE_HEADER = "===  Timeseries Tree  ===\n\n";
 
-  /** A thread will check whether the MTree is modified lately each such interval. Unit: second */
-  private static final long MTREE_SNAPSHOT_THREAD_CHECK_TIME = 600L;
-
   protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-  /** threshold total size of MTree */
-  private static final long MTREE_SIZE_THRESHOLD = config.getAllocateMemoryForSchema();
-
-  private static final int ESTIMATED_SERIES_SIZE = config.getEstimatedSeriesSize();
 
-  private boolean isRecovering;
   private boolean initialized;
-  private boolean allowToCreateNewSeries = true;
-
-  private AtomicLong totalSeriesNumber = new AtomicLong();
-
-  private final int mtreeSnapshotInterval;
-  private final long mtreeSnapshotThresholdTime;
-  private ScheduledExecutorService timedCreateMTreeSnapshotThread;
-  private ScheduledExecutorService timedForceMLogThread;
 
-  // the log file seriesPath
-  private String logFilePath;
-  private File logFile;
-  private MLogWriter logWriter;
-
-  private MTree mtree;
-  // device -> DeviceMNode
-  private LoadingCache<PartialPath, IMNode> mNodeCache;
-  private TagManager tagManager = TagManager.getInstance();
+  private TimeseriesStatistics timeseriesStatistics = TimeseriesStatistics.getInstance();
+  private IStorageGroupManager storageGroupManager = StorageGroupManager.getInstance();

Review comment:
       StorageGroupManager will be removed later, you need to maintain each MTree of Storage Group inside 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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47289048/badge)](https://coveralls.io/builds/47289048)
   
   Coverage increased (+0.02%) to 67.886% when pulling **ac3b24758cecbfd0f12c2e55d756f3157eaad772 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -169,6 +169,12 @@ public synchronized void replaceChild(String oldChildName, IMNode newChildNode)
       newChildNode.getAsEntityMNode().setUseTemplate(oldChildNode.isUseTemplate());
     }
 

Review comment:
       Just be curious, why do we need to clone the children from `oldChildNode` to `newChildNode`...




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/SGMManager.java
##########
@@ -0,0 +1,1765 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
+import org.apache.iotdb.db.exception.metadata.DifferentTemplateException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.NoTemplateOnMNodeException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.TemplateIsInUseException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeBelowSG;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.rescon.TimeseriesStatistics;
+import org.apache.iotdb.db.metadata.tag.TagManager;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.template.TemplateManager;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
+import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.db.utils.TypeInferenceUtils;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.google.gson.JsonObject;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
+
+/**
+ * This class takes the responsibility of serialization of all the metadata info and persistent it
+ * into files. This class contains all the interfaces to modify the metadata for delta system. All
+ * the operations will be insert into the logs temporary in case the downtime of the delta system.

Review comment:
       `will be inserted`




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/SGMManager.java
##########
@@ -0,0 +1,1762 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
+import org.apache.iotdb.db.exception.metadata.DifferentTemplateException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.NoTemplateOnMNodeException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.TemplateIsInUseException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.db.metadata.logfile.MLogReader;
+import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeBelowSG;
+import org.apache.iotdb.db.metadata.path.MeasurementPath;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.rescon.TimeseriesStatistics;
+import org.apache.iotdb.db.metadata.tag.TagManager;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.template.TemplateManager;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
+import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
+import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.db.utils.TypeInferenceUtils;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.google.gson.JsonObject;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
+
+/**
+ * This class takes the responsibility of serialization of all the metadata info of one certain
+ * storage group and persistent it into files. This class contains the interfaces to modify the
+ * metadata in storage group for delta system. All the operations will be inserted into the logs
+ * temporary in case the downtime of the delta system.
+ *
+ * <p>Since there are too many interfaces and methods in this class, we use code region to help
+ * manage code. The code region starts with //region and ends with //endregion. When using Intellij
+ * Idea to develop, it's easy to fold the code region and see code region overview by collapsing
+ * all.
+ *
+ * <p>The codes are divided into the following code regions:
+ *
+ * <ol>
+ *   <li>Interfaces and Implementation of initialization、snapshot、recover and clear
+ *   <li>Interfaces and Implementation for Timeseries operation
+ *   <li>Interfaces and Implementation for StorageGroup and TTL operation
+ *   <li>Interfaces for get and auto create device
+ *   <li>Interfaces for metadata info Query
+ *       <ol>
+ *         <li>Interfaces for metadata count
+ *         <li>Interfaces for level Node info Query
+ *         <li>Interfaces for StorageGroup and TTL info Query
+ *         <li>Interfaces for Entity/Device info Query
+ *         <li>Interfaces for timeseries, measurement and schema info Query
+ *       </ol>
+ *   <li>Interfaces and methods for MNode query
+ *   <li>Interfaces for alias and tag/attribute operations
+ *   <li>Interfaces only for Cluster module usage
+ *   <li>Interfaces and Implementation for InsertPlan process
+ *   <li>Interfaces and Implementation for Template operations
+ *   <li>TestOnly Interfaces
+ * </ol>
+ */
+@SuppressWarnings("java:S1135") // ignore todos
+public class SGMManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  /** A thread will check whether the MTree is modified lately each such interval. Unit: second */
+  private static final long MTREE_SNAPSHOT_THREAD_CHECK_TIME = 600L;
+
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private boolean isRecovering = true;
+  private boolean initialized = false;
+
+  private final int mtreeSnapshotInterval;
+  private final long mtreeSnapshotThresholdTime;
+  private ScheduledExecutorService timedCreateMTreeSnapshotThread;
+  private ScheduledExecutorService timedForceMLogThread;
+
+  private String sgSchemaDirPath;
+  private String storageGroupFullPath;
+
+  // the log file seriesPath
+  private String logFilePath;
+  private File logFile;
+  private MLogWriter logWriter;
+
+  private TimeseriesStatistics timeseriesStatistics = TimeseriesStatistics.getInstance();
+  private MTreeBelowSG mtree;
+  // device -> DeviceMNode
+  private LoadingCache<PartialPath, IMNode> mNodeCache;
+  private TagManager tagManager;
+
+  // region Interfaces and Implementation of initialization、snapshot、recover and clear
+  public SGMManager() {
+    mtreeSnapshotInterval = config.getMtreeSnapshotInterval();
+    mtreeSnapshotThresholdTime = config.getMtreeSnapshotThresholdTime() * 1000L;
+
+    int cacheSize = config.getmManagerCacheSize();
+    mNodeCache =
+        Caffeine.newBuilder()
+            .maximumSize(cacheSize)
+            .build(
+                new com.github.benmanes.caffeine.cache.CacheLoader<PartialPath, IMNode>() {
+                  @Override
+                  public @Nullable IMNode load(@NonNull PartialPath partialPath)
+                      throws MetadataException {
+
+                    return mtree.getNodeByPath(partialPath);
+                  }
+                });
+
+    if (config.isEnableMTreeSnapshot()) {
+      timedCreateMTreeSnapshotThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedCreateMTreeSnapshot");
+      timedCreateMTreeSnapshotThread.scheduleAtFixedRate(
+          this::checkMTreeModified,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          MTREE_SNAPSHOT_THREAD_CHECK_TIME,
+          TimeUnit.SECONDS);
+    }
+
+    if (config.getSyncMlogPeriodInMs() != 0) {
+      timedForceMLogThread =
+          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("timedForceMLogThread");
+
+      timedForceMLogThread.scheduleAtFixedRate(
+          this::forceMlog,
+          config.getSyncMlogPeriodInMs(),
+          config.getSyncMlogPeriodInMs(),
+          TimeUnit.MILLISECONDS);
+    }
+  }
+
+  // Because the writer will be used later and should not be closed here.
+  @SuppressWarnings("squid:S2093")
+  public synchronized void init(IStorageGroupMNode storageGroupMNode) {
+    if (initialized) {
+      return;
+    }
+
+    storageGroupFullPath = storageGroupMNode.getFullPath();
+
+    sgSchemaDirPath = config.getSchemaDir() + File.separator + storageGroupFullPath;
+    File sgSchemaFolder = SystemFileFactory.INSTANCE.getFile(sgSchemaDirPath);
+    if (!sgSchemaFolder.exists()) {
+      if (sgSchemaFolder.mkdirs()) {
+        logger.info("create storage group schema folder {}", sgSchemaFolder.getAbsolutePath());
+      } else {
+        logger.info(
+            "create storage group schema folder {} failed.", sgSchemaFolder.getAbsolutePath());
+      }
+    }
+    logFilePath = sgSchemaDirPath + File.separator + MetadataConstant.METADATA_LOG;
+
+    logFile = SystemFileFactory.INSTANCE.getFile(logFilePath);
+
+    try {
+      // do not write log when recover
+      isRecovering = true;
+
+      tagManager = new TagManager(sgSchemaDirPath);
+      tagManager.init();
+      mtree = new MTreeBelowSG(storageGroupMNode);
+      mtree.init();

Review comment:
       Fixed. Now the init() is inside the constructor.




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -169,6 +169,12 @@ public synchronized void replaceChild(String oldChildName, IMNode newChildNode)
       newChildNode.getAsEntityMNode().setUseTemplate(oldChildNode.isUseTemplate());
     }
 

Review comment:
       We define the parent node of measurementMNode as Device/Entity, thus there will be a MNode type change when creating timeseries under an existing InternalMNode. The type change is implemented as node replacement.




-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47416271/badge)](https://coveralls.io/builds/47416271)
   
   Coverage decreased (-0.2%) to 65.345% when pulling **b7284f5f4a444748335afe3c51ace50055a61c91 on MarcosZyk:schema_engine** into **1ff6db96bf688aff6024ef676d5de2bcd03436c0 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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(nodeNames[i])) {
+        // node b has child sg
+        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+          throw new StorageGroupAlreadySetException(path.getFullPath());
+        } else {
+          throw new StorageGroupAlreadySetException(path.getFullPath(), true);
+        }
+      } else {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        IStorageGroupMNode storageGroupMNode =

Review comment:
       I've changed the following code. Now there's a result check of addChild and the SGMManager will be initialized before the storageGroupMNode added  to MTree, which makes sure that any thread reach the storageGroup node won't get a null nor a uninitialized SGMManager.
   
   On the other hand, we've analyzed the replacement of parent node and it is ensured that the replacement won't happen on nodes above sg, thus the data of setStorageGroup won't lose.




-- 
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] MarcosZyk commented on pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   Hi, I have an idea about the concurrent issues which is possibly occurred during changing the structure of MTree. 
   
   The safest solution is to synchronize all methods that may invoke ```mnode.addChild``` and ```mnode.replaceChild```, including ```setStorageGroup``` in MTreeAboveSG and ```createTimeserie```, ```createAlignedTimeseries``` and ```getDeviceNodeWithAutoCreate``` in MTreeBelowSG. 
   
   This will make all ```setStorageGroup``` in parallel and make all timeseries and device creation in parallel in one storage group. This definitely affects the performance. 
   
   The creation of storage group is not that frequent, while the creation of timeseries is frequent. If the performance of timeseries creation decrease to a degree that we cannot accept, may be we could add a interface for atomic creation of batched timeseries under one certain device. I think this is worthy, since we can group batched creation of timeseries into one record in MLog, which increases the disk I/O.


-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47361112/badge)](https://coveralls.io/builds/47361112)
   
   Coverage decreased (-2.2%) to 65.628% when pulling **4880db021d25c21b678f1435718ab4f1b7c889b5 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47403849/badge)](https://coveralls.io/builds/47403849)
   
   Coverage decreased (-2.2%) to 65.7% when pulling **0f868ccdec5298c81fee1f1c73591b7a511ec578 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  private MTreeAboveSG mtree;
+
+  public StorageGroupManager() {
+    mtree = new MTreeAboveSG();
+  }
+
+  public synchronized void init() {
+    mtree.init();
+    File dir = new File(IoTDBDescriptor.getInstance().getConfig().getSchemaDir());
+    File[] sgDirs = dir.listFiles((dir1, name) -> name.startsWith(PATH_ROOT + PATH_SEPARATOR));
+    if (sgDirs != null) {
+      for (File sgDir : sgDirs) {
+        try {
+          setStorageGroup(new PartialPath(sgDir.getName()));
+        } catch (MetadataException e) {
+          logger.error("Cannot recover storage group from dir {} because", sgDir.getName(), e);
+        }
+      }
+    }
+  }
+
+  public synchronized void clear() {
+    for (SGMManager sgmManager : getAllSGMManagers()) {
+      sgmManager.clear();
+    }
+    if (mtree != null) {
+      mtree.clear();
+    }
+  }
+
+  @Override
+  public void setStorageGroup(PartialPath path) throws MetadataException {
+    IStorageGroupMNode storageGroupMNode = mtree.setStorageGroup(path);
+    SGMManager sgmManager = new SGMManager();
+    storageGroupMNode.setSGMManager(sgmManager);
+    sgmManager.init(storageGroupMNode);
+  }
+
+  @Override
+  public SGMManager getBelongedSGMManager(PartialPath path) throws MetadataException {

Review comment:
       Not sure whether this method is ok with lock-free. We need to ensure this method could be invoked in parallel.
   
   The root node of SGMManager will be changed when this method is invoked, we need to consider whether it is ok here without concurrent control.




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized

Review comment:
       Grammar issue.
   
   Let's check all the comments and ensure the grammar is OK.




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));

Review comment:
       Concurrent issue.
   
   If several threads execute this line in parallel with same noneName, there will be some concurrent issues here




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

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

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



[GitHub] [iotdb] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));

Review comment:
       Concurrent issue.
   
   If several threads execute this line in parallel with same noneName, there will be some concurrent issue here




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

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

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



[GitHub] [iotdb] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
##########
@@ -169,6 +169,12 @@ public synchronized void replaceChild(String oldChildName, IMNode newChildNode)
       newChildNode.getAsEntityMNode().setUseTemplate(oldChildNode.isUseTemplate());
     }
 
+    if (newChildNode.isStorageGroup() && oldChildNode.isStorageGroup()) {

Review comment:
       Fixed. I added a new interface, moveDataToNewNode(IMNode newMNode), to IMNode and the replace method will invoke the oldChildNode.moveDataToNewNode(newChildNode). The moveDataToNewNode will be overrided by different types of mnode.




-- 
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] MarcosZyk commented on pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   Let's conclude the concurrent issues and solutions.
   1. Concurrent deletion: all deletion is synchronized.
   2. Node replacement, which may cause subTree loss. The solution is the reference to  ```children``` attribute in old node will be directly copied to new node, which avoids the sutree loss.


-- 
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 #5205: [IOTDB-2675] Manage Metadata by Storage Group

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


   
   [![Coverage Status](https://coveralls.io/builds/47289064/badge)](https://coveralls.io/builds/47289064)
   
   Coverage increased (+0.03%) to 67.89% when pulling **ac3b24758cecbfd0f12c2e55d756f3157eaad772 on MarcosZyk:schema_engine** into **e31f129c993508c4a526501cde18dcb2ac8df95d 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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java
##########
@@ -177,14 +169,28 @@ public void init() throws IOException {
     File mtreeSnapshot = SystemFileFactory.INSTANCE.getFile(mtreeSnapshotPath);
     long time = System.currentTimeMillis();
     if (mtreeSnapshot.exists()) {
-      this.root = deserializeFrom(mtreeSnapshot).root;
-      logger.debug(
-          "spend {} ms to deserialize mtree from snapshot", System.currentTimeMillis() - time);
+      IStorageGroupMNode recoveredTree =
+          deserializeFrom(mtreeSnapshot, storageGroupMNode.getFullPath());
+      if (recoveredTree != null) {
+        this.storageGroupMNode.getParent().replaceChild(storageGroupMNode.getName(), recoveredTree);
+        this.storageGroupMNode = recoveredTree;
+        logger.debug(
+            "spend {} ms to deserialize mtree from snapshot", System.currentTimeMillis() - time);
+      }
     }
   }
 
+  public IStorageGroupMNode getStorageGroupMNode() {
+    return this.storageGroupMNode;
+  }
+
+  public void setStorageGroupMNode(IStorageGroupMNode storageGroupMNode) {
+    this.storageGroupMNode = storageGroupMNode;
+    levelOfSG = storageGroupMNode.getPartialPath().getNodeLength() - 1;
+  }
+
   public void clear() {
-    root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+    storageGroupMNode = null;
   }
 

Review comment:
       The info of SGM and MTreeAboveSG is all recorded in the sg folders' names.




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  private MTreeAboveSG mtree;
+
+  public StorageGroupManager() {
+    mtree = new MTreeAboveSG();
+  }
+
+  public synchronized void init() {
+    mtree.init();
+    File dir = new File(IoTDBDescriptor.getInstance().getConfig().getSchemaDir());
+    File[] sgDirs = dir.listFiles((dir1, name) -> name.startsWith(PATH_ROOT + PATH_SEPARATOR));
+    if (sgDirs != null) {
+      for (File sgDir : sgDirs) {
+        try {
+          setStorageGroup(new PartialPath(sgDir.getName()));
+        } catch (MetadataException e) {
+          logger.error("Cannot recover storage group from dir {} because", sgDir.getName(), e);
+        }
+      }
+    }
+  }
+
+  public synchronized void clear() {
+    for (SGMManager sgmManager : getAllSGMManagers()) {
+      sgmManager.clear();
+    }
+    if (mtree != null) {
+      mtree.clear();
+    }
+  }
+
+  @Override
+  public void setStorageGroup(PartialPath path) throws MetadataException {
+    IStorageGroupMNode storageGroupMNode = mtree.setStorageGroup(path);
+    SGMManager sgmManager = new SGMManager();
+    storageGroupMNode.setSGMManager(sgmManager);

Review comment:
       Inside the implementation of `setSGMManager`, the storageGroupMNode has been set as a variable of sgmManager. So why do we need to pass it again in the following `init()` ?
   
   Suggest to decouple the relation about storageGroupMNode and sgmManager. The circle dependency of these two class may make the codes to be complicated to read.




-- 
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] xingtanzjr commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata.mtree;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.MNodeAboveSGCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.collector.StorageGroupCollector;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.CounterTraverser;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.MNodeAboveSGLevelCounter;
+import org.apache.iotdb.db.metadata.mtree.traverser.counter.StorageGroupCounter;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+
+public class MTreeAboveSG {
+
+  private static final Logger logger = LoggerFactory.getLogger(MTreeAboveSG.class);
+  public static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+
+  private IMNode root;
+
+  public MTreeAboveSG() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  public void init() {}
+
+  public void clear() {
+    this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
+  }
+
+  /**
+   * Set storage group. Make sure check seriesPath before setting storage group
+   *
+   * @param path path
+   */
+  public IStorageGroupMNode setStorageGroup(PartialPath path) throws MetadataException {
+    String[] nodeNames = path.getNodes();
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    Template upperTemplate = cur.getSchemaTemplate();
+    int i = 1;
+    // e.g., path = root.a.b.sg, create internal nodes for a, b
+    while (i < nodeNames.length - 1) {
+      IMNode temp = cur.getChild(nodeNames[i]);
+      if (temp == null) {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        cur.addChild(nodeNames[i], new InternalMNode(cur, nodeNames[i]));
+      } else if (temp.isStorageGroup()) {
+        // before set storage group, check whether the exists or not
+        throw new StorageGroupAlreadySetException(temp.getFullPath());
+      }
+      cur = cur.getChild(nodeNames[i]);
+      upperTemplate = cur.getSchemaTemplate() == null ? upperTemplate : cur.getSchemaTemplate();
+      i++;
+    }
+
+    // synchronize check and add, we need addChild become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      if (cur.hasChild(nodeNames[i])) {
+        // node b has child sg
+        if (cur.getChild(nodeNames[i]).isStorageGroup()) {
+          throw new StorageGroupAlreadySetException(path.getFullPath());
+        } else {
+          throw new StorageGroupAlreadySetException(path.getFullPath(), true);
+        }
+      } else {
+        if (cur.isUseTemplate() && upperTemplate.hasSchema(nodeNames[i])) {
+          throw new PathAlreadyExistException(
+              cur.getPartialPath().concatNode(nodeNames[i]).getFullPath());
+        }
+        IStorageGroupMNode storageGroupMNode =
+            new StorageGroupMNode(
+                cur, nodeNames[i], IoTDBDescriptor.getInstance().getConfig().getDefaultTTL());
+        cur.addChild(nodeNames[i], storageGroupMNode);
+        return storageGroupMNode;
+      }
+    }
+  }
+
+  /** Delete a storage group */
+  public void deleteStorageGroup(PartialPath path) throws MetadataException {
+    IStorageGroupMNode cur = getStorageGroupNodeByStorageGroupPath(path);
+    // Suppose current system has root.a.b.sg1, root.a.sg2, and delete root.a.b.sg1
+    // delete the storage group node sg1
+    cur.getParent().deleteChild(cur.getName());
+  }
+
+  /**
+   * Check whether path is storage group or not
+   *
+   * <p>e.g., path = root.a.b.sg. if nor a and b is StorageGroupMNode and sg is a StorageGroupMNode
+   * path is a storage group
+   *
+   * @param path path
+   * @apiNote :for cluster
+   */
+  public boolean isStorageGroup(PartialPath path) {
+    String[] nodeNames = path.getNodes();
+    if (nodeNames.length <= 1 || !nodeNames[0].equals(IoTDBConstant.PATH_ROOT)) {
+      return false;
+    }
+    IMNode cur = root;
+    int i = 1;
+    while (i < nodeNames.length - 1) {
+      cur = cur.getChild(nodeNames[i]);
+      if (cur == null || cur.isStorageGroup()) {
+        return false;
+      }
+      i++;
+    }
+    cur = cur.getChild(nodeNames[i]);
+    return cur != null && cur.isStorageGroup();
+  }
+
+  /** Check whether the given path contains a storage group */
+  public boolean checkStorageGroupByPath(PartialPath path) {
+    String[] nodes = path.getNodes();
+    IMNode cur = root;
+    for (int i = 1; i < nodes.length; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        return false;
+      } else if (cur.isStorageGroup()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Get storage group path by path
+   *
+   * <p>e.g., root.sg1 is storage group, path is root.sg1.d1, return root.sg1
+   *
+   * @return storage group in the given path
+   */
+  public PartialPath getBelongedStorageGroup(PartialPath path) throws StorageGroupNotSetException {
+    String[] nodes = path.getNodes();
+    IMNode cur = root;
+    for (int i = 1; i < nodes.length; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        throw new StorageGroupNotSetException(path.getFullPath());
+      } else if (cur.isStorageGroup()) {
+        return cur.getPartialPath();
+      }
+    }
+    throw new StorageGroupNotSetException(path.getFullPath());
+  }
+
+  /**
+   * Get the storage group that given path pattern matches or belongs to.
+   *
+   * <p>Suppose we have (root.sg1.d1.s1, root.sg2.d2.s2), refer the following cases: 1. given path
+   * "root.sg1", ("root.sg1") will be returned. 2. given path "root.*", ("root.sg1", "root.sg2")
+   * will be returned. 3. given path "root.*.d1.s1", ("root.sg1", "root.sg2") will be returned.
+   *
+   * @param pathPattern a path pattern or a full path
+   * @return a list contains all storage groups related to given path
+   */
+  public List<PartialPath> getBelongedStorageGroups(PartialPath pathPattern)
+      throws MetadataException {
+    return collectStorageGroups(pathPattern, false, true);
+  }
+
+  /**
+   * Get all storage group that the given path pattern matches. If using prefix match, the path
+   * pattern is used to match prefix path. All timeseries start with the matched prefix path will be
+   * collected.
+   *
+   * @param pathPattern a path pattern or a full path
+   * @param isPrefixMatch if true, the path pattern is used to match prefix path
+   * @return a list contains all storage group names under given path pattern
+   */
+  public List<PartialPath> getMatchedStorageGroups(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    return collectStorageGroups(pathPattern, isPrefixMatch, false);
+  }
+
+  private List<PartialPath> collectStorageGroups(
+      PartialPath pathPattern, boolean isPrefixMatch, boolean collectInternal)
+      throws MetadataException {
+    List<PartialPath> result = new LinkedList<>();
+    StorageGroupCollector<List<PartialPath>> collector =
+        new StorageGroupCollector<List<PartialPath>>(root, pathPattern) {
+          @Override
+          protected void collectStorageGroup(IStorageGroupMNode node) {
+            result.add(node.getPartialPath());
+          }
+        };
+    collector.setCollectInternal(collectInternal);
+    collector.setPrefixMatch(isPrefixMatch);
+    collector.traverse();
+    return result;
+  }
+
+  /**
+   * Get all storage group names
+   *
+   * @return a list contains all distinct storage groups
+   */
+  public List<PartialPath> getAllStorageGroupPaths() {
+    List<PartialPath> res = new ArrayList<>();
+    Deque<IMNode> nodeStack = new ArrayDeque<>();
+    nodeStack.add(root);
+    while (!nodeStack.isEmpty()) {
+      IMNode current = nodeStack.pop();
+      if (current.isStorageGroup()) {
+        res.add(current.getPartialPath());
+      } else {
+        nodeStack.addAll(current.getChildren().values());
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Resolve the path or path pattern into StorageGroupName-FullPath pairs. Try determining the
+   * storage group using the children of a mNode. If one child is a storage group node, put a
+   * storageGroupName-fullPath pair into paths.
+   */
+  public Map<String, List<PartialPath>> groupPathByStorageGroup(PartialPath path)
+      throws MetadataException {
+    Map<String, List<PartialPath>> result = new HashMap<>();
+    StorageGroupCollector<Map<String, String>> collector =
+        new StorageGroupCollector<Map<String, String>>(root, path) {
+          @Override
+          protected void collectStorageGroup(IStorageGroupMNode node) {
+            PartialPath sgPath = node.getPartialPath();
+            result.put(sgPath.getFullPath(), path.alterPrefixPath(sgPath));
+          }
+        };
+    collector.setCollectInternal(true);
+    collector.traverse();
+    return result;
+  }
+
+  /**
+   * Get the count of storage group matching the given path. If using prefix match, the path pattern
+   * is used to match prefix path. All timeseries start with the matched prefix path will be
+   * counted.
+   *
+   * @param pathPattern a path pattern or a full path, may contain wildcard.
+   * @param isPrefixMatch if true, the path pattern is used to match prefix path
+   */
+  public int getStorageGroupNum(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    CounterTraverser counter = new StorageGroupCounter(root, pathPattern);
+    counter.setPrefixMatch(isPrefixMatch);
+    counter.traverse();
+    return counter.getCount();
+  }
+
+  /**
+   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg,
+   * device], throw exception Get storage group node, if the give path is not a storage group, throw
+   * exception
+   */
+  public IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
+      throws MetadataException {
+    IStorageGroupMNode node = getStorageGroupNodeByPath(path);
+    if (!node.getPartialPath().equals(path)) {
+      throw new MNodeTypeMismatchException(
+          path.getFullPath(), MetadataConstant.STORAGE_GROUP_MNODE_TYPE);
+    }
+
+    return node;
+  }
+
+  /**
+   * E.g., root.sg is storage group given [root, sg], return the MNode of root.sg given [root, sg,
+   * device], return the MNode of root.sg Get storage group node, the give path don't need to be
+   * storage group path.
+   */
+  public IStorageGroupMNode getStorageGroupNodeByPath(PartialPath path) throws MetadataException {
+    String[] nodes = path.getNodes();
+    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
+      throw new IllegalPathException(path.getFullPath());
+    }
+    IMNode cur = root;
+    for (int i = 1; i < nodes.length; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        break;
+      }
+      if (cur.isStorageGroup()) {
+        return cur.getAsStorageGroupMNode();
+      }
+    }
+    throw new StorageGroupNotSetException(path.getFullPath());
+  }
+
+  public List<IStorageGroupMNode> getInvolvedStorageGroupNodes(
+      PartialPath pathPattern, boolean isPrefixMatch) throws MetadataException {
+    List<IStorageGroupMNode> result = new ArrayList<>();
+    StorageGroupCollector<List<IStorageGroupMNode>> collector =
+        new StorageGroupCollector<List<IStorageGroupMNode>>(root, pathPattern) {
+          @Override
+          protected void collectStorageGroup(IStorageGroupMNode node) {
+            result.add(node);
+          }
+        };
+    collector.setCollectInternal(true);
+    collector.setPrefixMatch(isPrefixMatch);
+    collector.traverse();
+    return result;
+  }
+
+  /** Get all storage group MNodes */
+  public List<IStorageGroupMNode> getAllStorageGroupNodes() {
+    List<IStorageGroupMNode> ret = new ArrayList<>();
+    Deque<IMNode> nodeStack = new ArrayDeque<>();
+    nodeStack.add(root);
+    while (!nodeStack.isEmpty()) {
+      IMNode current = nodeStack.pop();
+      if (current.isStorageGroup()) {
+        ret.add(current.getAsStorageGroupMNode());
+      } else {
+        nodeStack.addAll(current.getChildren().values());
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Check whether the given path exists.
+   *
+   * @param path a full path or a prefix path
+   */
+  public boolean isPathExist(PartialPath path) {

Review comment:
       The name of this method is confusing.
   
   Actually, this method cannot judge whether a path exists or not. It can only tell the invoker whether the SG of the path exists or not. 




-- 
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] MarcosZyk commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
+
+  private MTreeAboveSG mtree;
+
+  private static class StorageGroupManagerHolder {
+
+    private static final StorageGroupManager INSTANCE = new StorageGroupManager();
+
+    private StorageGroupManagerHolder() {}
+  }
+
+  public static StorageGroupManager getInstance() {
+    return StorageGroupManagerHolder.INSTANCE;
+  }
+
+  private StorageGroupManager() {
+    mtree = new MTreeAboveSG();
+  }
+
+  public synchronized void init() {
+    mtree.init();
+    File dir = new File(IoTDBDescriptor.getInstance().getConfig().getSchemaDir());
+    File[] sgDirs = dir.listFiles((dir1, name) -> name.startsWith(PATH_ROOT + PATH_SEPARATOR));
+    if (sgDirs != null) {
+      for (File sgDir : sgDirs) {
+        try {
+          setStorageGroup(new PartialPath(sgDir.getName()));
+        } catch (MetadataException e) {
+          logger.error("Cannot recover storage group from dir {} because", sgDir.getName(), e);
+        }
+      }
+    }
+  }
+
+  public synchronized void clear() {
+    for (SGMManager sgmManager : getAllSGMManagers()) {
+      sgmManager.clear();
+    }
+    if (mtree != null) {
+      mtree.clear();
+    }
+  }
+
+  @Override
+  public void setStorageGroup(PartialPath path) throws MetadataException {
+    // mtree.setStorageGroup ensures that the storageGroupMNode
+    IStorageGroupMNode storageGroupMNode = mtree.setStorageGroup(path);
+    SGMManager sgmManager = new SGMManager();
+    storageGroupMNode.setSGMManager(sgmManager);
+    sgmManager.init(storageGroupMNode);
+  }
+
+  @Override
+  public SGMManager getBelongedSGMManager(PartialPath path) throws MetadataException {
+    return mtree.getStorageGroupNodeByPath(path).getSGMManager();
+  }
+
+  @Override
+  public SGMManager getSGMManagerByStorageGroupPath(PartialPath path) throws MetadataException {
+    return getStorageGroupNodeByStorageGroupPath(path).getSGMManager();
+  }
+
+  @Override
+  public List<SGMManager> getInvolvedSGMManagers(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    List<SGMManager> result = new ArrayList<>();
+    for (IStorageGroupMNode storageGroupMNode :
+        mtree.getInvolvedStorageGroupNodes(pathPattern, isPrefixMatch)) {
+      result.add(storageGroupMNode.getSGMManager());
+    }
+    return result;
+  }
+
+  @Override
+  public List<SGMManager> getAllSGMManagers() {
+    List<SGMManager> result = new ArrayList<>();
+    for (IStorageGroupMNode storageGroupMNode : mtree.getAllStorageGroupNodes()) {
+      result.add(storageGroupMNode.getSGMManager());
+    }
+    return result;
+  }
+
+  @Override
+  public void deleteStorageGroup(PartialPath storageGroup) throws MetadataException {
+    mtree.deleteStorageGroup(storageGroup);
+  }
+
+  @Override
+  public boolean isStorageGroup(PartialPath path) {
+    return mtree.isStorageGroup(path);
+  }
+
+  @Override
+  public boolean checkStorageGroupByPath(PartialPath path) {
+    return mtree.checkStorageGroupByPath(path);
+  }
+
+  @Override
+  public PartialPath getBelongedStorageGroup(PartialPath path) throws StorageGroupNotSetException {
+    return mtree.getBelongedStorageGroup(path);
+  }
+
+  @Override
+  public List<PartialPath> getBelongedStorageGroups(PartialPath pathPattern)
+      throws MetadataException {
+    return mtree.getBelongedStorageGroups(pathPattern);
+  }
+
+  @Override
+  public List<PartialPath> getMatchedStorageGroups(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    return mtree.getMatchedStorageGroups(pathPattern, isPrefixMatch);
+  }
+
+  @Override
+  public List<PartialPath> getAllStorageGroupPaths() {
+    return mtree.getAllStorageGroupPaths();
+  }
+
+  @Override
+  public Map<String, List<PartialPath>> groupPathByStorageGroup(PartialPath path)
+      throws MetadataException {
+    return mtree.groupPathByStorageGroup(path);
+  }
+
+  @Override
+  public int getStorageGroupNum(PartialPath pathPattern, boolean isPrefixMatch)
+      throws MetadataException {
+    return mtree.getStorageGroupNum(pathPattern, isPrefixMatch);
+  }
+
+  @Override
+  public IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
+      throws MetadataException {
+    return mtree.getStorageGroupNodeByStorageGroupPath(path);
+  }
+
+  @Override
+  public IStorageGroupMNode getStorageGroupNodeByPath(PartialPath path) throws MetadataException {
+    return mtree.getStorageGroupNodeByPath(path);
+  }
+
+  @Override
+  public List<IStorageGroupMNode> getAllStorageGroupNodes() {
+    return mtree.getAllStorageGroupNodes();
+  }
+
+  @Override
+  public boolean isPathExist(PartialPath path) {
+    return mtree.isPathExist(path);
+  }
+
+  @Override
+  public Pair<Integer, List<SGMManager>> getNodesCountInGivenLevel(
+      PartialPath pathPattern, int level, boolean isPrefixMatch) throws MetadataException {
+    Pair<Integer, Set<IStorageGroupMNode>> resultAboveSG =
+        mtree.getNodesCountInGivenLevel(pathPattern, level, isPrefixMatch);
+    List<SGMManager> sgmManagers = new LinkedList<>();
+    for (IStorageGroupMNode storageGroupMNode : resultAboveSG.right) {
+      sgmManagers.add(storageGroupMNode.getSGMManager());
+    }
+    return new Pair<>(resultAboveSG.left, sgmManagers);
+  }
+
+  @Override
+  public Pair<List<PartialPath>, List<SGMManager>> getNodesListInGivenLevel(
+      PartialPath pathPattern, int nodeLevel, MManager.StorageGroupFilter filter)
+      throws MetadataException {
+    Pair<List<PartialPath>, Set<IStorageGroupMNode>> resultAboveSG =
+        mtree.getNodesListInGivenLevel(pathPattern, nodeLevel, filter);
+    List<SGMManager> sgmManagers = new LinkedList<>();

Review comment:
       This method is used by the old cluster to finish task of  ```getMeasurementGroupByLevel```. This may be removed in future development of new cluster.




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