You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2015/09/28 03:10:38 UTC

[18/33] hive git commit: HIVE-11336: Support initial file option for new CLI [beeline-cli branch](Ferdinand Xu, reviewed by Xuefu Zhang)

HIVE-11336: Support initial file option for new CLI [beeline-cli branch](Ferdinand Xu, reviewed by Xuefu Zhang)


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

Branch: refs/heads/master
Commit: e28043fb4753b26daaa8a01da61ceda04b970645
Parents: edbd8db
Author: Ferdinand Xu <ch...@intel.com>
Authored: Mon Jul 27 22:34:47 2015 -0400
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Mon Jul 27 22:34:47 2015 -0400

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   | 44 +++++++++++++++-----
 .../org/apache/hive/beeline/BeeLineOpts.java    | 10 ++---
 2 files changed, 39 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e28043fb/beeline/src/java/org/apache/hive/beeline/BeeLine.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
index d4beefc..406059d 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -22,7 +22,6 @@
  */
 package org.apache.hive.beeline;
 
-import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
@@ -31,7 +30,6 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.InputStreamReader;
 import java.io.PrintStream;
 import java.io.SequenceInputStream;
 import java.lang.reflect.InvocationTargetException;
@@ -672,6 +670,10 @@ public class BeeLine implements Closeable {
 
     getOpts().setScriptFile(commandLine.getOptionValue("f"));
 
+    if (commandLine.getOptionValues("i") != null) {
+      getOpts().setInitFiles(commandLine.getOptionValues("i"));
+    }
+
     dbName = commandLine.getOptionValue("database");
     getOpts().setVerbose(Boolean.valueOf(commandLine.getOptionValue("verbose")));
     getOpts().setSilent(Boolean.valueOf(commandLine.getOptionValue("slient")));
@@ -747,7 +749,7 @@ public class BeeLine implements Closeable {
       pass = cl.getOptionValue("p");
     }
     url = cl.getOptionValue("u");
-    getOpts().setInitFile(cl.getOptionValue("i"));
+    getOpts().setInitFiles(cl.getOptionValues("i"));
     getOpts().setScriptFile(cl.getOptionValue("f"));
     if (cl.getOptionValues('e') != null) {
       commands = Arrays.asList(cl.getOptionValues('e'));
@@ -835,6 +837,8 @@ public class BeeLine implements Closeable {
         getOpts().setShowHeader(false);
         getOpts().setOutputFormat("dsv");
         getOpts().setDelimiterForDSV(' ');
+
+        processInitFiles(opts.getInitFiles());
       }
 
       if (getOpts().getScriptFile() != null) {
@@ -853,13 +857,15 @@ public class BeeLine implements Closeable {
   }
 
   int runInit() {
-    String initFile = getOpts().getInitFile();
-    if (initFile != null) {
-      info("Running init script " + initFile);
-      try {
-        return executeFile(initFile);
-      } finally {
-        exit = false;
+    String initFiles[] = getOpts().getInitFiles();
+    if (initFiles != null && initFiles.length != 0) {
+      for (String initFile : initFiles) {
+        info("Running init script " + initFile);
+        try {
+          return executeFile(initFile);
+        } finally {
+          exit = false;
+        }
       }
     }
     return ERRNO_OK;
@@ -920,6 +926,24 @@ public class BeeLine implements Closeable {
     }
   }
 
+  /**
+   * Only initial files specified by i option will be executed. The hiverc file will be processed by session manager.
+   *
+   * @param files
+   * @throws IOException
+   */
+  public void processInitFiles(String[] files) throws IOException {
+    if (files == null || files.length == 0) {
+      return;
+    }
+    for (String initFile : files) {
+      int rc = executeFile(initFile);
+      if (rc != 0) {
+        System.exit(rc);
+      }
+    }
+  }
+
   private int execute(ConsoleReader reader, boolean exitOnError) {
     String line;
     while (!exit) {

http://git-wip-us.apache.org/repos/asf/hive/blob/e28043fb/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
index fec95f2..d9f726d 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
@@ -93,7 +93,7 @@ class BeeLineOpts implements Completer {
   private String historyFile = new File(saveDir(), "history").getAbsolutePath();
 
   private String scriptFile = null;
-  private String initFile = null;
+  private String[] initFiles = null;
   private String authType = null;
   private char delimiterForDSV = DEFAULT_DELIMITER_FOR_DSV;
 
@@ -388,12 +388,12 @@ class BeeLineOpts implements Completer {
     return scriptFile;
   }
 
-  public String getInitFile() {
-    return initFile;
+  public String[] getInitFiles() {
+    return initFiles;
   }
 
-  public void setInitFile(String initFile) {
-    this.initFile = initFile;
+  public void setInitFiles(String[] initFiles) {
+    this.initFiles = initFiles;
   }
 
   public void setColor(boolean color) {