You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2022/05/10 05:36:20 UTC

[iotdb] branch master updated: Fix multi level sg creation bug (#5851)

This is an automated email from the ASF dual-hosted git repository.

qiaojialin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 0f1cd23fcd Fix multi level sg creation bug (#5851)
0f1cd23fcd is described below

commit 0f1cd23fcd8cb34934958d2dd8b005aeff366751
Author: Marcos_Zyk <38...@users.noreply.github.com>
AuthorDate: Tue May 10 13:36:14 2022 +0800

    Fix multi level sg creation bug (#5851)
---
 .../client/handlers/InitRegionHandler.java         |  7 +++++
 .../confignode/persistence/ClusterSchemaInfo.java  | 21 +++++++++-----
 .../iotdb/db/metadata/mtree/MTreeAboveSG.java      | 33 ++++++++++++++++------
 .../db/metadata/schemaregion/SchemaEngine.java     |  3 +-
 .../storagegroup/IStorageGroupSchemaManager.java   | 26 ++++++-----------
 .../storagegroup/StorageGroupSchemaManager.java    | 21 ++++----------
 6 files changed, 62 insertions(+), 49 deletions(-)

diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/client/handlers/InitRegionHandler.java b/confignode/src/main/java/org/apache/iotdb/confignode/client/handlers/InitRegionHandler.java
index aab587a3ae..4be95bb347 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/client/handlers/InitRegionHandler.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/client/handlers/InitRegionHandler.java
@@ -22,6 +22,8 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.rpc.TSStatusCode;
 
 import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.BitSet;
 import java.util.concurrent.CountDownLatch;
@@ -29,6 +31,8 @@ import java.util.concurrent.CountDownLatch;
 /** Only use this handler when initialize Region to set StorageGroup */
 public class InitRegionHandler implements AsyncMethodCallback<TSStatus> {
 
+  private static final Logger LOGGER = LoggerFactory.getLogger(InitRegionHandler.class);
+
   private final int index;
   private final BitSet bitSet;
   private final CountDownLatch latch;
@@ -45,12 +49,15 @@ public class InitRegionHandler implements AsyncMethodCallback<TSStatus> {
       synchronized (bitSet) {
         bitSet.set(index);
       }
+    } else {
+      LOGGER.error(tsStatus.toString());
     }
     latch.countDown();
   }
 
   @Override
   public void onError(Exception e) {
+    LOGGER.error(e.getMessage());
     latch.countDown();
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/ClusterSchemaInfo.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/ClusterSchemaInfo.java
index a6d4fe2ca4..3a58068ef1 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/ClusterSchemaInfo.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/ClusterSchemaInfo.java
@@ -96,7 +96,9 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       mTree.setStorageGroup(partialPathName);
 
       // Set StorageGroupSchema
-      mTree.getStorageGroupNodeByPath(partialPathName).setStorageGroupSchema(storageGroupSchema);
+      mTree
+          .getStorageGroupNodeByStorageGroupPath(partialPathName)
+          .setStorageGroupSchema(storageGroupSchema);
 
       result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
 
@@ -144,7 +146,10 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
     try {
       PartialPath path = new PartialPath(req.getStorageGroup());
       if (mTree.isStorageGroupAlreadySet(path)) {
-        mTree.getStorageGroupNodeByPath(path).getStorageGroupSchema().setTTL(req.getTTL());
+        mTree
+            .getStorageGroupNodeByStorageGroupPath(path)
+            .getStorageGroupSchema()
+            .setTTL(req.getTTL());
         result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
       } else {
         result.setCode(TSStatusCode.STORAGE_GROUP_NOT_EXIST.getStatusCode());
@@ -167,7 +172,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       PartialPath path = new PartialPath(req.getStorageGroup());
       if (mTree.isStorageGroupAlreadySet(path)) {
         mTree
-            .getStorageGroupNodeByPath(path)
+            .getStorageGroupNodeByStorageGroupPath(path)
             .getStorageGroupSchema()
             .setSchemaReplicationFactor(req.getSchemaReplicationFactor());
         result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
@@ -192,7 +197,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       PartialPath path = new PartialPath(req.getStorageGroup());
       if (mTree.isStorageGroupAlreadySet(path)) {
         mTree
-            .getStorageGroupNodeByPath(path)
+            .getStorageGroupNodeByStorageGroupPath(path)
             .getStorageGroupSchema()
             .setDataReplicationFactor(req.getDataReplicationFactor());
         result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
@@ -217,7 +222,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       PartialPath path = new PartialPath(req.getStorageGroup());
       if (mTree.isStorageGroupAlreadySet(path)) {
         mTree
-            .getStorageGroupNodeByPath(path)
+            .getStorageGroupNodeByStorageGroupPath(path)
             .getStorageGroupSchema()
             .setTimePartitionInterval(req.getTimePartitionInterval());
         result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
@@ -279,7 +284,8 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
       List<PartialPath> matchedPaths = mTree.getBelongedStorageGroups(patternPath);
       for (PartialPath path : matchedPaths) {
         schemaMap.put(
-            path.getFullPath(), mTree.getStorageGroupNodeByPath(path).getStorageGroupSchema());
+            path.getFullPath(),
+            mTree.getStorageGroupNodeByStorageGroupPath(path).getStorageGroupSchema());
       }
       result.setSchemaMap(schemaMap);
       result.setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()));
@@ -322,7 +328,8 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
     storageGroupReadWriteLock.readLock().lock();
     try {
       StorageGroupMNode mNode =
-          (StorageGroupMNode) mTree.getStorageGroupNodeByPath(new PartialPath(storageGroup));
+          (StorageGroupMNode)
+              mTree.getStorageGroupNodeByStorageGroupPath(new PartialPath(storageGroup));
       switch (type) {
         case SchemaRegion:
           result = mNode.getStorageGroupSchema().getSchemaRegionGroupIds();
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
index 8c1f77abad..2c13a2139b 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeAboveSG.java
@@ -329,19 +329,36 @@ public class MTreeAboveSG {
   }
 
   /**
-   * 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
+   * E.g., root.sg is storage group given [root, sg], if the give path is not a storage group, throw
    * exception
    */
-  public IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath path)
+  public IStorageGroupMNode getStorageGroupNodeByStorageGroupPath(PartialPath storageGroupPath)
       throws MetadataException {
-    IStorageGroupMNode node = getStorageGroupNodeByPath(path);
-    if (!node.getPartialPath().equals(path)) {
-      throw new MNodeTypeMismatchException(
-          path.getFullPath(), MetadataConstant.STORAGE_GROUP_MNODE_TYPE);
+    String[] nodes = storageGroupPath.getNodes();
+    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
+      throw new IllegalPathException(storageGroupPath.getFullPath());
+    }
+    IMNode cur = root;
+    for (int i = 1; i < nodes.length - 1; i++) {
+      cur = cur.getChild(nodes[i]);
+      if (cur == null) {
+        throw new StorageGroupNotSetException(storageGroupPath.getFullPath());
+      }
+      if (cur.isStorageGroup()) {
+        throw new StorageGroupAlreadySetException(cur.getFullPath());
+      }
     }
 
-    return node;
+    cur = cur.getChild(nodes[nodes.length - 1]);
+    if (cur == null) {
+      throw new StorageGroupNotSetException(storageGroupPath.getFullPath());
+    }
+    if (cur.isStorageGroup()) {
+      return cur.getAsStorageGroupMNode();
+    } else {
+      throw new MNodeTypeMismatchException(
+          storageGroupPath.getFullPath(), MetadataConstant.STORAGE_GROUP_MNODE_TYPE);
+    }
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaEngine.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaEngine.java
index 364c60ec68..2c53f441da 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaEngine.java
@@ -207,9 +207,8 @@ public class SchemaEngine {
   private ISchemaRegion createSchemaRegionWithoutExistenceCheck(
       PartialPath storageGroup, SchemaRegionId schemaRegionId) throws MetadataException {
     ISchemaRegion schemaRegion = null;
-    this.localStorageGroupSchemaManager.ensureStorageGroup(storageGroup);
     IStorageGroupMNode storageGroupMNode =
-        this.localStorageGroupSchemaManager.getStorageGroupNodeByStorageGroupPath(storageGroup);
+        this.localStorageGroupSchemaManager.ensureStorageGroupByStorageGroupPath(storageGroup);
     switch (this.schemaRegionStoredMode) {
       case Memory:
         schemaRegion = new SchemaRegionMemoryImpl(storageGroup, schemaRegionId, storageGroupMNode);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/IStorageGroupSchemaManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/IStorageGroupSchemaManager.java
index 1a9b3cdfc4..e8611221bf 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/IStorageGroupSchemaManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/IStorageGroupSchemaManager.java
@@ -46,14 +46,6 @@ public interface IStorageGroupSchemaManager {
    */
   void setStorageGroup(PartialPath path) throws MetadataException;
 
-  /**
-   * different with LocalConfigNode.ensureStorageGroup, this method won't init storageGroup
-   * resources.
-   *
-   * @param path storage group path
-   */
-  void ensureStorageGroup(PartialPath path) throws MetadataException;
-
   /**
    * Delete storage groups of given paths from MTree. Log format: "delete_storage_group,sg1,sg2,sg3"
    */
@@ -141,21 +133,21 @@ public interface IStorageGroupSchemaManager {
    */
   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();
 
+  /**
+   * Different with LocalConfigNode.ensureStorageGroup, this method won't init storageGroup
+   * resources and the input is the target storage group path.
+   *
+   * @param storageGroup storage group path
+   */
+  IStorageGroupMNode ensureStorageGroupByStorageGroupPath(PartialPath storageGroup)
+      throws MetadataException;
+
   /**
    * Check whether the storage group of given path is set. The path may be a prefix path of some
    * storage group. Besides, the given path may be also beyond the MTreeAboveSG scope, then return
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupSchemaManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupSchemaManager.java
index 81d983abc6..fe44e56bf4 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupSchemaManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupSchemaManager.java
@@ -28,7 +28,6 @@ import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
 import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
 import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
-import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
@@ -151,17 +150,13 @@ public class StorageGroupSchemaManager implements IStorageGroupSchemaManager {
   }
 
   @Override
-  public void ensureStorageGroup(PartialPath path) throws MetadataException {
+  public IStorageGroupMNode ensureStorageGroupByStorageGroupPath(PartialPath storageGroup)
+      throws MetadataException {
     try {
-      getBelongedStorageGroup(path);
+      return mtree.getStorageGroupNodeByStorageGroupPath(storageGroup);
     } catch (StorageGroupNotSetException e) {
-      if (!config.isAutoCreateSchemaEnabled()) {
-        throw e;
-      }
-      PartialPath storageGroupPath =
-          MetaUtils.getStorageGroupPathByLevel(path, config.getDefaultStorageGroupLevel());
       try {
-        setStorageGroup(storageGroupPath);
+        setStorageGroup(storageGroup);
       } catch (StorageGroupAlreadySetException storageGroupAlreadySetException) {
         // do nothing
         // concurrent timeseries creation may result concurrent ensureStorageGroup
@@ -173,6 +168,8 @@ public class StorageGroupSchemaManager implements IStorageGroupSchemaManager {
           throw storageGroupAlreadySetException;
         }
       }
+
+      return mtree.getStorageGroupNodeByStorageGroupPath(storageGroup);
     }
   }
 
@@ -246,12 +243,6 @@ public class StorageGroupSchemaManager implements IStorageGroupSchemaManager {
     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);