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/05/05 23:35:18 UTC

[GitHub] [iotdb] cmlmakahts opened a new pull request, #5811: [IOTDB-2919] Feature/Delete storage group

cmlmakahts opened a new pull request, #5811:
URL: https://github.com/apache/iotdb/pull/5811

   ## Test in 3 confignodes 3 datanodes
   
   ```sql
   set storage group to root.sg; 
   create timeseries root.sg.d.s1 with datatype=INT32,encoding=RLE,compression=snappy;
   create timeseries root.sg.d.s2 with datatype=INT32,encoding=RLE,compression=snappy;
   create timeseries root.sg.d.s3 with datatype=INT32,encoding=RLE,compression=snappy; 
   insert into root.sg.d(time,s1,s2,s3) values(1,1,2,3); 
   insert into root.sg.d(time,s1,s2,s3) values(2,1,2,3); 
   set storage group to root.sg1; 
   create timeseries root.sg1.d.s1 with datatype=INT32,encoding=RLE,compression=snappy;
   create timeseries root.sg1.d.s2 with datatype=INT32,encoding=RLE,compression=snappy;
   create timeseries root.sg1.d.s3 with datatype=INT32,encoding=RLE,compression=snappy; 
   insert into root.sg1.d(time,s1,s2,s3) values(1,1,2,3); 
   insert into root.sg1.d(time,s1,s2,s3) values(2,1,2,3); 
   set storage group to root.sg2; 
   create timeseries root.sg2.d.s1 with datatype=INT32,encoding=RLE,compression=snappy;
   create timeseries root.sg2.d.s2 with datatype=INT32,encoding=RLE,compression=snappy;
   create timeseries root.sg2.d.s3 with datatype=INT32,encoding=RLE,compression=snappy; 
   insert into root.sg2.d(time,s1,s2,s3) values(1,1,2,3); 
   insert into root.sg2.d(time,s1,s2,s3) values(2,1,2,3); 
   ```
   
   ### 【1. Insert Data】
   
   ![image](https://user-images.githubusercontent.com/82880298/167042950-eaaa4f63-daf6-4362-87db-e6ae9e5e48cb.png)
   
   ### 【2. Delete sg】
   ```sql
   delete storage group root.sg, root.*
   ```
   ![image](https://user-images.githubusercontent.com/82880298/167042982-ac43a1be-3480-4ee3-b345-543321582924.png)
   
   ### 【3. Insert Again】
   ![image](https://user-images.githubusercontent.com/82880298/167043145-e9aa2486-8a15-4f1d-af3c-d95095711951.png)
   
   ### 【4. Delete Partial】
   ```sql
   delete storage group root.sg
   ```
   ![image](https://user-images.githubusercontent.com/82880298/167043158-4caa3c1e-25f7-4c1e-a183-e4c99e210728.png)
   


-- 
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] cmlmakahts commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
cmlmakahts commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867482070


##########
thrift-confignode/src/main/thrift/confignode.thrift:
##########
@@ -53,8 +53,8 @@ struct TSetStorageGroupReq {
   1: required TStorageGroupSchema storageGroup
 }
 
-struct TDeleteStorageGroupReq {
-  1: required string storageGroup
+struct TDeleteStorageGroupsReq {
+  1: required list<string> prefixPathList

Review Comment:
   Now a deleteStorageGroup is added.



##########
thrift-confignode/src/main/thrift/confignode.thrift:
##########
@@ -53,8 +53,8 @@ struct TSetStorageGroupReq {
   1: required TStorageGroupSchema storageGroup
 }
 
-struct TDeleteStorageGroupReq {
-  1: required string storageGroup
+struct TDeleteStorageGroupsReq {
+  1: required list<string> prefixPathList

Review Comment:
   Now _deleteStorageGroup_ is added.



-- 
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] cmlmakahts commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
cmlmakahts commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867473620


##########
thrift-confignode/src/main/thrift/confignode.thrift:
##########
@@ -161,7 +161,7 @@ service ConfigIService {
 
   common.TSStatus setStorageGroup(TSetStorageGroupReq req)
 
-  common.TSStatus deleteStorageGroup(TDeleteStorageGroupReq req)
+  common.TSStatus deleteStorageGroups(TDeleteStorageGroupsReq req)

Review Comment:
   Fixed.



##########
confignode/src/main/java/org/apache/iotdb/confignode/procedure/DeleteStorageGroupProcedure.java:
##########
@@ -0,0 +1,247 @@
+/*
+ * 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.confignode.procedure;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.exception.runtime.ThriftSerDeException;
+import org.apache.iotdb.commons.utils.StatusUtils;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils;
+import org.apache.iotdb.confignode.consensus.request.write.DeleteStorageGroupReq;
+import org.apache.iotdb.confignode.persistence.PartitionInfo;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
+import org.apache.iotdb.mpp.rpc.thrift.InternalService;
+import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq;
+import org.apache.iotdb.procedure.Procedure;
+import org.apache.iotdb.procedure.StateMachineProcedure;
+import org.apache.iotdb.procedure.exception.ProcedureException;
+import org.apache.iotdb.procedure.exception.ProcedureSuspendedException;
+import org.apache.iotdb.procedure.exception.ProcedureYieldException;
+
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class DeleteStorageGroupProcedure
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, DeleteStorageGroupState> {
+  private static final Logger LOG = LoggerFactory.getLogger(Procedure.class);
+  private static final int retryThreshold = 5;
+
+  private static boolean byPassForTest = false;
+
+  @TestOnly
+  public static void setByPassForTest(boolean byPass) {
+    byPassForTest = byPass;
+  }
+
+  private TStorageGroupSchema deleteSgSchema;
+
+  public DeleteStorageGroupProcedure() {
+    super();
+  }
+
+  public DeleteStorageGroupProcedure(TStorageGroupSchema deleteSgSchema) {
+    super();
+    this.deleteSgSchema = deleteSgSchema;
+  }
+
+  public TStorageGroupSchema getDeleteSgSchema() {
+    return deleteSgSchema;
+  }
+
+  public void setDeleteSgSchema(TStorageGroupSchema deleteSgSchema) {
+    this.deleteSgSchema = deleteSgSchema;
+  }
+
+  @Override
+  protected Flow executeFromState(ConfigNodeProcedureEnv env, DeleteStorageGroupState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    if (deleteSgSchema == null) {
+      return Flow.NO_MORE_STATE;
+    }
+    String storageGroupName = deleteSgSchema.getName();
+    List<TConsensusGroupId> dataRegionGroupIds = deleteSgSchema.getDataRegionGroupIds();
+    List<TConsensusGroupId> schemaRegionGroupIds = deleteSgSchema.getSchemaRegionGroupIds();
+    List<TRegionReplicaSet> dataRegionReplicaSets =
+        new ArrayList<>(PartitionInfo.getInstance().getRegionReplicaSets(dataRegionGroupIds));
+    List<TRegionReplicaSet> schemaRegionReplicaSets =
+        new ArrayList<>(PartitionInfo.getInstance().getRegionReplicaSets(schemaRegionGroupIds));
+    try {
+      switch (state) {
+        case DELETE_STORAGE_GROUP_PREPEARE:
+          setNextState(DeleteStorageGroupState.DELETE_DATA_REGION);
+          break;
+        case DELETE_DATA_REGION:
+          LOG.info("Delete dataRegions of {}", storageGroupName);
+          if (byPassForTest || deleteRegion(env, dataRegionReplicaSets)) {
+            setNextState(DeleteStorageGroupState.DELETE_SCHEMA_REGION);
+          }
+          break;
+        case DELETE_SCHEMA_REGION:
+          LOG.info("Delete schemaRegions of {}", storageGroupName);
+          if (byPassForTest || deleteRegion(env, schemaRegionReplicaSets)) {
+            setNextState(DeleteStorageGroupState.DELETE_CONFIG);
+          }
+          break;
+        case DELETE_CONFIG:
+          LOG.info("Delete config info of {}", storageGroupName);
+          TSStatus status = deleteConfig(env, deleteSgSchema);
+          if (verifySucceed(status)) {
+            if (byPassForTest) {
+              return Flow.NO_MORE_STATE;
+            }
+            setNextState(DeleteStorageGroupState.INVALIDATE_CACHE);
+          } else if (getCycles() > retryThreshold) {
+            setFailure(new ProcedureException("Delete config info id failed, status is " + status));
+          }
+          break;
+        case INVALIDATE_CACHE:
+          LOG.info("Invalidate cache of {}", storageGroupName);
+          invalidateCache(env, storageGroupName);
+          return Flow.NO_MORE_STATE;
+      }
+    } catch (TException | IOException e) {
+      LOG.error(
+          "Retriable error trying to delete storage group {}, state {}",
+          storageGroupName,
+          state,
+          e);
+      if (getCycles() > retryThreshold) {
+        setFailure(new ProcedureException("State stack at " + state));
+      }
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  private TSStatus deleteConfig(ConfigNodeProcedureEnv env, TStorageGroupSchema deleteSgSchema) {
+    DeleteStorageGroupReq deleteStorageGroupReq = new DeleteStorageGroupReq(deleteSgSchema);
+    return env.getConfigManager()
+        .getClusterSchemaManager()
+        .deleteStorageGroup(deleteStorageGroupReq);
+  }
+
+  private boolean deleteRegion(
+      ConfigNodeProcedureEnv env, List<TRegionReplicaSet> regionReplicaSets) throws TException {
+    for (TRegionReplicaSet dataRegionReplicaSet : regionReplicaSets) {
+      TConsensusGroupId regionId = dataRegionReplicaSet.getRegionId();
+      InternalService.Client dataNodeClient = null;
+      try {
+        dataNodeClient = env.getDataNodeClient(dataRegionReplicaSet);
+        if (dataNodeClient != null) {
+          LOG.info("Get client of {} success", regionId);

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] cmlmakahts commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
cmlmakahts commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867473620


##########
thrift-confignode/src/main/thrift/confignode.thrift:
##########
@@ -161,7 +161,7 @@ service ConfigIService {
 
   common.TSStatus setStorageGroup(TSetStorageGroupReq req)
 
-  common.TSStatus deleteStorageGroup(TDeleteStorageGroupReq req)
+  common.TSStatus deleteStorageGroups(TDeleteStorageGroupsReq req)

Review Comment:
   Updated



-- 
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] cmlmakahts commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
cmlmakahts commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867482572


##########
confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/DeleteStorageGroupsReq.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.confignode.consensus.request.write;
+
+import java.util.List;
+import java.util.Objects;
+
+public class DeleteStorageGroupsReq {

Review Comment:
   This request does not work through consensus layer. Only _deleteStorageGroupProcedure_ needs to write consensus.



##########
server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java:
##########
@@ -58,7 +58,7 @@
 public class ConfigNodeClient {
   private static final Logger logger = LoggerFactory.getLogger(ConfigNodeClient.class);
 
-  private static final int TIMEOUT_MS = 10000;
+  private static final int TIMEOUT_MS = 1000000000;

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] qiaojialin merged pull request #5811: [IOTDB-2919] Feature/Delete storage group

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


-- 
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] cmlmakahts commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
cmlmakahts commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867481986


##########
confignode/src/main/java/org/apache/iotdb/confignode/persistence/ClusterSchemaInfo.java:
##########
@@ -282,6 +283,27 @@ public StorageGroupSchemaResp getMatchedStorageGroupSchemas(GetStorageGroupReq r
     return result;
   }
 
+  /** @return All StorageGroupSchemas that matches to the specific StorageGroup patterns */
+  public Map<String, TStorageGroupSchema> getDeleteStorageGroups(List<String> rawPathList) {
+    Map<String, TStorageGroupSchema> schemaMap = new HashMap<>();
+    storageGroupReadWriteLock.readLock().lock();
+    try {
+      for (String rawPath : rawPathList) {
+        PartialPath patternPath = new PartialPath(rawPath);
+        List<PartialPath> matchedPaths = mTree.getBelongedStorageGroups(patternPath);
+        for (PartialPath path : matchedPaths) {
+          schemaMap.put(
+              path.getFullPath(), mTree.getStorageGroupNodeByPath(path).getStorageGroupSchema());
+        }
+      }

Review Comment:
   This section is same as getStorageGroup. Key of _schemaMap_ is always storageGroup



-- 
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] cmlmakahts commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
cmlmakahts commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867482240


##########
thrift-confignode/src/main/thrift/confignode.thrift:
##########
@@ -161,7 +161,7 @@ service ConfigIService {
 
   common.TSStatus setStorageGroup(TSetStorageGroupReq req)
 
-  common.TSStatus deleteStorageGroup(TDeleteStorageGroupReq req)

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] CRZbulabula commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
CRZbulabula commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867981717


##########
confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/DeleteStorageGroupReq.java:
##########
@@ -18,40 +18,40 @@
  */
 package org.apache.iotdb.confignode.consensus.request.write;
 
-import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils;
 import org.apache.iotdb.confignode.consensus.request.ConfigRequest;
 import org.apache.iotdb.confignode.consensus.request.ConfigRequestType;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 
 import java.nio.ByteBuffer;
 import java.util.Objects;
 
 public class DeleteStorageGroupReq extends ConfigRequest {
 
-  private String storageGroup;
+  private TStorageGroupSchema storageGroup;

Review Comment:
   But the DeleteStorageGroupsReq uses List<String> storageGroups, why?



-- 
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] cmlmakahts commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
cmlmakahts commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867311451


##########
thrift-confignode/src/main/thrift/confignode.thrift:
##########
@@ -53,8 +53,8 @@ struct TSetStorageGroupReq {
   1: required TStorageGroupSchema storageGroup
 }
 
-struct TDeleteStorageGroupReq {
-  1: required string storageGroup
+struct TDeleteStorageGroupsReq {
+  1: required list<string> prefixPathList

Review Comment:
   Because there is only a _deleteStorageGroups_  in the rpc interface at previous version, so I just restore it as it is in previous. Should I implement the interface of delete a single storage group besides as we discussed ?
   
   ![image](https://user-images.githubusercontent.com/82880298/167241359-d2e569ff-5237-4ad5-83cb-15aaa6dcffc7.png)
   



-- 
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] cmlmakahts commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
cmlmakahts commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867313430


##########
confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/DeleteStorageGroupReq.java:
##########
@@ -18,40 +18,40 @@
  */
 package org.apache.iotdb.confignode.consensus.request.write;
 
-import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils;
 import org.apache.iotdb.confignode.consensus.request.ConfigRequest;
 import org.apache.iotdb.confignode.consensus.request.ConfigRequestType;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 
 import java.nio.ByteBuffer;
 import java.util.Objects;
 
 public class DeleteStorageGroupReq extends ConfigRequest {
 
-  private String storageGroup;
+  private TStorageGroupSchema storageGroup;

Review Comment:
   Because we need to delete region entries in regionMap by  region ids, and  delete partition entries in partition map by storageGroupName. It is more convienent to use this parameter. And here is the same object in DeleteStorageGroupProcedure.
   
   ![image](https://user-images.githubusercontent.com/82880298/167242277-07509da2-d622-4efd-b017-a9ae9e9847c7.png)
   



-- 
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] CRZbulabula commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
CRZbulabula commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867298215


##########
confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.confignode.manager;
+
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.commons.utils.StatusUtils;
+import org.apache.iotdb.confignode.conf.ConfigNodeConf;
+import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
+import org.apache.iotdb.confignode.procedure.ConfigProcedureStore;
+import org.apache.iotdb.confignode.procedure.DeleteStorageGroupProcedure;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
+import org.apache.iotdb.procedure.Procedure;
+import org.apache.iotdb.procedure.ProcedureExecutor;
+import org.apache.iotdb.procedure.scheduler.ProcedureScheduler;
+import org.apache.iotdb.procedure.scheduler.SimpleProcedureScheduler;
+import org.apache.iotdb.procedure.store.IProcedureStore;
+import org.apache.iotdb.procedure.store.ProcedureStore;
+import org.apache.iotdb.rpc.RpcUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ProcedureManager {

Review Comment:
   Don't forget to update this doc: [ConfigNode construction](https://y8dp9fjm8f.feishu.cn/docs/doccnUzGLXivqOoNjAo8Ole3cQe)



##########
server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java:
##########
@@ -58,7 +58,7 @@
 public class ConfigNodeClient {
   private static final Logger logger = LoggerFactory.getLogger(ConfigNodeClient.class);
 
-  private static final int TIMEOUT_MS = 10000;
+  private static final int TIMEOUT_MS = 1000000000;

Review Comment:
   Why the timeout is set so long?



##########
confignode/src/main/java/org/apache/iotdb/confignode/procedure/DeleteStorageGroupProcedure.java:
##########
@@ -0,0 +1,247 @@
+/*
+ * 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.confignode.procedure;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.exception.runtime.ThriftSerDeException;
+import org.apache.iotdb.commons.utils.StatusUtils;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils;
+import org.apache.iotdb.confignode.consensus.request.write.DeleteStorageGroupReq;
+import org.apache.iotdb.confignode.persistence.PartitionInfo;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
+import org.apache.iotdb.mpp.rpc.thrift.InternalService;
+import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq;
+import org.apache.iotdb.procedure.Procedure;
+import org.apache.iotdb.procedure.StateMachineProcedure;
+import org.apache.iotdb.procedure.exception.ProcedureException;
+import org.apache.iotdb.procedure.exception.ProcedureSuspendedException;
+import org.apache.iotdb.procedure.exception.ProcedureYieldException;
+
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class DeleteStorageGroupProcedure
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, DeleteStorageGroupState> {
+  private static final Logger LOG = LoggerFactory.getLogger(Procedure.class);
+  private static final int retryThreshold = 5;
+
+  private static boolean byPassForTest = false;
+
+  @TestOnly
+  public static void setByPassForTest(boolean byPass) {
+    byPassForTest = byPass;
+  }
+
+  private TStorageGroupSchema deleteSgSchema;
+
+  public DeleteStorageGroupProcedure() {
+    super();
+  }
+
+  public DeleteStorageGroupProcedure(TStorageGroupSchema deleteSgSchema) {
+    super();
+    this.deleteSgSchema = deleteSgSchema;
+  }
+
+  public TStorageGroupSchema getDeleteSgSchema() {
+    return deleteSgSchema;
+  }
+
+  public void setDeleteSgSchema(TStorageGroupSchema deleteSgSchema) {
+    this.deleteSgSchema = deleteSgSchema;
+  }
+
+  @Override
+  protected Flow executeFromState(ConfigNodeProcedureEnv env, DeleteStorageGroupState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    if (deleteSgSchema == null) {
+      return Flow.NO_MORE_STATE;
+    }
+    String storageGroupName = deleteSgSchema.getName();
+    List<TConsensusGroupId> dataRegionGroupIds = deleteSgSchema.getDataRegionGroupIds();
+    List<TConsensusGroupId> schemaRegionGroupIds = deleteSgSchema.getSchemaRegionGroupIds();
+    List<TRegionReplicaSet> dataRegionReplicaSets =
+        new ArrayList<>(PartitionInfo.getInstance().getRegionReplicaSets(dataRegionGroupIds));
+    List<TRegionReplicaSet> schemaRegionReplicaSets =
+        new ArrayList<>(PartitionInfo.getInstance().getRegionReplicaSets(schemaRegionGroupIds));
+    try {
+      switch (state) {
+        case DELETE_STORAGE_GROUP_PREPEARE:
+          setNextState(DeleteStorageGroupState.DELETE_DATA_REGION);
+          break;

Review Comment:
   Based on the results of our discussion, it is necessary to lock ClusterSchemaInfo and PartitionInfo for ConfigNode and Regions managed by DataNode during the preparation phase. Please leave a **TODO** in this place for future optimization.



##########
confignode/src/main/java/org/apache/iotdb/confignode/procedure/DeleteStorageGroupProcedure.java:
##########
@@ -0,0 +1,247 @@
+/*
+ * 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.confignode.procedure;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.exception.runtime.ThriftSerDeException;
+import org.apache.iotdb.commons.utils.StatusUtils;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils;
+import org.apache.iotdb.confignode.consensus.request.write.DeleteStorageGroupReq;
+import org.apache.iotdb.confignode.persistence.PartitionInfo;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
+import org.apache.iotdb.mpp.rpc.thrift.InternalService;
+import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq;
+import org.apache.iotdb.procedure.Procedure;
+import org.apache.iotdb.procedure.StateMachineProcedure;
+import org.apache.iotdb.procedure.exception.ProcedureException;
+import org.apache.iotdb.procedure.exception.ProcedureSuspendedException;
+import org.apache.iotdb.procedure.exception.ProcedureYieldException;
+
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class DeleteStorageGroupProcedure
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, DeleteStorageGroupState> {
+  private static final Logger LOG = LoggerFactory.getLogger(Procedure.class);
+  private static final int retryThreshold = 5;
+
+  private static boolean byPassForTest = false;
+
+  @TestOnly
+  public static void setByPassForTest(boolean byPass) {
+    byPassForTest = byPass;
+  }
+
+  private TStorageGroupSchema deleteSgSchema;
+
+  public DeleteStorageGroupProcedure() {
+    super();
+  }
+
+  public DeleteStorageGroupProcedure(TStorageGroupSchema deleteSgSchema) {
+    super();
+    this.deleteSgSchema = deleteSgSchema;
+  }
+
+  public TStorageGroupSchema getDeleteSgSchema() {
+    return deleteSgSchema;
+  }
+
+  public void setDeleteSgSchema(TStorageGroupSchema deleteSgSchema) {
+    this.deleteSgSchema = deleteSgSchema;
+  }
+
+  @Override
+  protected Flow executeFromState(ConfigNodeProcedureEnv env, DeleteStorageGroupState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    if (deleteSgSchema == null) {
+      return Flow.NO_MORE_STATE;
+    }
+    String storageGroupName = deleteSgSchema.getName();
+    List<TConsensusGroupId> dataRegionGroupIds = deleteSgSchema.getDataRegionGroupIds();
+    List<TConsensusGroupId> schemaRegionGroupIds = deleteSgSchema.getSchemaRegionGroupIds();
+    List<TRegionReplicaSet> dataRegionReplicaSets =
+        new ArrayList<>(PartitionInfo.getInstance().getRegionReplicaSets(dataRegionGroupIds));
+    List<TRegionReplicaSet> schemaRegionReplicaSets =
+        new ArrayList<>(PartitionInfo.getInstance().getRegionReplicaSets(schemaRegionGroupIds));
+    try {
+      switch (state) {
+        case DELETE_STORAGE_GROUP_PREPEARE:
+          setNextState(DeleteStorageGroupState.DELETE_DATA_REGION);
+          break;
+        case DELETE_DATA_REGION:
+          LOG.info("Delete dataRegions of {}", storageGroupName);
+          if (byPassForTest || deleteRegion(env, dataRegionReplicaSets)) {
+            setNextState(DeleteStorageGroupState.DELETE_SCHEMA_REGION);
+          }
+          break;
+        case DELETE_SCHEMA_REGION:
+          LOG.info("Delete schemaRegions of {}", storageGroupName);
+          if (byPassForTest || deleteRegion(env, schemaRegionReplicaSets)) {
+            setNextState(DeleteStorageGroupState.DELETE_CONFIG);
+          }
+          break;
+        case DELETE_CONFIG:
+          LOG.info("Delete config info of {}", storageGroupName);
+          TSStatus status = deleteConfig(env, deleteSgSchema);
+          if (verifySucceed(status)) {
+            if (byPassForTest) {
+              return Flow.NO_MORE_STATE;
+            }
+            setNextState(DeleteStorageGroupState.INVALIDATE_CACHE);
+          } else if (getCycles() > retryThreshold) {
+            setFailure(new ProcedureException("Delete config info id failed, status is " + status));
+          }
+          break;
+        case INVALIDATE_CACHE:
+          LOG.info("Invalidate cache of {}", storageGroupName);
+          invalidateCache(env, storageGroupName);
+          return Flow.NO_MORE_STATE;
+      }
+    } catch (TException | IOException e) {
+      LOG.error(
+          "Retriable error trying to delete storage group {}, state {}",
+          storageGroupName,
+          state,
+          e);
+      if (getCycles() > retryThreshold) {
+        setFailure(new ProcedureException("State stack at " + state));
+      }
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  private TSStatus deleteConfig(ConfigNodeProcedureEnv env, TStorageGroupSchema deleteSgSchema) {
+    DeleteStorageGroupReq deleteStorageGroupReq = new DeleteStorageGroupReq(deleteSgSchema);
+    return env.getConfigManager()
+        .getClusterSchemaManager()
+        .deleteStorageGroup(deleteStorageGroupReq);
+  }
+
+  private boolean deleteRegion(
+      ConfigNodeProcedureEnv env, List<TRegionReplicaSet> regionReplicaSets) throws TException {
+    for (TRegionReplicaSet dataRegionReplicaSet : regionReplicaSets) {
+      TConsensusGroupId regionId = dataRegionReplicaSet.getRegionId();
+      InternalService.Client dataNodeClient = null;

Review Comment:
   In order to make full use of client resources, here you can use the **ClientPool** implementation provided by the  org.apache.iotdb.confignode.client package. If the current PR is not convenient to modify, please leave a **TODO**



##########
thrift-confignode/src/main/thrift/confignode.thrift:
##########
@@ -161,7 +161,7 @@ service ConfigIService {
 
   common.TSStatus setStorageGroup(TSetStorageGroupReq req)
 
-  common.TSStatus deleteStorageGroup(TDeleteStorageGroupReq req)

Review Comment:
   Please keep this interface. If current PR is not convenient to implement this interface, leave a TODO in the ConfigNodeRPCServiceProcessor.



##########
confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConf.java:
##########
@@ -114,6 +114,19 @@ public class ConfigNodeConf {
   /** The initial number of DataRegions of each StorageGroup */
   private int initialDataRegionCount = 1;
 
+  /** Procedure directory, storage procedure wal */
+  private String procedureWalDir =
+      ConfigNodeConstant.DATA_DIR + File.separator + ConfigNodeConstant.PROCEDURE_FOLDER;
+
+  /** Procedure Evict ttl */
+  private int completedEvictTTL = 800;
+
+  /** Procedure completed clean interval */
+  private int completedCleanInterval = 30;
+
+  /** Procedure core worker threads size */
+  private int coreWorkerThreadsSize = Math.max(Runtime.getRuntime().availableProcessors() / 4, 16);
+

Review Comment:
   Are these parameters user-configurable? If so, please update iotdb-confignode.properties file.



##########
thrift-confignode/src/main/thrift/confignode.thrift:
##########
@@ -161,7 +161,7 @@ service ConfigIService {
 
   common.TSStatus setStorageGroup(TSetStorageGroupReq req)
 
-  common.TSStatus deleteStorageGroup(TDeleteStorageGroupReq req)
+  common.TSStatus deleteStorageGroups(TDeleteStorageGroupsReq req)

Review Comment:
   Don't forget to update this doc: [ConfigNode RPC interfaces for DataNode](https://y8dp9fjm8f.feishu.cn/docs/doccnCp13O7QDU7O6GMvagUwEDc)



##########
confignode/src/main/java/org/apache/iotdb/confignode/persistence/ClusterSchemaInfo.java:
##########
@@ -282,6 +283,27 @@ public StorageGroupSchemaResp getMatchedStorageGroupSchemas(GetStorageGroupReq r
     return result;
   }
 
+  /** @return All StorageGroupSchemas that matches to the specific StorageGroup patterns */
+  public Map<String, TStorageGroupSchema> getDeleteStorageGroups(List<String> rawPathList) {
+    Map<String, TStorageGroupSchema> schemaMap = new HashMap<>();
+    storageGroupReadWriteLock.readLock().lock();
+    try {
+      for (String rawPath : rawPathList) {
+        PartialPath patternPath = new PartialPath(rawPath);
+        List<PartialPath> matchedPaths = mTree.getBelongedStorageGroups(patternPath);
+        for (PartialPath path : matchedPaths) {
+          schemaMap.put(
+              path.getFullPath(), mTree.getStorageGroupNodeByPath(path).getStorageGroupSchema());
+        }
+      }

Review Comment:
   If a patternPath matches multiple StorageGroups, save only the last one? Please check this block~



##########
confignode/src/main/java/org/apache/iotdb/confignode/procedure/DeleteStorageGroupProcedure.java:
##########
@@ -0,0 +1,247 @@
+/*
+ * 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.confignode.procedure;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.exception.runtime.ThriftSerDeException;
+import org.apache.iotdb.commons.utils.StatusUtils;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils;
+import org.apache.iotdb.confignode.consensus.request.write.DeleteStorageGroupReq;
+import org.apache.iotdb.confignode.persistence.PartitionInfo;
+import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
+import org.apache.iotdb.mpp.rpc.thrift.InternalService;
+import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq;
+import org.apache.iotdb.procedure.Procedure;
+import org.apache.iotdb.procedure.StateMachineProcedure;
+import org.apache.iotdb.procedure.exception.ProcedureException;
+import org.apache.iotdb.procedure.exception.ProcedureSuspendedException;
+import org.apache.iotdb.procedure.exception.ProcedureYieldException;
+
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class DeleteStorageGroupProcedure
+    extends StateMachineProcedure<ConfigNodeProcedureEnv, DeleteStorageGroupState> {
+  private static final Logger LOG = LoggerFactory.getLogger(Procedure.class);
+  private static final int retryThreshold = 5;
+
+  private static boolean byPassForTest = false;
+
+  @TestOnly
+  public static void setByPassForTest(boolean byPass) {
+    byPassForTest = byPass;
+  }
+
+  private TStorageGroupSchema deleteSgSchema;
+
+  public DeleteStorageGroupProcedure() {
+    super();
+  }
+
+  public DeleteStorageGroupProcedure(TStorageGroupSchema deleteSgSchema) {
+    super();
+    this.deleteSgSchema = deleteSgSchema;
+  }
+
+  public TStorageGroupSchema getDeleteSgSchema() {
+    return deleteSgSchema;
+  }
+
+  public void setDeleteSgSchema(TStorageGroupSchema deleteSgSchema) {
+    this.deleteSgSchema = deleteSgSchema;
+  }
+
+  @Override
+  protected Flow executeFromState(ConfigNodeProcedureEnv env, DeleteStorageGroupState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    if (deleteSgSchema == null) {
+      return Flow.NO_MORE_STATE;
+    }
+    String storageGroupName = deleteSgSchema.getName();
+    List<TConsensusGroupId> dataRegionGroupIds = deleteSgSchema.getDataRegionGroupIds();
+    List<TConsensusGroupId> schemaRegionGroupIds = deleteSgSchema.getSchemaRegionGroupIds();
+    List<TRegionReplicaSet> dataRegionReplicaSets =
+        new ArrayList<>(PartitionInfo.getInstance().getRegionReplicaSets(dataRegionGroupIds));
+    List<TRegionReplicaSet> schemaRegionReplicaSets =
+        new ArrayList<>(PartitionInfo.getInstance().getRegionReplicaSets(schemaRegionGroupIds));
+    try {
+      switch (state) {
+        case DELETE_STORAGE_GROUP_PREPEARE:
+          setNextState(DeleteStorageGroupState.DELETE_DATA_REGION);
+          break;
+        case DELETE_DATA_REGION:
+          LOG.info("Delete dataRegions of {}", storageGroupName);
+          if (byPassForTest || deleteRegion(env, dataRegionReplicaSets)) {
+            setNextState(DeleteStorageGroupState.DELETE_SCHEMA_REGION);
+          }
+          break;
+        case DELETE_SCHEMA_REGION:
+          LOG.info("Delete schemaRegions of {}", storageGroupName);
+          if (byPassForTest || deleteRegion(env, schemaRegionReplicaSets)) {
+            setNextState(DeleteStorageGroupState.DELETE_CONFIG);
+          }
+          break;
+        case DELETE_CONFIG:
+          LOG.info("Delete config info of {}", storageGroupName);
+          TSStatus status = deleteConfig(env, deleteSgSchema);
+          if (verifySucceed(status)) {
+            if (byPassForTest) {
+              return Flow.NO_MORE_STATE;
+            }
+            setNextState(DeleteStorageGroupState.INVALIDATE_CACHE);
+          } else if (getCycles() > retryThreshold) {
+            setFailure(new ProcedureException("Delete config info id failed, status is " + status));
+          }
+          break;
+        case INVALIDATE_CACHE:
+          LOG.info("Invalidate cache of {}", storageGroupName);
+          invalidateCache(env, storageGroupName);
+          return Flow.NO_MORE_STATE;
+      }
+    } catch (TException | IOException e) {
+      LOG.error(
+          "Retriable error trying to delete storage group {}, state {}",
+          storageGroupName,
+          state,
+          e);
+      if (getCycles() > retryThreshold) {
+        setFailure(new ProcedureException("State stack at " + state));
+      }
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  private TSStatus deleteConfig(ConfigNodeProcedureEnv env, TStorageGroupSchema deleteSgSchema) {
+    DeleteStorageGroupReq deleteStorageGroupReq = new DeleteStorageGroupReq(deleteSgSchema);
+    return env.getConfigManager()
+        .getClusterSchemaManager()
+        .deleteStorageGroup(deleteStorageGroupReq);
+  }
+
+  private boolean deleteRegion(
+      ConfigNodeProcedureEnv env, List<TRegionReplicaSet> regionReplicaSets) throws TException {
+    for (TRegionReplicaSet dataRegionReplicaSet : regionReplicaSets) {
+      TConsensusGroupId regionId = dataRegionReplicaSet.getRegionId();
+      InternalService.Client dataNodeClient = null;
+      try {
+        dataNodeClient = env.getDataNodeClient(dataRegionReplicaSet);
+        if (dataNodeClient != null) {
+          LOG.info("Get client of {} success", regionId);

Review Comment:
   This log can be deleted. Because we should maintain all clients by ClientPool.



##########
thrift-confignode/src/main/thrift/confignode.thrift:
##########
@@ -53,8 +53,8 @@ struct TSetStorageGroupReq {
   1: required TStorageGroupSchema storageGroup
 }
 
-struct TDeleteStorageGroupReq {
-  1: required string storageGroup
+struct TDeleteStorageGroupsReq {
+  1: required list<string> prefixPathList

Review Comment:
   Does replace TDeleteStorageGroupReq with TDeleteStorageGroupsReq means the user can only currently use "delete StorageGroups"?



##########
confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/DeleteStorageGroupsReq.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.confignode.consensus.request.write;
+
+import java.util.List;
+import java.util.Objects;
+
+public class DeleteStorageGroupsReq {

Review Comment:
   Should extends ConfigRequest.



##########
confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/DeleteStorageGroupReq.java:
##########
@@ -18,40 +18,40 @@
  */
 package org.apache.iotdb.confignode.consensus.request.write;
 
-import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils;
 import org.apache.iotdb.confignode.consensus.request.ConfigRequest;
 import org.apache.iotdb.confignode.consensus.request.ConfigRequestType;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 
 import java.nio.ByteBuffer;
 import java.util.Objects;
 
 public class DeleteStorageGroupReq extends ConfigRequest {
 
-  private String storageGroup;
+  private TStorageGroupSchema storageGroup;

Review Comment:
   Why change this parameter? Isn't it enough to pass the StorageGroupName for the deletion request?



-- 
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] cmlmakahts commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
cmlmakahts commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r867311451


##########
thrift-confignode/src/main/thrift/confignode.thrift:
##########
@@ -53,8 +53,8 @@ struct TSetStorageGroupReq {
   1: required TStorageGroupSchema storageGroup
 }
 
-struct TDeleteStorageGroupReq {
-  1: required string storageGroup
+struct TDeleteStorageGroupsReq {
+  1: required list<string> prefixPathList

Review Comment:
   Because there is only _deleteStorageGroups_  in the rpc interface at previous version, so I just restore it as it is in previous. Should I implement the interface of delete a single storage group besides as we discussed ?
   
   ![image](https://user-images.githubusercontent.com/82880298/167241359-d2e569ff-5237-4ad5-83cb-15aaa6dcffc7.png)
   



-- 
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 diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r871501698


##########
confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConstant.java:
##########
@@ -42,6 +42,7 @@ public class ConfigNodeConstant {
   public static final String DATA_DIR = "data";
   public static final String CONF_DIR = "conf";
   public static final String CONSENSUS_FOLDER = "consensus";
+  public static final String PROCEDURE_FOLDER = "proc";

Review Comment:
   How about storing this under the system folder, reducing a parameter for users
   
   Besides, we could rename this to procedure, proc looks like process...



##########
confignode/src/assembly/resources/conf/iotdb-confignode.properties:
##########
@@ -194,4 +194,33 @@ target_confignode=0.0.0.0:22277
 # consensus_dir=data\\consensus
 # For Linux platform
 # If its prefix is "/", then the path is absolute. Otherwise, it is relative.
-# consensus_dir=data/consensus
\ No newline at end of file
+# consensus_dir=data/consensus
+
+# procedure wal dir
+# If this property is unset, system will save the data in the default relative path directory under the confignode folder(i.e., %CONFIGNODE_HOME%/data/consensus).
+# If it is absolute, system will save the data in exact location it points to.
+# If it is relative, system will save the data in the relative path directory it indicates under the confignode folder.
+# Note: If data_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path.
+# For windows platform
+# If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative.
+# proc_wal_dir=data\\proc
+# For Linux platform
+# If its prefix is "/", then the path is absolute. Otherwise, it is relative.
+# proc_wal_dir=data/proc
+####################
+### Procedure Configuration
+####################
+
+# Default number of worker thread count
+# Datatype: int
+#workerthreads_core_size=4
+
+# Default time interval of completed procedure cleaner work in, time unit is second
+# Datatype: int
+#completed_clean_interval=30

Review Comment:
   ```suggestion
   #procedure_completed_clean_interval=30
   ```



##########
confignode/src/assembly/resources/conf/iotdb-confignode.properties:
##########
@@ -194,4 +194,33 @@ target_confignode=0.0.0.0:22277
 # consensus_dir=data\\consensus
 # For Linux platform
 # If its prefix is "/", then the path is absolute. Otherwise, it is relative.
-# consensus_dir=data/consensus
\ No newline at end of file
+# consensus_dir=data/consensus
+
+# procedure wal dir
+# If this property is unset, system will save the data in the default relative path directory under the confignode folder(i.e., %CONFIGNODE_HOME%/data/consensus).
+# If it is absolute, system will save the data in exact location it points to.
+# If it is relative, system will save the data in the relative path directory it indicates under the confignode folder.
+# Note: If data_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path.
+# For windows platform
+# If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative.
+# proc_wal_dir=data\\proc
+# For Linux platform
+# If its prefix is "/", then the path is absolute. Otherwise, it is relative.
+# proc_wal_dir=data/proc
+####################
+### Procedure Configuration
+####################
+
+# Default number of worker thread count
+# Datatype: int
+#workerthreads_core_size=4

Review Comment:
   ```suggestion
   #procedure_workerthreads_core_size=4
   ```



##########
confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/DeleteStorageGroupsReq.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.confignode.consensus.request.write;
+
+import java.util.List;
+import java.util.Objects;
+
+public class DeleteStorageGroupsReq {

Review Comment:
   Better to distinguish between requests that send to consensus and those sent to the procedure.
   
   We could rename this to DeleteStorageGroupsProcedureParameter or so



##########
confignode/src/assembly/resources/conf/iotdb-confignode.properties:
##########
@@ -194,4 +194,33 @@ target_confignode=0.0.0.0:22277
 # consensus_dir=data\\consensus
 # For Linux platform
 # If its prefix is "/", then the path is absolute. Otherwise, it is relative.
-# consensus_dir=data/consensus
\ No newline at end of file
+# consensus_dir=data/consensus
+
+# procedure wal dir
+# If this property is unset, system will save the data in the default relative path directory under the confignode folder(i.e., %CONFIGNODE_HOME%/data/consensus).
+# If it is absolute, system will save the data in exact location it points to.
+# If it is relative, system will save the data in the relative path directory it indicates under the confignode folder.
+# Note: If data_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path.
+# For windows platform
+# If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative.
+# proc_wal_dir=data\\proc
+# For Linux platform
+# If its prefix is "/", then the path is absolute. Otherwise, it is relative.
+# proc_wal_dir=data/proc
+####################
+### Procedure Configuration
+####################
+
+# Default number of worker thread count
+# Datatype: int
+#workerthreads_core_size=4
+
+# Default time interval of completed procedure cleaner work in, time unit is second
+# Datatype: int
+#completed_clean_interval=30
+
+
+# Default ttl of completed procedure, time unit is second
+# Datatype: int
+#completed_evict_ttl=800

Review Comment:
   ```suggestion
   #procedure_completed_evict_ttl=800
   ```



-- 
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] cmlmakahts commented on a diff in pull request #5811: [IOTDB-2919] Feature/Delete storage group

Posted by GitBox <gi...@apache.org>.
cmlmakahts commented on code in PR #5811:
URL: https://github.com/apache/iotdb/pull/5811#discussion_r872041034


##########
confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/DeleteStorageGroupsReq.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.confignode.consensus.request.write;
+
+import java.util.List;
+import java.util.Objects;
+
+public class DeleteStorageGroupsReq {

Review Comment:
   Now it is replaced by a _List_.



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