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 2014/06/19 01:30:51 UTC

svn commit: r1603671 - in /hive/trunk: beeline/src/java/org/apache/hive/beeline/ beeline/src/main/resources/ itests/hive-unit/src/test/java/org/apache/hive/beeline/

Author: xuefu
Date: Wed Jun 18 23:30:51 2014
New Revision: 1603671

URL: http://svn.apache.org/r1603671
Log:
HIVE-6561: Beeline should accept -i option to Initializing a SQL file (Navis via Xuefu)

Modified:
    hive/trunk/beeline/src/java/org/apache/hive/beeline/BeeLine.java
    hive/trunk/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
    hive/trunk/beeline/src/java/org/apache/hive/beeline/Commands.java
    hive/trunk/beeline/src/main/resources/BeeLine.properties
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java

Modified: hive/trunk/beeline/src/java/org/apache/hive/beeline/BeeLine.java
URL: http://svn.apache.org/viewvc/hive/trunk/beeline/src/java/org/apache/hive/beeline/BeeLine.java?rev=1603671&r1=1603670&r2=1603671&view=diff
==============================================================================
--- hive/trunk/beeline/src/java/org/apache/hive/beeline/BeeLine.java (original)
+++ hive/trunk/beeline/src/java/org/apache/hive/beeline/BeeLine.java Wed Jun 18 23:30:51 2014
@@ -24,6 +24,7 @@ package org.apache.hive.beeline;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
 import java.io.EOFException;
 import java.io.File;
 import java.io.FileInputStream;
@@ -76,7 +77,9 @@ import jline.ClassNameCompletor;
 import jline.Completor;
 import jline.ConsoleReader;
 import jline.FileNameCompletor;
+import jline.History;
 import jline.SimpleCompletor;
+import org.apache.hadoop.io.IOUtils;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
@@ -101,7 +104,7 @@ import org.apache.commons.cli.ParseExcep
  * </ul>
  *
  */
