You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ai...@apache.org on 2016/12/07 16:21:33 UTC

hive git commit: HIVE-15275 "beeline -f " will throw NPE (Aihua Xu, reviewed by Vihang Karajgaonkar, Yongzhi Chen)

Repository: hive
Updated Branches:
  refs/heads/master 1a39cbfca -> 70cc5efac


HIVE-15275 "beeline -f <file>" will throw NPE (Aihua Xu, reviewed by Vihang Karajgaonkar, Yongzhi Chen)


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

Branch: refs/heads/master
Commit: 70cc5eface64b5417916e42312befc022f4a06c0
Parents: 1a39cbf
Author: Aihua Xu <ai...@apache.org>
Authored: Tue Nov 29 09:27:17 2016 -0500
Committer: Aihua Xu <ai...@apache.org>
Committed: Wed Dec 7 11:20:26 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   | 155 ++++++++-----------
 .../org/apache/hive/beeline/cli/HiveCli.java    |   6 +-
 .../apache/hive/beeline/TestBeeLineHistory.java |   9 +-
 3 files changed, 75 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/70cc5efa/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 e9111c4..65818dd 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -23,12 +23,10 @@
 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;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
@@ -80,7 +78,6 @@ import jline.console.completer.Completer;
 import jline.console.completer.StringsCompleter;
 import jline.console.completer.FileNameCompleter;
 import jline.console.ConsoleReader;
-import jline.console.history.History;
 import jline.console.history.FileHistory;
 
 import org.apache.commons.cli.CommandLine;
