You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ni...@apache.org on 2019/08/06 12:56:57 UTC

[kylin] 01/02: KYLIN-4108 Show slow query hit cube

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

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

commit 03a2be03de4dc294a4366e7e6e7da4ba6f7f22f0
Author: chao long <wa...@qq.com>
AuthorDate: Thu Aug 1 23:13:54 2019 +0800

    KYLIN-4108 Show slow query hit cube
---
 .../kylin/metadata/badquery/BadQueryEntry.java     |  13 +-
 .../badquery/BadQueryHistoryManagerTest.java       |  10 +-
 .../query/relnode/OLAPToEnumerableConverter.java   |   7 +
 .../kylin/query/util/QueryInfoCollector.java       |  55 ++++++
 .../kylin/rest/service/BadQueryDetector.java       |  17 +-
 .../apache/kylin/rest/service/QueryService.java    |   2 +
 .../kylin/rest/service/BadQueryDetectorTest.java   |  49 ++++-
 .../kylin/rest/service/QueryInfoCollectorTest.java | 200 +++++++++++++++++++++
 8 files changed, 339 insertions(+), 14 deletions(-)

diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/badquery/BadQueryEntry.java b/core-metadata/src/main/java/org/apache/kylin/metadata/badquery/BadQueryEntry.java
index 56b27ca..41b1d11 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/badquery/BadQueryEntry.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/badquery/BadQueryEntry.java
@@ -49,9 +49,11 @@ public class BadQueryEntry extends RootPersistentEntity implements Comparable<Ba
     private String user;
     @JsonProperty("query_id")
     private String queryId;
+    @JsonProperty("cube")
+    private String cube;
 
     public BadQueryEntry(String sql, String adj, long startTime, float runningSec, String server, String thread,
-            String user, String queryId) {
+            String user, String queryId, String cube) {
         this.updateRandomUuid();
         this.adj = adj;
         this.sql = sql;
@@ -61,6 +63,7 @@ public class BadQueryEntry extends RootPersistentEntity implements Comparable<Ba
         this.thread = thread;
         this.user = user;
         this.queryId = queryId;
+        this.cube = cube;
     }
 
     public BadQueryEntry() {
@@ -131,6 +134,14 @@ public class BadQueryEntry extends RootPersistentEntity implements Comparable<Ba
         this.thread = thread;
     }
 
