You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by lu...@apache.org on 2023/04/18 12:20:59 UTC

[doris] branch master updated: [improvement](resource-group) add test for resource group (#18575)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 5c076b738b [improvement](resource-group) add test for resource group (#18575)
5c076b738b is described below

commit 5c076b738b07a7b9543d0200cdca0d3d70f64ce7
Author: luozenglin <lu...@baidu.com>
AuthorDate: Tue Apr 18 20:20:50 2023 +0800

    [improvement](resource-group) add test for resource group (#18575)
    
    Co-authored-by: wangbo <yo...@qq.com>
---
 .../main/java/org/apache/doris/common/Config.java  |   3 +-
 .../resource/resourcegroup/ResourceGroup.java      |   2 +-
 .../resource/resourcegroup/ResourceGroupMgr.java   |  21 ++-
 .../resourcegroup/ResourceGroupMgrTest.java        | 146 +++++++++++++++++++++
 .../resource/resourcegroup/ResourceGroupTest.java  | 100 ++++++++++++++
 .../test_resource_group_tvf.groovy                 |   4 +
 .../test_resource_group.groovy}                    |  15 ++-
 7 files changed, 278 insertions(+), 13 deletions(-)

diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
index 2636c7650e..1942ec0834 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
@@ -1738,7 +1738,8 @@ public class Config extends ConfigBase {
     @ConfField
     public static boolean enable_pipeline_load = false;
 
-    @ConfField
+    // enable_resource_group should be immutable and temporarily set to mutable during the development test phase
+    @ConfField(mutable = true, masterOnly = true, expType = ExperimentalType.EXPERIMENTAL)
     public static boolean enable_resource_group = false;
 
     @ConfField(mutable = false, masterOnly = true)
diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/resourcegroup/ResourceGroup.java b/fe/fe-core/src/main/java/org/apache/doris/resource/resourcegroup/ResourceGroup.java
index bcf72cb3e2..2c3dfb4caa 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/resource/resourcegroup/ResourceGroup.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/resource/resourcegroup/ResourceGroup.java
@@ -65,7 +65,7 @@ public class ResourceGroup implements Writable {
         this.version = 0;
     }
 
-    public static ResourceGroup createResourceGroup(String name, Map<String, String> properties) throws DdlException {
+    public static ResourceGroup create(String name, Map<String, String> properties) throws DdlException {
         checkProperties(properties);
         return new ResourceGroup(Env.getCurrentEnv().getNextId(), name, properties);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/resourcegroup/ResourceGroupMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/resourcegroup/ResourceGroupMgr.java
index 058eee4471..37c0bc69e6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/resource/resourcegroup/ResourceGroupMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/resource/resourcegroup/ResourceGroupMgr.java
@@ -84,7 +84,7 @@ public class ResourceGroupMgr implements Writable, GsonPostProcessable {
     }
 
     public void init() {
-        if (Config.enable_resource_group) {
+        if (Config.enable_resource_group || Config.use_fuzzy_session_variable /* for github workflow */) {
             checkAndCreateDefaultGroup();
         }
     }
@@ -114,7 +114,7 @@ public class ResourceGroupMgr implements Writable, GsonPostProcessable {
             }
             Map<String, String> properties = Maps.newHashMap();
             properties.put(ResourceGroup.CPU_SHARE, "10");
-            defaultResourceGroup = ResourceGroup.createResourceGroup(DEFAULT_GROUP_NAME, properties);
+            defaultResourceGroup = ResourceGroup.create(DEFAULT_GROUP_NAME, properties);
             nameToResourceGroup.put(DEFAULT_GROUP_NAME, defaultResourceGroup);
             idToResourceGroup.put(defaultResourceGroup.getId(), defaultResourceGroup);
             Env.getCurrentEnv().getEditLog().logCreateResourceGroup(defaultResourceGroup);
@@ -127,8 +127,11 @@ public class ResourceGroupMgr implements Writable, GsonPostProcessable {
     }
 
     public void createResourceGroup(CreateResourceGroupStmt stmt) throws DdlException {
-        ResourceGroup resourceGroup = ResourceGroup.createResourceGroup(stmt.getResourceGroupName(),
-                stmt.getProperties());
+        if (!Config.enable_resource_group) {
+            throw new DdlException("unsupported feature now, coming soon.");
+        }
+
+        ResourceGroup resourceGroup = ResourceGroup.create(stmt.getResourceGroupName(), stmt.getProperties());
         String resourceGroupNameName = resourceGroup.getName();
         writeLock();
         try {
@@ -160,6 +163,16 @@ public class ResourceGroupMgr implements Writable, GsonPostProcessable {
         return procNode.fetchResult().getRows();
     }
 
+    // for ut
+    public Map<String, ResourceGroup> getNameToResourceGroup() {
+        return nameToResourceGroup;
+    }
+
+    // for ut
+    public Map<Long, ResourceGroup> getIdToResourceGroup() {
+        return idToResourceGroup;
+    }
+
     @Override
     public void write(DataOutput out) throws IOException {
         String json = GsonUtils.GSON.toJson(this);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/resource/resourcegroup/ResourceGroupMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/resource/resourcegroup/ResourceGroupMgrTest.java
new file mode 100644
index 0000000000..7c013b4cfb
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/resource/resourcegroup/ResourceGroupMgrTest.java
@@ -0,0 +1,146 @@
+// 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.doris.resource.resourcegroup;
+
+import org.apache.doris.analysis.CreateResourceGroupStmt;
+import org.apache.doris.catalog.Env;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.UserException;
+import org.apache.doris.persist.EditLog;
+import org.apache.doris.thrift.TPipelineResourceGroup;
+
+import com.google.common.collect.Maps;
+import mockit.Delegate;
+import mockit.Expectations;
+import mockit.Injectable;
+import mockit.Mocked;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ResourceGroupMgrTest {
+
+    @Injectable
+    private EditLog editLog;
+
+    @Mocked
+    private Env env;
+
+    private AtomicLong id = new AtomicLong(10);
+
+    @Before
+    public void setUp() throws DdlException {
+        new Expectations() {
+            {
+                env.getEditLog();
+                minTimes = 0;
+                result = editLog;
+
+                env.getNextId();
+                minTimes = 0;
+                result = new Delegate() {
+                    long delegate() {
+                        return id.addAndGet(1);
+                    }
+                };
+
+                editLog.logCreateResourceGroup((ResourceGroup) any);
+                minTimes = 0;
+
+                Env.getCurrentEnv();
+                minTimes = 0;
+                result = env;
+            }
+        };
+    }
+
+    @Test
+    public void testCreateResourceGroup() throws DdlException {
+        Config.enable_resource_group = true;
+        ResourceGroupMgr resourceGroupMgr = new ResourceGroupMgr();
+        Map<String, String> properties1 = Maps.newHashMap();
+        properties1.put(ResourceGroup.CPU_SHARE, "10");
+        String name1 = "g1";
+        CreateResourceGroupStmt stmt1 = new CreateResourceGroupStmt(false, name1, properties1);
+        resourceGroupMgr.createResourceGroup(stmt1);
+
+        Map<String, ResourceGroup> nameToRG = resourceGroupMgr.getNameToResourceGroup();
+        Assert.assertEquals(1, nameToRG.size());
+        Assert.assertTrue(nameToRG.containsKey(name1));
+        ResourceGroup group1 = nameToRG.get(name1);
+        Assert.assertEquals(name1, group1.getName());
+
+        Map<Long, ResourceGroup> idToRG = resourceGroupMgr.getIdToResourceGroup();
+        Assert.assertEquals(1, idToRG.size());
+        Assert.assertTrue(idToRG.containsKey(group1.getId()));
+
+        Map<String, String> properties2 = Maps.newHashMap();
+        properties2.put(ResourceGroup.CPU_SHARE, "20");
+        String name2 = "g2";
+        CreateResourceGroupStmt stmt2 = new CreateResourceGroupStmt(false, name2, properties2);
+        resourceGroupMgr.createResourceGroup(stmt2);
+
+        nameToRG = resourceGroupMgr.getNameToResourceGroup();
+        Assert.assertEquals(2, nameToRG.size());
+        Assert.assertTrue(nameToRG.containsKey(name2));
+        ResourceGroup group2 = nameToRG.get(name2);
+        idToRG = resourceGroupMgr.getIdToResourceGroup();
+        Assert.assertEquals(2, idToRG.size());
+        Assert.assertTrue(idToRG.containsKey(group2.getId()));
+
+        try {
+            resourceGroupMgr.createResourceGroup(stmt2);
+            Assert.fail();
+        } catch (DdlException e) {
+            Assert.assertTrue(e.getMessage().contains("already exist"));
+        }
+
+        CreateResourceGroupStmt stmt3 = new CreateResourceGroupStmt(true, name2, properties2);
+        resourceGroupMgr.createResourceGroup(stmt3);
+        Assert.assertEquals(2, resourceGroupMgr.getIdToResourceGroup().size());
+        Assert.assertEquals(2, resourceGroupMgr.getNameToResourceGroup().size());
+    }
+
+    @Test
+    public void testGetResourceGroup() throws UserException {
+        Config.enable_resource_group = true;
+        ResourceGroupMgr resourceGroupMgr = new ResourceGroupMgr();
+        Map<String, String> properties1 = Maps.newHashMap();
+        properties1.put(ResourceGroup.CPU_SHARE, "10");
+        String name1 = "g1";
+        CreateResourceGroupStmt stmt1 = new CreateResourceGroupStmt(false, name1, properties1);
+        resourceGroupMgr.createResourceGroup(stmt1);
+        List<TPipelineResourceGroup> tResourceGroups1 = resourceGroupMgr.getResourceGroup(name1);
+        Assert.assertEquals(1, tResourceGroups1.size());
+        TPipelineResourceGroup tResourceGroup1 = tResourceGroups1.get(0);
+        Assert.assertEquals(name1, tResourceGroup1.getName());
+        Assert.assertTrue(tResourceGroup1.getProperties().containsKey(ResourceGroup.CPU_SHARE));
+
+        try {
+            resourceGroupMgr.getResourceGroup("g2");
+            Assert.fail();
+        } catch (UserException e) {
+            Assert.assertTrue(e.getMessage().contains("does not exist"));
+        }
+    }
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/resource/resourcegroup/ResourceGroupTest.java b/fe/fe-core/src/test/java/org/apache/doris/resource/resourcegroup/ResourceGroupTest.java
new file mode 100644
index 0000000000..978d28b8f1
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/resource/resourcegroup/ResourceGroupTest.java
@@ -0,0 +1,100 @@
+// 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.doris.resource.resourcegroup;
+
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.proc.BaseProcResult;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+
+public class ResourceGroupTest {
+
+    @Test
+    public void testCreateNormal() throws DdlException {
+        Map<String, String> properties1 = Maps.newHashMap();
+        properties1.put(ResourceGroup.CPU_SHARE, "10");
+        String name1 = "g1";
+        ResourceGroup group1 = ResourceGroup.create(name1, properties1);
+        Assert.assertEquals(name1, group1.getName());
+        Assert.assertEquals(1, group1.getProperties().size());
+        Assert.assertTrue(group1.getProperties().containsKey(ResourceGroup.CPU_SHARE));
+    }
+
+    @Test(expected = DdlException.class)
+    public void testNotSupportProperty() throws DdlException {
+        Map<String, String> properties1 = Maps.newHashMap();
+        properties1.put(ResourceGroup.CPU_SHARE, "10");
+        properties1.put("share", "10");
+        String name1 = "g1";
+        ResourceGroup.create(name1, properties1);
+    }
+
+    @Test(expected = DdlException.class)
+    public void testRequiredProperty() throws DdlException {
+        Map<String, String> properties1 = Maps.newHashMap();
+        String name1 = "g1";
+        ResourceGroup.create(name1, properties1);
+    }
+
+    @Test
+    public void testCpuShareValue() {
+        Map<String, String> properties1 = Maps.newHashMap();
+        properties1.put(ResourceGroup.CPU_SHARE, "0");
+        String name1 = "g1";
+        try {
+            ResourceGroup.create(name1, properties1);
+            Assert.fail();
+        } catch (DdlException e) {
+            Assert.assertTrue(e.getMessage().contains(ResourceGroup.CPU_SHARE + " requires a positive integer."));
+        }
+
+        properties1.put(ResourceGroup.CPU_SHARE, "cpu");
+        try {
+            ResourceGroup.create(name1, properties1);
+            Assert.fail();
+        } catch (DdlException e) {
+            Assert.assertTrue(e.getMessage().contains(ResourceGroup.CPU_SHARE + " requires a positive integer."));
+        }
+    }
+
+    @Test
+    public void testGetProcNodeData() throws DdlException {
+        Map<String, String> properties1 = Maps.newHashMap();
+        properties1.put(ResourceGroup.CPU_SHARE, "10");
+        String name1 = "g1";
+        ResourceGroup group1 = ResourceGroup.create(name1, properties1);
+
+        BaseProcResult result = new BaseProcResult();
+        group1.getProcNodeData(result);
+        List<List<String>> rows = result.getRows();
+        Assert.assertEquals(1, rows.size());
+        List<List<String>> expectedRows = Lists.newArrayList();
+        expectedRows.add(Lists.newArrayList(String.valueOf(group1.getId()), name1, ResourceGroup.CPU_SHARE, "10"));
+        for (int i = 0; i < expectedRows.size(); ++i) {
+            for (int j = 0; j < expectedRows.get(i).size(); ++j) {
+                Assert.assertEquals(expectedRows.get(i).get(j), rows.get(i).get(j));
+            }
+        }
+    }
+}
diff --git a/regression-test/suites/correctness_p0/table_valued_function/test_resource_group_tvf.groovy b/regression-test/suites/correctness_p0/table_valued_function/test_resource_group_tvf.groovy
index fe0f2a9ab6..a0293a0ee6 100644
--- a/regression-test/suites/correctness_p0/table_valued_function/test_resource_group_tvf.groovy
+++ b/regression-test/suites/correctness_p0/table_valued_function/test_resource_group_tvf.groovy
@@ -18,9 +18,13 @@
 // This suit test the `resource_groups` tvf
 // TO DO
 suite("test_resource_groups_tvf") {
+    sql """ADMIN SET FRONTEND CONFIG ("experimental_enable_resource_group" = "true");"""
+
     def name1 = "test";
 	sql "create resource group if not exists ${name1} properties('cpu_share'='10');"
     List<List<Object>> table =  sql """ select * from resource_groups(); """
     assertTrue(table.size() > 0)
     assertTrue(table[0].size == 4) // column should be 4
+
+    sql """ADMIN SET FRONTEND CONFIG ("experimental_enable_resource_group" = "false");"""
 }
\ No newline at end of file
diff --git a/regression-test/suites/correctness_p0/table_valued_function/test_resource_group_tvf.groovy b/regression-test/suites/resource_group_p0/test_resource_group.groovy
similarity index 71%
copy from regression-test/suites/correctness_p0/table_valued_function/test_resource_group_tvf.groovy
copy to regression-test/suites/resource_group_p0/test_resource_group.groovy
index fe0f2a9ab6..8dc7135547 100644
--- a/regression-test/suites/correctness_p0/table_valued_function/test_resource_group_tvf.groovy
+++ b/regression-test/suites/resource_group_p0/test_resource_group.groovy
@@ -14,13 +14,14 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+suite("test_resource_group") {
+	sql """ADMIN SET FRONTEND CONFIG ("experimental_enable_resource_group" = "true");"""
 
-// This suit test the `resource_groups` tvf
-// TO DO
-suite("test_resource_groups_tvf") {
-    def name1 = "test";
+	def name1 = "g1";
 	sql "create resource group if not exists ${name1} properties('cpu_share'='10');"
-    List<List<Object>> table =  sql """ select * from resource_groups(); """
-    assertTrue(table.size() > 0)
-    assertTrue(table[0].size == 4) // column should be 4
+	List<List<Object>> results = sql "show resource groups;"
+    assertTrue(results.size() >= 2)
+    assertEquals(4, results[0].size())
+
+	sql """ADMIN SET FRONTEND CONFIG ("experimental_enable_resource_group" = "false");"""
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org