You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2014/10/30 17:22:48 UTC

svn commit: r1635536 [2/28] - in /hive/branches/spark: ./ accumulo-handler/ accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/columns/ accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/columns/ accumulo-handler/src/test/org/apache/hadoo...

Modified: hive/branches/spark/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java (original)
+++ hive/branches/spark/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java Thu Oct 30 16:22:33 2014
@@ -17,7 +17,19 @@
  */
 package org.apache.hive.beeline;
 
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
 import java.util.IllegalFormatException;
+import java.util.List;
 
 public class HiveSchemaHelper {
   public static final String DB_DERBY = "derby";
@@ -26,6 +38,56 @@ public class HiveSchemaHelper {
   public static final String DB_POSTGRACE = "postgres";
   public static final String DB_ORACLE = "oracle";
 
+  /***
+   * Get JDBC connection to metastore db
+   *
+   * @param userName metastore connection username
+   * @param password metastore connection password
+   * @param printInfo print connection parameters
+   * @param hiveConf hive config object
+   * @return metastore connection object
+   * @throws org.apache.hadoop.hive.metastore.api.MetaException
+   */
+  public static Connection getConnectionToMetastore(String userName,
+      String password, boolean printInfo, HiveConf hiveConf)
+      throws HiveMetaException {
+    try {
+      String connectionURL = getValidConfVar(
+          HiveConf.ConfVars.METASTORECONNECTURLKEY, hiveConf);
+      String driver = getValidConfVar(
+          HiveConf.ConfVars.METASTORE_CONNECTION_DRIVER, hiveConf);
+      if (printInfo) {
+        System.out.println("Metastore connection URL:\t " + connectionURL);
+        System.out.println("Metastore Connection Driver :\t " + driver);
+        System.out.println("Metastore connection User:\t " + userName);
+      }
+      if ((userName == null) || userName.isEmpty()) {
+        throw new HiveMetaException("UserName empty ");
+      }
+
+      // load required JDBC driver
+      Class.forName(driver);
+
+      // Connect using the JDBC URL and user/pass from conf
+      return DriverManager.getConnection(connectionURL, userName, password);
+    } catch (IOException e) {
+      throw new HiveMetaException("Failed to get schema version.", e);
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to get schema version.", e);
+    } catch (ClassNotFoundException e) {
+      throw new HiveMetaException("Failed to load driver", e);
+    }
+  }
+
+  public static String getValidConfVar(HiveConf.ConfVars confVar, HiveConf hiveConf)
+      throws IOException {
+    String confVarStr = hiveConf.get(confVar.varname);
+    if (confVarStr == null || confVarStr.isEmpty()) {
+      throw new IOException("Empty " + confVar.varname);
+    }
+    return confVarStr;
+  }
+
   public interface NestedScriptParser {
 
     public enum CommandType {
@@ -57,7 +119,7 @@ public class HiveSchemaHelper {
     public boolean isNestedScript(String dbCommand);
 
     /***
-     * Find if the given command is should be passed to DB
+     * Find if the given command should not be passed to DB
      * @param dbCommand
      * @return
      */
@@ -80,8 +142,16 @@ public class HiveSchemaHelper {
      * @return
      */
     public boolean needsQuotedIdentifier();
-  }
 
+    /***
+     * Flatten the nested upgrade script into a buffer
+     * @param scriptDir upgrade script directory
+     * @param scriptFile upgrade script file
+     * @return string of sql commands
+     */
+    public String buildCommand(String scriptDir, String scriptFile)
+        throws IllegalFormatException, IOException;
+  }
 
   /***
    * Base implemenation of NestedScriptParser
@@ -89,6 +159,18 @@ public class HiveSchemaHelper {
    *
    */
   private static abstract class AbstractCommandParser implements NestedScriptParser {
+    private List<String> dbOpts;
+    private String msUsername;
+    private String msPassword;
+    private HiveConf hiveConf;
+
+    public AbstractCommandParser(String dbOpts, String msUsername, String msPassword,
+        HiveConf hiveConf) {
+      setDbOpts(dbOpts);
+      this.msUsername = msUsername;
+      this.msPassword = msPassword;
+      this.hiveConf = hiveConf;
+    }
 
     @Override
     public boolean isPartialCommand(String dbCommand) throws IllegalArgumentException{
@@ -127,13 +209,84 @@ public class HiveSchemaHelper {
     public boolean needsQuotedIdentifier() {
       return false;
     }
-  }
 
+    @Override
+    public String buildCommand(
+      String scriptDir, String scriptFile) throws IllegalFormatException, IOException {
+      BufferedReader bfReader =
+          new BufferedReader(new FileReader(scriptDir + File.separatorChar + scriptFile));
+      String currLine;
+      StringBuilder sb = new StringBuilder();
+      String currentCommand = null;
+      while ((currLine = bfReader.readLine()) != null) {
+        currLine = currLine.trim();
+        if (currLine.isEmpty()) {
+          continue; // skip empty lines
+        }
+
+        if (currentCommand == null) {
+          currentCommand = currLine;
+        } else {
+          currentCommand = currentCommand + " " + currLine;
+        }
+        if (isPartialCommand(currLine)) {
+          // if its a partial line, continue collecting the pieces
+          continue;
+        }
+
+        // if this is a valid executable command then add it to the buffer
+        if (!isNonExecCommand(currentCommand)) {
+          currentCommand = cleanseCommand(currentCommand);
+
+          if (isNestedScript(currentCommand)) {
+            // if this is a nested sql script then flatten it
+            String currScript = getScriptName(currentCommand);
+            sb.append(buildCommand(scriptDir, currScript));
+          } else {
+            // Now we have a complete statement, process it
+            // write the line to buffer
+            sb.append(currentCommand);
+            sb.append(System.getProperty("line.separator"));
+          }
+        }
+        currentCommand = null;
+      }
+      bfReader.close();
+      return sb.toString();
+    }
+
+    private void setDbOpts(String dbOpts) {
+      if (dbOpts != null) {
+        this.dbOpts = Lists.newArrayList(dbOpts.split(","));
+      }
+    }
+
+    protected List<String> getDbOpts() {
+      return dbOpts;
+    }
+
+    protected String getMsUsername() {
+      return msUsername;
+    }
+
+    protected String getMsPassword() {
+      return msPassword;
+    }
+
+    protected HiveConf getHiveConf() {
+      return hiveConf;
+    }
+  }
 
   // Derby commandline parser
   public static class DerbyCommandParser extends AbstractCommandParser {
     private static String DERBY_NESTING_TOKEN = "RUN";
 
+    public DerbyCommandParser(String dbOpts, String msUsername, String msPassword,
+        HiveConf hiveConf) {
+      super(dbOpts, msUsername, msPassword, hiveConf);
+    }
+
     @Override
     public String getScriptName(String dbCommand) throws IllegalArgumentException {
 
@@ -154,13 +307,17 @@ public class HiveSchemaHelper {
     }
   }
 
-
   // MySQL parser
   public static class MySqlCommandParser extends AbstractCommandParser {
     private static final String MYSQL_NESTING_TOKEN = "SOURCE";
     private static final String DELIMITER_TOKEN = "DELIMITER";
     private String delimiter = DEFAUTL_DELIMITER;
 
+    public MySqlCommandParser(String dbOpts, String msUsername, String msPassword,
+        HiveConf hiveConf) {
+      super(dbOpts, msUsername, msPassword, hiveConf);
+    }
+
     @Override
     public boolean isPartialCommand(String dbCommand) throws IllegalArgumentException{
       boolean isPartial = super.isPartialCommand(dbCommand);
@@ -213,6 +370,11 @@ public class HiveSchemaHelper {
   public static class PostgresCommandParser extends AbstractCommandParser {
     private static String POSTGRES_NESTING_TOKEN = "\\i";
 
+    public PostgresCommandParser(String dbOpts, String msUsername, String msPassword,
+        HiveConf hiveConf) {
+      super(dbOpts, msUsername, msPassword, hiveConf);
+    }
+
     @Override
     public String getScriptName(String dbCommand) throws IllegalArgumentException {
       String[] tokens = dbCommand.split(" ");
@@ -237,6 +399,12 @@ public class HiveSchemaHelper {
   //Oracle specific parser
   public static class OracleCommandParser extends AbstractCommandParser {
     private static String ORACLE_NESTING_TOKEN = "@";
+
+    public OracleCommandParser(String dbOpts, String msUsername, String msPassword,
+        HiveConf hiveConf) {
+      super(dbOpts, msUsername, msPassword, hiveConf);
+    }
+
     @Override
     public String getScriptName(String dbCommand) throws IllegalArgumentException {
       if (!isNestedScript(dbCommand)) {
@@ -255,6 +423,12 @@ public class HiveSchemaHelper {
   //MSSQL specific parser
   public static class MSSQLCommandParser extends AbstractCommandParser {
     private static String MSSQL_NESTING_TOKEN = ":r";
+
+    public MSSQLCommandParser(String dbOpts, String msUsername, String msPassword,
+        HiveConf hiveConf) {
+      super(dbOpts, msUsername, msPassword, hiveConf);
+    }
+
     @Override
     public String getScriptName(String dbCommand) throws IllegalArgumentException {
       String[] tokens = dbCommand.split(" ");
@@ -271,18 +445,24 @@ public class HiveSchemaHelper {
   }
 
   public static NestedScriptParser getDbCommandParser(String dbName) {
+    return getDbCommandParser(dbName, null, null, null, null);
+  }
+
+  public static NestedScriptParser getDbCommandParser(String dbName,
+      String dbOpts, String msUsername, String msPassword,
+      HiveConf hiveConf) {
     if (dbName.equalsIgnoreCase(DB_DERBY)) {
-      return new DerbyCommandParser();
+      return new DerbyCommandParser(dbOpts, msUsername, msPassword, hiveConf);
     } else if (dbName.equalsIgnoreCase(DB_MSSQL)) {
-      return new MSSQLCommandParser();
+      return new MSSQLCommandParser(dbOpts, msUsername, msPassword, hiveConf);
     } else if (dbName.equalsIgnoreCase(DB_MYSQL)) {
-      return new MySqlCommandParser();
+      return new MySqlCommandParser(dbOpts, msUsername, msPassword, hiveConf);
     } else if (dbName.equalsIgnoreCase(DB_POSTGRACE)) {
-      return new PostgresCommandParser();
+      return new PostgresCommandParser(dbOpts, msUsername, msPassword, hiveConf);
     } else if (dbName.equalsIgnoreCase(DB_ORACLE)) {
-      return new OracleCommandParser();
+      return new OracleCommandParser(dbOpts, msUsername, msPassword, hiveConf);
     } else {
       throw new IllegalArgumentException("Unknown dbType " + dbName);
     }
   }
-}
+}
\ No newline at end of file

Modified: hive/branches/spark/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java (original)
+++ hive/branches/spark/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java Thu Oct 30 16:22:33 2014
@@ -17,20 +17,16 @@
  */
 package org.apache.hive.beeline;
 
-import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.File;
-import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.sql.Connection;
-import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
-import java.util.IllegalFormatException;
 import java.util.List;
 
 import org.apache.commons.cli.CommandLine;
@@ -56,6 +52,7 @@ public class HiveSchemaTool {
   private String passWord = null;
   private boolean dryRun = false;
   private boolean verbose = false;
+  private String dbOpts = null;
   private final HiveConf hiveConf;
   private final String dbType;
   private final MetaStoreSchemaInfo metaStoreSchemaInfo;
@@ -65,7 +62,7 @@ public class HiveSchemaTool {
   }
 
   public HiveSchemaTool(String hiveHome, HiveConf hiveConf, String dbType)
-        throws HiveMetaException {
+      throws HiveMetaException {
     if (hiveHome == null || hiveHome.isEmpty()) {
       throw new HiveMetaException("No Hive home directory provided");
     }
@@ -101,12 +98,27 @@ public class HiveSchemaTool {
     this.verbose = verbose;
   }
 
+  public void setDbOpts(String dbOpts) {
+    this.dbOpts = dbOpts;
+  }
+
   private static void printAndExit(Options cmdLineOptions) {
     HelpFormatter formatter = new HelpFormatter();
     formatter.printHelp("schemaTool", cmdLineOptions);
     System.exit(1);
   }
 
+  private Connection getConnectionToMetastore(boolean printInfo)
+      throws HiveMetaException {
+    return HiveSchemaHelper.getConnectionToMetastore(userName,
+        passWord, printInfo, hiveConf);
+  }
+
+  private NestedScriptParser getDbCommandParser(String dbType) {
+    return HiveSchemaHelper.getDbCommandParser(dbType, dbOpts, userName,
+        passWord, hiveConf);
+  }
+
   /***
    * Print Hive version and schema version
    * @throws MetaException
@@ -121,9 +133,9 @@ public class HiveSchemaTool {
 
   // read schema version from metastore
   private String getMetaStoreSchemaVersion(Connection metastoreConn)
-        throws HiveMetaException {
+      throws HiveMetaException {
     String versionQuery;
-    if (HiveSchemaHelper.getDbCommandParser(dbType).needsQuotedIdentifier()) {
+    if (getDbCommandParser(dbType).needsQuotedIdentifier()) {
       versionQuery = "select t.\"SCHEMA_VERSION\" from \"VERSION\" t";
     } else {
       versionQuery = "select t.SCHEMA_VERSION from VERSION t";
@@ -152,40 +164,6 @@ public class HiveSchemaTool {
     }
   }
 
-  /***
-   * get JDBC connection to metastore db
-   *
-   * @param printInfo print connection parameters
-   * @return
-   * @throws MetaException
-   */
-  private Connection getConnectionToMetastore(boolean printInfo)
-        throws HiveMetaException {
-    try {
-      String connectionURL = getValidConfVar(ConfVars.METASTORECONNECTURLKEY);
-      String driver = getValidConfVar(ConfVars.METASTORE_CONNECTION_DRIVER);
-      if (printInfo) {
-        System.out.println("Metastore connection URL:\t " + connectionURL);
-        System.out.println("Metastore Connection Driver :\t " + driver);
-        System.out.println("Metastore connection User:\t " + userName);
-      }
-      if ((userName == null) || userName.isEmpty()) {
-        throw new HiveMetaException("UserName empty ");
-      }
-
-      // load required JDBC driver
-      Class.forName(driver);
-
-      // Connect using the JDBC URL and user/pass from conf
-      return DriverManager.getConnection(connectionURL, userName, passWord);
-    } catch (IOException e) {
-      throw new HiveMetaException("Failed to get schema version.", e);
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get schema version.", e);
-    } catch (ClassNotFoundException e) {
-      throw new HiveMetaException("Failed to load driver", e);
-    }
-  }
 
   /**
    * check if the current schema version in metastore matches the Hive version
@@ -196,8 +174,8 @@ public class HiveSchemaTool {
     if (dryRun) {
       return;
     }
-    String newSchemaVersion =
-        getMetaStoreSchemaVersion(getConnectionToMetastore(false));
+    String newSchemaVersion = getMetaStoreSchemaVersion(
+        getConnectionToMetastore(false));
     // verify that the new version is added to schema
     if (!MetaStoreSchemaInfo.getHiveSchemaVersion().equalsIgnoreCase(newSchemaVersion)) {
       throw new HiveMetaException("Found unexpected schema version " + newSchemaVersion);
@@ -209,7 +187,8 @@ public class HiveSchemaTool {
    * @throws MetaException
    */
   public void doUpgrade() throws HiveMetaException {
-    String fromVersion = getMetaStoreSchemaVersion(getConnectionToMetastore(false));
+    String fromVersion = getMetaStoreSchemaVersion(
+        getConnectionToMetastore(false));
     if (fromVersion == null || fromVersion.isEmpty()) {
       throw new HiveMetaException("Schema version not stored in the metastore. " +
           "Metastore schema is too old or corrupt. Try specifying the version manually");
@@ -240,6 +219,7 @@ public class HiveSchemaTool {
       for (String scriptFile : upgradeScripts) {
         System.out.println("Upgrade script " + scriptFile);
         if (!dryRun) {
+          runPreUpgrade(scriptDir, scriptFile);
           runBeeLine(scriptDir, scriptFile);
           System.out.println("Completed " + scriptFile);
         }
@@ -291,58 +271,47 @@ public class HiveSchemaTool {
     }
   }
 
-  // Flatten the nested upgrade script into a buffer
-  public static String buildCommand(NestedScriptParser dbCommandParser,
-        String scriptDir, String scriptFile) throws IllegalFormatException, IOException {
-
-    BufferedReader bfReader =
-        new BufferedReader(new FileReader(scriptDir + File.separatorChar + scriptFile));
-    String currLine;
-    StringBuilder sb = new StringBuilder();
-    String currentCommand = null;
-    while ((currLine = bfReader.readLine()) != null) {
-      currLine = currLine.trim();
-      if (currLine.isEmpty()) {
-        continue; // skip empty lines
-      }
-
-      if (currentCommand == null) {
-        currentCommand = currLine;
-      } else {
-        currentCommand = currentCommand + " " + currLine;
-      }
-      if (dbCommandParser.isPartialCommand(currLine)) {
-        // if its a partial line, continue collecting the pieces
-        continue;
-      }
-
-      // if this is a valid executable command then add it to the buffer
-      if (!dbCommandParser.isNonExecCommand(currentCommand)) {
-        currentCommand = dbCommandParser.cleanseCommand(currentCommand);
-
-        if (dbCommandParser.isNestedScript(currentCommand)) {
-          // if this is a nested sql script then flatten it
-          String currScript = dbCommandParser.getScriptName(currentCommand);
-          sb.append(buildCommand(dbCommandParser, scriptDir, currScript));
-        } else {
-          // Now we have a complete statement, process it
-          // write the line to buffer
-          sb.append(currentCommand);
-          sb.append(System.getProperty("line.separator"));
+  /**
+   *  Run pre-upgrade scripts corresponding to a given upgrade script,
+   *  if any exist. The errors from pre-upgrade are ignored.
+   *  Pre-upgrade scripts typically contain setup statements which
+   *  may fail on some database versions and failure is ignorable.
+   *
+   *  @param scriptDir upgrade script directory name
+   *  @param scriptFile upgrade script file name
+   */
+  private void runPreUpgrade(String scriptDir, String scriptFile) {
+    for (int i = 0;; i++) {
+      String preUpgradeScript =
+          MetaStoreSchemaInfo.getPreUpgradeScriptName(i, scriptFile);
+      File preUpgradeScriptFile = new File(scriptDir, preUpgradeScript);
+      if (!preUpgradeScriptFile.isFile()) {
+        break;
+      }
+
+      try {
+        runBeeLine(scriptDir, preUpgradeScript);
+        System.out.println("Completed " + preUpgradeScript);
+      } catch (Exception e) {
+        // Ignore the pre-upgrade script errors
+        System.err.println("Warning in pre-upgrade script " + preUpgradeScript + ": "
+            + e.getMessage());
+        if (verbose) {
+          e.printStackTrace();
         }
       }
-      currentCommand = null;
     }
-    bfReader.close();
-    return sb.toString();
   }
 
-  // run beeline on the given metastore scrip, flatten the nested scripts into single file
-  private void runBeeLine(String scriptDir, String scriptFile) throws IOException {
-    NestedScriptParser dbCommandParser =
-        HiveSchemaHelper.getDbCommandParser(dbType);
+  /***
+   * Run beeline with the given metastore script. Flatten the nested scripts
+   * into single file.
+   */
+  private void runBeeLine(String scriptDir, String scriptFile)
+      throws IOException, HiveMetaException {
+    NestedScriptParser dbCommandParser = getDbCommandParser(dbType);
     // expand the nested script
-    String sqlCommands = buildCommand(dbCommandParser, scriptDir, scriptFile);
+    String sqlCommands = dbCommandParser.buildCommand(scriptDir, scriptFile);
     File tmpFile = File.createTempFile("schematool", ".sql");
     tmpFile.deleteOnExit();
 
@@ -360,9 +329,11 @@ public class HiveSchemaTool {
   public void runBeeLine(String sqlScriptFile) throws IOException {
     List<String> argList = new ArrayList<String>();
     argList.add("-u");
-    argList.add(getValidConfVar(ConfVars.METASTORECONNECTURLKEY));
+    argList.add(HiveSchemaHelper.getValidConfVar(
+        ConfVars.METASTORECONNECTURLKEY, hiveConf));
     argList.add("-d");
-    argList.add(getValidConfVar(ConfVars.METASTORE_CONNECTION_DRIVER));
+    argList.add(HiveSchemaHelper.getValidConfVar(
+        ConfVars.METASTORE_CONNECTION_DRIVER, hiveConf));
     argList.add("-n");
     argList.add(userName);
     argList.add("-p");
@@ -384,14 +355,6 @@ public class HiveSchemaTool {
     }
   }
 
-  private String getValidConfVar(ConfVars confVar) throws IOException {
-    String confVarStr = hiveConf.get(confVar.varname);
-    if (confVarStr == null || confVarStr.isEmpty()) {
-      throw new IOException("Empty " + confVar.varname);
-    }
-    return confVarStr;
-  }
-
   // Create the required command line options
   @SuppressWarnings("static-access")
   private static void initOptions(Options cmdLineOptions) {
@@ -423,6 +386,9 @@ public class HiveSchemaTool {
     Option dbTypeOpt = OptionBuilder.withArgName("databaseType")
                 .hasArgs().withDescription("Metastore database type")
                 .create("dbType");
+    Option dbOpts = OptionBuilder.withArgName("databaseOpts")
+                .hasArgs().withDescription("Backend DB specific options")
+                .create("dbOpts");
     Option dryRunOpt = new Option("dryRun", "list SQL scripts (no execute)");
     Option verboseOpt = new Option("verbose", "only print SQL statements");
 
@@ -432,6 +398,7 @@ public class HiveSchemaTool {
     cmdLineOptions.addOption(passwdOpt);
     cmdLineOptions.addOption(dbTypeOpt);
     cmdLineOptions.addOption(verboseOpt);
+    cmdLineOptions.addOption(dbOpts);
     cmdLineOptions.addOptionGroup(optGroup);
   }
 
@@ -488,7 +455,9 @@ public class HiveSchemaTool {
       if (line.hasOption("verbose")) {
         schemaTool.setVerbose(true);
       }
-
+      if (line.hasOption("dbOpts")) {
+        schemaTool.setDbOpts(line.getOptionValue("dbOpts"));
+      }
       if (line.hasOption("info")) {
         schemaTool.showInfo();
       } else if (line.hasOption("upgradeSchema")) {

Modified: hive/branches/spark/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java (original)
+++ hive/branches/spark/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java Thu Oct 30 16:22:33 2014
@@ -51,11 +51,14 @@ class SeparatedValuesOutputFormat implem
       char newDel = beeLine.getOpts().getDelimiterForDSV();
       // if delimiter changed, rebuild the csv preference
       if (newDel != curDel) {
+        // "" is passed as the end of line symbol in following function, as
+        // beeline itself adds newline
         csvPreference = new CsvPreference.Builder('"', newDel, "").build();
       }
     }
   }
 
+  @Override
   public int print(Rows rows) {
     updateCsvPreference();
 

Modified: hive/branches/spark/beeline/src/main/resources/BeeLine.properties
URL: http://svn.apache.org/viewvc/hive/branches/spark/beeline/src/main/resources/BeeLine.properties?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/beeline/src/main/resources/BeeLine.properties (original)
+++ hive/branches/spark/beeline/src/main/resources/BeeLine.properties Thu Oct 30 16:22:33 2014
@@ -68,7 +68,7 @@ help-procedures: List all the procedures
 help-tables: List all the tables in the database
 help-columns: List all the columns for the specified table
 help-properties: Connect to the database specified in the properties file(s)
-help-outputformat: Set the output format for displaying results (table,vertical,csv,dsv,tsv,xmlattrs,xmlelements)
+help-outputformat: Set the output format for displaying results (table,vertical,csv2,dsv,tsv2,xmlattrs,xmlelements, and deprecated formats(csv, tsv))
 help-delimiterForDSV: Set the delimiter for dsv output format
 help-nullemptystring: Set to true to get historic behavior of printing null as empty string. Default is false.
 
@@ -167,7 +167,8 @@ cmd-usage: Usage: java org.apache.hive.c
 \  --maxColumnWidth=MAXCOLWIDTH    the maximum width to use when displaying columns\n \
 \  --silent=[true/false]           be more silent\n \
 \  --autosave=[true/false]         automatically save preferences\n \
-\  --outputformat=[table/vertical/csv/tsv/dsv]   format mode for result display\n \
+\  --outputformat=[table/vertical/csv2/tsv2/dsv/csv/tsv]  format mode for result display\n \
+\                                  Note that csv, and tsv are deprecated - use csv2, tsv2 instead\n\
 \  --truncateTable=[true/false]    truncate table column when it exceeds length\n \
 \  --delimiterForDSV=DELIMITER     specify the delimiter for delimiter-separated values output format (default: |)\n \
 \  --isolation=LEVEL               set the transaction isolation level\n \

Modified: hive/branches/spark/bin/ext/hiveserver2.cmd
URL: http://svn.apache.org/viewvc/hive/branches/spark/bin/ext/hiveserver2.cmd?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/bin/ext/hiveserver2.cmd (original)
+++ hive/branches/spark/bin/ext/hiveserver2.cmd Thu Oct 30 16:22:33 2014
@@ -59,10 +59,7 @@ if [%1]==[hiveserver2_help] goto :hivese
 
 if [%1]==[hiveserver2_catservice] goto :hiveserver2_catservice
 
-if [%1]==[hiveserver2_catcmd] goto :hiveserver2_catcmd
-
 :hiveserver2
-  echo "Starting Hive Thrift Server"
 
   @rem hadoop 20 or newer - skip the aux_jars option and hiveconf
   call %HIVE_BIN_PATH%\ext\util\execHiveCmd.cmd %CLASS%
@@ -78,21 +75,11 @@ goto :EOF
 @echo   ^<id^>HiveServer2^</id^>
 @echo   ^<name^>HiveServer2^</name^>
 @echo   ^<description^>Hadoop HiveServer2 Service^</description^>
-@echo   ^<executable^>%SystemRoot%\system32\cmd.exe^</executable^>
-@echo   ^<arguments^>/c %HIVE_BIN_PATH%\ext\hs2service.cmd ^</arguments^>
+@echo   ^<executable^>%JAVA_HOME%\bin\java^</executable^>
+@echo   ^<arguments^>%JAVA_HEAP_MAX% %HADOOP_OPTS% -classpath %CLASSPATH%;%HIVE_HBASE_PATH% %CLASS% -hiveconf hive.hadoop.classpath=%HIVE_LIB%\* -hiveconf hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory -hiveconf hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator -hiveconf hive.metastore.uris=" " %HIVE_OPTS%^</arguments^>
 @echo ^</service^>
 goto :EOF
 
-
-:hiveserver2_catcmd
-if not defined HADOOP_CLASSPATH (
-  @echo set HADOOP_CLASSPATH=%HIVE_LIB%\*
-  ) else (
-  @echo set HADOOP_CLASSPATH=%HADOOP_CLASSPATH%;%HIVE_LIB%\*
-  )
-@echo %JAVA_HOME%\bin\java %JAVA_HEAP_MAX% %HADOOP_OPTS% -classpath %CLASSPATH%;%HIVE_HBASE_PATH% %CLASS% -hiveconf hive.metastore.uris=" " -hiveconf hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory -hiveconf hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator  %HIVE_OPTS%
-goto :EOF
-
 :AddToHiveHbasePath
 if not defined HIVE_HBASE_PATH (
    set HIVE_HBASE_PATH=%1

Modified: hive/branches/spark/bin/ext/hiveserver2.sh
URL: http://svn.apache.org/viewvc/hive/branches/spark/bin/ext/hiveserver2.sh?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/bin/ext/hiveserver2.sh (original)
+++ hive/branches/spark/bin/ext/hiveserver2.sh Thu Oct 30 16:22:33 2014
@@ -17,7 +17,6 @@ THISSERVICE=hiveserver2
 export SERVICE_LIST="${SERVICE_LIST}${THISSERVICE} "
 
 hiveserver2() {
-  echo "Starting HiveServer2"
   CLASS=org.apache.hive.service.server.HiveServer2
   if $cygwin; then
     HIVE_LIB=`cygpath -w "$HIVE_LIB"`

Modified: hive/branches/spark/bin/hive
URL: http://svn.apache.org/viewvc/hive/branches/spark/bin/hive?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/bin/hive (original)
+++ hive/branches/spark/bin/hive Thu Oct 30 16:22:33 2014
@@ -240,7 +240,9 @@ if [[ -n $HBASE_BIN ]] ; then
 fi
 
 if [ "${AUX_PARAM}" != "" ]; then
-  HIVE_OPTS="$HIVE_OPTS --hiveconf hive.aux.jars.path=${AUX_PARAM}"
+  if [[ "$SERVICE" != beeline ]]; then
+    HIVE_OPTS="$HIVE_OPTS --hiveconf hive.aux.jars.path=${AUX_PARAM}"
+  fi
   AUX_JARS_CMD_LINE="-libjars ${AUX_PARAM}"
 fi
 

Modified: hive/branches/spark/bin/hive.cmd
URL: http://svn.apache.org/viewvc/hive/branches/spark/bin/hive.cmd?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/bin/hive.cmd (original)
+++ hive/branches/spark/bin/hive.cmd Thu Oct 30 16:22:33 2014
@@ -284,9 +284,6 @@ if defined CATSERVICE (
 	) else (
 	  call %HADOOP_HOME%\libexec\hadoop-config.cmd
 	)
-  if %TORUN% == hiveserver2 (
-        call %HIVE_BIN_PATH%\ext\hiveserver2.cmd hiveserver2_catcmd > %HIVE_BIN_PATH%\ext\hs2service.cmd
-  )  
 	call %HIVE_BIN_PATH%\ext\%TORUN%.cmd %TORUN%%CATSERVICE% %*
 	goto :EOF
 )

Modified: hive/branches/spark/cli/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/spark/cli/pom.xml?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/cli/pom.xml (original)
+++ hive/branches/spark/cli/pom.xml Thu Oct 30 16:22:33 2014
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>0.14.0-SNAPSHOT</version>
+    <version>0.15.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

Modified: hive/branches/spark/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (original)
+++ hive/branches/spark/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java Thu Oct 30 16:22:33 2014
@@ -60,6 +60,7 @@ import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 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;
@@ -125,7 +126,7 @@ public class CliDriver {
         ret = 1;
       } else {
         try {
-          this.processFile(cmd_1);
+          ret = processFile(cmd_1);
         } catch (IOException e) {
           console.printError("Failed processing file "+ cmd_1 +" "+ e.getLocalizedMessage(),
             stringifyException(e));
@@ -384,6 +385,7 @@ public class CliDriver {
 
           // First, kill any running MR jobs
           HadoopJobExecHelper.killRunningJobs();
+          TezJobExecHelper.killRunningJobs();
           HiveInterruptUtils.interrupt();
         }
       });

Modified: hive/branches/spark/cli/src/test/org/apache/hadoop/hive/cli/TestRCFileCat.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/cli/src/test/org/apache/hadoop/hive/cli/TestRCFileCat.java?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/cli/src/test/org/apache/hadoop/hive/cli/TestRCFileCat.java (original)
+++ hive/branches/spark/cli/src/test/org/apache/hadoop/hive/cli/TestRCFileCat.java Thu Oct 30 16:22:33 2014
@@ -25,6 +25,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -89,19 +90,19 @@ public class TestRCFileCat {
 
 
     try {
-      String[] params = {"--verbose","file://" + template.getAbsolutePath() };
+      String[] params = {"--verbose","file://" + template.toURI().getPath() };
 
       assertEquals(0, fileCat.run(params));
       assertTrue(dataOut.toString().contains("123\t456\t789\t1000\t5.3\thive and hadoop\t\tNULL"));
       assertTrue(dataOut.toString().contains("100\t200\t123\t1000\t5.3\thive and hadoop\t\tNULL"));
       assertTrue(dataOut.toString().contains("200\t400\t678\t1000\t4.8\thive and hadoop\t\tTEST"));
       dataOut.reset();
-       params = new String[] { "--start=-10","--file-sizes","file://" + template.getAbsolutePath() };
+       params = new String[] { "--start=-10","--file-sizes", "file://" + template.toURI().getPath() };
       assertEquals(0, fileCat.run(params));
       assertTrue(dataOut.toString().contains("File size (uncompressed): 105. File size (compressed): 134. Number of rows: 3."));
       dataOut.reset();
 
-      params = new String[] {"--start=0", "--column-sizes","file://" + template.getAbsolutePath() };
+      params = new String[] {"--start=0", "--column-sizes","file://" + template.toURI().getPath() };
       assertEquals(0, fileCat.run(params));
       assertTrue(dataOut.toString().contains("0\t9\t17"));
       assertTrue(dataOut.toString().contains("1\t9\t17"));
@@ -112,7 +113,8 @@ public class TestRCFileCat {
 
 
       dataOut.reset();
-      params = new String[] {"--start=0", "--column-sizes-pretty","file://" + template.getAbsolutePath() };
+      params = new String[] {"--start=0", "--column-sizes-pretty",
+          "file://" + template.toURI().getPath() };
       assertEquals(0, fileCat.run(params));
       assertTrue(dataOut.toString().contains("Column 0: Uncompressed size: 9 Compressed size: 17"));
       assertTrue(dataOut.toString().contains("Column 1: Uncompressed size: 9 Compressed size: 17"));
@@ -127,7 +129,8 @@ public class TestRCFileCat {
           "[--column-sizes | --column-sizes-pretty] [--file-sizes] fileName"));
 
       dataErr.reset();
-      params = new String[] { "--fakeParameter","file://" + template.getAbsolutePath()};
+      params = new String[] { "--fakeParameter",
+          "file://" + template.toURI().getPath()};
       assertEquals(-1, fileCat.run(params));
       assertTrue(dataErr.toString().contains("RCFileCat [--start=start_offet] [--length=len] [--verbose] " +
           "[--column-sizes | --column-sizes-pretty] [--file-sizes] fileName"));

Modified: hive/branches/spark/common/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/spark/common/pom.xml?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/common/pom.xml (original)
+++ hive/branches/spark/common/pom.xml Thu Oct 30 16:22:33 2014
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>0.14.0-SNAPSHOT</version>
+    <version>0.15.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

Modified: hive/branches/spark/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/branches/spark/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Thu Oct 30 16:22:33 2014
@@ -53,6 +53,8 @@ import org.apache.hadoop.security.UserGr
 import org.apache.hadoop.util.Shell;
 import org.apache.hive.common.HiveCompat;
 
+import com.google.common.base.Joiner;
+
 /**
  * Hive Configuration.
  */
@@ -75,11 +77,9 @@ public class HiveConf extends Configurat
   private static final Map<String, ConfVars> metaConfs = new HashMap<String, ConfVars>();
   private final List<String> restrictList = new ArrayList<String>();
 
-  private boolean isWhiteListRestrictionEnabled = false;
-  private final List<String> modWhiteList = new ArrayList<String>();
+  private Pattern modWhiteListPattern = null;
   private boolean isSparkConfigUpdated = false;
 
-
   public boolean getSparkConfigUpdated() {
     return isSparkConfigUpdated;
   }
@@ -249,9 +249,9 @@ public class HiveConf extends Configurat
         "The compression codec and other options are determined from Hadoop config variables mapred.output.compress*"),
     COMPRESSINTERMEDIATECODEC("hive.intermediate.compression.codec", "", ""),
     COMPRESSINTERMEDIATETYPE("hive.intermediate.compression.type", "", ""),
-    BYTESPERREDUCER("hive.exec.reducers.bytes.per.reducer", (long) (1000 * 1000 * 1000),
-        "size per reducer.The default is 1G, i.e if the input size is 10G, it will use 10 reducers."),
-    MAXREDUCERS("hive.exec.reducers.max", 999,
+    BYTESPERREDUCER("hive.exec.reducers.bytes.per.reducer", (long) (256 * 1000 * 1000),
+        "size per reducer.The default is 256Mb, i.e if the input size is 1G, it will use 4 reducers."),
+    MAXREDUCERS("hive.exec.reducers.max", 1009,
         "max number of reducers will be used. If the one specified in the configuration parameter mapred.reduce.tasks is\n" +
         "negative, Hive will use this one as the max number of reducers when automatically determine number of reducers."),
     PREEXECHOOKS("hive.exec.pre.hooks", "",
@@ -281,8 +281,9 @@ public class HiveConf extends Configurat
     DYNAMICPARTITIONING("hive.exec.dynamic.partition", true,
         "Whether or not to allow dynamic partitions in DML/DDL."),
     DYNAMICPARTITIONINGMODE("hive.exec.dynamic.partition.mode", "strict",
-        "In strict mode, the user must specify at least one static partition \n" +
-        "in case the user accidentally overwrites all partitions."),
+        "In strict mode, the user must specify at least one static partition\n" +
+        "in case the user accidentally overwrites all partitions.\n" +
+        "In nonstrict mode all partitions are allowed to be dynamic."),
     DYNAMICPARTITIONMAXPARTS("hive.exec.max.dynamic.partitions", 1000,
         "Maximum number of dynamic partitions allowed to be created in total."),
     DYNAMICPARTITIONMAXPARTSPERNODE("hive.exec.max.dynamic.partitions.pernode", 100,
@@ -321,7 +322,7 @@ public class HiveConf extends Configurat
         "When hive.exec.mode.local.auto is true, the number of tasks should less than this for local mode."),
 
     DROPIGNORESNONEXISTENT("hive.exec.drop.ignorenonexistent", true,
-        "Do not report an error if DROP TABLE/VIEW specifies a non-existent table/view"),
+        "Do not report an error if DROP TABLE/VIEW/Index specifies a non-existent table/view/index"),
 
     HIVEIGNOREMAPJOINHINT("hive.ignore.mapjoin.hint", true, "Ignore the mapjoin hint"),
 
@@ -422,13 +423,19 @@ public class HiveConf extends Configurat
         "The delegation token store implementation. Set to org.apache.hadoop.hive.thrift.ZooKeeperTokenStore for load-balanced cluster."),
     METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_ZK_CONNECTSTR(
         "hive.cluster.delegation.token.store.zookeeper.connectString", "",
-        "The ZooKeeper token store connect string."),
+        "The ZooKeeper token store connect string. You can re-use the configuration value\n" +
+        "set in hive.zookeeper.quorum, by leaving this parameter unset."),
     METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_ZK_ZNODE(
-        "hive.cluster.delegation.token.store.zookeeper.znode", "/hive/cluster/delegation",
-        "The root path for token store data."),
+        "hive.cluster.delegation.token.store.zookeeper.znode", "/hivedelegation",
+        "The root path for token store data. Note that this is used by both HiveServer2 and\n" +
+        "MetaStore to store delegation Token. One directory gets created for each of them.\n" +
+        "The final directory names would have the servername appended to it (HIVESERVER2,\n" +
+        "METASTORE)."),
     METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_ZK_ACL(
         "hive.cluster.delegation.token.store.zookeeper.acl", "",
-        "ACL for token store entries. List comma separated all server principals for the cluster."),
+        "ACL for token store entries. Comma separated list of ACL entries. For example:\n" +
+        "sasl:hive/host1@MY.DOMAIN:cdrwa,sasl:hive/host2@MY.DOMAIN:cdrwa\n" +
+        "Defaults to all permissions for the hiveserver2/metastore process user."),
     METASTORE_CACHE_PINOBJTYPES("hive.metastore.cache.pinobjtypes", "Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order",
         "List of comma separated metastore object types that should be pinned in the cache"),
     METASTORE_CONNECTION_POOLING_TYPE("datanucleus.connectionPoolingType", "BONECP",
@@ -501,20 +508,33 @@ public class HiveConf extends Configurat
         "However, it doesn't work correctly with integral values that are not normalized (e.g. have\n" +
         "leading zeroes, like 0012). If metastore direct SQL is enabled and works, this optimization\n" +
         "is also irrelevant."),
-    METASTORE_TRY_DIRECT_SQL("hive.metastore.try.direct.sql", true, ""),
-    METASTORE_TRY_DIRECT_SQL_DDL("hive.metastore.try.direct.sql.ddl", true, ""),
+    METASTORE_TRY_DIRECT_SQL("hive.metastore.try.direct.sql", true,
+        "Whether the Hive metastore should try to use direct SQL queries instead of the\n" +
+        "DataNucleus for certain read paths. This can improve metastore performance when\n" +
+        "fetching many partitions or column statistics by orders of magnitude; however, it\n" +
+        "is not guaranteed to work on all RDBMS-es and all versions. In case of SQL failures,\n" +
+        "the metastore will fall back to the DataNucleus, so it's safe even if SQL doesn't\n" +
+        "work for all queries on your datastore. If all SQL queries fail (for example, your\n" +
+        "metastore is backed by MongoDB), you might want to disable this to save the\n" +
+        "try-and-fall-back cost."),
+    METASTORE_TRY_DIRECT_SQL_DDL("hive.metastore.try.direct.sql.ddl", true,
+        "Same as hive.metastore.try.direct.sql, for read statements within a transaction that\n" +
+        "modifies metastore data. Due to non-standard behavior in Postgres, if a direct SQL\n" +
+        "select query has incorrect syntax or something similar inside a transaction, the\n" +
+        "entire transaction will fail and fall-back to DataNucleus will not be possible. You\n" +
+        "should disable the usage of direct SQL inside transactions if that happens in your case."),
     METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES(
         "hive.metastore.disallow.incompatible.col.type.changes", false,
-        "If true (default is false), ALTER TABLE operations which change the type of \n" +
-        "a column (say STRING) to an incompatible type (say MAP<STRING, STRING>) are disallowed.  \n" +
+        "If true (default is false), ALTER TABLE operations which change the type of a\n" +
+        "column (say STRING) to an incompatible type (say MAP) are disallowed.\n" +
         "RCFile default SerDe (ColumnarSerDe) serializes the values in such a way that the\n" +
         "datatypes can be converted from string to any type. The map is also serialized as\n" +
-        "a string, which can be read as a string as well. However, with any binary \n" +
+        "a string, which can be read as a string as well. However, with any binary\n" +
         "serialization, this is not true. Blocking the ALTER TABLE prevents ClassCastExceptions\n" +
-        "when subsequently trying to access old partitions. \n" +
+        "when subsequently trying to access old partitions.\n" +
         "\n" +
-        "Primitive types like INT, STRING, BIGINT, etc are compatible with each other and are \n" +
-        "not blocked.  \n" +
+        "Primitive types like INT, STRING, BIGINT, etc., are compatible with each other and are\n" +
+        "not blocked.\n" +
         "\n" +
         "See HIVE-4409 for more details."),
 
@@ -587,8 +607,10 @@ public class HiveConf extends Configurat
     HIVEJOBNAMELENGTH("hive.jobname.length", 50, "max jobname length"),
 
     // hive jar
-    HIVEJAR("hive.jar.path", "", ""),
-    HIVEAUXJARS("hive.aux.jars.path", "", ""),
+    HIVEJAR("hive.jar.path", "",
+        "The location of hive_cli.jar that is used when submitting jobs in a separate jvm."),
+    HIVEAUXJARS("hive.aux.jars.path", "",
+        "The location of the plugin jars that contain implementations of user defined functions and serdes."),
 
     // reloadable jars
     HIVERELOADABLEJARS("hive.reloadable.aux.jars.path", "",
@@ -596,9 +618,9 @@ public class HiveConf extends Configurat
             + "used as the auxiliary classes like creating a UDF or SerDe."),
 
     // hive added files and jars
-    HIVEADDEDFILES("hive.added.files.path", "", ""),
-    HIVEADDEDJARS("hive.added.jars.path", "", ""),
-    HIVEADDEDARCHIVES("hive.added.archives.path", "", ""),
+    HIVEADDEDFILES("hive.added.files.path", "", "This an internal parameter."),
+    HIVEADDEDJARS("hive.added.jars.path", "", "This an internal parameter."),
+    HIVEADDEDARCHIVES("hive.added.archives.path", "", "This an internal parameter."),
 
     HIVE_CURRENT_DATABASE("hive.current.database", "", "Database name used by current session. Internal usage only.", true),
 
@@ -607,8 +629,6 @@ public class HiveConf extends Configurat
         new TimeValidator(TimeUnit.SECONDS),
         "How long to run autoprogressor for the script/UDTF operators.\n" +
         "Set to 0 for forever."),
-    HIVETABLENAME("hive.table.name", "", ""),
-    HIVEPARTITIONNAME("hive.partition.name", "", ""),
     HIVESCRIPTAUTOPROGRESS("hive.script.auto.progress", false,
         "Whether Hive Transform/Map/Reduce Clause should automatically send progress information to TaskTracker \n" +
         "to avoid the task getting killed because of inactivity.  Hive sends progress information when the script is \n" +
@@ -619,6 +639,10 @@ public class HiveConf extends Configurat
         "transform function (the custom mapper/reducer that the user has specified in the query)"),
     HIVESCRIPTTRUNCATEENV("hive.script.operator.truncate.env", false,
         "Truncate each environment variable for external script in scripts operator to 20KB (to fit system limits)"),
+    HIVESCRIPT_ENV_BLACKLIST("hive.script.operator.env.blacklist",
+        "hive.txn.valid.txns,hive.script.operator.env.blacklist",
+        "Comma separated list of keys from the configuration file not to convert to environment " +
+        "variables when envoking the script operator"),
     HIVEMAPREDMODE("hive.mapred.mode", "nonstrict",
         "The mode in which the Hive operations are being performed. \n" +
         "In strict mode, some risky queries are not allowed to run. They include:\n" +
@@ -650,7 +674,7 @@ public class HiveConf extends Configurat
         "How many rows in the joining tables (except the streaming table) should be cached in memory."),
 
     // CBO related
-    HIVE_CBO_ENABLED("hive.cbo.enable", false, "Flag to control enabling Cost Based Optimizations using Optiq framework."),
+    HIVE_CBO_ENABLED("hive.cbo.enable", false, "Flag to control enabling Cost Based Optimizations using Calcite framework."),
 
     // hive.mapjoin.bucket.cache.size has been replaced by hive.smbjoin.cache.row,
     // need to remove by hive .13. Also, do not change default (see SMB operator)
@@ -820,10 +844,10 @@ public class HiveConf extends Configurat
         "if hive.merge.mapfiles is true, and for map-reduce jobs if hive.merge.mapredfiles is true."),
     HIVEMERGERCFILEBLOCKLEVEL("hive.merge.rcfile.block.level", true, ""),
     HIVEMERGEORCFILESTRIPELEVEL("hive.merge.orcfile.stripe.level", true,
-        "When hive.merge.mapfiles or hive.merge.mapredfiles is enabled while writing a\n" +
-        " table with ORC file format, enabling this config will do stripe level fast merge\n" +
-        " for small ORC files. Note that enabling this config will not honor padding tolerance\n" +
-        " config (hive.exec.orc.block.padding.tolerance)."),
+        "When hive.merge.mapfiles, hive.merge.mapredfiles or hive.merge.tezfiles is enabled\n" +
+        "while writing a table with ORC file format, enabling this config will do stripe-level\n" +
+        "fast merge for small ORC files. Note that enabling this config will not honor the\n" +
+        "padding tolerance config (hive.exec.orc.block.padding.tolerance)."),
 
     HIVEUSEEXPLICITRCFILEHEADER("hive.exec.rcfile.use.explicit.header", true,
         "If this is set the header for RCFiles will simply be RCF.  If this is not\n" +
@@ -839,28 +863,37 @@ public class HiveConf extends Configurat
     HIVE_ORC_FILE_MEMORY_POOL("hive.exec.orc.memory.pool", 0.5f,
         "Maximum fraction of heap that can be used by ORC file writers"),
     HIVE_ORC_WRITE_FORMAT("hive.exec.orc.write.format", null,
-        "Define the version of the file to write"),
+        "Define the version of the file to write. Possible values are 0.11 and 0.12.\n" +
+        "If this parameter is not defined, ORC will use the run length encoding (RLE)\n" +
+        "introduced in Hive 0.12. Any value other than 0.11 results in the 0.12 encoding."),
     HIVE_ORC_DEFAULT_STRIPE_SIZE("hive.exec.orc.default.stripe.size",
         64L * 1024 * 1024,
-        "Define the default ORC stripe size"),
+        "Define the default ORC stripe size, in bytes."),
     HIVE_ORC_DEFAULT_BLOCK_SIZE("hive.exec.orc.default.block.size", 256L * 1024 * 1024,
         "Define the default file system block size for ORC files."),
 
     HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD("hive.exec.orc.dictionary.key.size.threshold", 0.8f,
         "If the number of keys in a dictionary is greater than this fraction of the total number of\n" +
         "non-null rows, turn off dictionary encoding.  Use 1 to always use dictionary encoding."),
-    HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE("hive.exec.orc.default.row.index.stride", 10000, "Define the default ORC index stride"),
+    HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE("hive.exec.orc.default.row.index.stride", 10000,
+        "Define the default ORC index stride in number of rows. (Stride is the number of rows\n" +
+        "an index entry represents.)"),
     HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK("hive.orc.row.index.stride.dictionary.check", true,
         "If enabled dictionary check will happen after first row index stride (default 10000 rows)\n" +
         "else dictionary check will happen before writing first stripe. In both cases, the decision\n" +
         "to use dictionary or not will be retained thereafter."),
-    HIVE_ORC_DEFAULT_BUFFER_SIZE("hive.exec.orc.default.buffer.size", 256 * 1024, "Define the default ORC buffer size"),
-    HIVE_ORC_DEFAULT_BLOCK_PADDING("hive.exec.orc.default.block.padding", true, "Define the default block padding"),
+    HIVE_ORC_DEFAULT_BUFFER_SIZE("hive.exec.orc.default.buffer.size", 256 * 1024,
+        "Define the default ORC buffer size, in bytes."),
+    HIVE_ORC_DEFAULT_BLOCK_PADDING("hive.exec.orc.default.block.padding", true,
+        "Define the default block padding, which pads stripes to the HDFS block boundaries."),
     HIVE_ORC_BLOCK_PADDING_TOLERANCE("hive.exec.orc.block.padding.tolerance", 0.05f,
-        "Define the tolerance for block padding as a percentage of stripe size.\n" +
-        "For the defaults of 64Mb ORC stripe and 256Mb HDFS blocks, a maximum of 3.2Mb will be reserved for padding within the 256Mb block. \n" +
-        "In that case, if the available size within the block is more than 3.2Mb, a new smaller stripe will be inserted to fit within that space. \n" +
-        "This will make sure that no stripe written will cross block boundaries and cause remote reads within a node local task."),
+        "Define the tolerance for block padding as a decimal fraction of stripe size (for\n" +
+        "example, the default value 0.05 is 5% of the stripe size). For the defaults of 64Mb\n" +
+        "ORC stripe and 256Mb HDFS blocks, the default block padding tolerance of 5% will\n" +
+        "reserve a maximum of 3.2Mb for padding within the 256Mb block. In that case, if the\n" +
+        "available size within the block is more than 3.2Mb, a new smaller stripe will be\n" +
+        "inserted to fit within that space. This will make sure that no stripe written will\n" +
+        "cross block boundaries and cause remote reads within a node local task."),
     HIVE_ORC_DEFAULT_COMPRESS("hive.exec.orc.default.compress", "ZLIB", "Define the default compression codec for ORC file"),
 
     HIVE_ORC_ENCODING_STRATEGY("hive.exec.orc.encoding.strategy", "SPEED", new StringSet("SPEED", "COMPRESSION"),
@@ -883,7 +916,8 @@ public class HiveConf extends Configurat
         "If ORC reader encounters corrupt data, this value will be used to determine\n" +
         "whether to skip the corrupt data or throw exception. The default behavior is to throw exception."),
 
-    HIVE_ORC_ZEROCOPY("hive.exec.orc.zerocopy", false, "Use zerocopy reads with ORC."),
+    HIVE_ORC_ZEROCOPY("hive.exec.orc.zerocopy", false,
+        "Use zerocopy reads with ORC. (This requires Hadoop 2.3 or later.)"),
 
     HIVE_LAZYSIMPLE_EXTENDED_BOOLEAN_LITERAL("hive.lazysimple.extended_boolean_literal", false,
         "LazySimpleSerde uses this property to determine if it treats 'T', 't', 'F', 'f',\n" +
@@ -968,6 +1002,9 @@ public class HiveConf extends Configurat
 
     HIVETEZCONTAINERSIZE("hive.tez.container.size", -1,
         "By default Tez will spawn containers of the size of a mapper. This can be used to overwrite."),
+    HIVETEZCPUVCORES("hive.tez.cpu.vcores", -1,
+        "By default Tez will ask for however many cpus map-reduce is configured to use per container.\n" +
+        "This can be used to overwrite."),
     HIVETEZJAVAOPTS("hive.tez.java.opts", null,
         "By default Tez will use the Java options from map tasks. This can be used to overwrite."),
     HIVETEZLOGLEVEL("hive.tez.log.level", "INFO",
@@ -1117,8 +1154,10 @@ public class HiveConf extends Configurat
     HIVESTATSAUTOGATHER("hive.stats.autogather", true,
         "A flag to gather statistics automatically during the INSERT OVERWRITE command."),
     HIVESTATSDBCLASS("hive.stats.dbclass", "fs", new PatternSet("jdbc(:.*)", "hbase", "counter", "custom", "fs"),
-        "The storage that stores temporary Hive statistics. Currently, jdbc, hbase, counter and custom type are supported."
-    ), // StatsSetupConst.StatDB
+        "The storage that stores temporary Hive statistics. In filesystem based statistics collection ('fs'), \n" +
+        "each task writes statistics it has collected in a file on the filesystem, which will be aggregated \n" +
+        "after the job has finished. Supported values are fs (filesystem), jdbc:database (where database \n" +
+        "can be derby, mysql, etc.), hbase, counter, and custom as defined in StatsSetupConst.java."), // StatsSetupConst.StatDB
     HIVESTATSJDBCDRIVER("hive.stats.jdbcdriver",
         "org.apache.derby.jdbc.EmbeddedDriver",
         "The JDBC driver for the database that stores temporary Hive statistics."),
@@ -1251,13 +1290,16 @@ public class HiveConf extends Configurat
 
      // Zookeeper related configs
     HIVE_ZOOKEEPER_QUORUM("hive.zookeeper.quorum", "",
-        "List of ZooKeeper servers to talk to. This is needed for: " +
-        "1. Read/write locks - when hive.lock.manager is set to " +
-        "org.apache.hadoop.hive.ql.lockmgr.zookeeper.ZooKeeperHiveLockManager, " +
-        "2. When HiveServer2 supports service discovery via Zookeeper."),
+        "List of ZooKeeper servers to talk to. This is needed for: \n" +
+        "1. Read/write locks - when hive.lock.manager is set to \n" +
+        "org.apache.hadoop.hive.ql.lockmgr.zookeeper.ZooKeeperHiveLockManager, \n" +
+        "2. When HiveServer2 supports service discovery via Zookeeper.\n" +
+        "3. For delegation token storage if zookeeper store is used, if\n" +
+        "hive.cluster.delegation.token.store.zookeeper.connectString is not set"),
+
     HIVE_ZOOKEEPER_CLIENT_PORT("hive.zookeeper.client.port", "2181",
-        "The port of ZooKeeper servers to talk to. " +
-        "If the list of Zookeeper servers specified in hive.zookeeper.quorum," +
+        "The port of ZooKeeper servers to talk to.\n" +
+        "If the list of Zookeeper servers specified in hive.zookeeper.quorum\n" +
         "does not contain port numbers, this value is used."),
     HIVE_ZOOKEEPER_SESSION_TIMEOUT("hive.zookeeper.session.timeout", 600*1000,
         "ZooKeeper client's session timeout. The client is disconnected, and as a result, all locks released, \n" +
@@ -1269,42 +1311,72 @@ public class HiveConf extends Configurat
 
     // Transactions
     HIVE_TXN_MANAGER("hive.txn.manager",
-        "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager", ""),
+        "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager",
+        "Set to org.apache.hadoop.hive.ql.lockmgr.DbTxnManager as part of turning on Hive\n" +
+        "transactions, which also requires appropriate settings for hive.compactor.initiator.on,\n" +
+        "hive.compactor.worker.threads, hive.support.concurrency (true), hive.enforce.bucketing\n" +
+        "(true), and hive.exec.dynamic.partition.mode (nonstrict).\n" +
+        "The default DummyTxnManager replicates pre-Hive-0.13 behavior and provides\n" +
+        "no transactions."),
     HIVE_TXN_TIMEOUT("hive.txn.timeout", "300s", new TimeValidator(TimeUnit.SECONDS),
         "time after which transactions are declared aborted if the client has not sent a heartbeat."),
 
     HIVE_TXN_MAX_OPEN_BATCH("hive.txn.max.open.batch", 1000,
         "Maximum number of transactions that can be fetched in one call to open_txns().\n" +
-        "Increasing this will decrease the number of delta files created when\n" +
-        "streaming data into Hive.  But it will also increase the number of\n" +
-        "open transactions at any given time, possibly impacting read performance."),
+        "This controls how many transactions streaming agents such as Flume or Storm open\n" +
+        "simultaneously. The streaming agent then writes that number of entries into a single\n" +
+        "file (per Flume agent or Storm bolt). Thus increasing this value decreases the number\n" +
+        "of delta files created by streaming agents. But it also increases the number of open\n" +
+        "transactions that Hive has to track at any given time, which may negatively affect\n" +
+        "read performance."),
 
     HIVE_COMPACTOR_INITIATOR_ON("hive.compactor.initiator.on", false,
-        "Whether to run the compactor's initiator thread in this metastore instance or not."),
+        "Whether to run the initiator and cleaner threads on this metastore instance or not.\n" +
+        "Set this to true on one instance of the Thrift metastore service as part of turning\n" +
+        "on Hive transactions. For a complete list of parameters required for turning on\n" +
+        "transactions, see hive.txn.manager."),
 
     HIVE_COMPACTOR_WORKER_THREADS("hive.compactor.worker.threads", 0,
-        "Number of compactor worker threads to run on this metastore instance."),
+        "How many compactor worker threads to run on this metastore instance. Set this to a\n" +
+        "positive number on one or more instances of the Thrift metastore service as part of\n" +
+        "turning on Hive transactions. For a complete list of parameters required for turning\n" +
+        "on transactions, see hive.txn.manager.\n" +
+        "Worker threads spawn MapReduce jobs to do compactions. They do not do the compactions\n" +
+        "themselves. Increasing the number of worker threads will decrease the time it takes\n" +
+        "tables or partitions to be compacted once they are determined to need compaction.\n" +
+        "It will also increase the background load on the Hadoop cluster as more MapReduce jobs\n" +
+        "will be running in the background."),
 
     HIVE_COMPACTOR_WORKER_TIMEOUT("hive.compactor.worker.timeout", "86400s",
         new TimeValidator(TimeUnit.SECONDS),
-        "Time before a given compaction in working state is declared a failure\n" +
-        "and returned to the initiated state."),
+        "Time in seconds after which a compaction job will be declared failed and the\n" +
+        "compaction re-queued."),
 
     HIVE_COMPACTOR_CHECK_INTERVAL("hive.compactor.check.interval", "300s",
         new TimeValidator(TimeUnit.SECONDS),
-        "Time between checks to see if any partitions need compacted.\n" +
-        "This should be kept high because each check for compaction requires many calls against the NameNode."),
+        "Time in seconds between checks to see if any tables or partitions need to be\n" +
+        "compacted. This should be kept high because each check for compaction requires\n" +
+        "many calls against the NameNode.\n" +
+        "Decreasing this value will reduce the time it takes for compaction to be started\n" +
+        "for a table or partition that requires compaction. However, checking if compaction\n" +
+        "is needed requires several calls to the NameNode for each table or partition that\n" +
+        "has had a transaction done on it since the last major compaction. So decreasing this\n" +
+        "value will increase the load on the NameNode."),
 
     HIVE_COMPACTOR_DELTA_NUM_THRESHOLD("hive.compactor.delta.num.threshold", 10,
-        "Number of delta files that must exist in a directory before the compactor will attempt\n" +
-        "a minor compaction."),
+        "Number of delta directories in a table or partition that will trigger a minor\n" +
+        "compaction."),
 
     HIVE_COMPACTOR_DELTA_PCT_THRESHOLD("hive.compactor.delta.pct.threshold", 0.1f,
-        "Percentage (by size) of base that deltas can be before major compaction is initiated."),
+        "Percentage (fractional) size of the delta files relative to the base that will trigger\n" +
+        "a major compaction. (1.0 = 100%, so the default 0.1 = 10%.)"),
 
     HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD("hive.compactor.abortedtxn.threshold", 1000,
-        "Number of aborted transactions involving a particular table or partition before major\n" +
-        "compaction is initiated."),
+        "Number of aborted transactions involving a given table or partition that will trigger\n" +
+        "a major compaction."),
+
+    HIVE_COMPACTOR_CLEANER_RUN_INTERVAL("hive.compactor.cleaner.run.interval", "5000ms",
+        new TimeValidator(TimeUnit.MILLISECONDS), "Time between runs of the cleaner thread"),
 
     // For HBase storage handler
     HIVE_HBASE_WAL_ENABLED("hive.hbase.wal.enabled", true,
@@ -1324,10 +1396,11 @@ public class HiveConf extends Configurat
 
     HIVEOUTERJOINSUPPORTSFILTERS("hive.outerjoin.supports.filters", true, ""),
 
-    HIVEFETCHTASKCONVERSION("hive.fetch.task.conversion", "more", new StringSet("minimal", "more"),
+    HIVEFETCHTASKCONVERSION("hive.fetch.task.conversion", "more", new StringSet("none", "minimal", "more"),
         "Some select queries can be converted to single FETCH task minimizing latency.\n" +
         "Currently the query should be single sourced not having any subquery and should not have\n" +
         "any aggregations or distincts (which incurs RS), lateral views and joins.\n" +
+        "0. none : disable hive.fetch.task.conversion\n" +
         "1. minimal : SELECT STAR, FILTER on partition columns, LIMIT only\n" +
         "2. more    : SELECT, FILTER, LIMIT only (support TABLESAMPLE and virtual columns)"
     ),
@@ -1351,7 +1424,12 @@ public class HiveConf extends Configurat
         "The SerDe used by FetchTask to serialize the fetch output."),
 
     HIVEEXPREVALUATIONCACHE("hive.cache.expr.evaluation", true,
-        "If true, evaluation result of deterministic expression referenced twice or more will be cached."),
+        "If true, the evaluation result of a deterministic expression referenced twice or more\n" +
+        "will be cached.\n" +
+        "For example, in a filter condition like '.. where key + 10 = 100 or key + 10 = 0'\n" +
+        "the expression 'key + 10' will be evaluated/cached once and reused for the following\n" +
+        "expression ('key + 10 = 0'). Currently, this is applied only to expressions in select\n" +
+        "or filter operators."),
 
     // Hive Variables
     HIVEVARIABLESUBSTITUTE("hive.variable.substitute", true,
@@ -1377,9 +1455,11 @@ public class HiveConf extends Configurat
         "interface org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider."),
     HIVE_METASTORE_AUTHORIZATION_MANAGER("hive.security.metastore.authorization.manager",
         "org.apache.hadoop.hive.ql.security.authorization.DefaultHiveMetastoreAuthorizationProvider",
-        "authorization manager class name to be used in the metastore for authorization.\n" +
-        "The user defined authorization class should implement interface \n" +
-        "org.apache.hadoop.hive.ql.security.authorization.HiveMetastoreAuthorizationProvider. "),
+        "Names of authorization manager classes (comma separated) to be used in the metastore\n" +
+        "for authorization. The user defined authorization class should implement interface\n" +
+        "org.apache.hadoop.hive.ql.security.authorization.HiveMetastoreAuthorizationProvider.\n" +
+        "All authorization manager classes have to successfully authorize the metastore API\n" +
+        "call for the command execution to be allowed."),
     HIVE_METASTORE_AUTHORIZATION_AUTH_READS("hive.security.metastore.authorization.auth.reads", true,
         "If this is true, metastore authorizer authorizes read actions on database, table"),
     HIVE_METASTORE_AUTHENTICATOR_MANAGER("hive.security.metastore.authenticator.manager",
@@ -1399,15 +1479,30 @@ public class HiveConf extends Configurat
         "the privileges automatically granted to some roles whenever a table gets created.\n" +
         "An example like \"roleX,roleY:select;roleZ:create\" will grant select privilege to roleX and roleY,\n" +
         "and grant create privilege to roleZ whenever a new table created."),
-    HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS("hive.security.authorization.createtable.owner.grants", "",
-        "the privileges automatically granted to the owner whenever a table gets created.\n" +
-        "An example like \"select,drop\" will grant select and drop privilege to the owner of the table"),
+    HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS("hive.security.authorization.createtable.owner.grants",
+        "",
+        "The privileges automatically granted to the owner whenever a table gets created.\n" +
+        "An example like \"select,drop\" will grant select and drop privilege to the owner\n" +
+        "of the table. Note that the default gives the creator of a table no access to the\n" +
+        "table (but see HIVE-8067)."),
 
-    // if this is not set default value is added by sql standard authorizer.
+    // if this is not set default value is set during config initialization
     // Default value can't be set in this constructor as it would refer names in other ConfVars
     // whose constructor would not have been called
-    HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST("hive.security.authorization.sqlstd.confwhitelist", "",
-        "interal variable. List of modifiable configurations by user."),
+    HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST(
+        "hive.security.authorization.sqlstd.confwhitelist", "",
+        "List of comma separated Java regexes. Configurations parameters that match these\n" +
+        "regexes can be modified by user when SQL standard authorization is enabled.\n" +
+        "To get the default value, use the 'set <param>' command.\n" +
+        "Note that the hive.conf.restricted.list checks are still enforced after the white list\n" +
+        "check"),
+
+    HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST_APPEND(
+        "hive.security.authorization.sqlstd.confwhitelist.append", "",
+        "List of comma separated Java regexes, to be appended to list set in\n" +
+        "hive.security.authorization.sqlstd.confwhitelist. Using this list instead\n" +
+        "of updating the original list means that you can append to the defaults\n" +
+        "set by SQL standard authorization instead of replacing it entirely."),
 
     HIVE_CLI_PRINT_HEADER("hive.cli.print.header", false, "Whether to print the names of the columns in query output."),
 
@@ -1460,6 +1555,10 @@ public class HiveConf extends Configurat
         "If the property is set, the value must be a valid URI (java.net.URI, e.g. \"file:///tmp/my-logging.properties\"), \n" +
         "which you can then extract a URL from and pass to PropertyConfigurator.configure(URL)."),
 
+    HIVE_LOG_EXPLAIN_OUTPUT("hive.log.explain.output", false,
+        "Whether to log explain output for every query.\n" +
+        "When enabled, will log EXPLAIN EXTENDED output for the query at INFO log4j log level."),
+
     // prefix used to auto generated column aliases (this should be started with '_')
     HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL("hive.autogen.columnalias.prefix.label", "_c",
         "String used as a prefix when auto generating column alias.\n" +
@@ -1538,12 +1637,13 @@ public class HiveConf extends Configurat
     // binary transport settings
     HIVE_SERVER2_THRIFT_PORT("hive.server2.thrift.port", 10000,
         "Port number of HiveServer2 Thrift interface when hive.server2.transport.mode is 'binary'."),
-    // hadoop.rpc.protection being set to a higher level than HiveServer2
-    // does not make sense in most situations.
-    // HiveServer2 ignores hadoop.rpc.protection in favor of hive.server2.thrift.sasl.qop.
-    HIVE_SERVER2_THRIFT_SASL_QOP("hive.server2.thrift.sasl.qop", "auth", new StringSet("auth", "auth-int", "auth-conf"),
-        "Sasl QOP value; Set it to one of following values to enable higher levels of\n" +
-        " protection for HiveServer2 communication with clients.\n" +
+    HIVE_SERVER2_THRIFT_SASL_QOP("hive.server2.thrift.sasl.qop", "auth",
+        new StringSet("auth", "auth-int", "auth-conf"),
+        "Sasl QOP value; set it to one of following values to enable higher levels of\n" +
+        "protection for HiveServer2 communication with clients.\n" +
+        "Setting hadoop.rpc.protection to a higher level than HiveServer2 does not\n" +
+        "make sense in most situations. HiveServer2 ignores hadoop.rpc.protection in favor\n" +
+        "of hive.server2.thrift.sasl.qop.\n" +
         "  \"auth\" - authentication only (default)\n" +
         "  \"auth-int\" - authentication plus integrity protection\n" +
         "  \"auth-conf\" - authentication plus integrity and confidentiality protection\n" +
@@ -1561,7 +1661,7 @@ public class HiveConf extends Configurat
         "Number of threads in the async thread pool for HiveServer2"),
     HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT("hive.server2.async.exec.shutdown.timeout", "10s",
         new TimeValidator(TimeUnit.SECONDS),
-        "Maximum time for which HiveServer2 shutdown will wait for async"),
+        "How long HiveServer2 shutdown will wait for async threads to terminate."),
     HIVE_SERVER2_ASYNC_EXEC_WAIT_QUEUE_SIZE("hive.server2.async.exec.wait.queue.size", 100,
         "Size of the wait queue for async thread pool in HiveServer2.\n" +
         "After hitting this limit, the async thread pool will reject new requests."),
@@ -1581,7 +1681,9 @@ public class HiveConf extends Configurat
         "  LDAP: LDAP/AD based authentication\n" +
         "  KERBEROS: Kerberos/GSSAPI authentication\n" +
         "  CUSTOM: Custom authentication provider\n" +
-        "          (Use with property hive.server2.custom.authentication.class)"),
+        "          (Use with property hive.server2.custom.authentication.class)\n" +
+        "  PAM: Pluggable authentication module\n" +
+        "  NOSASL:  Raw transport"),
     HIVE_SERVER2_ALLOW_USER_SUBSTITUTION("hive.server2.allow.user.substitution", true,
         "Allow alternate user to be specified as part of HiveServer2 open connection request."),
     HIVE_SERVER2_KERBEROS_KEYTAB("hive.server2.authentication.kerberos.keytab", "",
@@ -1629,9 +1731,18 @@ public class HiveConf extends Configurat
         "  HIVE : Exposes Hive's native table types like MANAGED_TABLE, EXTERNAL_TABLE, VIRTUAL_VIEW\n" +
         "  CLASSIC : More generic types like TABLE and VIEW"),
     HIVE_SERVER2_SESSION_HOOK("hive.server2.session.hook", "", ""),
-    HIVE_SERVER2_USE_SSL("hive.server2.use.SSL", false, ""),
-    HIVE_SERVER2_SSL_KEYSTORE_PATH("hive.server2.keystore.path", "", ""),
-    HIVE_SERVER2_SSL_KEYSTORE_PASSWORD("hive.server2.keystore.password", "", ""),
+    HIVE_SERVER2_USE_SSL("hive.server2.use.SSL", false,
+        "Set this to true for using SSL encryption in HiveServer2."),
+    HIVE_SERVER2_SSL_KEYSTORE_PATH("hive.server2.keystore.path", "",
+        "SSL certificate keystore location."),
+    HIVE_SERVER2_SSL_KEYSTORE_PASSWORD("hive.server2.keystore.password", "",
+        "SSL certificate keystore password."),
+    HIVE_SERVER2_MAP_FAIR_SCHEDULER_QUEUE("hive.server2.map.fair.scheduler.queue", true,
+        "If the YARN fair scheduler is configured and HiveServer2 is running in non-impersonation mode,\n" +
+        "this setting determines the user for fair scheduler queue mapping.\n" +
+        "If set to true (default), the logged-in user determines the fair scheduler queue\n" +
+        "for submitted jobs, so that map reduce resource usage can be tracked by user.\n" +
+        "If set to false, all Hive jobs go to the 'hive' user's queue."),
 
     HIVE_SECURITY_COMMAND_WHITELIST("hive.security.command.whitelist", "set,reset,dfs,add,list,delete,reload,compile",
         "Comma separated list of non-SQL Hive commands users are authorized to execute"),
@@ -1731,8 +1842,10 @@ public class HiveConf extends Configurat
     HIVE_VECTORIZATION_GROUPBY_FLUSH_PERCENT("hive.vectorized.groupby.flush.percent", (float) 0.1,
         "Percent of entries in the group by aggregation hash flushed when the memory threshold is exceeded."),
 
-
     HIVE_TYPE_CHECK_ON_INSERT("hive.typecheck.on.insert", true, ""),
+    HIVE_HADOOP_CLASSPATH("hive.hadoop.classpath", null,
+        "For Windows OS, we need to pass HIVE_HADOOP_CLASSPATH Java parameter while starting HiveServer2 \n" +
+        "using \"-hiveconf hive.hadoop.classpath=%HIVE_LIB%\"."),
 
     HIVE_RPC_QUERY_PLAN("hive.rpc.query.plan", false,
         "Whether to send the query plan via local resource or RPC"),
@@ -1799,12 +1912,17 @@ public class HiveConf extends Configurat
         "of reducers that tez specifies."),
     TEZ_DYNAMIC_PARTITION_PRUNING(
         "hive.tez.dynamic.partition.pruning", true,
-        "When dynamic pruning is enabled, joins on partition keys will be processed by sending events from the processing " +
-        "vertices to the tez application master. These events will be used to prune unnecessary partitions."),
+        "When dynamic pruning is enabled, joins on partition keys will be processed by sending\n" +
+        "events from the processing vertices to the Tez application master. These events will be\n" +
+        "used to prune unnecessary partitions."),
     TEZ_DYNAMIC_PARTITION_PRUNING_MAX_EVENT_SIZE("hive.tez.dynamic.partition.pruning.max.event.size", 1*1024*1024L,
         "Maximum size of events sent by processors in dynamic pruning. If this size is crossed no pruning will take place."),
     TEZ_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE("hive.tez.dynamic.partition.pruning.max.data.size", 100*1024*1024L,
-        "Maximum total data size of events in dynamic pruning.")
+        "Maximum total data size of events in dynamic pruning."),
+    TEZ_SMB_NUMBER_WAVES(
+        "hive.tez.smb.number.waves",
+        (float) 0.5,
+        "The number of waves in which to run the SMB join. Account for cluster being occupied. Ideally should be 1 wave.")
     ;
 
     public final String varname;
@@ -2030,8 +2148,9 @@ public class HiveConf extends Configurat
   }
 
   public void verifyAndSet(String name, String value) throws IllegalArgumentException {
-    if (isWhiteListRestrictionEnabled) {
-      if (!modWhiteList.contains(name)) {
+    if (modWhiteListPattern != null) {
+      Matcher wlMatcher = modWhiteListPattern.matcher(name);
+      if (!wlMatcher.matches()) {
         throw new IllegalArgumentException("Cannot modify " + name + " at runtime. "
             + "It is not in list of params that are allowed to be modified at runtime");
       }
@@ -2102,7 +2221,7 @@ public class HiveConf extends Configurat
 
   public static TimeUnit unitFor(String unit, TimeUnit defaultUnit) {
     unit = unit.trim().toLowerCase();
-    if (unit.isEmpty()) {
+    if (unit.isEmpty() || unit.equals("l")) {
       if (defaultUnit == null) {
         throw new IllegalArgumentException("Time unit is not specified");
       }
@@ -2369,11 +2488,146 @@ public class HiveConf extends Configurat
         unset(key);
       }
     }
+
+    setupSQLStdAuthWhiteList();
+
     // setup list of conf vars that are not allowed to change runtime
     setupRestrictList();
+
   }
 
   /**
+   * If the config whitelist param for sql standard authorization is not set, set it up here.
+   */
+  private void setupSQLStdAuthWhiteList() {
+    String whiteListParamsStr = getVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST);
+    if (whiteListParamsStr == null || whiteListParamsStr.trim().isEmpty()) {
+      // set the default configs in whitelist
+      whiteListParamsStr = getSQLStdAuthDefaultWhiteListPattern();
+    }
+    setVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST, whiteListParamsStr);
+  }
+
+  private static String getSQLStdAuthDefaultWhiteListPattern() {
+    // create the default white list from list of safe config params
+    // and regex list
+    String confVarPatternStr = Joiner.on("|").join(convertVarsToRegex(sqlStdAuthSafeVarNames));
+    String regexPatternStr = Joiner.on("|").join(sqlStdAuthSafeVarNameRegexes);
+    return regexPatternStr + "|" + confVarPatternStr;
+  }
+
+  /**
+   * @param paramList  list of parameter strings
+   * @return list of parameter strings with "." replaced by "\."
+   */
+  private static String[] convertVarsToRegex(String[] paramList) {
+    String[] regexes = new String[paramList.length];
+    for(int i=0; i<paramList.length; i++) {
+      regexes[i] = paramList[i].replace(".", "\\." );
+    }
+    return regexes;
+  }
+
+  /**
+   * Default list of modifiable config parameters for sql standard authorization
+   * For internal use only.
+   */
+  private static final String [] sqlStdAuthSafeVarNames = new String [] {
+    ConfVars.BYTESPERREDUCER.varname,
+    ConfVars.CLIENT_STATS_COUNTERS.varname,
+    ConfVars.DEFAULTPARTITIONNAME.varname,
+    ConfVars.DROPIGNORESNONEXISTENT.varname,
+    ConfVars.HIVECOUNTERGROUP.varname,
+    ConfVars.HIVEENFORCEBUCKETING.varname,
+    ConfVars.HIVEENFORCEBUCKETMAPJOIN.varname,
+    ConfVars.HIVEENFORCESORTING.varname,
+    ConfVars.HIVEENFORCESORTMERGEBUCKETMAPJOIN.varname,
+    ConfVars.HIVEEXPREVALUATIONCACHE.varname,
+    ConfVars.HIVEGROUPBYSKEW.varname,
+    ConfVars.HIVEHASHTABLELOADFACTOR.varname,
+    ConfVars.HIVEHASHTABLETHRESHOLD.varname,
+    ConfVars.HIVEIGNOREMAPJOINHINT.varname,
+    ConfVars.HIVELIMITMAXROWSIZE.varname,
+    ConfVars.HIVEMAPREDMODE.varname,
+    ConfVars.HIVEMAPSIDEAGGREGATE.varname,
+    ConfVars.HIVEOPTIMIZEMETADATAQUERIES.varname,
+    ConfVars.HIVEROWOFFSET.varname,
+    ConfVars.HIVEVARIABLESUBSTITUTE.varname,
+    ConfVars.HIVEVARIABLESUBSTITUTEDEPTH.varname,
+    ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_INCLUDEFUNCNAME.varname,
+    ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL.varname,
+    ConfVars.HIVE_CHECK_CROSS_PRODUCT.varname,
+    ConfVars.HIVE_COMPAT.varname,
+    ConfVars.HIVE_CONCATENATE_CHECK_INDEX.varname,
+    ConfVars.HIVE_DISPLAY_PARTITION_COLUMNS_SEPARATELY.varname,
+    ConfVars.HIVE_ERROR_ON_EMPTY_PARTITION.varname,
+    ConfVars.HIVE_EXECUTION_ENGINE.varname,
+    ConfVars.HIVE_EXIM_URI_SCHEME_WL.varname,
+    ConfVars.HIVE_FILE_MAX_FOOTER.varname,
+    ConfVars.HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES.varname,
+    ConfVars.HIVE_INSERT_INTO_MULTILEVEL_DIRS.varname,
+    ConfVars.HIVE_LOCALIZE_RESOURCE_NUM_WAIT_ATTEMPTS.varname,
+    ConfVars.HIVE_MULTI_INSERT_MOVE_TASKS_SHARE_DEPENDENCIES.varname,
+    ConfVars.HIVE_QUOTEDID_SUPPORT.varname,
+    ConfVars.HIVE_RESULTSET_USE_UNIQUE_COLUMN_NAMES.varname,
+    ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS.varname,
+    ConfVars.JOB_DEBUG_CAPTURE_STACKTRACES.varname,
+    ConfVars.JOB_DEBUG_TIMEOUT.varname,
+    ConfVars.MAXCREATEDFILES.varname,
+    ConfVars.MAXREDUCERS.varname,
+    ConfVars.OUTPUT_FILE_EXTENSION.varname,
+    ConfVars.SHOW_JOB_FAIL_DEBUG_INFO.varname,
+    ConfVars.TASKLOG_DEBUG_TIMEOUT.varname,
+  };
+
+  /**
+   * Default list of regexes for config parameters that are modifiable with
+   * sql standard authorization enabled
+   */
+  static final String [] sqlStdAuthSafeVarNameRegexes = new String [] {
+    "hive\\.auto\\..*",
+    "hive\\.cbo\\..*",
+    "hive\\.convert\\..*",
+    "hive\\.exec\\..*\\.dynamic\\.partitions\\..*",
+    "hive\\.exec\\.compress\\..*",
+    "hive\\.exec\\.infer\\..*",
+    "hive\\.exec\\.mode.local\\..*",
+    "hive\\.exec\\.orc\\..*",
+    "hive\\.fetch.task\\..*",
+    "hive\\.hbase\\..*",
+    "hive\\.index\\..*",
+    "hive\\.index\\..*",
+    "hive\\.intermediate\\..*",
+    "hive\\.join\\..*",
+    "hive\\.limit\\..*",
+    "hive\\.mapjoin\\..*",
+    "hive\\.merge\\..*",
+    "hive\\.optimize\\..*",
+    "hive\\.orc\\..*",
+    "hive\\.outerjoin\\..*",
+    "hive\\.ppd\\..*",
+    "hive\\.prewarm\\..*",
+    "hive\\.skewjoin\\..*",
+    "hive\\.smbjoin\\..*",
+    "hive\\.stats\\..*",
+    "hive\\.tez\\..*",
+    "hive\\.vectorized\\..*",
+    "mapred\\.map\\..*",
+    "mapred\\.reduce\\..*",
+    "mapred\\.output\\.compression\\.codec",
+    "mapreduce\\.job\\.reduce\\.slowstart\\.completedmaps",
+    "mapreduce\\.job\\.queuename",
+    "mapreduce\\.input\\.fileinputformat\\.split\\.minsize",
+    "mapreduce\\.map\\..*",
+    "mapreduce\\.reduce\\..*",
+    "tez\\.am\\..*",
+    "tez\\.task\\..*",
+    "tez\\.runtime\\..*",
+  };
+
+
+
+  /**
    * Apply system properties to this object if the property name is defined in ConfVars
    * and the value is non-null and not an empty string.
    */
@@ -2515,26 +2769,16 @@ public class HiveConf extends Configurat
   }
 
   /**
-   * Set if whitelist check is enabled for parameter modification
-   *
-   * @param isEnabled
-   */
-  @LimitedPrivate(value = { "Currently only for use by HiveAuthorizer" })
-  public void setIsModWhiteListEnabled(boolean isEnabled) {
-    this.isWhiteListRestrictionEnabled = isEnabled;
-  }
-
-  /**
-   * Add config parameter name to whitelist of parameters that can be modified
+   * Set white list of parameters that are allowed to be modified
    *
-   * @param paramname
+   * @param paramNameRegex
    */
   @LimitedPrivate(value = { "Currently only for use by HiveAuthorizer" })
-  public void addToModifiableWhiteList(String paramname) {
-    if (paramname == null) {
+  public void setModifiableWhiteListRegex(String paramNameRegex) {
+    if (paramNameRegex == null) {
       return;
     }
-    modWhiteList.add(paramname);
+    modWhiteListPattern = Pattern.compile(paramNameRegex);
   }
 
   /**

Copied: hive/branches/spark/common/src/test/org/apache/hadoop/hive/common/TestValidTxnImpl.java (from r1633910, hive/trunk/common/src/test/org/apache/hadoop/hive/common/TestValidTxnImpl.java)
URL: http://svn.apache.org/viewvc/hive/branches/spark/common/src/test/org/apache/hadoop/hive/common/TestValidTxnImpl.java?p2=hive/branches/spark/common/src/test/org/apache/hadoop/hive/common/TestValidTxnImpl.java&p1=hive/trunk/common/src/test/org/apache/hadoop/hive/common/TestValidTxnImpl.java&r1=1633910&r2=1635536&rev=1635536&view=diff
==============================================================================
--- hive/trunk/common/src/test/org/apache/hadoop/hive/common/TestValidTxnImpl.java (original)
+++ hive/branches/spark/common/src/test/org/apache/hadoop/hive/common/TestValidTxnImpl.java Thu Oct 30 16:22:33 2014
@@ -64,7 +64,6 @@ public class TestValidTxnImpl {
     for (int i = 0; i < 1000; i++) exceptions[i] = i + 100;
     ValidTxnList txnList = new ValidTxnListImpl(exceptions, 2000);
     String str = txnList.writeToString();
-    Assert.assertEquals('C', str.charAt(0));
     ValidTxnList newList = new ValidTxnListImpl();
     newList.readFromString(str);
     for (int i = 0; i < 100; i++) Assert.assertTrue(newList.isTxnCommitted(i));
@@ -79,7 +78,6 @@ public class TestValidTxnImpl {
     for (int i = 0; i < 1000; i++) exceptions[i] = i + 100;
     ValidTxnList txnList = new ValidTxnListImpl(exceptions, 2000);
     String str = txnList.writeToString();
-    Assert.assertEquals('C', str.charAt(0));
     Configuration conf = new Configuration();
     conf.set(ValidTxnList.VALID_TXNS_KEY, str);
     File tmpFile = File.createTempFile("TestValidTxnImpl", "readWriteConfig");

Modified: hive/branches/spark/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java (original)
+++ hive/branches/spark/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java Thu Oct 30 16:22:33 2014
@@ -24,6 +24,8 @@ import org.apache.hive.common.util.HiveT
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.concurrent.TimeUnit;
+
 
 /**
  * TestHiveConf
@@ -95,4 +97,24 @@ public class TestHiveConf {
       Assert.assertTrue(i == HiveConf.getPositionFromInternalName(HiveConf.getColumnInternalName(i)));
     }
   }
+
+  @Test
+  public void testUnitFor() throws Exception {
+    Assert.assertEquals(TimeUnit.SECONDS, HiveConf.unitFor("L", TimeUnit.SECONDS));
+    Assert.assertEquals(TimeUnit.MICROSECONDS, HiveConf.unitFor("", TimeUnit.MICROSECONDS));
+    Assert.assertEquals(TimeUnit.DAYS, HiveConf.unitFor("d", null));
+    Assert.assertEquals(TimeUnit.DAYS, HiveConf.unitFor("days", null));
+    Assert.assertEquals(TimeUnit.HOURS, HiveConf.unitFor("h", null));
+    Assert.assertEquals(TimeUnit.HOURS, HiveConf.unitFor("hours", null));
+    Assert.assertEquals(TimeUnit.MINUTES, HiveConf.unitFor("m", null));
+    Assert.assertEquals(TimeUnit.MINUTES, HiveConf.unitFor("minutes", null));
+    Assert.assertEquals(TimeUnit.SECONDS, HiveConf.unitFor("s", null));
+    Assert.assertEquals(TimeUnit.SECONDS, HiveConf.unitFor("seconds", null));
+    Assert.assertEquals(TimeUnit.MILLISECONDS, HiveConf.unitFor("ms", null));
+    Assert.assertEquals(TimeUnit.MILLISECONDS, HiveConf.unitFor("msecs", null));
+    Assert.assertEquals(TimeUnit.MICROSECONDS, HiveConf.unitFor("us", null));
+    Assert.assertEquals(TimeUnit.MICROSECONDS, HiveConf.unitFor("useconds", null));
+    Assert.assertEquals(TimeUnit.NANOSECONDS, HiveConf.unitFor("ns", null));
+    Assert.assertEquals(TimeUnit.NANOSECONDS, HiveConf.unitFor("nsecs", null));
+  }
 }

Modified: hive/branches/spark/contrib/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/spark/contrib/pom.xml?rev=1635536&r1=1635535&r2=1635536&view=diff
==============================================================================
--- hive/branches/spark/contrib/pom.xml (original)
+++ hive/branches/spark/contrib/pom.xml Thu Oct 30 16:22:33 2014
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hive</groupId>
     <artifactId>hive</artifactId>
-    <version>0.14.0-SNAPSHOT</version>
+    <version>0.15.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>