+    public String getCube() {
+        return cube;
+    }
+
+    public void setCube(String cube) {
+        this.cube = cube;
+    }
+
     @Override
     public int compareTo(BadQueryEntry obj) {
         int comp = Long.compare(this.startTime, obj.startTime);
diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/badquery/BadQueryHistoryManagerTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/badquery/BadQueryHistoryManagerTest.java
index 436eb10..a3aa890 100644
--- a/core-metadata/src/test/java/org/apache/kylin/metadata/badquery/BadQueryHistoryManagerTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/metadata/badquery/BadQueryHistoryManagerTest.java
@@ -67,7 +67,7 @@ public class BadQueryHistoryManagerTest extends LocalFileMetadataTestCase {
         KylinConfig kylinConfig = getTestConfig();
         BadQueryHistoryManager manager = BadQueryHistoryManager.getInstance(kylinConfig);
         BadQueryEntry entry = new BadQueryEntry("sql", "adj", 1459362239992L, 100, "server", "t-0", "user",
-                RandomUtil.randomUUID().toString());
+                RandomUtil.randomUUID().toString(), "cube");
         BadQueryHistory history = manager.upsertEntryToProject(entry, "default");
         NavigableSet<BadQueryEntry> entries = history.getEntries();
         assertEquals(4, entries.size());
@@ -84,7 +84,7 @@ public class BadQueryHistoryManagerTest extends LocalFileMetadataTestCase {
 
         for (int i = 0; i < kylinConfig.getBadQueryHistoryNum(); i++) {
             BadQueryEntry tmp = new BadQueryEntry("sql", "adj", 1459362239993L + i, 100 + i, "server", "t-0", "user",
-                    RandomUtil.randomUUID().toString());
+                    RandomUtil.randomUUID().toString(), "cube");
             history = manager.upsertEntryToProject(tmp, "default");
         }
         assertEquals(kylinConfig.getBadQueryHistoryNum(), history.getEntries().size());
@@ -97,14 +97,14 @@ public class BadQueryHistoryManagerTest extends LocalFileMetadataTestCase {
 
         String queryId = RandomUtil.randomUUID().toString();
         manager.upsertEntryToProject(
-                new BadQueryEntry("sql", "adj", 1459362239000L, 100, "server", "t-0", "user", queryId),
+                new BadQueryEntry("sql", "adj", 1459362239000L, 100, "server", "t-0", "user", queryId, "cube"),
                 "default");
         BadQueryHistory history = manager.upsertEntryToProject(
-                new BadQueryEntry("sql", "adj2", 1459362239000L, 120, "server2", "t-1", "user", queryId), "default");
+                new BadQueryEntry("sql", "adj2", 1459362239000L, 120, "server2", "t-1", "user", queryId, "cube"), "default");
 
         NavigableSet<BadQueryEntry> entries = history.getEntries();
         BadQueryEntry newEntry = entries
-                .floor(new BadQueryEntry("sql", "adj2", 1459362239000L, 120, "server2", "t-1", "user", queryId));
+                .floor(new BadQueryEntry("sql", "adj2", 1459362239000L, 120, "server2", "t-1", "user", queryId, "cube"));
         System.out.println(newEntry);
         assertEquals("adj2", newEntry.getAdj());
         assertEquals("server2", newEntry.getServer());
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
index ebfb6f4..e3b2272 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
@@ -19,6 +19,7 @@
 package org.apache.kylin.query.relnode;
 
 import java.util.List;
+import java.util.stream.Collectors;
 
 import org.apache.calcite.adapter.enumerable.EnumerableRel;
 import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
@@ -40,6 +41,7 @@ import org.apache.kylin.query.security.QueryInterceptor;
 import org.apache.kylin.query.security.QueryInterceptorUtil;
 
 import com.google.common.collect.Lists;
+import org.apache.kylin.query.util.QueryInfoCollector;
 
 /**
  * If you're renaming this class, please keep it ending with OLAPToEnumerableConverter
@@ -93,6 +95,11 @@ public class OLAPToEnumerableConverter extends ConverterImpl implements Enumerab
 
         RealizationChooser.selectRealization(contexts);
 
+        QueryInfoCollector.current().setCubeNames(contexts.stream()
+                .filter(olapContext -> olapContext.realization != null)
+                .map(olapContext -> olapContext.realization.getCanonicalName())
+                .collect(Collectors.toList()));
+
         doAccessControl(contexts);
 
         // rewrite query if necessary
diff --git a/query/src/main/java/org/apache/kylin/query/util/QueryInfoCollector.java b/query/src/main/java/org/apache/kylin/query/util/QueryInfoCollector.java
new file mode 100644
index 0000000..d8f6171
--- /dev/null
+++ b/query/src/main/java/org/apache/kylin/query/util/QueryInfoCollector.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.query.util;
+
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+
+import com.google.common.collect.Lists;
+
+public class QueryInfoCollector {
+    private List<String> cubeNames;
+
+    private static final ThreadLocal<QueryInfoCollector> queryInfoThreadLocal = ThreadLocal.withInitial(QueryInfoCollector::new);
+
+    public static QueryInfoCollector current() {
+        return queryInfoThreadLocal.get();
+    }
+
+    public static void reset() {
+        queryInfoThreadLocal.remove();
+    }
+
+    private QueryInfoCollector() {
+        this.cubeNames = Lists.newArrayList();
+    }
+
+    public List<String> getCubeNames() {
+        return cubeNames;
+    }
+
+    public void setCubeNames(List<String> cubeNames) {
+        this.cubeNames = cubeNames;
+    }
+
+    public String getCubeNameString() {
+        return StringUtils.join(this.cubeNames, ",");
+    }
+}
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java b/server-base/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
index 3f3db45..edabbb5 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentMap;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.metadata.badquery.BadQueryEntry;
 import org.apache.kylin.metadata.badquery.BadQueryHistoryManager;
+import org.apache.kylin.query.util.QueryInfoCollector;
 import org.apache.kylin.rest.request.SQLRequest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -102,7 +103,7 @@ public class BadQueryDetector extends Thread {
         for (Notifier notifier : notifiers) {
             try {
                 notifier.badQueryFound(adj, runningSec, //
-                        e.startTime, e.sqlRequest.getProject(), e.sqlRequest.getSql(), e.user, e.thread, e.queryId);
+                        e.startTime, e.sqlRequest.getProject(), e.sqlRequest.getSql(), e.user, e.thread, e.queryId, e.collector);
             } catch (Exception ex) {
                 logger.error("", ex);
             }
@@ -110,7 +111,7 @@ public class BadQueryDetector extends Thread {
     }
 
     public void queryStart(Thread thread, SQLRequest sqlRequest, String user, String queryId) {
-        runningQueries.put(thread, new Entry(sqlRequest, user, thread, queryId));
+        runningQueries.put(thread, new Entry(sqlRequest, user, thread, queryId, QueryInfoCollector.current()));
     }
 
     public void queryEnd(Thread thread) {
@@ -193,13 +194,13 @@ public class BadQueryDetector extends Thread {
 
     public interface Notifier {
         void badQueryFound(String adj, float runningSec, long startTime, String project, String sql, String user,
-                Thread t, String queryId);
+                Thread t, String queryId, QueryInfoCollector collector);
     }
 
     private class LoggerNotifier implements Notifier {
         @Override
         public void badQueryFound(String adj, float runningSec, long startTime, String project, String sql, String user,
-                Thread t, String queryId) {
+                Thread t, String queryId, QueryInfoCollector collector) {
             logger.info("{} query has been running {} seconds (project:{}, thread: 0x{}, user:{}, query id:{}) -- {}",
                     adj, runningSec, project, Long.toHexString(t.getId()), user, queryId, sql);
         }
@@ -220,10 +221,10 @@ public class BadQueryDetector extends Thread {
 
         @Override
         public void badQueryFound(String adj, float runningSec, long startTime, String project, String sql, String user,
-                Thread t, String queryId) {
+                Thread t, String queryId, QueryInfoCollector collector) {
             try {
                 BadQueryEntry entry = new BadQueryEntry(sql, adj, startTime, runningSec, serverHostname, t.getName(),
-                        user, queryId);
+                        user, queryId, collector.getCubeNameString());
                 badQueryManager.upsertEntryToProject(entry, project);
             } catch (IOException e) {
                 logger.error("Error in bad query persistence.", e);
@@ -237,13 +238,15 @@ public class BadQueryDetector extends Thread {
         final Thread thread;
         final String user;
         final String queryId;
+        final QueryInfoCollector collector;
 
-        Entry(SQLRequest sqlRequest, String user, Thread thread, String queryId) {
+        Entry(SQLRequest sqlRequest, String user, Thread thread, String queryId, QueryInfoCollector collector) {
             this.sqlRequest = sqlRequest;
             this.startTime = System.currentTimeMillis();
             this.thread = thread;
             this.user = user;
             this.queryId = queryId;
+            this.collector = collector;
         }
 
         @Override
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 59b2b61..171bd07 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -97,6 +97,7 @@ import org.apache.kylin.metrics.MetricsManager;
 import org.apache.kylin.query.QueryConnection;
 import org.apache.kylin.query.relnode.OLAPContext;
 import org.apache.kylin.query.util.PushDownUtil;
+import org.apache.kylin.query.util.QueryInfoCollector;
 import org.apache.kylin.query.util.QueryUtil;
 import org.apache.kylin.query.util.TempStatementUtil;
 import org.apache.kylin.rest.constant.Constant;
@@ -446,6 +447,7 @@ public class QueryService extends BasicService {
         } finally {
             BackdoorToggles.cleanToggles();
             QueryContextFacade.resetCurrent();
+            QueryInfoCollector.reset();
         }
     }
 
diff --git a/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java b/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
index ce1bedc..d1e2af9 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
@@ -20,12 +20,21 @@ package org.apache.kylin.rest.service;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.NavigableSet;
+import java.util.Set;
 
+import com.google.common.collect.Lists;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.common.util.RandomUtil;
 import org.apache.kylin.metadata.badquery.BadQueryEntry;
+import org.apache.kylin.metadata.badquery.BadQueryHistory;
+import org.apache.kylin.metadata.badquery.BadQueryHistoryManager;
+import org.apache.kylin.query.util.QueryInfoCollector;
 import org.apache.kylin.rest.request.SQLRequest;
 import org.junit.After;
 import org.junit.Before;
@@ -55,7 +64,7 @@ public class BadQueryDetectorTest extends LocalFileMetadataTestCase {
         badQueryDetector.registerNotifier(new BadQueryDetector.Notifier() {
             @Override
             public void badQueryFound(String adj, float runningSec, long startTime, String project, String sql,
-                    String user, Thread t, String queryId) {
+                    String user, Thread t, String queryId, QueryInfoCollector collect) {
                 alerts.add(new String[] { adj, sql });
             }
         });
@@ -82,4 +91,42 @@ public class BadQueryDetectorTest extends LocalFileMetadataTestCase {
         // end notifies a Pushdown
         assertArrayEquals(new String[] { BadQueryEntry.ADJ_PUSHDOWN, mockSql }, alerts.get(1));
     }
+
+    @Test
+    public void testSlowQuery() throws InterruptedException, IOException {
+        int alertMB = BadQueryDetector.getSystemAvailMB() * 2;
+        int alertRunningSec = 2;
+        String mockSql = "select * from just_a_test";
+
+        BadQueryDetector badQueryDetector = new BadQueryDetector(alertRunningSec * 1000, alertMB, alertRunningSec, 1000);
+        badQueryDetector.start();
+
+        SQLRequest sqlRequest = new SQLRequest();
+        sqlRequest.setProject("test_project");
+        sqlRequest.setSql(mockSql);
+        badQueryDetector.queryStart(Thread.currentThread(), sqlRequest, "user", RandomUtil.randomUUID().toString());
+
+        try {
+            Thread.sleep(1000);
+
+            QueryInfoCollector.current().setCubeNames(Lists.newArrayList("[CUBE[name=TEST_CUBE]]"));
+
+            Thread.sleep((alertRunningSec * 2 + 1) * 1000);
+
+            BadQueryHistory badQueryHistory = BadQueryHistoryManager.getInstance(KylinConfig.getInstanceFromEnv()).getBadQueriesForProject("test_project");
+            Set entries = badQueryHistory.getEntries();
+
+            assertEquals(1, entries.size());
+
+            BadQueryEntry entry = (BadQueryEntry) ((NavigableSet) entries).pollFirst();
+
+            assertNotNull(entry);
+            assertEquals(BadQueryEntry.ADJ_SLOW, entry.getAdj());
+            assertEquals("[CUBE[name=TEST_CUBE]]", entry.getCube());
+        } finally {
+            badQueryDetector.queryEnd(Thread.currentThread(), BadQueryEntry.ADJ_SLOW);
+            badQueryDetector.interrupt();
+            QueryInfoCollector.reset();
+        }
+    }
 }
diff --git a/server/src/test/java/org/apache/kylin/rest/service/QueryInfoCollectorTest.java b/server/src/test/java/org/apache/kylin/rest/service/QueryInfoCollectorTest.java
new file mode 100644
index 0000000..91cf190
--- /dev/null
+++ b/server/src/test/java/org/apache/kylin/rest/service/QueryInfoCollectorTest.java
@@ -0,0 +1,200 @@
+/*
+ * 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.kylin.rest.service;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.QueryContextFacade;
+import org.apache.kylin.common.debug.BackdoorToggles;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.metadata.realization.RealizationStatusEnum;
+import org.apache.kylin.query.QueryConnection;
+import org.apache.kylin.query.util.QueryInfoCollector;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class QueryInfoCollectorTest extends LocalFileMetadataTestCase {
+    private Connection connection = null;
+    private Statement statement = null;
+    private ResultSet resultSet = null;
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testQueryInfoCollector() throws Exception {
+        prepareContexts();
+        enableCube("ci_inner_join_cube", "ci_left_join_cube");
+
+        try {
+            connection = QueryConnection.getConnection("default");
+            statement = connection.createStatement();
+            String sql = "select count(*) as cnt1 from test_kylin_fact inner join test_account on seller_id = account_id\n" +
+                    "union all\n" +
+                    "select count(*) as cnt2 from test_kylin_fact left join test_account on seller_id = account_id";
+            resultSet = statement.executeQuery(sql);
+
+            Assert.assertNotNull(resultSet);
+
+            List<String> cubes = QueryInfoCollector.current().getCubeNames();
+
+            Assert.assertTrue(cubes.contains("HYBRID[name=ci_inner_join_hybrid]"));
+            Assert.assertTrue(cubes.contains("CUBE[name=ci_left_join_cube]"));
+        } finally {
+            cleanContexts();
+            close();
+        }
+    }
+
+    @Test
+    public void testQueryInfoCollectorReset() throws Exception {
+        prepareContexts();
+        enableCube("ci_left_join_cube");
+
+        ExecutorService executorService = Executors.newSingleThreadExecutor();
+        try {
+            String project = "default";
+            String expectedCube = "CUBE[name=ci_left_join_cube]";
+
+            String sqlWithCube = "select count(*) from test_kylin_fact";
+            FutureTask<String> queryTask1 = new FutureTask<String>(new QueryCallable(sqlWithCube, project, false));
+            executorService.submit(queryTask1);
+
+            String cubeName1 = queryTask1.get(2, TimeUnit.MINUTES);
+
+            Assert.assertTrue(queryTask1.isDone());
+            Assert.assertEquals(expectedCube, cubeName1);
+
+            String sqlNoCube = "select * from test_account";
+            FutureTask<String> queryTask2 = new FutureTask<String>(new QueryCallable(sqlNoCube, project, true));
+            executorService.submit(queryTask2);
+
+            String cubeName2 = queryTask2.get(2, TimeUnit.MINUTES);
+
+            Assert.assertTrue(queryTask2.isDone());
+            Assert.assertEquals(cubeName1, cubeName2);
+
+            FutureTask<String> queryTask3 = new FutureTask<String>(new QueryCallable(sqlNoCube, project, true));
+            executorService.submit(queryTask3);
+
+            String cubeName3 = queryTask3.get(2, TimeUnit.MINUTES);
+
+            Assert.assertTrue(queryTask3.isDone());
+            Assert.assertEquals("", cubeName3);
+        } finally {
+            executorService.shutdown();
+            cleanContexts();
+        }
+    }
+
+    class QueryCallable implements Callable<String> {
+        private String sql;
+        private String project;
+        private boolean reset;
+
+        public QueryCallable(String sql, String project, boolean reset) {
+            this.sql = sql;
+            this.project = project;
+            this.reset = reset;
+        }
+
+        @Override
+        public String call() throws Exception {
+            Connection connection = QueryConnection.getConnection(project);
+            Statement statement = connection.createStatement();
+
+            try {
+                statement.executeQuery(sql);
+                return QueryInfoCollector.current().getCubeNameString();
+            } catch (Exception e) {
+                return QueryInfoCollector.current().getCubeNameString();
+            } finally {
+                if (reset) {
+                    QueryInfoCollector.reset();
+                }
+            }
+        }
+    }
+
+    private void cleanContexts() {
+        QueryContextFacade.resetCurrent();
+        QueryInfoCollector.reset();
+        BackdoorToggles.cleanToggles();
+    }
+
+    private void prepareContexts() {
+        QueryContextFacade.resetCurrent();
+        BackdoorToggles.addToggle(BackdoorToggles.DEBUG_TOGGLE_PREPARE_ONLY, "true");
+    }
+
+    private void enableCube(String... cubes) throws IOException {
+        CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+        for (String cube : cubes) {
+            CubeInstance cubeInstance = cubeManager.getCube(cube);
+            cubeManager.updateCubeStatus(cubeInstance, RealizationStatusEnum.READY);
+        }
+    }
+
+    private void close() {
+        if (resultSet != null) {
+            try {
+                resultSet.close();
+            } catch (SQLException e) {
+                // ignore
+            }
+        }
+        if (statement != null) {
+            try {
+                statement.close();
+            } catch (SQLException e) {
+                // ignore
+            }
+        }
+        if (connection != null) {
+            try {
+                connection.close();
+            } catch (SQLException e) {
+                // ignore
+            }
+        }
+    }
+}