You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2016/02/29 12:29:20 UTC

[1/2] kylin git commit: KYLIN-1054 Support Hive client Beeline

Repository: kylin
Updated Branches:
  refs/heads/2.x-staging 7df1cc74d -> 4d6043f1c


KYLIN-1054 Support Hive client Beeline


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/9c77a5eb
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/9c77a5eb
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/9c77a5eb

Branch: refs/heads/2.x-staging
Commit: 9c77a5ebe955e708c51f9ea9c3cf3d747e880cd7
Parents: 7df1cc7
Author: lidongsjtu <li...@apache.org>
Authored: Mon Feb 29 19:26:56 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Mon Feb 29 19:26:56 2016 +0800

----------------------------------------------------------------------
 build/bin/find-hive-dependency.sh               |  11 +-
 .../apache/kylin/common/KylinConfigBase.java    |  12 +-
 .../test_case_data/sandbox/kylin.properties     |   3 +
 .../kylin/provision/BuildIIWithStream.java      |  17 ++-
 .../kylin/source/hive/HiveCmdBuilder.java       | 106 ++++++++++++++++++
 .../apache/kylin/source/hive/HiveMRInput.java   |  25 +++--
 .../kylin/source/hive/HiveCmdBuilderTest.java   |  78 +++++++++++++
 .../storage/hbase/steps/DeprecatedGCStep.java   |   6 +-
 .../storage/hbase/util/HiveCmdBuilder.java      | 109 +++++++++++++++++++
 .../storage/hbase/util/StorageCleanupJob.java   |  24 ++--
 10 files changed, 351 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/9c77a5eb/build/bin/find-hive-dependency.sh
----------------------------------------------------------------------
diff --git a/build/bin/find-hive-dependency.sh b/build/bin/find-hive-dependency.sh
index 5994dda..171c5b1 100644
--- a/build/bin/find-hive-dependency.sh
+++ b/build/bin/find-hive-dependency.sh
@@ -17,7 +17,16 @@
 # limitations under the License.
 #
 
-hive_env=`hive -e set | grep 'env:CLASSPATH'`
+client_mode=`sh ${KYLIN_HOME}/bin/get-properties.sh kylin.hive.client`
+hive_env=
+
+if [ "${client_mode}" == "beeline" ]
+then
+    beeline_params=`sh ${KYLIN_HOME}/bin/get-properties.sh kylin.hive.beeline.params`
+    hive_env=`beeline ${beeline_params} --outputformat=dsv -e set | grep 'env:CLASSPATH'`
+else
+    hive_env=`hive -e set | grep 'env:CLASSPATH'`
+fi
 
 hive_classpath=`echo $hive_env | grep 'env:CLASSPATH' | awk -F '=' '{print $2}'`
 arr=(`echo $hive_classpath | cut -d ":"  --output-delimiter=" " -f 1-`)

http://git-wip-us.apache.org/repos/asf/kylin/blob/9c77a5eb/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 0cee9f8..826a28c 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -367,7 +367,7 @@ public class KylinConfigBase implements Serializable {
     public double getCubeAlgorithmAutoThreshold() {
         return Double.parseDouble(getOptional("kylin.cube.algorithm.auto.threshold", "8"));
     }
-    
+
     public int getCubeAggrGroupMaxSize() {
         return Integer.parseInt(getOptional("kylin.cube.aggrgroup.max.size", "12"));
     }