-public class BeeLine {
+public class BeeLine implements Closeable {
   private static final ResourceBundle resourceBundle =
       ResourceBundle.getBundle(BeeLine.class.getSimpleName());
   private final BeeLineSignalHandler signalHandler = null;
@@ -123,6 +126,8 @@ public class BeeLine {
   private List<String> batch = null;
   private final Reflector reflector;
 
+  private History history;
+
   private static final Options options = new Options();
 
   public static final String BEELINE_DEFAULT_JDBC_DRIVER = "org.apache.hive.jdbc.HiveDriver";
@@ -291,6 +296,13 @@ public class BeeLine {
         .withDescription("the authentication type")
         .create('a'));
 
+    // -i <init file>
+    options.addOption(OptionBuilder
+        .hasArg()
+        .withArgName("init")
+        .withDescription("script file for initialization")
+        .create('i'));
+
     // -e <query>
     options.addOption(OptionBuilder
         .hasArgs()
@@ -298,7 +310,7 @@ public class BeeLine {
         .withDescription("query that should be executed")
         .create('e'));
 
-    // -f <file>
+    // -f <script file>
     options.addOption(OptionBuilder
         .hasArg()
         .withArgName("file")
@@ -618,7 +630,7 @@ public class BeeLine {
       return false;
     }
 
-    String driver = null, user = null, pass = null, url = null, cmd = null;
+    String driver = null, user = null, pass = null, url = null;
     String auth = null;
 
 
@@ -643,6 +655,7 @@ public class BeeLine {
     getOpts().setAuthType(auth);
     pass = cl.getOptionValue("p");
     url = cl.getOptionValue("u");
+    getOpts().setInitFile(cl.getOptionValue("i"));
     getOpts().setScriptFile(cl.getOptionValue("f"));
     if (cl.getOptionValues('e') != null) {
       commands = Arrays.asList(cl.getOptionValues('e'));
@@ -674,7 +687,6 @@ public class BeeLine {
       dispatch("!properties " + i.next());
     }
 
-
     if (commands.size() > 0) {
       // for single command execute, disable color
       getOpts().setColor(false);
@@ -697,7 +709,6 @@ public class BeeLine {
    * global variable <code>exit</code> is true.
    */
   public int begin(String[] args, InputStream inputStream) throws IOException {
-    int status = ERRNO_OK;
     try {
       // load the options first, so we can override on the command line
       getOpts().load();
@@ -705,55 +716,72 @@ public class BeeLine {
       // nothing
     }
 
-    if (!(initArgs(args))) {
-      usage();
-      return ERRNO_ARGS;
+    try {
+      if (!initArgs(args)) {
+        usage();
+        return ERRNO_ARGS;
+      }
+
+      if (getOpts().getScriptFile() != null) {
+        return executeFile(getOpts().getScriptFile());
+      }
+      try {
+        info(getApplicationTitle());
+      } catch (Exception e) {
+        // ignore
+      }
+      ConsoleReader reader = getConsoleReader(inputStream);
+      return execute(reader, false);
+    } finally {
+      close();
     }
+  }
 
-    ConsoleReader reader = null;
-    boolean runningScript = (getOpts().getScriptFile() != null);
-    if (runningScript) {
+  int runInit() {
+    String initFile = getOpts().getInitFile();
+    if (initFile != null) {
+      info("Running init script " + initFile);
       try {
-        FileInputStream scriptStream = new FileInputStream(getOpts().getScriptFile());
-        reader = getConsoleReader(scriptStream);
-      } catch (Throwable t) {
-        handleException(t);
-        commands.quit(null);
-        status = ERRNO_OTHER;
+        return executeFile(initFile);
+      } finally {
+        exit = false;
       }
-    } else {
-      reader = getConsoleReader(inputStream);
     }
+    return ERRNO_OK;
+  }
 
+  private int executeFile(String fileName) {
+    FileInputStream initStream = null;
     try {
-      info(getApplicationTitle());
-    } catch (Exception e) {
-      // ignore
+      initStream = new FileInputStream(fileName);
+      return execute(getConsoleReader(initStream), true);
+    } catch (Throwable t) {
+      handleException(t);
+      return ERRNO_OTHER;
+    } finally {
+      IOUtils.closeStream(initStream);
+      consoleReader = null;
+      output("");   // dummy new line
     }
+  }
 
+  private int execute(ConsoleReader reader, boolean exitOnError) {
     while (!exit) {
       try {
         // Execute one instruction; terminate on executing a script if there is an error
-        if (!dispatch(reader.readLine(getPrompt())) && runningScript) {
-          commands.quit(null);
-          status = ERRNO_OTHER;
+        if (!dispatch(reader.readLine(getPrompt())) && exitOnError) {
+          return ERRNO_OTHER;
         }
-      } catch (EOFException eof) {
-        // CTRL-D
-        commands.quit(null);
       } catch (Throwable t) {
         handleException(t);
-        status = ERRNO_OTHER;
+        return ERRNO_OTHER;
       }
     }
-    // ### NOTE jvs 10-Aug-2004: Clean up any outstanding
-    // connections automatically.
-    commands.closeall(null);
-    return status;
+    return ERRNO_OK;
   }
 
+  @Override
   public void close() {
-    commands.quit(null);
     commands.closeall(null);
   }
 
@@ -822,7 +850,7 @@ public class BeeLine {
    * Dispatch the specified line to the appropriate {@link CommandHandler}.
    *
    * @param line
-   *          the commmand-line to dispatch
+   *          the command-line to dispatch
    * @return true if the command was "successful"
    */
   boolean dispatch(String line) {
@@ -1434,6 +1462,8 @@ public class BeeLine {
 
     if (e instanceof SQLException) {
       handleSQLException((SQLException) e);
+    } else if (e instanceof EOFException) {
+      setExit(true);  // CTRL-D
     } else if (!(getOpts().getVerbose())) {
       if (e.getMessage() == null) {
         error(e.getClass().getName());

Modified: hive/trunk/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
URL: http://svn.apache.org/viewvc/hive/trunk/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java?rev=1603671&r1=1603670&r2=1603671&view=diff
==============================================================================
--- hive/trunk/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java (original)
+++ hive/trunk/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java Wed Jun 18 23:30:51 2014
@@ -86,6 +86,7 @@ class BeeLineOpts implements Completor {
   private String historyFile = new File(saveDir(), "history").getAbsolutePath();
 
   private String scriptFile = null;
+  private String initFile = null;
   private String authType = null;
 
 
@@ -358,6 +359,14 @@ class BeeLineOpts implements Completor {
     return scriptFile;
   }
 
+  public String getInitFile() {
+    return initFile;
+  }
+
+  public void setInitFile(String initFile) {
+    this.initFile = initFile;
+  }
+
   public void setColor(boolean color) {
     this.color = color;
   }
@@ -481,6 +490,5 @@ class BeeLineOpts implements Completor {
   public void setHiveConfVariables(Map<String, String> hiveConfVariables) {
     this.hiveConfVariables = hiveConfVariables;
   }
-
 }
 

Modified: hive/trunk/beeline/src/java/org/apache/hive/beeline/Commands.java
URL: http://svn.apache.org/viewvc/hive/trunk/beeline/src/java/org/apache/hive/beeline/Commands.java?rev=1603671&r1=1603670&r2=1603671&view=diff
==============================================================================
--- hive/trunk/beeline/src/java/org/apache/hive/beeline/Commands.java (original)
+++ hive/trunk/beeline/src/java/org/apache/hive/beeline/Commands.java Wed Jun 18 23:30:51 2014
@@ -964,6 +964,7 @@ public class Commands {
       beeLine.getDatabaseConnections().setConnection(
           new DatabaseConnection(beeLine, driver, url, props));
       beeLine.getDatabaseConnection().getConnection();
+      beeLine.runInit();
 
       beeLine.setCompletions();
       return true;

Modified: hive/trunk/beeline/src/main/resources/BeeLine.properties
URL: http://svn.apache.org/viewvc/hive/trunk/beeline/src/main/resources/BeeLine.properties?rev=1603671&r1=1603670&r2=1603671&view=diff
==============================================================================
--- hive/trunk/beeline/src/main/resources/BeeLine.properties (original)
+++ hive/trunk/beeline/src/main/resources/BeeLine.properties Wed Jun 18 23:30:51 2014
@@ -143,8 +143,9 @@ cmd-usage: Usage: java org.apache.hive.c
 \  -n <username>                   the username to connect as\n \
 \  -p <password>                   the password to connect as\n \
 \  -d <driver class>               the driver class to use\n \
+\  -i <init file>                  script file for initialization\n \
 \  -e <query>                      query that should be executed\n \
-\  -f <file>                       script file that should be executed\n \
+\  -f <exec file>                  script file that should be executed\n \
 \  --hiveconf property=value       Use value for given property\n \
 \  --hivevar name=value            hive variable name and value\n \
 \                                  This is Hive specific settings in which variables\n \

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java?rev=1603671&r1=1603670&r2=1603671&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java Wed Jun 18 23:30:51 2014
@@ -25,7 +25,9 @@ import static org.junit.Assert.fail;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
+import java.io.InputStream;
 import java.io.PrintStream;
+import java.io.StringBufferInputStream;
 import java.io.UnsupportedEncodingException;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -138,21 +140,22 @@ public class TestBeeLineWithArgs {
   }
 
   /**
-   * Execute a script with "beeline -f"
-   * @param scriptFileName The name of the script to execute
-   * @throws Any exception while executing
+   * Execute a script with "beeline -f or -i"
+   *
    * @return The stderr and stdout from running the script
    */
-  private String testCommandLineScript(List<String> argList) throws Throwable {
+  private String testCommandLineScript(List<String> argList, InputStream inputStream)
+      throws Throwable {
     BeeLine beeLine = new BeeLine();
     ByteArrayOutputStream os = new ByteArrayOutputStream();
     PrintStream beelineOutputStream = new PrintStream(os);
     beeLine.setOutputStream(beelineOutputStream);
     beeLine.setErrorStream(beelineOutputStream);
     String[] args = argList.toArray(new String[argList.size()]);
-    beeLine.begin(args, null);
+    beeLine.begin(args, inputStream);
     String output = os.toString("UTF8");
 
+    beeLine.close();
     return output;
   }
 
@@ -169,9 +172,6 @@ public class TestBeeLineWithArgs {
   private void testScriptFile(String testName, String scriptText, String expectedPattern,
       boolean shouldMatch, List<String> argList) throws Throwable {
 
-    long startTime = System.currentTimeMillis();
-    System.out.println(">>> STARTED " + testName);
-
     // Put the script content in a temp file
     File scriptFile = File.createTempFile(testName, "temp");
     scriptFile.deleteOnExit();
@@ -179,15 +179,34 @@ public class TestBeeLineWithArgs {
     os.print(scriptText);
     os.close();
 
-    argList.add("-f");
-    argList.add(scriptFile.getAbsolutePath());
+    System.out.println(">>> STARTED -f " + testName);
+    {
+      List<String> copy = new ArrayList<String>(argList);
+      copy.add("-f");
+      copy.add(scriptFile.getAbsolutePath());
+
+      String output = testCommandLineScript(copy, null);
+      boolean matches = output.contains(expectedPattern);
+      if (shouldMatch != matches) {
+        //failed
+        fail(testName + ": Output" + output + " should" +  (shouldMatch ? "" : " not") +
+            " contain " + expectedPattern);
+      }
+    }
 
-    String output = testCommandLineScript(argList);
-    boolean matches = output.contains(expectedPattern);
-    if (shouldMatch != matches) {
-      //failed
-      fail(testName + ": Output" + output + " should" +  (shouldMatch ? "" : " not") +
-          " contain " + expectedPattern);
+    System.out.println(">>> STARTED -i " + testName);
+    {
+      List<String> copy = new ArrayList<String>(argList);
+      copy.add("-i");
+      copy.add(scriptFile.getAbsolutePath());
+
+      String output = testCommandLineScript(copy, new StringBufferInputStream("!quit\n"));
+      boolean matches = output.contains(expectedPattern);
+      if (shouldMatch != matches) {
+        //failed
+        fail(testName + ": Output" + output + " should" +  (shouldMatch ? "" : " not") +
+            " contain " + expectedPattern);
+      }
     }
     scriptFile.delete();
   }
@@ -385,7 +404,7 @@ public class TestBeeLineWithArgs {
     argList.add(scriptFile.getAbsolutePath());
 
     try {
-        String output = testCommandLineScript(argList);
+        String output = testCommandLineScript(argList, null);
       long elapsedTime = (System.currentTimeMillis() - startTime)/1000;
       String time = "(" + elapsedTime + "s)";
       if (output.contains(EXPECTED_PATTERN)) {