You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2020/01/31 16:13:30 UTC

[GitHub] [incubator-doris] morningman opened a new pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

morningman opened a new pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826
 
 
   This CL implements a simulated FE process and a simulated BE service. 
   You can view their specific usage methods at 
   
   `fe/src/test/java/org/apache/doris/utframe/DemoTest.java`
   
   At the same time, I modified the configuration of the maven-surefire-plugin plugin,
   so that each unit test runs in a separate JVM, which can avoid conflicts caused by 
   various singleton classes in FE.
   
   Starting a separate jvm for each unit test will bring about 30% extra time overhead. 
   However, you can control the number of concurrency of unit tests by setting the `forkCount`
    configuration of the maven-surefire-plugin plugin in `fe/ pom.xml`. The default configuration 
   is still 1 for easy viewing of the output log. If set to 3, the entire FE unit test run time is about
    4 minutes.
   
   ISSUE: #2825

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373952452
 
 

 ##########
 File path: fe/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java
 ##########
 @@ -0,0 +1,166 @@
+// 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.utframe;
+
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.Pair;
+import org.apache.doris.planner.OlapScanNode;
+import org.apache.doris.planner.PlanFragment;
+import org.apache.doris.planner.Planner;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.StmtExecutor;
+import org.apache.doris.thrift.TNetworkAddress;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultBeThriftServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultHeartbeatServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultPBackendServiceImpl;
+import org.apache.doris.utframe.MockedFrontend.EnvVarNotSetException;
+import org.apache.doris.utframe.MockedFrontend.FeStartException;
+import org.apache.doris.utframe.MockedFrontend.NotInitException;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/*
+ * This demo is mainly used to confirm that 
+ * repeatedly starting FE and BE in 2 UnitTest will not cause conflict
+ */
+public class AnotherDemoTest {
+
+    private static int fe_http_port;
+    private static int fe_rpc_port;
+    private static int fe_query_port;
+    private static int fe_edit_log_port;
+
+    private static int be_heartbeat_port;
+    private static int be_thrift_port;
+    private static int be_brpc_port;
+    private static int be_http_port;
+
+    // use a unique dir so that it won't be conflict with other unit test which
+    // may also start a Mocked Frontend
+    private static String runningDir = "fe/mocked/AnotherDemoTest";
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373932467
 
 

 ##########
 File path: fe/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java
 ##########
 @@ -0,0 +1,166 @@
+// 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.utframe;
+
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.Pair;
+import org.apache.doris.planner.OlapScanNode;
+import org.apache.doris.planner.PlanFragment;
+import org.apache.doris.planner.Planner;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.StmtExecutor;
+import org.apache.doris.thrift.TNetworkAddress;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultBeThriftServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultHeartbeatServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultPBackendServiceImpl;
+import org.apache.doris.utframe.MockedFrontend.EnvVarNotSetException;
+import org.apache.doris.utframe.MockedFrontend.FeStartException;
+import org.apache.doris.utframe.MockedFrontend.NotInitException;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/*
+ * This demo is mainly used to confirm that 
+ * repeatedly starting FE and BE in 2 UnitTest will not cause conflict
+ */
+public class AnotherDemoTest {
+
+    private static int fe_http_port;
+    private static int fe_rpc_port;
+    private static int fe_query_port;
+    private static int fe_edit_log_port;
+
+    private static int be_heartbeat_port;
+    private static int be_thrift_port;
+    private static int be_brpc_port;
+    private static int be_http_port;
+
+    // use a unique dir so that it won't be conflict with other unit test which
+    // may also start a Mocked Frontend
+    private static String runningDir = "fe/mocked/AnotherDemoTest";
 
 Review comment:
   Use a uuid?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373940556
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Catalog.java
 ##########
 @@ -635,7 +642,23 @@ private void unlock() {
         }
     }
 
+    public String getBdbDir() {
+        return bdbDir;
+    }
+
+    public String getImageDir() {
+        return imageDir;
+    }
+
     public void initialize(String[] args) throws Exception {
+        // set meta dir first.
+        // we already set these varialbes in constructor. but Catalog is a singleton class.
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373942889
 
 

 ##########
 File path: fe/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java
 ##########
 @@ -0,0 +1,166 @@
+// 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.utframe;
+
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.Pair;
+import org.apache.doris.planner.OlapScanNode;
+import org.apache.doris.planner.PlanFragment;
+import org.apache.doris.planner.Planner;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.StmtExecutor;
+import org.apache.doris.thrift.TNetworkAddress;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultBeThriftServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultHeartbeatServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultPBackendServiceImpl;
+import org.apache.doris.utframe.MockedFrontend.EnvVarNotSetException;
+import org.apache.doris.utframe.MockedFrontend.FeStartException;
+import org.apache.doris.utframe.MockedFrontend.NotInitException;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/*
+ * This demo is mainly used to confirm that 
+ * repeatedly starting FE and BE in 2 UnitTest will not cause conflict
+ */
+public class AnotherDemoTest {
+
+    private static int fe_http_port;
+    private static int fe_rpc_port;
+    private static int fe_query_port;
+    private static int fe_edit_log_port;
+
+    private static int be_heartbeat_port;
+    private static int be_thrift_port;
+    private static int be_brpc_port;
+    private static int be_http_port;
+
+    // use a unique dir so that it won't be conflict with other unit test which
+    // may also start a Mocked Frontend
+    private static String runningDir = "fe/mocked/AnotherDemoTest";
 
 Review comment:
   Using test class name as dir name is more readable. But in case there are 2 tests with same name, I will add a uuid next to it.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373928012
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Catalog.java
 ##########
 @@ -635,7 +642,23 @@ private void unlock() {
         }
     }
 
+    public String getBdbDir() {
+        return bdbDir;
+    }
+
+    public String getImageDir() {
+        return imageDir;
+    }
+
     public void initialize(String[] args) throws Exception {
+        // set meta dir first.
+        // we already set these varialbes in constructor. but Catalog is a singleton class.
 
 Review comment:
   ```suggestion
           // we already set these variables in constructor. but Catalog is a singleton class.
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373940106
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/PaloFe.java
 ##########
 @@ -51,25 +51,36 @@
 public class PaloFe {
     private static final Logger LOG = LogManager.getLogger(PaloFe.class);
 
-    // entrance for palo frontend
+    public static final String DORIS_HOME_DIR = System.getenv("DORIS_HOME");
+    public static final String PID_DIR = System.getenv("PID_DIR");
+
     public static void main(String[] args) {
+        start(DORIS_HOME_DIR, PID_DIR, args);
+    }
+
+    // entrance for doris frontend
+    public static void start(String dorisHomeDir, String pidDir, String[] args) {
+        if (Strings.isNullOrEmpty(dorisHomeDir)) {
+            System.out.println("env DORIS_HOME is not set");
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman merged pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
morningman merged pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373940664
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/qe/ResultReceiver.java
 ##########
 @@ -76,7 +76,9 @@ public RowBatch getNext(Status status) throws TException {
                         throw new TimeoutException("query timeout");
                     }
                     try {
+                        LOG.debug("cmy before get next");
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373929470
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/qe/ResultReceiver.java
 ##########
 @@ -76,7 +76,9 @@ public RowBatch getNext(Status status) throws TException {
                         throw new TimeoutException("query timeout");
                     }
                     try {
+                        LOG.debug("cmy before get next");
 
 Review comment:
   removed?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373935376
 
 

 ##########
 File path: fe/src/test/java/org/apache/doris/utframe/DemoTest.java
 ##########
 @@ -0,0 +1,193 @@
+// 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.utframe;
+
+import org.apache.doris.alter.AlterJobV2;
+import org.apache.doris.analysis.AlterTableStmt;
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.Pair;
+import org.apache.doris.planner.OlapScanNode;
+import org.apache.doris.planner.PlanFragment;
+import org.apache.doris.planner.Planner;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.StmtExecutor;
+import org.apache.doris.thrift.TNetworkAddress;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultBeThriftServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultHeartbeatServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultPBackendServiceImpl;
+import org.apache.doris.utframe.MockedFrontend.EnvVarNotSetException;
+import org.apache.doris.utframe.MockedFrontend.FeStartException;
+import org.apache.doris.utframe.MockedFrontend.NotInitException;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/*
+ * This demo shows how to run unit test with mocked FE and BE.
+ * It will
+ *  1. start a mocked FE and a mocked BE.
+ *  2. Create a database and a tbl.
+ *  3. Make a schema change to tbl.
+ *  4. send a query and get query plan
+ */
+public class DemoTest {
+
+    private static int fe_http_port;
+    private static int fe_rpc_port;
+    private static int fe_query_port;
+    private static int fe_edit_log_port;
+
+    private static int be_heartbeat_port;
+    private static int be_thrift_port;
+    private static int be_brpc_port;
+    private static int be_http_port;
+    // use a unique dir so that it won't be conflict with other unit test which
+    // may also start a Mocked Frontend
+    private static String runningDir = "fe/mocked/DemoTest";
+
+    @BeforeClass
+    public static void beforeClass() throws EnvVarNotSetException, IOException,
+            FeStartException, NotInitException, DdlException, InterruptedException {
+        // get DORIS_HOME
+        final String dorisHome = System.getenv("DORIS_HOME");
+        if (Strings.isNullOrEmpty(dorisHome)) {
+            throw new EnvVarNotSetException("env DORIS_HOME is not set");
+        }
+
+        getRandomPort();
+
+        // start fe in "DORIS_HOME/fe/mocked/"
+        MockedFrontend frontend = MockedFrontend.getInstance();
+        Map<String, String> feConfMap = Maps.newHashMap();
+        // set additional fe config
+        feConfMap.put("http_port", String.valueOf(fe_http_port));
+        feConfMap.put("rpc_port", String.valueOf(fe_rpc_port));
+        feConfMap.put("query_port", String.valueOf(fe_query_port));
+        feConfMap.put("edit_log_port", String.valueOf(fe_edit_log_port));
+        feConfMap.put("tablet_create_timeout_second", "10");
+        frontend.init(dorisHome + "/" + runningDir, feConfMap);
+        frontend.start(new String[0]);
+
+        // start be
+        MockedBackend backend = MockedBackendFactory.createBackend("127.0.0.1",
+                be_heartbeat_port, be_thrift_port, be_brpc_port, be_http_port,
+                new DefaultHeartbeatServiceImpl(be_thrift_port, be_http_port, be_brpc_port),
+                new DefaultBeThriftServiceImpl(), new DefaultPBackendServiceImpl());
+        backend.setFeAddress(new TNetworkAddress("127.0.0.1", frontend.getRpcPort()));
+        backend.start();
+
+        // add be
+        List<Pair<String, Integer>> bes = Lists.newArrayList();
+        bes.add(Pair.create(backend.getHost(), backend.getHeartbeatPort()));
+        Catalog.getCurrentSystemInfo().addBackends(bes, false, "default_cluster");
+
+        // sleep to wait first heartbeat
+        Thread.sleep(6000);
+    }
+
+    // generate all port from between 20000 ~ 30000
+    private static void getRandomPort() {
 
 Review comment:
   `getRandomPort`  is duplicate with `AnotherDemoTest`. could move to `UtFrameUtils `

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373942498
 
 

 ##########
 File path: fe/src/test/java/org/apache/doris/utframe/DemoTest.java
 ##########
 @@ -0,0 +1,193 @@
+// 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.utframe;
+
+import org.apache.doris.alter.AlterJobV2;
+import org.apache.doris.analysis.AlterTableStmt;
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.Pair;
+import org.apache.doris.planner.OlapScanNode;
+import org.apache.doris.planner.PlanFragment;
+import org.apache.doris.planner.Planner;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.StmtExecutor;
+import org.apache.doris.thrift.TNetworkAddress;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultBeThriftServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultHeartbeatServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultPBackendServiceImpl;
+import org.apache.doris.utframe.MockedFrontend.EnvVarNotSetException;
+import org.apache.doris.utframe.MockedFrontend.FeStartException;
+import org.apache.doris.utframe.MockedFrontend.NotInitException;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/*
+ * This demo shows how to run unit test with mocked FE and BE.
+ * It will
+ *  1. start a mocked FE and a mocked BE.
+ *  2. Create a database and a tbl.
+ *  3. Make a schema change to tbl.
+ *  4. send a query and get query plan
+ */
+public class DemoTest {
+
+    private static int fe_http_port;
+    private static int fe_rpc_port;
+    private static int fe_query_port;
+    private static int fe_edit_log_port;
+
+    private static int be_heartbeat_port;
+    private static int be_thrift_port;
+    private static int be_brpc_port;
+    private static int be_http_port;
+    // use a unique dir so that it won't be conflict with other unit test which
+    // may also start a Mocked Frontend
+    private static String runningDir = "fe/mocked/DemoTest";
+
+    @BeforeClass
+    public static void beforeClass() throws EnvVarNotSetException, IOException,
+            FeStartException, NotInitException, DdlException, InterruptedException {
+        // get DORIS_HOME
+        final String dorisHome = System.getenv("DORIS_HOME");
+        if (Strings.isNullOrEmpty(dorisHome)) {
+            throw new EnvVarNotSetException("env DORIS_HOME is not set");
+        }
+
+        getRandomPort();
+
+        // start fe in "DORIS_HOME/fe/mocked/"
+        MockedFrontend frontend = MockedFrontend.getInstance();
+        Map<String, String> feConfMap = Maps.newHashMap();
+        // set additional fe config
+        feConfMap.put("http_port", String.valueOf(fe_http_port));
+        feConfMap.put("rpc_port", String.valueOf(fe_rpc_port));
+        feConfMap.put("query_port", String.valueOf(fe_query_port));
+        feConfMap.put("edit_log_port", String.valueOf(fe_edit_log_port));
+        feConfMap.put("tablet_create_timeout_second", "10");
+        frontend.init(dorisHome + "/" + runningDir, feConfMap);
+        frontend.start(new String[0]);
+
+        // start be
+        MockedBackend backend = MockedBackendFactory.createBackend("127.0.0.1",
+                be_heartbeat_port, be_thrift_port, be_brpc_port, be_http_port,
+                new DefaultHeartbeatServiceImpl(be_thrift_port, be_http_port, be_brpc_port),
+                new DefaultBeThriftServiceImpl(), new DefaultPBackendServiceImpl());
+        backend.setFeAddress(new TNetworkAddress("127.0.0.1", frontend.getRpcPort()));
+        backend.start();
+
+        // add be
+        List<Pair<String, Integer>> bes = Lists.newArrayList();
+        bes.add(Pair.create(backend.getHost(), backend.getHeartbeatPort()));
+        Catalog.getCurrentSystemInfo().addBackends(bes, false, "default_cluster");
+
+        // sleep to wait first heartbeat
+        Thread.sleep(6000);
+    }
+
+    // generate all port from between 20000 ~ 30000
+    private static void getRandomPort() {
 
 Review comment:
   Which ports are used is all depends on the caller. Here is just an example to get random ports. And ports set here are all static members of the certain Test class. So the only thing 
   that can be put to `UtFrameUtils` is:
   
   ```
   Random r = new Random(System.currentTimeMillis());
   int basePort = 20000 + r.nextInt(9000);
   ```
   
   It seems unnecessary. 
   
   `AnotherDemoTest` is just a copy of `DemoTest`, so don't worry about this part of duplicate codes.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on issue #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on issue #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#issuecomment-581256872
 
 
   Nice Job!

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373952531
 
 

 ##########
 File path: fe/src/test/java/org/apache/doris/utframe/DemoTest.java
 ##########
 @@ -0,0 +1,193 @@
+// 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.utframe;
+
+import org.apache.doris.alter.AlterJobV2;
+import org.apache.doris.analysis.AlterTableStmt;
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.Pair;
+import org.apache.doris.planner.OlapScanNode;
+import org.apache.doris.planner.PlanFragment;
+import org.apache.doris.planner.Planner;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.StmtExecutor;
+import org.apache.doris.thrift.TNetworkAddress;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultBeThriftServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultHeartbeatServiceImpl;
+import org.apache.doris.utframe.MockedBackendFactory.DefaultPBackendServiceImpl;
+import org.apache.doris.utframe.MockedFrontend.EnvVarNotSetException;
+import org.apache.doris.utframe.MockedFrontend.FeStartException;
+import org.apache.doris.utframe.MockedFrontend.NotInitException;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/*
+ * This demo shows how to run unit test with mocked FE and BE.
+ * It will
+ *  1. start a mocked FE and a mocked BE.
+ *  2. Create a database and a tbl.
+ *  3. Make a schema change to tbl.
+ *  4. send a query and get query plan
+ */
+public class DemoTest {
+
+    private static int fe_http_port;
+    private static int fe_rpc_port;
+    private static int fe_query_port;
+    private static int fe_edit_log_port;
+
+    private static int be_heartbeat_port;
+    private static int be_thrift_port;
+    private static int be_brpc_port;
+    private static int be_http_port;
+    // use a unique dir so that it won't be conflict with other unit test which
+    // may also start a Mocked Frontend
+    private static String runningDir = "fe/mocked/DemoTest";
+
+    @BeforeClass
+    public static void beforeClass() throws EnvVarNotSetException, IOException,
+            FeStartException, NotInitException, DdlException, InterruptedException {
+        // get DORIS_HOME
+        final String dorisHome = System.getenv("DORIS_HOME");
+        if (Strings.isNullOrEmpty(dorisHome)) {
+            throw new EnvVarNotSetException("env DORIS_HOME is not set");
+        }
+
+        getRandomPort();
+
+        // start fe in "DORIS_HOME/fe/mocked/"
+        MockedFrontend frontend = MockedFrontend.getInstance();
+        Map<String, String> feConfMap = Maps.newHashMap();
+        // set additional fe config
+        feConfMap.put("http_port", String.valueOf(fe_http_port));
+        feConfMap.put("rpc_port", String.valueOf(fe_rpc_port));
+        feConfMap.put("query_port", String.valueOf(fe_query_port));
+        feConfMap.put("edit_log_port", String.valueOf(fe_edit_log_port));
+        feConfMap.put("tablet_create_timeout_second", "10");
+        frontend.init(dorisHome + "/" + runningDir, feConfMap);
+        frontend.start(new String[0]);
+
+        // start be
+        MockedBackend backend = MockedBackendFactory.createBackend("127.0.0.1",
+                be_heartbeat_port, be_thrift_port, be_brpc_port, be_http_port,
+                new DefaultHeartbeatServiceImpl(be_thrift_port, be_http_port, be_brpc_port),
+                new DefaultBeThriftServiceImpl(), new DefaultPBackendServiceImpl());
+        backend.setFeAddress(new TNetworkAddress("127.0.0.1", frontend.getRpcPort()));
+        backend.start();
+
+        // add be
+        List<Pair<String, Integer>> bes = Lists.newArrayList();
+        bes.add(Pair.create(backend.getHost(), backend.getHeartbeatPort()));
+        Catalog.getCurrentSystemInfo().addBackends(bes, false, "default_cluster");
+
+        // sleep to wait first heartbeat
+        Thread.sleep(6000);
+    }
+
+    // generate all port from between 20000 ~ 30000
+    private static void getRandomPort() {
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2826: [UnitTest] Support starting mocked FE and BE process in unit test
URL: https://github.com/apache/incubator-doris/pull/2826#discussion_r373927500
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/PaloFe.java
 ##########
 @@ -51,25 +51,36 @@
 public class PaloFe {
     private static final Logger LOG = LogManager.getLogger(PaloFe.class);
 
-    // entrance for palo frontend
+    public static final String DORIS_HOME_DIR = System.getenv("DORIS_HOME");
+    public static final String PID_DIR = System.getenv("PID_DIR");
+
     public static void main(String[] args) {
+        start(DORIS_HOME_DIR, PID_DIR, args);
+    }
+
+    // entrance for doris frontend
+    public static void start(String dorisHomeDir, String pidDir, String[] args) {
+        if (Strings.isNullOrEmpty(dorisHomeDir)) {
+            System.out.println("env DORIS_HOME is not set");
 
 Review comment:
   System.err

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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