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

[10/43] hive git commit: HIVE-6791: Support variable substition for Beeline shell command (Ferdinand Xu, reviewed by Xuefu Zhang)

HIVE-6791: Support variable substition for Beeline shell command (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/102b23b1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/102b23b1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/102b23b1

Branch: refs/heads/llap
Commit: 102b23b16bf26cbf439009b4b95542490a082710
Parents: 00e0d55
Author: Ferdinand Xu <ch...@intel.com>
Authored: Sun Jun 28 20:42:54 2015 -0400
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Sun Jun 28 20:42:54 2015 -0400

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   |  75 +---
 .../java/org/apache/hive/beeline/Commands.java  | 430 ++++++++++++++-----
 .../apache/hive/beeline/cli/TestHiveCli.java    |  75 +++-
 .../org/apache/hadoop/hive/cli/CliDriver.java   |  24 +-
 .../hadoop/hive/conf/HiveVariableSource.java    |  24 ++
 .../hadoop/hive/conf/VariableSubstitution.java  |  70 +++
 .../hive/conf/TestVariableSubstitution.java     |  63 +++
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  10 +-
 .../ql/parse/ColumnStatsSemanticAnalyzer.java   |  14 +-
 .../hive/ql/parse/VariableSubstitution.java     |  60 ---
 .../ql/processors/AddResourceProcessor.java     |  11 +-
 .../hive/ql/processors/CompileProcessor.java    |  11 +-
 .../ql/processors/DeleteResourceProcessor.java  |  11 +-
 .../hadoop/hive/ql/processors/DfsProcessor.java |  11 +-
 .../hadoop/hive/ql/processors/SetProcessor.java |  32 +-
 .../service/cli/operation/SQLOperation.java     |  10 +-
 16 files changed, 651 insertions(+), 280 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/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 66fe322..1d468eb 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -87,6 +87,7 @@ import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hive.beeline.cli.CliOptionsProcessor;
 import org.apache.hive.jdbc.Utils;
@@ -830,7 +831,7 @@ public class BeeLine implements Closeable {
       }
 
       if (getOpts().getScriptFile() != null) {
-        return executeFile(getOpts().getScriptFile(), false);
+        return executeFile(getOpts().getScriptFile());
       }
       try {
         info(getApplicationTitle());
@@ -849,7 +850,7 @@ public class BeeLine implements Closeable {
     if (initFile != null) {
       info("Running init script " + initFile);
       try {
-        return executeFile(initFile, false);
+        return executeFile(initFile);
       } finally {
         exit = false;
       }
@@ -884,7 +885,7 @@ public class BeeLine implements Closeable {
     return ERRNO_OK;
   }
 
-  private int executeFile(String fileName, boolean isSourceCMD) {
+  private int executeFile(String fileName) {
     FileInputStream initStream = null;
     try {
       initStream = new FileInputStream(fileName);
@@ -894,31 +895,8 @@ public class BeeLine implements Closeable {
       return ERRNO_OTHER;
     } finally {
       IOUtils.closeStream(initStream);
-      if(!isSourceCMD) {
-        consoleReader = null;
-        output("");   // dummy new line
-      }
-    }
-  }
-
-  private boolean isSourceCMD(String cmd) {
-    if (cmd == null || cmd.isEmpty())
-      return false;
-    String[] tokens = tokenizeCmd(cmd);
-    return tokens[0].equalsIgnoreCase("!source");
-  }
-
-  private boolean sourceFile(String cmd) {
-    String[] tokens = tokenizeCmd(cmd);
-    String cmd_1 = getFirstCmd(cmd, tokens[0].length());
-    File sourceFile = new File(cmd_1);
-    if (!sourceFile.isFile()) {
-      return false;
-    } else {
-      boolean ret = (executeFile(cmd_1, true) == ERRNO_OK);
-      // For source command, we should not exit even when meeting some empty line.
-      setExit(false);
-      return ret;
+      consoleReader = null;
+      output("");   // dummy new line
     }
   }
 
@@ -933,10 +911,6 @@ public class BeeLine implements Closeable {
 
         // trim line
         line = (line == null) ? null : line.trim();
-        if (!isBeeLine) {
-          line = cliToBeelineCmd(line);
-        }
-
         if (!dispatch(line) && exitOnError) {
           return ERRNO_OTHER;
         }
@@ -1043,31 +1017,6 @@ public class BeeLine implements Closeable {
   }
 
   /**
-   * Extract and clean up the first command in the input.
-   */
-  private String getFirstCmd(String cmd, int length) {
-    return cmd.substring(length).trim();
-  }
-
-  private String cliToBeelineCmd(String cmd) {
-    if (cmd == null)
-      return null;
-    String[] tokens = tokenizeCmd(cmd);
-    if (cmd.equalsIgnoreCase("quit") || cmd.equalsIgnoreCase("exit")
-	|| cmd.equalsIgnoreCase("quit;") || cmd.equals("exit;")) {
-      return null;
-    } else if (tokens[0].equalsIgnoreCase("source")) {
-      return COMMAND_PREFIX + cmd;
-    } else if (cmd.startsWith("!")) {
-      String shell_cmd = cmd.substring(1);
-      return "!sh " + shell_cmd;
-    } else { // local mode
-      // command like dfs
-      return cmd;
-    }
-  }
-
-  /**
    * Dispatch the specified line to the appropriate {@link CommandHandler}.
    *
    * @param line
@@ -1089,10 +1038,6 @@ public class BeeLine implements Closeable {
       return true;
     }
 
-    if(isSourceCMD(line)){
-      return sourceFile(line);
-    }
-
     line = line.trim();
 
     // save it to the current script, if any
@@ -2121,4 +2066,12 @@ public class BeeLine implements Closeable {
   protected Reflector getReflector() {
     return reflector;
   }
+
+  public boolean isBeeLine() {
+    return isBeeLine;
+  }
+
+  public void setBeeLine(boolean isBeeLine) {
+    this.isBeeLine = isBeeLine;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/beeline/src/java/org/apache/hive/beeline/Commands.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/Commands.java b/beeline/src/java/org/apache/hive/beeline/Commands.java
index aaf6aec..d490273 100644
--- a/beeline/src/java/org/apache/hive/beeline/Commands.java
+++ b/beeline/src/java/org/apache/hive/beeline/Commands.java
@@ -22,6 +22,10 @@
  */
 package org.apache.hive.beeline;
 
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveVariableSource;
+import org.apache.hadoop.hive.conf.SystemVariables;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.io.IOUtils;
 
 import java.io.BufferedReader;
@@ -32,7 +36,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.lang.reflect.Method;
-import java.net.MalformedURLException;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.sql.CallableStatement;
@@ -44,9 +47,11 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.SQLWarning;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.TreeSet;
@@ -219,9 +224,8 @@ public class Commands {
 
 
   public boolean exportedkeys(String line) throws Exception {
-    return metadata("getExportedKeys", new String[] {
-        beeLine.getConnection().getCatalog(), null,
-        arg1(line, "table name"),});
+    return metadata("getExportedKeys",
+        new String[] { beeLine.getConnection().getCatalog(), null, arg1(line, "table name"), });
   }
 
 
@@ -709,10 +713,303 @@ public class Commands {
     return execute(line, false, false);
   }
 
+  /**
+   * This method is used for retrieving the latest configuration from hive server2.
+   * It uses the set command processor.
+   *
+   * @return
+   */
+  private Map<String, String> getHiveVariables() {
+    Map<String, String> result = new HashMap<>();
+    BufferedRows rows = getConfInternal();
+    while (rows.hasNext()) {
+      Rows.Row row = (Rows.Row) rows.next();
+      if (!row.isMeta) {
+        result.put(row.values[0], row.values[1]);
+      }
+    }
+    return result;
+  }
+
+  private HiveConf getHiveConf() {
+    HiveConf conf = new HiveConf();
+    BufferedRows rows = getConfInternal();
+    while (rows.hasNext()) {
+      addConf((Rows.Row) rows.next(), conf);
+    }
+    return conf;
+  }
+
+  private BufferedRows getConfInternal() {
+    Statement stmnt = null;
+    BufferedRows rows = null;
+    try {
+      stmnt = beeLine.createStatement();
+      boolean hasResults = stmnt.execute("set");
+      if (hasResults) {
+        ResultSet rs = stmnt.getResultSet();
+        rows = new BufferedRows(beeLine, rs);
+      }
+    } catch (SQLException e) {
+      beeLine.error(e);
+      if (stmnt != null) {
+        try {
+          stmnt.close();
+        } catch (SQLException e1) {
+          beeLine.error(e1);
+        }
+      }
+    }
+    return rows;
+  }
+
+  private void addConf(Rows.Row r, HiveConf hiveConf) {
+    if (r.isMeta) {
+      return;
+    }
+    if (r.values == null || r.values[0] == null || r.values[0].isEmpty()) {
+      return;
+    }
+    String val = r.values[0];
+    if (r.values[0].startsWith(SystemVariables.SYSTEM_PREFIX) || r.values[0]
+        .startsWith(SystemVariables.ENV_PREFIX)) {
+      return;
+    } else {
+      String[] kv = val.split("=", 2);
+      hiveConf.set(kv[0], kv[1]);
+    }
+  }
+
+  /**
+   * Extract and clean up the first command in the input.
+   */
+  private String getFirstCmd(String cmd, int length) {
+    return cmd.substring(length).trim();
+  }
+
+  private String[] tokenizeCmd(String cmd) {
+    return cmd.split("\\s+");
+  }
+
+  private boolean isSourceCMD(String cmd) {
+    if (cmd == null || cmd.isEmpty())
+      return false;
+    String[] tokens = tokenizeCmd(cmd);
+    return tokens[0].equalsIgnoreCase("!source");
+  }
+
+  private boolean sourceFile(String cmd) {
+    String[] tokens = tokenizeCmd(cmd);
+    String cmd_1 = getFirstCmd(cmd, tokens[0].length());
+
+    cmd_1 = substituteVariables(getHiveConf(), cmd_1);
+    File sourceFile = new File(cmd_1);
+    if (!sourceFile.isFile()) {
+      return false;
+    } else {
+      boolean ret;
+      try {
+        ret = sourceFileInternal(sourceFile);
+      } catch (IOException e) {
+        beeLine.error(e);
+        return false;
+      }
+      return ret;
+    }
+  }
+
+  private boolean sourceFileInternal(File sourceFile) throws IOException {
+    BufferedReader reader = null;
+    try {
+      reader = new BufferedReader(new FileReader(sourceFile));
+      String extra = reader.readLine();
+      String lines = null;
+      while (extra != null) {
+        if (beeLine.isComment(extra)) {
+          continue;
+        }
+        if (lines == null) {
+          lines = extra;
+        } else {
+          lines += "\n" + extra;
+        }
+        extra = reader.readLine();
+      }
+      String[] cmds = lines.split(";");
+      for (String c : cmds) {
+        if (!executeInternal(c, false)) {
+          return false;
+        }
+      }
+    } finally {
+      if (reader != null) {
+        reader.close();
+      }
+    }
+    return true;
+  }
+
+  private String cliToBeelineCmd(String cmd) {
+    if (cmd == null)
+      return null;
+    String[] tokens = tokenizeCmd(cmd);
+    if (tokens[0].equalsIgnoreCase("source")) {
+      return BeeLine.COMMAND_PREFIX + cmd;
+    } else if (cmd.startsWith("!")) {
+      String shell_cmd = cmd.substring(1);
+      return "!sh " + shell_cmd;
+    } else { // local mode
+      // command like dfs
+      return cmd;
+    }
+  }
+
+  // Return false only occurred error when execution the sql and the sql should follow the rules
+  // of beeline.
+  private boolean executeInternal(String sql, boolean call) {
+    if (sql == null || sql.length() == 0) {
+      return true;
+    }
+
+    if (beeLine.isComment(sql)) {
+      //skip this and rest cmds in the line
+      return true;
+    }
+
+    if (sql.startsWith(BeeLine.COMMAND_PREFIX)) {
+      sql = sql.substring(1);
+    }
+
+    String prefix = call ? "call" : "sql";
+
+    if (sql.startsWith(prefix)) {
+      sql = sql.substring(prefix.length());
+    }
+
+    // batch statements?
+    if (beeLine.getBatch() != null) {
+      beeLine.getBatch().add(sql);
+      return true;
+    }
+
+    try {
+      Statement stmnt = null;
+      boolean hasResults;
+      Thread logThread = null;
+
+      try {
+        long start = System.currentTimeMillis();
+
+        if (call) {
+          stmnt = beeLine.getDatabaseConnection().getConnection().prepareCall(sql);
+          hasResults = ((CallableStatement) stmnt).execute();
+        } else {
+          stmnt = beeLine.createStatement();
+          if (beeLine.getOpts().isSilent()) {
+            hasResults = stmnt.execute(sql);
+          } else {
+            logThread = new Thread(createLogRunnable(stmnt));
+            logThread.setDaemon(true);
+            logThread.start();
+            hasResults = stmnt.execute(sql);
+            logThread.interrupt();
+            logThread.join(DEFAULT_QUERY_PROGRESS_THREAD_TIMEOUT);
+          }
+        }
+
+        beeLine.showWarnings();
+
+        if (hasResults) {
+          do {
+            ResultSet rs = stmnt.getResultSet();
+            try {
+              int count = beeLine.print(rs);
+              long end = System.currentTimeMillis();
+
+              beeLine.info(
+                  beeLine.loc("rows-selected", count) + " " + beeLine.locElapsedTime(end - start));
+            } finally {
+              if (logThread != null) {
+                logThread.join(DEFAULT_QUERY_PROGRESS_THREAD_TIMEOUT);
+                showRemainingLogsIfAny(stmnt);
+                logThread = null;
+              }
+              rs.close();
+            }
+          } while (BeeLine.getMoreResults(stmnt));
+        } else {
+          int count = stmnt.getUpdateCount();
+          long end = System.currentTimeMillis();
+          beeLine.info(
+              beeLine.loc("rows-affected", count) + " " + beeLine.locElapsedTime(end - start));
+        }
+      } finally {
+        if (logThread != null) {
+          if (!logThread.isInterrupted()) {
+            logThread.interrupt();
+          }
+          logThread.join(DEFAULT_QUERY_PROGRESS_THREAD_TIMEOUT);
+          showRemainingLogsIfAny(stmnt);
+        }
+        if (stmnt != null) {
+          stmnt.close();
+        }
+      }
+    } catch (Exception e) {
+      return beeLine.error(e);
+    }
+    beeLine.showWarnings();
+    return true;
+  }
+
+  public String handleMultiLineCmd(String line) throws IOException {
+    //When using -e, console reader is not initialized and command is a single line
+    while (beeLine.getConsoleReader() != null && !(line.trim().endsWith(";")) && beeLine.getOpts()
+        .isAllowMultiLineCommand()) {
+
+      if (!beeLine.getOpts().isSilent()) {
+        StringBuilder prompt = new StringBuilder(beeLine.getPrompt());
+        for (int i = 0; i < prompt.length() - 1; i++) {
+          if (prompt.charAt(i) != '>') {
+            prompt.setCharAt(i, i % 2 == 0 ? '.' : ' ');
+          }
+        }
+      }
+
+      String extra;
+      if (beeLine.getOpts().isSilent() && beeLine.getOpts().getScriptFile() != null) {
+        extra = beeLine.getConsoleReader().readLine(null, jline.console.ConsoleReader.NULL_MASK);
+      } else {
+        extra = beeLine.getConsoleReader().readLine(beeLine.getPrompt());
+      }
+
+      if (extra == null) { //it happens when using -f and the line of cmds does not end with ;
+        break;
+      }
+      if (!beeLine.isComment(extra)) {
+        line += "\n" + extra;
+      }
+    }
+    return line;
+  }
+
   public boolean sql(String line, boolean entireLineAsCommand) {
     return execute(line, false, entireLineAsCommand);
   }
 
+  private String substituteVariables(HiveConf conf, String line) {
+    if (!beeLine.isBeeLine()) {
+      // Substitution is only supported in non-beeline mode.
+      return new VariableSubstitution(new HiveVariableSource() {
+        @Override
+        public Map<String, String> getHiveVariable() {
+          return getHiveVariables();
+        }
+      }).substitute(conf, line);
+    }
+    return line;
+  }
+
   public boolean sh(String line) {
     if (line == null || line.length() == 0) {
       return false;
@@ -723,9 +1020,7 @@ public class Commands {
     }
 
     line = line.substring("sh".length()).trim();
-
-    // Support variable substitution. HIVE-6791.
-    // line = new VariableSubstitution().substitute(new HiveConf(BeeLine.class), line.trim());
+    line = substituteVariables(getHiveConf(), line.trim());
 
     try {
       ShellCmdExecutor executor = new ShellCmdExecutor(line, beeLine.getOutputStream(),
@@ -738,7 +1033,6 @@ public class Commands {
       return true;
     } catch (Exception e) {
       beeLine.error("Exception raised from Shell command " + e);
-      beeLine.error(e);
       return false;
     }
   }
@@ -760,33 +1054,7 @@ public class Commands {
 
     // use multiple lines for statements not terminated by ";"
     try {
-      //When using -e, console reader is not initialized and command is a single line
-      while (beeLine.getConsoleReader() != null && !(line.trim().endsWith(";"))
-        && beeLine.getOpts().isAllowMultiLineCommand()) {
-
-        if (!beeLine.getOpts().isSilent()) {
-          StringBuilder prompt = new StringBuilder(beeLine.getPrompt());
-          for (int i = 0; i < prompt.length() - 1; i++) {
-            if (prompt.charAt(i) != '>') {
-              prompt.setCharAt(i, i % 2 == 0 ? '.' : ' ');
-            }
-          }
-        }
-
-        String extra = null;
-        if (beeLine.getOpts().isSilent() && beeLine.getOpts().getScriptFile() != null) {
-          extra = beeLine.getConsoleReader().readLine(null, jline.console.ConsoleReader.NULL_MASK);
-        } else {
-          extra = beeLine.getConsoleReader().readLine(beeLine.getPrompt());
-        }
-
-        if (extra == null) { //it happens when using -f and the line of cmds does not end with ;
-          break;
-        }
-        if (!beeLine.isComment(extra)) {
-          line += "\n" + extra;
-        }
-      }
+      line = handleMultiLineCmd(line);
     } catch (Exception e) {
       beeLine.handleException(e);
     }
@@ -806,93 +1074,23 @@ public class Commands {
     for (int i = 0; i < cmds.length; i++) {
       String sql = cmds[i].trim();
       if (sql.length() != 0) {
-        if (beeLine.isComment(sql)) {
-          //skip this and rest cmds in the line
-          break;
-        }
-        if (sql.startsWith(BeeLine.COMMAND_PREFIX)) {
-          sql = sql.substring(1);
-        }
-
-        String prefix = call ? "call" : "sql";
-
-        if (sql.startsWith(prefix)) {
-          sql = sql.substring(prefix.length());
+        if (!beeLine.isBeeLine()) {
+          sql = cliToBeelineCmd(sql);
+          if (sql.equalsIgnoreCase("quit") || sql.equalsIgnoreCase("exit")) {
+            beeLine.setExit(true);
+            return true;
+          }
         }
 
-        // batch statements?
-        if (beeLine.getBatch() != null) {
-          beeLine.getBatch().add(sql);
+        // is source CMD
+        if (isSourceCMD(sql)) {
+          sourceFile(sql);
           continue;
         }
 
-        try {
-          Statement stmnt = null;
-          boolean hasResults;
-          Thread logThread = null;
-
-          try {
-            long start = System.currentTimeMillis();
-
-            if (call) {
-              stmnt = beeLine.getDatabaseConnection().getConnection().prepareCall(sql);
-              hasResults = ((CallableStatement) stmnt).execute();
-            } else {
-              stmnt = beeLine.createStatement();
-              if (beeLine.getOpts().isSilent()) {
-                hasResults = stmnt.execute(sql);
-              } else {
-                logThread = new Thread(createLogRunnable(stmnt));
-                logThread.setDaemon(true);
-                logThread.start();
-                hasResults = stmnt.execute(sql);
-                logThread.interrupt();
-                logThread.join(DEFAULT_QUERY_PROGRESS_THREAD_TIMEOUT);
-              }
-            }
-
-            beeLine.showWarnings();
-
-            if (hasResults) {
-              do {
-                ResultSet rs = stmnt.getResultSet();
-                try {
-                  int count = beeLine.print(rs);
-                  long end = System.currentTimeMillis();
-
-                  beeLine.info(beeLine.loc("rows-selected", count) + " "
-                      + beeLine.locElapsedTime(end - start));
-                } finally {
-                  if (logThread != null) {
-                    logThread.join(DEFAULT_QUERY_PROGRESS_THREAD_TIMEOUT);
-                    showRemainingLogsIfAny(stmnt);
-                    logThread = null;
-                  }
-                  rs.close();
-                }
-              } while (BeeLine.getMoreResults(stmnt));
-            } else {
-              int count = stmnt.getUpdateCount();
-              long end = System.currentTimeMillis();
-              beeLine.info(beeLine.loc("rows-affected", count)
-                  + " " + beeLine.locElapsedTime(end - start));
-            }
-          } finally {
-            if (logThread != null) {
-              if (!logThread.isInterrupted()) {
-                logThread.interrupt();
-              }
-              logThread.join(DEFAULT_QUERY_PROGRESS_THREAD_TIMEOUT);
-              showRemainingLogsIfAny(stmnt);
-            }
-            if (stmnt != null) {
-              stmnt.close();
-            }
-          }
-        } catch (Exception e) {
-          return beeLine.error(e);
+        if (!executeInternal(sql, call)) {
+          return false;
         }
-        beeLine.showWarnings();
       }
     }
     return true;

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
----------------------------------------------------------------------
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 6cbb030..ff8ab17 100644
--- a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
+++ b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
  * 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.
@@ -42,6 +42,10 @@ public class TestHiveCli {
 
   private final static String SOURCE_CONTEXT =
       "create table if not exists test.testSrcTbl(a string, b string);";
+  private final static String SOURCE_CONTEXT2 =
+      "create table if not exists test.testSrcTbl2(a string);";
+  private final static String SOURCE_CONTEXT3 =
+      "create table if not exists test.testSrcTbl3(a string);";
   final static String CMD =
       "create database if not exists test;\ncreate table if not exists test.testTbl(a string, b "
           + "string);\n";
@@ -60,7 +64,7 @@ public class TestHiveCli {
         inputStream = IOUtils.toInputStream(input);
       }
       ret = cli.runWithArgs(args, inputStream);
-    } catch (IOException e) {
+    } catch (Throwable e) {
       LOG.error("Failed to execute command due to the error: " + e);
     } finally {
       if (retCode != ret) {
@@ -77,48 +81,81 @@ public class TestHiveCli {
     Assert.assertTrue(output.contains(keywords));
   }
 
-  @Test public void testInValidCmd() {
-    verifyCMD("!lss\n", "Failed to execute lss", errS, null, ERRNO_OK);
+  @Test
+  public void testInValidCmd() {
+    verifyCMD("!lss\n", "Unknown command: lss", errS, null, ERRNO_OK);
   }
 
-  @Test public void testHelp() {
+  @Test
+  public void testHelp() {
     verifyCMD(null, "usage: hive", os, new String[] { "-H" }, ERRNO_ARGS);
   }
 
-  @Test public void testInvalidDatabaseOptions() {
-    verifyCMD("\nshow tables\nquit\n", "Database does not exist: invalidDB", errS,
+  @Test
+  public void testInvalidDatabaseOptions() {
+    verifyCMD("\nshow tables;\nquit;\n", "Database does not exist: invalidDB", errS,
         new String[] { "--database", "invalidDB" }, ERRNO_OK);
   }
 
-  @Test public void testDatabaseOptions() {
+  @Test
+  public void testDatabaseOptions() {
     verifyCMD("\nshow tables;\nquit;", "testTbl", os, new String[] { "--database", "test" },
         ERRNO_OK);
   }
 
-  @Test public void testSourceCmd() {
+  @Test
+  public void testSourceCmd() {
     File f = generateTmpFile(SOURCE_CONTEXT);
-    verifyCMD("source " + f.getPath() + "\n" + "desc testSrcTbl\n" + "quit\n", "col_name", os,
+    verifyCMD("source " + f.getPath() + ";" + "desc testSrcTbl;\nquit;\n", "col_name", os,
         new String[] { "--database", "test" }, ERRNO_OK);
+    f.delete();
   }
 
-  @Test public void testSqlFromCmd() {
+  @Test
+  public void testSourceCmd2() {
+    File f = generateTmpFile(SOURCE_CONTEXT3);
+    verifyCMD("source " + f.getPath() + ";" + "desc testSrcTbl3;\nquit;\n", "col_name", os,
+        new String[] { "--database", "test" }, ERRNO_OK);
+    f.delete();
+  }
+
+  @Test
+  public void testSqlFromCmd() {
     verifyCMD(null, "", os, new String[] { "-e", "show databases;" }, ERRNO_OK);
   }
 
-  @Test public void testSqlFromCmdWithDBName() {
+  @Test
+  public void testSqlFromCmdWithDBName() {
     verifyCMD(null, "testTbl", os, new String[] { "-e", "show tables;", "--database", "test" },
         ERRNO_OK);
   }
 
-  @Test public void testInvalidOptions() {
+  @Test
+  public void testInvalidOptions() {
     verifyCMD(null, "The '-e' and '-f' options cannot be specified simultaneously", errS,
         new String[] { "-e", "show tables;", "-f", "path/to/file" }, ERRNO_ARGS);
   }
 
-  @Test public void testInvalidOptions2() {
+  @Test
+  public void testInvalidOptions2() {
     verifyCMD(null, "Unrecognized option: -k", errS, new String[] { "-k" }, ERRNO_ARGS);
   }
 
+  @Test
+  public void testVariables() {
+    verifyCMD("set system:xxx=5;\nset system:yyy=${system:xxx};\nset system:yyy;", "", os, null,
+        ERRNO_OK);
+  }
+
+  @Test
+  public void testVariablesForSource() {
+    File f = generateTmpFile(SOURCE_CONTEXT2);
+    verifyCMD(
+        "set hiveconf:zzz=" + f.getAbsolutePath() + ";\nsource ${hiveconf:zzz};\ndesc testSrcTbl2;",
+        "col_name", os, new String[] { "--database", "test" }, ERRNO_OK);
+    f.delete();
+  }
+
   private void redirectOutputStream() {
     // Setup output stream to redirect output to
     os = new ByteArrayOutputStream();
@@ -152,13 +189,15 @@ public class TestHiveCli {
     return file;
   }
 
-  @Before public void setup() {
+  @Before
+  public void setup() {
     cli = new HiveCli();
     redirectOutputStream();
     initFromFile();
   }
 
-  @After public void tearDown() {
+  @After
+  public void tearDown() {
     tmp.delete();
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
----------------------------------------------------------------------
diff --git a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
index d62fd5c..4b52578 100644
--- a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
+++ b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
@@ -61,7 +61,9 @@ import org.apache.hadoop.hive.common.cli.ShellCmdExecutor;
 import org.apache.hadoop.hive.common.io.CachingPrintStream;
 import org.apache.hadoop.hive.common.io.FetchConverter;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveVariableSource;
 import org.apache.hadoop.hive.conf.Validator;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
@@ -69,7 +71,6 @@ import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHelper;
 import org.apache.hadoop.hive.ql.exec.tez.TezJobExecHelper;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
-import org.apache.hadoop.hive.ql.parse.VariableSubstitution;
 import org.apache.hadoop.hive.ql.processors.CommandProcessor;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
@@ -127,7 +128,12 @@ public class CliDriver {
 
     } else if (tokens[0].equalsIgnoreCase("source")) {
       String cmd_1 = getFirstCmd(cmd_trimmed, tokens[0].length());
-      cmd_1 = new VariableSubstitution().substitute(ss.getConf(), cmd_1);
+      cmd_1 = new VariableSubstitution(new HiveVariableSource() {
+        @Override
+        public Map<String, String> getHiveVariable() {
+          return SessionState.get().getHiveVariables();
+        }
+      }).substitute(ss.getConf(), cmd_1);
 
       File sourceFile = new File(cmd_1);
       if (! sourceFile.isFile()){
@@ -145,7 +151,12 @@ public class CliDriver {
     } else if (cmd_trimmed.startsWith("!")) {
 
       String shell_cmd = cmd_trimmed.substring(1);
-      shell_cmd = new VariableSubstitution().substitute(ss.getConf(), shell_cmd);
+      shell_cmd = new VariableSubstitution(new HiveVariableSource() {
+        @Override
+        public Map<String, String> getHiveVariable() {
+          return SessionState.get().getHiveVariables();
+        }
+      }).substitute(ss.getConf(), shell_cmd);
 
       // shell_cmd = "/bin/bash -c \'" + shell_cmd + "\'";
       try {
@@ -671,7 +682,12 @@ public class CliDriver {
 
     // read prompt configuration and substitute variables.
     prompt = conf.getVar(HiveConf.ConfVars.CLIPROMPT);
-    prompt = new VariableSubstitution().substitute(conf, prompt);
+    prompt = new VariableSubstitution(new HiveVariableSource() {
+      @Override
+      public Map<String, String> getHiveVariable() {
+        return SessionState.get().getHiveVariables();
+      }
+    }).substitute(conf, prompt);
     prompt2 = spacesForString(prompt);
 
     SessionState.start(ss);

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/common/src/java/org/apache/hadoop/hive/conf/HiveVariableSource.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveVariableSource.java b/common/src/java/org/apache/hadoop/hive/conf/HiveVariableSource.java
new file mode 100644
index 0000000..86f7a9c
--- /dev/null
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveVariableSource.java
@@ -0,0 +1,24 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hadoop.hive.conf;
+
+import java.util.Map;
+
+public interface HiveVariableSource {
+  Map<String, String> getHiveVariable();
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/common/src/java/org/apache/hadoop/hive/conf/VariableSubstitution.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/VariableSubstitution.java b/common/src/java/org/apache/hadoop/hive/conf/VariableSubstitution.java
new file mode 100644
index 0000000..e1f53ba
--- /dev/null
+++ b/common/src/java/org/apache/hadoop/hive/conf/VariableSubstitution.java
@@ -0,0 +1,70 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hadoop.hive.conf;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+import java.util.Map;
+
+public class VariableSubstitution extends SystemVariables {
+  private static final Log l4j = LogFactory.getLog(VariableSubstitution.class);
+
+  private HiveVariableSource hiveVariableSource;
+
+  public VariableSubstitution(HiveVariableSource hiveVariableSource) {
+    this.hiveVariableSource = hiveVariableSource;
+  }
+
+  /**
+   * The super method will handle with the case of substitutions for system variables,
+   * hive conf variables and env variables. In this method, it will retrieve the hive
+   * variables using hiveVariableSource.
+   *
+   * @param conf
+   * @param var
+   * @return
+   */
+  @Override
+  protected String getSubstitute(Configuration conf, String var) {
+    String val = super.getSubstitute(conf, var);
+    if (val == null && hiveVariableSource != null) {
+      Map<String, String> vars = hiveVariableSource.getHiveVariable();
+      if (var.startsWith(HIVEVAR_PREFIX)) {
+        val = vars.get(var.substring(HIVEVAR_PREFIX.length()));
+      } else {
+        val = vars.get(var);
+      }
+    }
+    return val;
+  }
+
+  public String substitute(HiveConf conf, String expr) {
+    if (expr == null) {
+      return expr;
+    }
+    if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEVARIABLESUBSTITUTE)) {
+      l4j.debug("Substitution is on: " + expr);
+    } else {
+      return expr;
+    }
+    int depth = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVEVARIABLESUBSTITUTEDEPTH);
+    return substitute(conf, expr, depth);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/common/src/test/org/apache/hadoop/hive/conf/TestVariableSubstitution.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestVariableSubstitution.java b/common/src/test/org/apache/hadoop/hive/conf/TestVariableSubstitution.java
new file mode 100644
index 0000000..faa07a7
--- /dev/null
+++ b/common/src/test/org/apache/hadoop/hive/conf/TestVariableSubstitution.java
@@ -0,0 +1,63 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hadoop.hive.conf;
+
+import junit.framework.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestVariableSubstitution {
+  private static class LocalMySource {
+    final Map<String, String> map = new HashMap<>();
+
+    public void put(String k, String v) {
+      map.put(k, v);
+    }
+
+    public String get(String k) {
+      return map.get(k);
+    }
+  }
+
+  private static LocalMySource getMySource() {
+    return localSource.get();
+  }
+
+  private static ThreadLocal<LocalMySource> localSource = new ThreadLocal<LocalMySource>() {
+    @Override protected LocalMySource initialValue() {
+      return new LocalMySource();
+    }
+  };
+
+  @Test public void testVariableSource() throws InterruptedException {
+    final VariableSubstitution variableSubstitution =
+        new VariableSubstitution(new HiveVariableSource() {
+          @Override public Map<String, String> getHiveVariable() {
+            return TestVariableSubstitution.getMySource().map;
+          }
+        });
+
+    String v = variableSubstitution.substitute(new HiveConf(), "${a}");
+    Assert.assertEquals("${a}", v);
+    TestVariableSubstitution.getMySource().put("a", "b");
+    v = variableSubstitution.substitute(new HiveConf(), "${a}");
+    Assert.assertEquals("b", v);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 669e6be..140d6aa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -40,6 +40,8 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.conf.HiveVariableSource;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -96,7 +98,6 @@ import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
 import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.SemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.parse.VariableSubstitution;
 import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -385,7 +386,12 @@ public class Driver implements CommandProcessor {
     SessionState.get().setupQueryCurrentTimestamp();
 
     try {
-      command = new VariableSubstitution().substitute(conf,command);
+      command = new VariableSubstitution(new HiveVariableSource() {
+        @Override
+        public Map<String, String> getHiveVariable() {
+          return SessionState.get().getHiveVariables();
+        }
+      }).substitute(conf, command);
       ctx = new Context(conf);
       ctx.setTryCount(getTryCount());
       ctx.setCmd(command);

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
index a5f0a7f..da2a902 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
@@ -28,12 +28,13 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.conf.HiveVariableSource;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde.serdeConstants;
@@ -216,10 +217,10 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
       //for other usually not used types, just quote the value
       returnVal = "'" + partVal + "'";
     }
-    
+
     return returnVal;
   }
-  
+
   private String getColTypeOf (String partKey) throws SemanticException{
 
     for (FieldSchema fs : tbl.getPartitionKeys()) {
@@ -333,7 +334,12 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     }
 
     rewrittenQuery = rewrittenQueryBuilder.toString();
-    rewrittenQuery = new VariableSubstitution().substitute(conf, rewrittenQuery);
+    rewrittenQuery = new VariableSubstitution(new HiveVariableSource() {
+      @Override
+      public Map<String, String> getHiveVariable() {
+        return SessionState.get().getHiveVariables();
+      }
+    }).substitute(conf, rewrittenQuery);
     return rewrittenQuery;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/ql/src/java/org/apache/hadoop/hive/ql/parse/VariableSubstitution.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/VariableSubstitution.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/VariableSubstitution.java
deleted file mode 100644
index e8b1d96..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/VariableSubstitution.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * 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.hadoop.hive.ql.parse;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.conf.SystemVariables;
-
-import java.util.Map;
-
-public class VariableSubstitution extends SystemVariables {
-
-  private static final Log l4j = LogFactory.getLog(VariableSubstitution.class);
-
-  @Override
-  protected String getSubstitute(Configuration conf, String var) {
-    String val = super.getSubstitute(conf, var);
-    if (val == null && SessionState.get() != null) {
-      Map<String,String> vars = SessionState.get().getHiveVariables();
-      if (var.startsWith(HIVEVAR_PREFIX)) {
-        val =  vars.get(var.substring(HIVEVAR_PREFIX.length()));
-      } else {
-        val = vars.get(var);
-      }
-    }
-    return val;
-  }
-
-  public String substitute(HiveConf conf, String expr) {
-    if (expr == null) {
-      return expr;
-    }
-    if (HiveConf.getBoolVar(conf, ConfVars.HIVEVARIABLESUBSTITUTE)) {
-      l4j.debug("Substitution is on: " + expr);
-    } else {
-      return expr;
-    }
-    int depth = HiveConf.getIntVar(conf, ConfVars.HIVEVARIABLESUBSTITUTEDEPTH);
-    return substitute(conf, expr, depth);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/ql/src/java/org/apache/hadoop/hive/ql/processors/AddResourceProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/AddResourceProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/AddResourceProcessor.java
index 0558c53..d2ac993 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/AddResourceProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/AddResourceProcessor.java
@@ -19,11 +19,13 @@
 package org.apache.hadoop.hive.ql.processors;
 
 import java.util.Arrays;
+import java.util.Map;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.parse.VariableSubstitution;
+import org.apache.hadoop.hive.conf.HiveVariableSource;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
@@ -45,7 +47,12 @@ public class AddResourceProcessor implements CommandProcessor {
   @Override
   public CommandProcessorResponse run(String command) {
     SessionState ss = SessionState.get();
-    command = new VariableSubstitution().substitute(ss.getConf(),command);
+    command = new VariableSubstitution(new HiveVariableSource() {
+      @Override
+      public Map<String, String> getHiveVariable() {
+        return SessionState.get().getHiveVariables();
+      }
+    }).substitute(ss.getConf(),command);
     String[] tokens = command.split("\\s+");
     SessionState.ResourceType t;
     if (tokens.length < 2

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/ql/src/java/org/apache/hadoop/hive/ql/processors/CompileProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/CompileProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/CompileProcessor.java
index 25ce168..7b79f64 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/CompileProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/CompileProcessor.java
@@ -24,6 +24,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.charset.Charset;
 import java.util.Arrays;
+import java.util.Map;
 import java.util.StringTokenizer;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -32,9 +33,10 @@ import org.apache.commons.compress.archivers.jar.JarArchiveOutputStream;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveVariableSource;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.parse.VariableSubstitution;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
@@ -142,7 +144,12 @@ public class CompileProcessor implements CommandProcessor {
   @VisibleForTesting
   void parse(SessionState ss) throws CompileProcessorException {
     if (ss != null){
-      command = new VariableSubstitution().substitute(ss.getConf(), command);
+      command = new VariableSubstitution(new HiveVariableSource() {
+        @Override
+        public Map<String, String> getHiveVariable() {
+          return SessionState.get().getHiveVariables();
+        }
+      }).substitute(ss.getConf(), command);
     }
     if (command == null || command.length() == 0) {
       throw new CompileProcessorException("Command was empty");

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/ql/src/java/org/apache/hadoop/hive/ql/processors/DeleteResourceProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/DeleteResourceProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/DeleteResourceProcessor.java
index 9052c82..736fa9c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/DeleteResourceProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/DeleteResourceProcessor.java
@@ -19,11 +19,13 @@
 package org.apache.hadoop.hive.ql.processors;
 
 import java.util.Arrays;
+import java.util.Map;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.parse.VariableSubstitution;
+import org.apache.hadoop.hive.conf.HiveVariableSource;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
@@ -44,7 +46,12 @@ public class DeleteResourceProcessor implements CommandProcessor {
   @Override
   public CommandProcessorResponse run(String command) {
     SessionState ss = SessionState.get();
-    command = new VariableSubstitution().substitute(ss.getConf(),command);
+    command = new VariableSubstitution(new HiveVariableSource() {
+      @Override
+      public Map<String, String> getHiveVariable() {
+        return SessionState.get().getHiveVariables();
+      }
+    }).substitute(ss.getConf(), command);
     String[] tokens = command.split("\\s+");
 
     SessionState.ResourceType t;

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/ql/src/java/org/apache/hadoop/hive/ql/processors/DfsProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/DfsProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/DfsProcessor.java
index cc0414d..c3d5f81 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/DfsProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/DfsProcessor.java
@@ -20,14 +20,16 @@ package org.apache.hadoop.hive.ql.processors;
 
 import java.io.PrintStream;
 import java.util.Arrays;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.hive.conf.HiveVariableSource;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
-import org.apache.hadoop.hive.ql.parse.VariableSubstitution;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
@@ -65,7 +67,12 @@ public class DfsProcessor implements CommandProcessor {
 
     try {
       SessionState ss = SessionState.get();
-      command = new VariableSubstitution().substitute(ss.getConf(),command);
+      command = new VariableSubstitution(new HiveVariableSource() {
+        @Override
+        public Map<String, String> getHiveVariable() {
+          return SessionState.get().getHiveVariables();
+        }
+      }).substitute(ss.getConf(), command);
 
       String[] tokens = command.split("\\s+");
       CommandProcessorResponse authErrResp =

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
index d271d6d..e5f5f71 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
@@ -30,10 +30,11 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveVariableSource;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.ql.parse.VariableSubstitution;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 /**
@@ -121,17 +122,33 @@ public class SetProcessor implements CommandProcessor {
       return 1;
     } else if (varname.startsWith(SYSTEM_PREFIX)){
       String propName = varname.substring(SYSTEM_PREFIX.length());
-      System.getProperties().setProperty(propName, new VariableSubstitution().substitute(ss.getConf(),varvalue));
+      System.getProperties()
+          .setProperty(propName, new VariableSubstitution(new HiveVariableSource() {
+            @Override
+            public Map<String, String> getHiveVariable() {
+              return SessionState.get().getHiveVariables();
+            }
+          }).substitute(ss.getConf(), varvalue));
     } else if (varname.startsWith(HIVECONF_PREFIX)){
       String propName = varname.substring(HIVECONF_PREFIX.length());
       setConf(varname, propName, varvalue, false);
     } else if (varname.startsWith(HIVEVAR_PREFIX)) {
       String propName = varname.substring(HIVEVAR_PREFIX.length());
-      ss.getHiveVariables().put(propName, new VariableSubstitution().substitute(ss.getConf(),varvalue));
+      ss.getHiveVariables().put(propName, new VariableSubstitution(new HiveVariableSource() {
+        @Override
+        public Map<String, String> getHiveVariable() {
+          return SessionState.get().getHiveVariables();
+        }
+      }).substitute(ss.getConf(), varvalue));
     } else if (varname.startsWith(METACONF_PREFIX)) {
       String propName = varname.substring(METACONF_PREFIX.length());
       Hive hive = Hive.get(ss.getConf());
-      hive.setMetaConf(propName, new VariableSubstitution().substitute(ss.getConf(), varvalue));
+      hive.setMetaConf(propName, new VariableSubstitution(new HiveVariableSource() {
+        @Override
+        public Map<String, String> getHiveVariable() {
+          return SessionState.get().getHiveVariables();
+        }
+      }).substitute(ss.getConf(), varvalue));
     } else {
       setConf(varname, varname, varvalue, true);
       if (varname.equals(HiveConf.ConfVars.HIVE_SESSION_HISTORY_ENABLED.toString())) {
@@ -145,7 +162,12 @@ public class SetProcessor implements CommandProcessor {
   private static void setConf(String varname, String key, String varvalue, boolean register)
         throws IllegalArgumentException {
     HiveConf conf = SessionState.get().getConf();
-    String value = new VariableSubstitution().substitute(conf, varvalue);
+    String value = new VariableSubstitution(new HiveVariableSource() {
+      @Override
+      public Map<String, String> getHiveVariable() {
+        return SessionState.get().getHiveVariables();
+      }
+    }).substitute(conf, varvalue);
     if (conf.getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) {
       HiveConf.ConfVars confVars = HiveConf.getConfVars(key);
       if (confVars != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/102b23b1/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index cc9df76..f4334e4 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -32,6 +32,8 @@ import java.util.concurrent.RejectedExecutionException;
 
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveVariableSource;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
@@ -39,7 +41,6 @@ import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.exec.ExplainTask;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.ql.parse.VariableSubstitution;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.OperationLog;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -105,7 +106,12 @@ public class SQLOperation extends ExecuteStatementOperation {
       // For now, we disable the test attempts.
       driver.setTryCount(Integer.MAX_VALUE);
 
-      String subStatement = new VariableSubstitution().substitute(sqlOperationConf, statement);
+      String subStatement = new VariableSubstitution(new HiveVariableSource() {
+        @Override
+        public Map<String, String> getHiveVariable() {
+          return SessionState.get().getHiveVariables();
+        }
+      }).substitute(sqlOperationConf, statement);
       response = driver.compileAndRespond(subStatement);
       if (0 != response.getResponseCode()) {
         throw toSQLException("Error while compiling statement", response);