You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ne...@apache.org on 2023/04/20 03:29:07 UTC

[iotdb] branch master updated: [IOTDB-5773] fix set space quota problem (#9596)

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

neuyilan 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 98adb10391 [IOTDB-5773] fix set space quota problem (#9596)
98adb10391 is described below

commit 98adb10391cd2afb6933c89797635992fafc2f83
Author: 任宇华 <79...@users.noreply.github.com>
AuthorDate: Thu Apr 20 11:29:02 2023 +0800

    [IOTDB-5773] fix set space quota problem (#9596)
---
 .../confignode/manager/ClusterQuotaManager.java    |  18 +-
 .../confignode/persistence/quota/QuotaInfo.java    |  36 +-
 .../iotdb/it/env/cluster/MppCommonConfig.java      |   6 +
 .../it/env/cluster/MppSharedCommonConfig.java      |   7 +
 .../iotdb/it/env/remote/RemoteCommonConfig.java    |   5 +
 .../org/apache/iotdb/itbase/env/CommonConfig.java  |   2 +
 .../iotdb/db/it/quotas/IoTDBSpaceQuotaIT.java      | 535 +++++++++++++++++++++
 .../apache/iotdb/commons/conf/IoTDBConstant.java   |  32 +-
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |   2 +-
 .../config/sys/quota/ShowSpaceQuotaTask.java       |   7 +-
 .../config/sys/quota/ShowThrottleQuotaTask.java    |  34 +-
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       |  88 ++--
 12 files changed, 695 insertions(+), 77 deletions(-)

diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterQuotaManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterQuotaManager.java
index 580aeae7f8..6dfbec20a3 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterQuotaManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterQuotaManager.java
@@ -24,6 +24,7 @@ import org.apache.iotdb.common.rpc.thrift.TSetSpaceQuotaReq;
 import org.apache.iotdb.common.rpc.thrift.TSetThrottleQuotaReq;
 import org.apache.iotdb.common.rpc.thrift.TSpaceQuota;
 import org.apache.iotdb.common.rpc.thrift.TThrottleQuota;
+import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.confignode.client.DataNodeRequestType;
 import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool;
 import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler;
@@ -76,6 +77,7 @@ public class ClusterQuotaManager {
           TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode(),
           "The used quota exceeds the preset quota. Please set a larger value.");
     }
+    // TODO: Datanode failed to receive rpc
     ConsensusWriteResponse response =
         configManager
             .getConsensusManager()
@@ -107,9 +109,19 @@ public class ClusterQuotaManager {
     for (String database : req.getDatabase()) {
       if (quotaInfo.getSpaceQuotaLimit().containsKey(database)) {
         TSpaceQuota spaceQuota = quotaInfo.getSpaceQuotaUsage().get(database);
-        if (spaceQuota.getDeviceNum() > req.getSpaceLimit().getDeviceNum()
-            || spaceQuota.getTimeserieNum() > req.getSpaceLimit().getTimeserieNum()
-            || spaceQuota.getDiskSize() > req.getSpaceLimit().getDiskSize()) {
+        if (req.getSpaceLimit().getDeviceNum() != IoTDBConstant.UNLIMITED_VALUE
+            && req.getSpaceLimit().getDeviceNum() != IoTDBConstant.DEFAULT_VALUE
+            && spaceQuota.getDeviceNum() > req.getSpaceLimit().getDeviceNum()) {
+          return false;
+        }
+        if (req.getSpaceLimit().getTimeserieNum() != IoTDBConstant.UNLIMITED_VALUE
+            && req.getSpaceLimit().getTimeserieNum() != IoTDBConstant.DEFAULT_VALUE
+            && spaceQuota.getTimeserieNum() > req.getSpaceLimit().getTimeserieNum()) {
+          return false;
+        }
+        if (req.getSpaceLimit().getDiskSize() != IoTDBConstant.UNLIMITED_VALUE
+            && req.getSpaceLimit().getDiskSize() != IoTDBConstant.DEFAULT_VALUE
+            && spaceQuota.getDiskSize() > req.getSpaceLimit().getDiskSize()) {
           return false;
         }
       }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/quota/QuotaInfo.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/quota/QuotaInfo.java
index 7ca7d67d3d..645ee94289 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/quota/QuotaInfo.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/quota/QuotaInfo.java
@@ -24,6 +24,7 @@ import org.apache.iotdb.common.rpc.thrift.TSpaceQuota;
 import org.apache.iotdb.common.rpc.thrift.TThrottleQuota;
 import org.apache.iotdb.common.rpc.thrift.TTimedQuota;
 import org.apache.iotdb.common.rpc.thrift.ThrottleType;
+import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.snapshot.SnapshotProcessor;
 import org.apache.iotdb.confignode.consensus.request.write.quota.SetSpaceQuotaPlan;
 import org.apache.iotdb.confignode.consensus.request.write.quota.SetThrottleQuotaPlan;
@@ -64,26 +65,26 @@ public class QuotaInfo implements SnapshotProcessor {
   public TSStatus setSpaceQuota(SetSpaceQuotaPlan setSpaceQuotaPlan) {
     for (String database : setSpaceQuotaPlan.getPrefixPathList()) {
       TSpaceQuota spaceQuota = setSpaceQuotaPlan.getSpaceLimit();
-      // “0” means that the user has not reset the value of the space quota type
+      // “DEFAULT_VALUE” means that the user has not reset the value of the space quota type
       // So the old values are still used
       if (spaceQuotaLimit.containsKey(database)) {
-        if (spaceQuota.getDeviceNum() == 0) {
+        if (spaceQuota.getDeviceNum() == IoTDBConstant.DEFAULT_VALUE) {
           spaceQuota.setDeviceNum(spaceQuotaLimit.get(database).getDeviceNum());
         }
-        if (spaceQuota.getTimeserieNum() == 0) {
+        if (spaceQuota.getTimeserieNum() == IoTDBConstant.DEFAULT_VALUE) {
           spaceQuota.setTimeserieNum(spaceQuotaLimit.get(database).getTimeserieNum());
         }
-        if (spaceQuota.getDiskSize() == 0) {
+        if (spaceQuota.getDiskSize() == IoTDBConstant.DEFAULT_VALUE) {
           spaceQuota.setDiskSize(spaceQuotaLimit.get(database).getDiskSize());
         }
-        if (spaceQuota.getDeviceNum() == -1) {
-          spaceQuota.setDeviceNum(0);
+        if (spaceQuota.getDeviceNum() == IoTDBConstant.UNLIMITED_VALUE) {
+          spaceQuota.setDeviceNum(IoTDBConstant.DEFAULT_VALUE);
         }
-        if (spaceQuota.getTimeserieNum() == -1) {
-          spaceQuota.setTimeserieNum(0);
+        if (spaceQuota.getTimeserieNum() == IoTDBConstant.UNLIMITED_VALUE) {
+          spaceQuota.setTimeserieNum(IoTDBConstant.DEFAULT_VALUE);
         }
-        if (spaceQuota.getDiskSize() == -1) {
-          spaceQuota.setDiskSize(0);
+        if (spaceQuota.getDiskSize() == IoTDBConstant.UNLIMITED_VALUE) {
+          spaceQuota.setDiskSize(IoTDBConstant.DEFAULT_VALUE);
         }
       }
       if (!spaceQuotaUsage.containsKey(database)) {
@@ -99,16 +100,18 @@ public class QuotaInfo implements SnapshotProcessor {
     String userName = setThrottleQuotaPlan.getUserName();
     if (throttleQuotaLimit.containsKey(setThrottleQuotaPlan.getUserName())) {
       // about memory
-      if (setThrottleQuotaPlan.getThrottleQuota().getMemLimit() == -1) {
-        throttleQuotaLimit.get(userName).setMemLimit(0);
-      } else if (setThrottleQuotaPlan.getThrottleQuota().getMemLimit() != 0) {
+      if (setThrottleQuotaPlan.getThrottleQuota().getMemLimit() == IoTDBConstant.UNLIMITED_VALUE) {
+        throttleQuotaLimit.get(userName).setMemLimit(IoTDBConstant.DEFAULT_VALUE);
+      } else if (setThrottleQuotaPlan.getThrottleQuota().getMemLimit()
+          != IoTDBConstant.DEFAULT_VALUE) {
         throttleQuotaLimit.get(userName).setMemLimit(throttleQuota.getMemLimit());
       }
 
       // about cpu
-      if (setThrottleQuotaPlan.getThrottleQuota().getCpuLimit() == -1) {
-        throttleQuotaLimit.get(userName).setCpuLimit(0);
-      } else if (setThrottleQuotaPlan.getThrottleQuota().getCpuLimit() != 0) {
+      if (setThrottleQuotaPlan.getThrottleQuota().getCpuLimit() == IoTDBConstant.UNLIMITED_VALUE) {
+        throttleQuotaLimit.get(userName).setCpuLimit(IoTDBConstant.DEFAULT_VALUE);
+      } else if (setThrottleQuotaPlan.getThrottleQuota().getCpuLimit()
+          != IoTDBConstant.DEFAULT_VALUE) {
         throttleQuotaLimit.get(userName).setCpuLimit(throttleQuota.getCpuLimit());
       }
       if (!throttleQuota.getThrottleLimit().isEmpty()) {
@@ -218,6 +221,7 @@ public class QuotaInfo implements SnapshotProcessor {
       spaceQuota.setTimeserieNum(ReadWriteIOUtils.readLong(fileInputStream));
       spaceQuota.setDiskSize(ReadWriteIOUtils.readLong(fileInputStream));
       spaceQuotaLimit.put(path, spaceQuota);
+      spaceQuotaUsage.put(path, new TSpaceQuota());
       size--;
     }
   }
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
index fe9f3a6767..6e3c8d365d 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
@@ -340,4 +340,10 @@ public class MppCommonConfig extends MppBaseConfig implements CommonConfig {
     setProperty("write_memory_proportion", writeMemoryProportion);
     return this;
   }
+
+  @Override
+  public CommonConfig setQuotaEnable(boolean quotaEnable) {
+    setProperty("quota_enable", String.valueOf(quotaEnable));
+    return this;
+  }
 }
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
index 275ed1f8f8..199274140e 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
@@ -350,4 +350,11 @@ public class MppSharedCommonConfig implements CommonConfig {
     cnConfig.setWriteMemoryProportion(writeMemoryProportion);
     return this;
   }
+
+  @Override
+  public CommonConfig setQuotaEnable(boolean quotaEnable) {
+    dnConfig.setQuotaEnable(quotaEnable);
+    cnConfig.setQuotaEnable(quotaEnable);
+    return this;
+  }
 }
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
index 526df5dbab..e6af4826a8 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
@@ -251,4 +251,9 @@ public class RemoteCommonConfig implements CommonConfig {
   public CommonConfig setWriteMemoryProportion(String writeMemoryProportion) {
     return this;
   }
+
+  @Override
+  public CommonConfig setQuotaEnable(boolean quotaEnable) {
+    return this;
+  }
 }
diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
index 4792b8b3e7..bb39edba20 100644
--- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
@@ -112,4 +112,6 @@ public interface CommonConfig {
   CommonConfig setSchemaMemoryAllocate(String schemaMemoryAllocate);
 
   CommonConfig setWriteMemoryProportion(String writeMemoryProportion);
+
+  CommonConfig setQuotaEnable(boolean quotaEnable);
 }
diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/quotas/IoTDBSpaceQuotaIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/quotas/IoTDBSpaceQuotaIT.java
new file mode 100644
index 0000000000..4071689964
--- /dev/null
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/quotas/IoTDBSpaceQuotaIT.java
@@ -0,0 +1,535 @@
+/*
+ * 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.it.quotas;
+
+import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.it.env.EnvFactory;
+import org.apache.iotdb.it.framework.IoTDBTestRunner;
+import org.apache.iotdb.itbase.category.ClusterIT;
+import org.apache.iotdb.itbase.category.LocalStandaloneIT;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(IoTDBTestRunner.class)
+@Category({LocalStandaloneIT.class, ClusterIT.class})
+public class IoTDBSpaceQuotaIT {
+
+  @Before
+  public void setUp() throws Exception {
+    EnvFactory.getEnv().getConfig().getDataNodeCommonConfig().setQuotaEnable(true);
+    EnvFactory.getEnv().initClusterEnvironment();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    EnvFactory.getEnv().getConfig().getDataNodeCommonConfig().setQuotaEnable(false);
+    EnvFactory.getEnv().cleanClusterEnvironment();
+  }
+
+  @Test
+  public void timeseriesNumExceedTest() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota timeseries=3 on root.sg0");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status1 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status2 with datatype=BOOLEAN,encoding=PLAIN;");
+      Thread.sleep(ConfigNodeDescriptor.getInstance().getConf().getHeartbeatIntervalInMs() * 2);
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status3 with datatype=BOOLEAN,encoding=PLAIN;");
+    } catch (SQLException | InterruptedException throwables) {
+      Assert.assertEquals(
+          "1700: The number of timeSeries has reached the upper limit", throwables.getMessage());
+    }
+  }
+
+  @Test
+  public void devicesNumExceedTest() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota devices=3 on root.sg0");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf02.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf03.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      Thread.sleep(ConfigNodeDescriptor.getInstance().getConf().getHeartbeatIntervalInMs() * 2);
+      adminStmt.execute(
+          "create timeseries root.sg0.wf04.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+    } catch (SQLException | InterruptedException throwables) {
+      Assert.assertEquals(
+          "1700: The number of devices has reached the upper limit", throwables.getMessage());
+    }
+  }
+
+  @Test
+  public void setSpaceQuotaTest0() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota timeseries=3 on root.sg0");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status1 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status2 with datatype=BOOLEAN,encoding=PLAIN;");
+      Thread.sleep(ConfigNodeDescriptor.getInstance().getConf().getHeartbeatIntervalInMs() * 2);
+      adminStmt.execute("set space quota timeseries=4 on root.sg0");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status3 with datatype=BOOLEAN,encoding=PLAIN;");
+    } catch (SQLException | InterruptedException throwables) {
+      Assert.fail(throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota devices=3 on root.sg1");
+      adminStmt.execute(
+          "create timeseries root.sg1.wf01.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg1.wf02.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg1.wf03.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      Thread.sleep(ConfigNodeDescriptor.getInstance().getConf().getHeartbeatIntervalInMs() * 2);
+      adminStmt.execute("set space quota devices=4 on root.sg1");
+      adminStmt.execute(
+          "create timeseries root.sg1.wf04.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+    } catch (SQLException | InterruptedException throwables) {
+      Assert.fail(throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota disk=100M on root.sg2");
+      adminStmt.execute("set space quota disk=200M on root.sg2");
+    } catch (SQLException throwables) {
+      Assert.fail(throwables.getMessage());
+    }
+  }
+
+  @Test
+  public void setSpaceQuotaTest1() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota timeseries=3 on root.sg0");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status1 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status2 with datatype=BOOLEAN,encoding=PLAIN;");
+      Thread.sleep(ConfigNodeDescriptor.getInstance().getConf().getHeartbeatIntervalInMs() * 2);
+      adminStmt.execute("set space quota timeseries=2 on root.sg0");
+    } catch (SQLException | InterruptedException throwables) {
+      Assert.assertEquals(
+          "301: The used quota exceeds the preset quota. Please set a larger value.",
+          throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota devices=3 on root.sg1");
+      adminStmt.execute(
+          "create timeseries root.sg1.wf01.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg1.wf02.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg1.wf03.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      Thread.sleep(ConfigNodeDescriptor.getInstance().getConf().getHeartbeatIntervalInMs() * 2);
+      adminStmt.execute("set space quota devices=2 on root.sg1");
+    } catch (SQLException | InterruptedException throwables) {
+      Assert.assertEquals(
+          "301: The used quota exceeds the preset quota. Please set a larger value.",
+          throwables.getMessage());
+    }
+  }
+
+  @Test
+  public void setSpaceQuotaTest2() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota timeseries=0 on root.sg0");
+    } catch (SQLException throwables) {
+      Assert.assertEquals(
+          "701: Please set the number of timeseries greater than 0", throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota devices=0 on root.sg1");
+    } catch (SQLException throwables) {
+      Assert.assertEquals(
+          "701: Please set the number of devices greater than 0", throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota disk=0M on root.sg2");
+    } catch (SQLException throwables) {
+      Assert.assertEquals("701: Please set the disk size greater than 0", throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota timeseries=0,devices=0,disk=0M on root.sg3");
+    } catch (SQLException throwables) {
+      Assert.assertEquals(
+          "701: Please set the number of devices greater than 0", throwables.getMessage());
+    }
+  }
+
+  @Test
+  public void setSpaceQuotaTest3() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota timeseries=2 on root.sg0");
+      adminStmt.execute("set space quota timeseries='unlimited' on root.sg0");
+    } catch (SQLException throwables) {
+      Assert.fail(throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota devices=2 on root.sg1");
+      adminStmt.execute("set space quota devices='unlimited' on root.sg1");
+    } catch (SQLException throwables) {
+      Assert.fail(throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota disk=10M on root.sg2");
+      adminStmt.execute("set space quota disk='unlimited' on root.sg2");
+    } catch (SQLException throwables) {
+      Assert.fail(throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota timeseries=2,devices=2,disk=10M on root.sg3");
+      adminStmt.execute(
+          "set space quota timeseries='unlimited',devices='unlimited',disk='unlimited' on root.sg3");
+    } catch (SQLException throwables) {
+      Assert.fail(throwables.getMessage());
+    }
+  }
+
+  @Test
+  public void setSpaceQuotaTest4() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota devices=3,timeseries=5 on root.sg0;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt02.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt03.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt04.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status1 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status2 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status3 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute("set space quota devices=4,timeseries=6 on root.sg0;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt05.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status5 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute("set space quota devices='unlimited',timeseries='unlimited' on root.sg0;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt06.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status6 with datatype=BOOLEAN,encoding=PLAIN;");
+    } catch (SQLException throwables) {
+      Assert.fail(throwables.getMessage());
+    }
+  }
+
+  @Test
+  public void setSpaceQuotaTest5() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota devices=3,timeseries=5,disk='100M' on root.sg0;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt02.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt02.status1 with datatype=BOOLEAN,encoding=PLAIN;");
+      Thread.sleep(2000);
+      ResultSet resultSet1 = adminStmt.executeQuery("show space quota root.sg0;");
+      String ans1 =
+          "root.sg0,diskSize,0.09765625G,0.0G"
+              + ",\n"
+              + "root.sg0,deviceNum,3,2"
+              + ",\n"
+              + "root.sg0,timeSeriesNum,5,3"
+              + ",\n";
+      validateResultSet(resultSet1, ans1);
+    } catch (InterruptedException | SQLException e) {
+      Assert.fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void setSpaceQuotaTest6() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota disk='100K' on root.sg0;");
+    } catch (SQLException e) {
+      Assert.assertEquals(
+          "701: When setting the disk size, the unit is incorrect. Please use 'M', 'G', 'P', 'T' as the unit",
+          e.getMessage());
+    }
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota disk='100M' on root.sg0;");
+      ResultSet resultSet1 = adminStmt.executeQuery("show space quota root.sg0;");
+      String ans1 =
+          "root.sg0,diskSize,0.09765625G,0.0G"
+              + ",\n"
+              + "root.sg0,deviceNum,unlimited,0"
+              + ",\n"
+              + "root.sg0,timeSeriesNum,unlimited,0"
+              + ",\n";
+      validateResultSet(resultSet1, ans1);
+    } catch (SQLException e) {
+      Assert.fail(e.getMessage());
+    }
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota disk='100G' on root.sg1;");
+      ResultSet resultSet1 = adminStmt.executeQuery("show space quota root.sg1;");
+      String ans1 =
+          "root.sg1,diskSize,100.0G,0.0G"
+              + ",\n"
+              + "root.sg1,deviceNum,unlimited,0"
+              + ",\n"
+              + "root.sg1,timeSeriesNum,unlimited,0"
+              + ",\n";
+      validateResultSet(resultSet1, ans1);
+    } catch (SQLException e) {
+      Assert.fail(e.getMessage());
+    }
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota disk='100T' on root.sg2;");
+      ResultSet resultSet1 = adminStmt.executeQuery("show space quota root.sg2;");
+      String ans1 =
+          "root.sg2,diskSize,102400.0G,0.0G"
+              + ",\n"
+              + "root.sg2,deviceNum,unlimited,0"
+              + ",\n"
+              + "root.sg2,timeSeriesNum,unlimited,0"
+              + ",\n";
+      validateResultSet(resultSet1, ans1);
+    } catch (SQLException e) {
+      Assert.fail(e.getMessage());
+    }
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota disk='100P' on root.sg3;");
+      ResultSet resultSet1 = adminStmt.executeQuery("show space quota root.sg3;");
+      String ans1 =
+          "root.sg3,diskSize,1.048576E8G,0.0G"
+              + ",\n"
+              + "root.sg3,deviceNum,unlimited,0"
+              + ",\n"
+              + "root.sg3,timeSeriesNum,unlimited,0"
+              + ",\n";
+      validateResultSet(resultSet1, ans1);
+    } catch (SQLException e) {
+      Assert.fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void setSpaceQuotaTest7() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute(
+          "create timeseries root.sg0.wf01.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf02.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf03.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf04.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute(
+          "create timeseries root.sg0.wf05.wt01.status0 with datatype=BOOLEAN,encoding=PLAIN;");
+      adminStmt.execute("set space quota devices=5 on root.sg0;");
+      adminStmt.execute("set space quota disk='5g' on root.sg0;");
+      adminStmt.execute("set space quota devices=10 on root.sg0;");
+    } catch (SQLException e) {
+      Assert.fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void setSpaceQuotaTest8() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota timeseries=-5 on root.sg0");
+    } catch (SQLException throwables) {
+      Assert.assertEquals(
+          "701: Please set the number of timeseries greater than 0", throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota devices=-6 on root.sg1");
+    } catch (SQLException throwables) {
+      Assert.assertEquals(
+          "701: Please set the number of devices greater than 0", throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota disk='-7M' on root.sg2");
+    } catch (SQLException throwables) {
+      Assert.assertEquals("701: Please set the disk size greater than 0", throwables.getMessage());
+    }
+
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota timeseries=-5,devices=6,disk=0M on root.sg3");
+    } catch (SQLException throwables) {
+      Assert.assertEquals(
+          "701: Please set the number of timeseries greater than 0", throwables.getMessage());
+    }
+  }
+
+  @Test
+  public void setSpaceQuotaTest9() {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota timeserie=5 on root.sg0");
+    } catch (SQLException throwables) {
+      Assert.assertEquals("701: Wrong space quota type: timeserie", throwables.getMessage());
+    }
+  }
+
+  @Test
+  public void showSpaceQuotaTest0() throws SQLException {
+    IoTDBDescriptor.getInstance().getConfig().setQuotaEnable(true);
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota devices=3,timeseries=5,disk='100M' on root.sg1,root.sg2;");
+      ResultSet resultSet1 = adminStmt.executeQuery("show space quota;");
+      String ans1 =
+          "root.sg1,diskSize,0.09765625G,0.0G"
+              + ",\n"
+              + "root.sg1,deviceNum,3,0"
+              + ",\n"
+              + "root.sg1,timeSeriesNum,5,0"
+              + ",\n"
+              + "root.sg2,diskSize,0.09765625G,0.0G"
+              + ",\n"
+              + "root.sg2,deviceNum,3,0"
+              + ",\n"
+              + "root.sg2,timeSeriesNum,5,0"
+              + ",\n";
+      validateResultSet(resultSet1, ans1);
+      adminStmt.execute(
+          "set space quota devices='unlimited',timeseries='unlimited',disk='unlimited' on root.sg1;");
+      ResultSet resultSet2 = adminStmt.executeQuery("show space quota root.sg1;");
+      String ans2 =
+          "root.sg1,diskSize,unlimited,0.0G"
+              + ",\n"
+              + "root.sg1,deviceNum,unlimited,0"
+              + ",\n"
+              + "root.sg1,timeSeriesNum,unlimited,0"
+              + ",\n";
+      validateResultSet(resultSet2, ans2);
+      IoTDBDescriptor.getInstance().getConfig().setQuotaEnable(false);
+    }
+  }
+
+  @Test
+  public void showSpaceQuotaTest1() throws SQLException {
+    try (Connection adminCon = EnvFactory.getEnv().getConnection();
+        Statement adminStmt = adminCon.createStatement()) {
+      adminStmt.execute("set space quota devices=3,timeseries=5,disk='100M' on root.sg4,root.sg5;");
+      ResultSet resultSet1 = adminStmt.executeQuery("show space quota root.sg4,root.sg5;");
+      String ans1 =
+          "root.sg4,diskSize,0.09765625G,0.0G"
+              + ",\n"
+              + "root.sg4,deviceNum,3,0"
+              + ",\n"
+              + "root.sg4,timeSeriesNum,5,0"
+              + ",\n"
+              + "root.sg5,diskSize,0.09765625G,0.0G"
+              + ",\n"
+              + "root.sg5,deviceNum,3,0"
+              + ",\n"
+              + "root.sg5,timeSeriesNum,5,0"
+              + ",\n";
+      validateResultSet(resultSet1, ans1);
+    }
+  }
+
+  private void validateResultSet(ResultSet set, String ans) throws SQLException {
+    try {
+      StringBuilder builder = new StringBuilder();
+      ResultSetMetaData metaData = set.getMetaData();
+      int colNum = metaData.getColumnCount();
+      while (set.next()) {
+        for (int i = 1; i <= colNum; i++) {
+          builder.append(set.getString(i)).append(",");
+        }
+        builder.append("\n");
+      }
+      String result = builder.toString();
+      assertEquals(ans.length(), result.length());
+      List<String> ansLines = Arrays.asList(ans.split("\n"));
+      List<String> resultLines = Arrays.asList(result.split("\n"));
+      assertEquals(ansLines.size(), resultLines.size());
+      for (String resultLine : resultLines) {
+        assertTrue(ansLines.contains(resultLine));
+      }
+    } finally {
+      set.close();
+    }
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
index 078efc15d2..344bf05893 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
@@ -103,9 +103,11 @@ public class IoTDBConstant {
   public static final String IOTDB_THREADPOOL_PACKAGE = "org.apache.iotdb.threadpool";
   public static final String JMX_TYPE = "type";
 
-  public static final long GB = 1024 * 1024 * 1024L;
-  public static final long MB = 1024 * 1024L;
-  public static final long KB = 1024L;
+  public static final long PB = 1L << 50;
+  public static final long TB = 1L << 40;
+  public static final long GB = 1L << 30;
+  public static final long MB = 1L << 20;
+  public static final long KB = 1L << 10;
 
   public static final String IOTDB_HOME = "IOTDB_HOME";
 
@@ -289,6 +291,30 @@ public class IoTDBConstant {
   public static final String REQUEST_TYPE_READ = "read";
   public static final String REQUEST_TYPE_WRITE = "write";
   public static final String REQ_UNIT = "req";
+  public static final String REQ_SPLIT_UNIT = "req/";
+  public static final int UNLIMITED_VALUE = -1;
+  public static final int DEFAULT_VALUE = 0;
+  public static final float B_FLOAT = 1024.0F;
+
+  // SizeUnit
+  public static final String B_UNIT = "B";
+  public static final String KB_UNIT = "K";
+  public static final String MB_UNIT = "M";
+  public static final String GB_UNIT = "G";
+  public static final String TB_UNIT = "T";
+  public static final String PB_UNIT = "P";
+
+  // Time
+  public static final int SEC = 1000;
+  public static final int MIN = 60 * 1000;
+  public static final int HOUR = 60 * 60 * 1000;
+  public static final int DAY = 24 * 60 * 60 * 1000;
+
+  // TimeUnit
+  public static final String SEC_UNIT = "sec";
+  public static final String MIN_UNIT = "min";
+  public static final String HOUR_UNIT = "hour";
+  public static final String DAY_UNIT = "day";
 
   // client version number
   public enum ClientVersion {
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 4e39691ed0..97dd9ccdf7 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -1078,7 +1078,7 @@ public class IoTDBConfig {
   /** The maximum number of threads that can be used to execute subtasks in PipeSubtaskExecutor */
   private int pipeMaxThreadNum = 5;
 
-  /** multi-tenancy */
+  /** Resource control */
   private boolean quotaEnable = false;
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/quota/ShowSpaceQuotaTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/quota/ShowSpaceQuotaTask.java
index 7bd7e68044..961a9ccda3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/quota/ShowSpaceQuotaTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/quota/ShowSpaceQuotaTask.java
@@ -75,12 +75,15 @@ public class ShowSpaceQuotaTask implements IConfigTask {
                   Binary.valueOf(
                       spaceQuotaEntry.getValue().getDiskSize() == 0
                           ? IoTDBConstant.QUOTA_UNLIMITED
-                          : spaceQuotaEntry.getValue().getDiskSize() + "M"));
+                          : spaceQuotaEntry.getValue().getDiskSize() / IoTDBConstant.B_FLOAT
+                              + IoTDBConstant.GB_UNIT));
           builder
               .getColumnBuilder(3)
               .writeBinary(
                   Binary.valueOf(
-                      resp.getSpaceQuotaUsage().get(spaceQuotaEntry.getKey()).getDiskSize() + "M"));
+                      resp.getSpaceQuotaUsage().get(spaceQuotaEntry.getKey()).getDiskSize()
+                              / IoTDBConstant.B_FLOAT
+                          + IoTDBConstant.GB_UNIT));
           builder.declarePosition();
         }
         if (spaceQuotaEntry.getValue().getDeviceNum() != -1) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/quota/ShowThrottleQuotaTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/quota/ShowThrottleQuotaTask.java
index 11bb7d0d59..8533e59a58 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/quota/ShowThrottleQuotaTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/quota/ShowThrottleQuotaTask.java
@@ -89,7 +89,9 @@ public class ShowThrottleQuotaTask implements IConfigTask {
           builder
               .getColumnBuilder(2)
               .writeBinary(
-                  Binary.valueOf(throttleQuota.getValue().getMemLimit() / 1024 / 1024 + "M"));
+                  Binary.valueOf(
+                      throttleQuota.getValue().getMemLimit() / IoTDBConstant.KB / IoTDBConstant.KB
+                          + IoTDBConstant.MB_UNIT));
           builder.getColumnBuilder(3).writeBinary(Binary.valueOf(IoTDBConstant.REQUEST_TYPE_READ));
           builder.declarePosition();
         }
@@ -151,19 +153,19 @@ public class ShowThrottleQuotaTask implements IConfigTask {
             + File.separator
             + toTimeUnit(timedQuota.getTimeUnit());
       case IoTDBConstant.REQUEST_SIZE_PER_UNIT_TIME:
-        if (timedQuota.getSoftLimit() < 1024) {
+        if (timedQuota.getSoftLimit() < IoTDBConstant.KB) {
           return timedQuota.getSoftLimit()
-              + "B"
+              + IoTDBConstant.B_UNIT
               + File.separator
               + toTimeUnit(timedQuota.getTimeUnit());
-        } else if (timedQuota.getSoftLimit() < 1024 * 1024) {
-          return timedQuota.getSoftLimit() / 1024
-              + "K"
+        } else if (timedQuota.getSoftLimit() < IoTDBConstant.MB) {
+          return timedQuota.getSoftLimit() / IoTDBConstant.KB
+              + IoTDBConstant.KB_UNIT
               + File.separator
               + toTimeUnit(timedQuota.getTimeUnit());
         } else {
-          return timedQuota.getSoftLimit() / 1024 / 1024
-              + "M"
+          return timedQuota.getSoftLimit() / IoTDBConstant.KB / IoTDBConstant.KB
+              + IoTDBConstant.MB_UNIT
               + File.separator
               + toTimeUnit(timedQuota.getTimeUnit());
         }
@@ -174,14 +176,14 @@ public class ShowThrottleQuotaTask implements IConfigTask {
 
   private static String toTimeUnit(long timeUnit) {
     switch ((int) timeUnit) {
-      case 1000:
-        return "sec";
-      case 60 * 1000:
-        return "min";
-      case 60 * 60 * 1000:
-        return "hour";
-      case 24 * 60 * 60 * 1000:
-        return "day";
+      case IoTDBConstant.SEC:
+        return IoTDBConstant.SEC_UNIT;
+      case IoTDBConstant.MIN:
+        return IoTDBConstant.MIN_UNIT;
+      case IoTDBConstant.HOUR:
+        return IoTDBConstant.HOUR_UNIT;
+      case IoTDBConstant.DAY:
+        return IoTDBConstant.DAY_UNIT;
       default:
         throw new RuntimeException("Wrong unit type");
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
index 344da21ff1..3a731ebfb0 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
@@ -3450,9 +3450,25 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
           parseAttributeValue(attributePair.attributeValue()));
     }
 
+    quotas
+        .keySet()
+        .forEach(
+            quotaType -> {
+              switch (quotaType) {
+                case IoTDBConstant.COLUMN_DEVICES:
+                  break;
+                case IoTDBConstant.COLUMN_TIMESERIES:
+                  break;
+                case IoTDBConstant.SPACE_QUOTA_DISK:
+                  break;
+                default:
+                  throw new SemanticException("Wrong space quota type: " + quotaType);
+              }
+            });
+
     if (quotas.containsKey(IoTDBConstant.COLUMN_DEVICES)) {
       if (quotas.get(IoTDBConstant.COLUMN_DEVICES).equals(IoTDBConstant.QUOTA_UNLIMITED)) {
-        setSpaceQuotaStatement.setDeviceNum(-1);
+        setSpaceQuotaStatement.setDeviceNum(IoTDBConstant.UNLIMITED_VALUE);
       } else if (Long.parseLong(quotas.get(IoTDBConstant.COLUMN_DEVICES)) <= 0) {
         throw new SemanticException("Please set the number of devices greater than 0");
       } else {
@@ -3462,7 +3478,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     }
     if (quotas.containsKey(IoTDBConstant.COLUMN_TIMESERIES)) {
       if (quotas.get(IoTDBConstant.COLUMN_TIMESERIES).equals(IoTDBConstant.QUOTA_UNLIMITED)) {
-        setSpaceQuotaStatement.setTimeSeriesNum(-1);
+        setSpaceQuotaStatement.setTimeSeriesNum(IoTDBConstant.UNLIMITED_VALUE);
       } else if (Long.parseLong(quotas.get(IoTDBConstant.COLUMN_TIMESERIES)) <= 0) {
         throw new SemanticException("Please set the number of timeseries greater than 0");
       } else {
@@ -3472,7 +3488,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     }
     if (quotas.containsKey(IoTDBConstant.SPACE_QUOTA_DISK)) {
       if (quotas.get(IoTDBConstant.SPACE_QUOTA_DISK).equals(IoTDBConstant.QUOTA_UNLIMITED)) {
-        setSpaceQuotaStatement.setDiskSize(-1);
+        setSpaceQuotaStatement.setDiskSize(IoTDBConstant.UNLIMITED_VALUE);
       } else {
         setSpaceQuotaStatement.setDiskSize(
             parseSpaceQuotaSizeUnit(quotas.get(IoTDBConstant.SPACE_QUOTA_DISK)));
@@ -3502,9 +3518,9 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
       TTimedQuota timedQuota;
       String request = quotas.get(IoTDBConstant.REQUEST_NUM_PER_UNIT_TIME);
       if (request.equals(IoTDBConstant.QUOTA_UNLIMITED)) {
-        timedQuota = new TTimedQuota(1000, Long.MAX_VALUE);
+        timedQuota = new TTimedQuota(IoTDBConstant.SEC, Long.MAX_VALUE);
       } else {
-        String[] split = request.toLowerCase().split("req/");
+        String[] split = request.toLowerCase().split(IoTDBConstant.REQ_SPLIT_UNIT);
         if (Long.parseLong(split[0]) < 0) {
           throw new SemanticException("Please set the number of requests greater than 0");
         }
@@ -3532,7 +3548,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
       TTimedQuota timedQuota;
       String size = quotas.get(IoTDBConstant.REQUEST_SIZE_PER_UNIT_TIME);
       if (size.equals(IoTDBConstant.QUOTA_UNLIMITED)) {
-        timedQuota = new TTimedQuota(1000, Long.MAX_VALUE);
+        timedQuota = new TTimedQuota(IoTDBConstant.SEC, Long.MAX_VALUE);
       } else {
         String[] split = size.toLowerCase().split("/");
         timedQuota =
@@ -3559,7 +3575,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     if (quotas.containsKey(IoTDBConstant.MEMORY_SIZE_PER_READ)) {
       String mem = quotas.get(IoTDBConstant.MEMORY_SIZE_PER_READ);
       if (mem.equals(IoTDBConstant.QUOTA_UNLIMITED)) {
-        setThrottleQuotaStatement.setMemLimit(-1);
+        setThrottleQuotaStatement.setMemLimit(IoTDBConstant.UNLIMITED_VALUE);
       } else {
         setThrottleQuotaStatement.setMemLimit(parseThrottleQuotaSizeUnit(mem));
       }
@@ -3568,7 +3584,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     if (quotas.containsKey(IoTDBConstant.CPU_NUMBER_PER_READ)) {
       String cpuLimit = quotas.get(IoTDBConstant.CPU_NUMBER_PER_READ);
       if (cpuLimit.contains(IoTDBConstant.QUOTA_UNLIMITED)) {
-        setThrottleQuotaStatement.setCpuLimit(-1);
+        setThrottleQuotaStatement.setCpuLimit(IoTDBConstant.UNLIMITED_VALUE);
       } else {
         int cpuNum = Integer.parseInt(cpuLimit);
         if (cpuNum <= 0) {
@@ -3595,14 +3611,14 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
 
   private long parseThrottleQuotaTimeUnit(String timeUnit) {
     switch (timeUnit.toLowerCase()) {
-      case "sec":
-        return 1000;
-      case "min":
-        return 60 * 1000;
-      case "hour":
-        return 60 * 60 * 1000;
-      case "day":
-        return 24 * 60 * 60 * 1000;
+      case IoTDBConstant.SEC_UNIT:
+        return IoTDBConstant.SEC;
+      case IoTDBConstant.MIN_UNIT:
+        return IoTDBConstant.MIN;
+      case IoTDBConstant.HOUR_UNIT:
+        return IoTDBConstant.HOUR;
+      case IoTDBConstant.DAY_UNIT:
+        return IoTDBConstant.DAY;
       default:
         throw new SemanticException(
             "When setting the request, the unit is incorrect. Please use 'sec', 'min', 'hour', 'day' as the unit");
@@ -3615,19 +3631,19 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     if (size <= 0) {
       throw new SemanticException("Please set the size greater than 0");
     }
-    switch (unit.toLowerCase()) {
-      case "b":
+    switch (unit.toUpperCase()) {
+      case IoTDBConstant.B_UNIT:
         return size;
-      case "k":
-        return size * 1024;
-      case "m":
-        return size * 1024 * 1024;
-      case "g":
-        return size * 1024 * 1024 * 1024;
-      case "t":
-        return size * 1024 * 1024 * 1024 * 1024;
-      case "p":
-        return size * 1024 * 1024 * 1024 * 1024 * 1024;
+      case IoTDBConstant.KB_UNIT:
+        return size * IoTDBConstant.KB;
+      case IoTDBConstant.MB_UNIT:
+        return size * IoTDBConstant.MB;
+      case IoTDBConstant.GB_UNIT:
+        return size * IoTDBConstant.GB;
+      case IoTDBConstant.TB_UNIT:
+        return size * IoTDBConstant.TB;
+      case IoTDBConstant.PB_UNIT:
+        return size * IoTDBConstant.PB;
       default:
         throw new SemanticException(
             "When setting the size/time, the unit is incorrect. Please use 'B', 'K', 'M', 'G', 'P', 'T' as the unit");
@@ -3640,15 +3656,15 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     if (disk <= 0) {
       throw new SemanticException("Please set the disk size greater than 0");
     }
-    switch (unit.toLowerCase()) {
-      case "m":
+    switch (unit.toUpperCase()) {
+      case IoTDBConstant.MB_UNIT:
         return disk;
-      case "g":
-        return disk * 1024;
-      case "t":
-        return disk * 1024 * 1024;
-      case "p":
-        return disk * 1024 * 1024 * 1024;
+      case IoTDBConstant.GB_UNIT:
+        return disk * IoTDBConstant.KB;
+      case IoTDBConstant.TB_UNIT:
+        return disk * IoTDBConstant.MB;
+      case IoTDBConstant.PB_UNIT:
+        return disk * IoTDBConstant.GB;
       default:
         throw new SemanticException(
             "When setting the disk size, the unit is incorrect. Please use 'M', 'G', 'P', 'T' as the unit");