@@ -533,7 +533,7 @@ public class KylinConfigBase implements Serializable {
     public String getMailSender() {
         return getOptional("mail.sender", "");
     }
-    
+
     public boolean isWebCrossDomainEnabled() {
         return Boolean.parseBoolean(getOptional("crossdomain.enable", "true"));
     }
@@ -542,4 +542,12 @@ public class KylinConfigBase implements Serializable {
         return getMetadataUrl();
     }
 
+    public String getHiveClientMode() {
+        return getOptional("kylin.hive.client", "cli");
+    }
+
+    public String getHiveBeelineParams() {
+        return getOptional("kylin.hive.beeline.params", "");
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/9c77a5eb/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
index bf161fc..a304cab 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -20,6 +20,9 @@ kylin.storage.url=hbase
 # Temp folder in hdfs, make sure user has the right access to the hdfs directory
 kylin.hdfs.working.dir=/kylin
 
+# Parameters for beeline client
+kylin.hive.beeline.params=--hiveconf hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' -u 'jdbc:hive2://localhost:10000'
+
 kylin.job.mapreduce.default.reduce.input.mb=500
 
 # If true, job engine will not assume that hadoop CLI reside on the same server as it self

http://git-wip-us.apache.org/repos/asf/kylin/blob/9c77a5eb/kylin-it/src/test/java/org/apache/kylin/provision/BuildIIWithStream.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildIIWithStream.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildIIWithStream.java
index 8436687..9b7cd14 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildIIWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildIIWithStream.java
@@ -74,6 +74,7 @@ import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
+import org.apache.kylin.source.hive.HiveCmdBuilder;
 import org.apache.kylin.source.hive.HiveTableReader;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.ii.IICreateHTableJob;
@@ -147,15 +148,13 @@ public class BuildIIWithStream {
         }
 
         ShellExecutable step = new ShellExecutable();
-        StringBuffer buf = new StringBuffer();
-        buf.append("hive -e \"");
-        buf.append(useDatabaseHql + "\n");
-        buf.append(dropTableHql + "\n");
-        buf.append(createTableHql + "\n");
-        buf.append(insertDataHqls + "\n");
-        buf.append("\"");
-
-        step.setCmd(buf.toString());
+        HiveCmdBuilder hiveCmdBuilder = new HiveCmdBuilder();
+        hiveCmdBuilder.addStatement(useDatabaseHql);
+        hiveCmdBuilder.addStatement(dropTableHql);
+        hiveCmdBuilder.addStatement(createTableHql);
+        hiveCmdBuilder.addStatement(insertDataHqls);
+
+        step.setCmd(hiveCmdBuilder.build());
         logger.info(step.getCmd());
         step.setName(ExecutableConstants.STEP_NAME_CREATE_FLAT_HIVE_TABLE);
         kylinConfig.getCliCommandExecutor().execute(step.getCmd(), null);

http://git-wip-us.apache.org/repos/asf/kylin/blob/9c77a5eb/source-hive/src/main/java/org/apache/kylin/source/hive/HiveCmdBuilder.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveCmdBuilder.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveCmdBuilder.java
new file mode 100644
index 0000000..f510780
--- /dev/null
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveCmdBuilder.java
@@ -0,0 +1,106 @@
+/*
+ * 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.source.hive;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+public class HiveCmdBuilder {
+    private static final Logger logger = LoggerFactory.getLogger(HiveCmdBuilder.class);
+
+    public enum HiveClientMode {
+        CLI, BEELINE
+    }
+
+    private HiveClientMode clientMode;
+    private KylinConfig kylinConfig;
+    final private ArrayList<String> statements = Lists.newArrayList();
+
+    public HiveCmdBuilder() {
+        kylinConfig = KylinConfig.getInstanceFromEnv();
+        clientMode = HiveClientMode.valueOf(kylinConfig.getHiveClientMode().toUpperCase());
+    }
+
+    public String build() {
+        StringBuffer buf = new StringBuffer();
+
+        switch (clientMode) {
+        case CLI:
+            buf.append("hive -e \"");
+            for (String statement : statements) {
+                buf.append(statement).append("\n");
+            }
+            buf.append("\"");
+            break;
+        case BEELINE:
+            BufferedWriter bw = null;
+            try {
+                File tmpHql = File.createTempFile("beeline_", ".hql");
+                StringBuffer hqlBuf = new StringBuffer();
+                bw = new BufferedWriter(new FileWriter(tmpHql));
+                for (String statement : statements) {
+                    bw.write(statement);
+                    bw.newLine();
+
+                    hqlBuf.append(statement).append("\n");
+                }
+                buf.append("beeline ");
+                buf.append(kylinConfig.getHiveBeelineParams());
+                buf.append(" -f ");
+                buf.append(tmpHql.getAbsolutePath());
+                buf.append(";rm -f ");
+                buf.append(tmpHql.getAbsolutePath());
+
+                logger.info("The statements to execute in beeline: \n" + hqlBuf);
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            } finally {
+                IOUtils.closeQuietly(bw);
+            }
+            break;
+        default:
+            throw new RuntimeException("Hive client cannot be recognized: " + clientMode);
+        }
+
+        return buf.toString();
+    }
+
+    public void reset() {
+        statements.clear();
+    }
+
+    public void addStatement(String statement) {
+        statements.add(statement);
+    }
+
+    @Override
+    public String toString() {
+        return build();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/9c77a5eb/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index b8d1333..873641d 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hive.hcatalog.data.HCatRecord;
@@ -123,15 +124,14 @@ public class HiveMRInput implements IMRInput {
             }
 
             ShellExecutable step = new ShellExecutable();
-            StringBuilder buf = new StringBuilder();
-            buf.append("hive -e \"");
-            buf.append(useDatabaseHql + "\n");
-            buf.append(dropTableHql + "\n");
-            buf.append(createTableHql + "\n");
-            buf.append(insertDataHqls + "\n");
-            buf.append("\"");
-
-            step.setCmd(buf.toString());
+
+            HiveCmdBuilder hiveCmdBuilder = new HiveCmdBuilder();
+            hiveCmdBuilder.addStatement(useDatabaseHql);
+            hiveCmdBuilder.addStatement(dropTableHql);
+            hiveCmdBuilder.addStatement(createTableHql);
+            hiveCmdBuilder.addStatement(insertDataHqls);
+
+            step.setCmd(hiveCmdBuilder.build());
             step.setName(ExecutableConstants.STEP_NAME_CREATE_FLAT_HIVE_TABLE);
 
             return step;
@@ -164,10 +164,11 @@ public class HiveMRInput implements IMRInput {
 
             final String hiveTable = this.getIntermediateTableIdentity();
             if (config.isHiveKeepFlatTable() == false && StringUtils.isNotEmpty(hiveTable)) {
-                final String dropSQL = "USE " + context.getConfig().getHiveDatabaseForIntermediateTable() + ";" + " DROP TABLE IF EXISTS  " + hiveTable + ";";
-                final String dropHiveCMD = "hive -e \"" + dropSQL + "\"";
+                final HiveCmdBuilder hiveCmdBuilder = new HiveCmdBuilder();
+                hiveCmdBuilder.addStatement("USE " + context.getConfig().getHiveDatabaseForIntermediateTable() + ";");
+                hiveCmdBuilder.addStatement("DROP TABLE IF EXISTS  " + hiveTable + ";");
                 try {
-                    config.getCliCommandExecutor().execute(dropHiveCMD);
+                    config.getCliCommandExecutor().execute(hiveCmdBuilder.build());
                     output.append("Hive table " + hiveTable + " is dropped. \n");
 
                     Path externalDataPath = new Path(getExternalDataPath());

http://git-wip-us.apache.org/repos/asf/kylin/blob/9c77a5eb/source-hive/src/test/java/org/apache/kylin/source/hive/HiveCmdBuilderTest.java
----------------------------------------------------------------------
diff --git a/source-hive/src/test/java/org/apache/kylin/source/hive/HiveCmdBuilderTest.java b/source-hive/src/test/java/org/apache/kylin/source/hive/HiveCmdBuilderTest.java
new file mode 100644
index 0000000..70c11b3
--- /dev/null
+++ b/source-hive/src/test/java/org/apache/kylin/source/hive/HiveCmdBuilderTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.source.hive;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Created by dongli on 2/22/16.
+ */
+public class HiveCmdBuilderTest {
+
+    @Before
+    public void setup() {
+        System.setProperty("KYLIN_CONF", "../examples/test_case_data/localmeta");
+    }
+
+    @After
+    public void after() throws Exception {
+        System.clearProperty("kylin.hive.client");
+        System.clearProperty("kylin.hive.beeline.params");
+    }
+
+    @Test
+    public void testHiveCLI() {
+        System.setProperty("kylin.hive.client", "cli");
+
+        HiveCmdBuilder hiveCmdBuilder = new HiveCmdBuilder();
+        hiveCmdBuilder.addStatement("USE default;");
+        hiveCmdBuilder.addStatement("DROP TABLE test;");
+        hiveCmdBuilder.addStatement("SHOW\n TABLES;");
+
+        assertEquals("hive -e \"USE default;\nDROP TABLE test;\nSHOW\n TABLES;\n\"", hiveCmdBuilder.build());
+    }
+
+    @Test
+    public void testBeeline() throws IOException {
+        System.setProperty("kylin.hive.client", "beeline");
+        System.setProperty("kylin.hive.beeline.params", "-u jdbc_url");
+
+        HiveCmdBuilder hiveCmdBuilder = new HiveCmdBuilder();
+        hiveCmdBuilder.addStatement("USE default;");
+        hiveCmdBuilder.addStatement("DROP TABLE test;");
+        hiveCmdBuilder.addStatement("SHOW\n TABLES;");
+
+        String cmd = hiveCmdBuilder.build();
+        assertTrue(cmd.startsWith("beeline -u jdbc_url -f") && cmd.contains(";rm -f"));
+
+        String hqlFile = cmd.substring(cmd.lastIndexOf("-f ") + 3).trim();
+        String hqlStatement = FileUtils.readFileToString(new File(hqlFile));
+        assertEquals("USE default;\nDROP TABLE test;\nSHOW\n TABLES;\n", hqlStatement);
+
+        FileUtils.forceDelete(new File(hqlFile));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/9c77a5eb/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
index bba16c3..735f967 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
@@ -38,6 +38,7 @@ import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableContext;
 import org.apache.kylin.job.execution.ExecuteResult;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.storage.hbase.util.HiveCmdBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -86,8 +87,9 @@ public class DeprecatedGCStep extends AbstractExecutable {
         final String hiveTable = this.getOldHiveTable();
         if (StringUtils.isNotEmpty(hiveTable)) {
             final String dropSQL = "USE " + context.getConfig().getHiveDatabaseForIntermediateTable() + ";" + " DROP TABLE IF EXISTS  " + hiveTable + ";";
-            final String dropHiveCMD = "hive -e \"" + dropSQL + "\"";
-            context.getConfig().getCliCommandExecutor().execute(dropHiveCMD);
+            final HiveCmdBuilder hiveCmdBuilder = new HiveCmdBuilder();
+            hiveCmdBuilder.addStatement(dropSQL);
+            context.getConfig().getCliCommandExecutor().execute(hiveCmdBuilder.build());
             output.append("Dropped Hive table " + hiveTable + " \n");
         }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/9c77a5eb/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HiveCmdBuilder.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HiveCmdBuilder.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HiveCmdBuilder.java
new file mode 100644
index 0000000..f1ca4de
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HiveCmdBuilder.java
@@ -0,0 +1,109 @@
+/*
+ * 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.storage.hbase.util;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Created by dongli on 2/29/16.
+ */
+public class HiveCmdBuilder {
+    private static final Logger logger = LoggerFactory.getLogger(HiveCmdBuilder.class);
+
+    public enum HiveClientMode {
+        CLI, BEELINE
+    }
+
+    private HiveClientMode clientMode;
+    private KylinConfig kylinConfig;
+    final private ArrayList<String> statements = Lists.newArrayList();
+
+    public HiveCmdBuilder() {
+        kylinConfig = KylinConfig.getInstanceFromEnv();
+        clientMode = HiveClientMode.valueOf(kylinConfig.getHiveClientMode().toUpperCase());
+    }
+
+    public String build() {
+        StringBuffer buf = new StringBuffer();
+
+        switch (clientMode) {
+            case CLI:
+                buf.append("hive -e \"");
+                for (String statement : statements) {
+                    buf.append(statement).append("\n");
+                }
+                buf.append("\"");
+                break;
+            case BEELINE:
+                BufferedWriter bw = null;
+                try {
+                    File tmpHql = File.createTempFile("beeline_", ".hql");
+                    StringBuffer hqlBuf = new StringBuffer();
+                    bw = new BufferedWriter(new FileWriter(tmpHql));
+                    for (String statement : statements) {
+                        bw.write(statement);
+                        bw.newLine();
+
+                        hqlBuf.append(statement).append("\n");
+                    }
+                    buf.append("beeline ");
+                    buf.append(kylinConfig.getHiveBeelineParams());
+                    buf.append(" -f ");
+                    buf.append(tmpHql.getAbsolutePath());
+                    buf.append(";rm -f ");
+                    buf.append(tmpHql.getAbsolutePath());
+
+                    logger.info("The statements to execute in beeline: \n" + hqlBuf);
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
+                } finally {
+                    IOUtils.closeQuietly(bw);
+                }
+                break;
+            default:
+                throw new RuntimeException("Hive client cannot be recognized: " + clientMode);
+        }
+
+        return buf.toString();
+    }
+
+    public void reset() {
+        statements.clear();
+    }
+
+    public void addStatement(String statement) {
+        statements.add(statement);
+    }
+
+    @Override
+    public String toString() {
+        return build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/9c77a5eb/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
index 2137f57..c010d51 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
@@ -240,13 +240,11 @@ public class StorageCleanupJob extends AbstractHadoopJob {
         final int uuidLength = 36;
         
         final String useDatabaseHql = "USE " + config.getHiveDatabaseForIntermediateTable() + ";";
-        StringBuilder buf = new StringBuilder();
-        buf.append("hive -e \"");
-        buf.append(useDatabaseHql);
-        buf.append("show tables " + "\'kylin_intermediate_*\'" + "; ");
-        buf.append("\"");
-        
-        Pair<Integer, String> result = cmdExec.execute(buf.toString());
+        final HiveCmdBuilder hiveCmdBuilder = new HiveCmdBuilder();
+        hiveCmdBuilder.addStatement(useDatabaseHql);
+        hiveCmdBuilder.addStatement("show tables " + "\'kylin_intermediate_*\'" + "; ");
+
+        Pair<Integer, String> result = cmdExec.execute(hiveCmdBuilder.build());
 
         String outputStr = result.getSecond();
         BufferedReader reader = new BufferedReader(new StringReader(outputStr));
@@ -282,17 +280,15 @@ public class StorageCleanupJob extends AbstractHadoopJob {
         }
 
         if (delete == true) {
-            buf.delete(0, buf.length());
-            buf.append("hive -e \"");
-            buf.append(useDatabaseHql);
+            hiveCmdBuilder.reset();
+            hiveCmdBuilder.addStatement(useDatabaseHql);
             for (String delHive : allHiveTablesNeedToBeDeleted) {
-                buf.append("drop table if exists " + delHive + "; ");
+                hiveCmdBuilder.addStatement("drop table if exists " + delHive + "; ");
                 logger.info("Remove " + delHive + " from hive tables.");
             }
-            buf.append("\"");
-            
+
             try {
-                cmdExec.execute(buf.toString());
+                cmdExec.execute(hiveCmdBuilder.build());
             } catch (IOException e) {
                 e.printStackTrace();
             }


[2/2] kylin git commit: KYLIN-1445 Check HIVE_CONF directory before startup Kylin instance

Posted by li...@apache.org.
KYLIN-1445 Check HIVE_CONF directory before startup Kylin instance


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/4d6043f1
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/4d6043f1
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/4d6043f1

Branch: refs/heads/2.x-staging
Commit: 4d6043f1c66a358f3c5e57e9cb14861401f09b28
Parents: 9c77a5e
Author: lidongsjtu <li...@apache.org>
Authored: Wed Feb 24 15:42:52 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Mon Feb 29 19:27:13 2016 +0800

----------------------------------------------------------------------
 build/bin/find-hive-dependency.sh | 24 +++++++++++++++++++++---
 1 file changed, 21 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4d6043f1/build/bin/find-hive-dependency.sh
----------------------------------------------------------------------
diff --git a/build/bin/find-hive-dependency.sh b/build/bin/find-hive-dependency.sh
index 171c5b1..87f18dd 100644
--- a/build/bin/find-hive-dependency.sh
+++ b/build/bin/find-hive-dependency.sh
@@ -32,6 +32,13 @@ hive_classpath=`echo $hive_env | grep 'env:CLASSPATH' | awk -F '=' '{print $2}'`
 arr=(`echo $hive_classpath | cut -d ":"  --output-delimiter=" " -f 1-`)
 hive_conf_path=
 hive_exec_path=
+
+if [ -n "$HIVE_CONF" ]
+then
+    echo "HIVE_CONF is set to: $HIVE_CONF, use it to locate hive configurations."
+    hive_conf_path=$HIVE_CONF
+fi
+
 for data in ${arr[@]}
 do
     result=`echo $data | grep -e 'hive-exec[a-z0-9A-Z\.-]*jar'`
@@ -39,13 +46,24 @@ do
     then
         hive_exec_path=$data
     fi
-    result=`echo $data | grep -e 'hive[^/]*/conf'`
-    if [ $result ]
+
+    # in some versions of hive config is not in hive's classpath, find it separately
+    if [ -z "$hive_conf_path" ]
     then
-        hive_conf_path=$data
+        result=`echo $data | grep -e 'hive[^/]*/conf'`
+        if [ $result ]
+        then
+            hive_conf_path=$data
+        fi
     fi
 done
 
+if [ -z "$hive_conf_path" ]
+then
+    echo "Couldn't find hive configuration directory. Please set HIVE_CONF to the path which contains hive-site.xml."
+    exit 1
+fi
+
 # in some versions of hive hcatalog is not in hive's classpath, find it separately
 if [ -z "$HCAT_HOME" ]
 then