diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java index d2b8590..c47b9fe 100644 --- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java +++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java @@ -22,6 +22,7 @@ */ package org.apache.hive.beeline; +import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.Closeable; @@ -30,6 +31,7 @@ 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; @@ -87,6 +89,8 @@ import org.apache.commons.cli.OptionBuilder; import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.io.IOUtils; import org.apache.hive.beeline.cli.CliOptionsProcessor; @@ -889,9 +893,22 @@ public int defaultConnect(boolean exitOnError) { } private int executeFile(String fileName) { - FileInputStream initStream = null; + InputStream initStream = null; try { - initStream = new FileInputStream(fileName); + if (!isBeeLine) { + org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(fileName); + FileSystem fs; + HiveConf conf = getCommands().getHiveConf(true); + if (!path.toUri().isAbsolute()) { + fs = FileSystem.getLocal(conf); + path = fs.makeQualified(path); + } else { + fs = FileSystem.get(path.toUri(), conf); + } + initStream = fs.open(path); + } else { + initStream = new FileInputStream(fileName); + } return execute(getConsoleReader(initStream), !getOpts().getForce()); } catch (Throwable t) { handleException(t); @@ -973,7 +990,7 @@ public ConsoleReader getConsoleReader(InputStream inputStream) throws IOExceptio handleException(e); } - if (inputStream instanceof FileInputStream) { + if (inputStream instanceof FileInputStream || inputStream instanceof FSDataInputStream) { // from script.. no need to load history and no need of completer, either return consoleReader; } diff --git a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java index 542f1ee..c465eac 100644 --- a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java +++ b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java @@ -79,7 +79,8 @@ private void verifyCMD(String CMD, String keywords, OutputStream os, String[] op executeCMD(options, CMD, retCode); String output = os.toString(); LOG.debug(output); - Assert.assertTrue("The expected keyword " + keywords + "doesn't occur in the output: " + output, + Assert.assertTrue( + "The expected keyword \"" + keywords + "\" doesn't occur in the output: " + output, output.contains(keywords)); }