@@ -146,7 +143,7 @@ public class BeeLine implements Closeable {
   private String dbName = null;
   private String currentDatabase = null;
 
-  private History history;
+  private FileHistory history;
   // Indicates if this instance of beeline is running in compatibility mode, or beeline mode
   private boolean isBeeLine = true;
 
@@ -517,14 +514,17 @@ public class BeeLine implements Closeable {
   public static void mainWithInputRedirection(String[] args, InputStream inputStream)
       throws IOException {
     BeeLine beeLine = new BeeLine();
-    int status = beeLine.begin(args, inputStream);
+    try {
+      int status = beeLine.begin(args, inputStream);
 
-    if (!Boolean.getBoolean(BeeLineOpts.PROPERTY_NAME_EXIT)) {
-        System.exit(status);
+      if (!Boolean.getBoolean(BeeLineOpts.PROPERTY_NAME_EXIT)) {
+          System.exit(status);
+      }
+    } finally {
+      beeLine.close();
     }
   }
 
-
   public BeeLine() {
     this(true);
   }
@@ -539,12 +539,11 @@ public class BeeLine implements Closeable {
 
 
   Connection getConnection() throws SQLException {
-    if (getDatabaseConnections().current() == null) {
-      throw new IllegalArgumentException(loc("no-current-connection"));
-    }
-    if (getDatabaseConnections().current().getConnection() == null) {
+    if (getDatabaseConnections().current() == null
+        || getDatabaseConnections().current().getConnection() == null) {
       throw new IllegalArgumentException(loc("no-current-connection"));
     }
+
     return getDatabaseConnections().current().getConnection();
   }
 
@@ -983,38 +982,36 @@ public class BeeLine implements Closeable {
       // nothing
     }
 
-    try {
-      //this method also initializes the consoleReader which is
-      //needed by initArgs for certain execution paths
-      ConsoleReader reader = initializeConsoleReader(inputStream);
-      if (isBeeLine) {
-        int code = initArgs(args);
-        if (code != 0) {
-          return code;
-        }
-      } else {
-        int code = initArgsFromCliVars(args);
-        if (code != 0 || exit) {
-          return code;
-        }
-        defaultConnect(false);
-      }
+    setupHistory();
 
-      if (getOpts().isHelpAsked()) {
-        return 0;
-      }
-      if (getOpts().getScriptFile() != null) {
-        return executeFile(getOpts().getScriptFile());
+    //this method also initializes the consoleReader which is
+    //needed by initArgs for certain execution paths
+    ConsoleReader reader = initializeConsoleReader(inputStream);
+    if (isBeeLine) {
+      int code = initArgs(args);
+      if (code != 0) {
+        return code;
       }
-      try {
-        info(getApplicationTitle());
-      } catch (Exception e) {
-        // ignore
+    } else {
+      int code = initArgsFromCliVars(args);
+      if (code != 0 || exit) {
+        return code;
       }
-      return execute(reader, false);
-    } finally {
-        close();
+      defaultConnect(false);
+    }
+
+    if (getOpts().isHelpAsked()) {
+      return 0;
     }
+    if (getOpts().getScriptFile() != null) {
+      return executeFile(getOpts().getScriptFile());
+    }
+    try {
+      info(getApplicationTitle());
+    } catch (Exception e) {
+      // ignore
+    }
+    return execute(reader, false);
   }
 
   /*
@@ -1120,7 +1117,7 @@ public class BeeLine implements Closeable {
   }
 
   private int executeFile(String fileName) {
-    InputStream initStream = null;
+    InputStream fileStream = null;
     try {
       if (!isBeeLine) {
         org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(fileName);
@@ -1132,17 +1129,16 @@ public class BeeLine implements Closeable {
         } else {
           fs = FileSystem.get(path.toUri(), conf);
         }
-        initStream = fs.open(path);
+        fileStream = fs.open(path);
       } else {
-        initStream = new FileInputStream(fileName);
+        fileStream = new FileInputStream(fileName);
       }
-      return execute(initializeConsoleReader(initStream), !getOpts().getForce());
+      return execute(initializeConsoleReader(fileStream), !getOpts().getForce());
     } catch (Throwable t) {
       handleException(t);
       return ERRNO_OTHER;
     } finally {
-      IOUtils.closeStream(initStream);
-      consoleReader = null;
+      IOUtils.closeStream(fileStream);
       output("");   // dummy new line
     }
   }
@@ -1181,6 +1177,25 @@ public class BeeLine implements Closeable {
     commands.closeall(null);
   }
 
+  private void setupHistory() throws IOException {
+    if (this.history != null) {
+       return;
+    }
+
+    this.history = new FileHistory(new File(getOpts().getHistoryFile()));
+    // add shutdown hook to flush the history to history file
+    ShutdownHookManager.addShutdownHook(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          history.flush();
+        } catch (IOException e) {
+          error(e);
+        }
+      }
+    });
+  }
+
   public ConsoleReader initializeConsoleReader(InputStream inputStream) throws IOException {
     if (inputStream != null) {
       // ### NOTE: fix for sf.net bug 879425.
@@ -1197,29 +1212,9 @@ public class BeeLine implements Closeable {
     //disable the expandEvents for the purpose of backward compatibility
     consoleReader.setExpandEvents(false);
 
-    // setup history
-    ByteArrayOutputStream hist = new ByteArrayOutputStream();
-    if (new File(getOpts().getHistoryFile()).isFile()) {
-      try {
-        // save the current contents of the history buffer. This gets
-        // around a bug in JLine where setting the output before the
-        // input will clobber the history input, but setting the
-        // input before the output will cause the previous commands
-        // to not be saved to the buffer.
-        try (FileInputStream historyIn = new FileInputStream(getOpts().getHistoryFile())) {
-          int n;
-          while ((n = historyIn.read()) != -1) {
-            hist.write(n);
-          }
-        }
-      } catch (Exception e) {
-        handleException(e);
-      }
-    }
-
     try {
       // now set the output for the history
-      consoleReader.setHistory(new FileHistory(new File(getOpts().getHistoryFile())));
+      consoleReader.setHistory(this.history);
     } catch (Exception e) {
       handleException(e);
     }
@@ -1228,32 +1223,6 @@ public class BeeLine implements Closeable {
       // from script.. no need to load history and no need of completer, either
       return consoleReader;
     }
-    try {
-      // now load in the previous history
-      if (hist != null) {
-        History h = consoleReader.getHistory();
-        if (!(h instanceof FileHistory)) {
-          consoleReader.getHistory().add(hist.toString());
-        }
-      }
-    } catch (Exception e) {
-        handleException(e);
-    }
-
-    // add shutdown hook to flush the history to history file
-    ShutdownHookManager.addShutdownHook(new Runnable() {
-        @Override
-        public void run() {
-            History h = consoleReader.getHistory();
-            if (h instanceof FileHistory) {
-                try {
-                    ((FileHistory) h).flush();
-                } catch (IOException e) {
-                    error(e);
-                }
-            }
-        }
-    });
 
     consoleReader.addCompleter(new BeeLineCompleter(this));
     return consoleReader;

http://git-wip-us.apache.org/repos/asf/hive/blob/70cc5efa/beeline/src/java/org/apache/hive/beeline/cli/HiveCli.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/cli/HiveCli.java b/beeline/src/java/org/apache/hive/beeline/cli/HiveCli.java
index 1e7f068..13fea29 100644
--- a/beeline/src/java/org/apache/hive/beeline/cli/HiveCli.java
+++ b/beeline/src/java/org/apache/hive/beeline/cli/HiveCli.java
@@ -32,6 +32,10 @@ public class HiveCli {
 
   public int runWithArgs(String[] cmd, InputStream inputStream) throws IOException {
     beeLine = new BeeLine(false);
-    return beeLine.begin(cmd, inputStream);
+    try {
+      return beeLine.begin(cmd, inputStream);
+    } finally {
+      beeLine.close();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/70cc5efa/beeline/src/test/org/apache/hive/beeline/TestBeeLineHistory.java
----------------------------------------------------------------------
diff --git a/beeline/src/test/org/apache/hive/beeline/TestBeeLineHistory.java b/beeline/src/test/org/apache/hive/beeline/TestBeeLineHistory.java
index 5f99a0e..623e667 100644
--- a/beeline/src/test/org/apache/hive/beeline/TestBeeLineHistory.java
+++ b/beeline/src/test/org/apache/hive/beeline/TestBeeLineHistory.java
@@ -22,6 +22,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.PrintStream;
 import java.io.PrintWriter;
+import java.lang.reflect.Method;
 
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -58,11 +59,14 @@ public class TestBeeLineHistory {
     BeeLine beeline = new BeeLine();
     beeline.getOpts().setHistoryFile(fileName);
     beeline.setOutputStream(ops);
+    Method method = beeline.getClass().getDeclaredMethod("setupHistory");
+    method.setAccessible(true);
+    method.invoke(beeline);
     beeline.initializeConsoleReader(null);
     beeline.dispatch("!history");
     String output = os.toString("UTF-8");
     int numHistories = output.split("\n").length;
-    Assert.assertEquals(numHistories, 10);
+    Assert.assertEquals(10, numHistories);
     beeline.close();
   }
 
@@ -73,6 +77,9 @@ public class TestBeeLineHistory {
     BeeLine beeline = new BeeLine();
     beeline.getOpts().setHistoryFile(fileName);
     beeline.setOutputStream(ops);
+    Method method = beeline.getClass().getDeclaredMethod("setupHistory");
+    method.setAccessible(true);
+    method.invoke(beeline);
     beeline.initializeConsoleReader(null);
     beeline.dispatch("!history");
     String output = os.toString("UTF-8");