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/14 23:36:33 UTC

svn commit: r1631893 - in /hive/trunk: beeline/src/java/org/apache/hive/beeline/ itests/hive-unit/src/test/java/org/apache/hive/beeline/ metastore/scripts/upgrade/postgres/ metastore/src/java/org/apache/hadoop/hive/metastore/

Author: brock
Date: Tue Oct 14 21:36:32 2014
New Revision: 1631893

URL: http://svn.apache.org/r1631893
Log:
HIVE-8374 - schematool fails on Postgres versions < 9.2 (Mohit Sabharwal via Brock)

Added:
    hive/trunk/metastore/scripts/upgrade/postgres/pre-0-upgrade-0.12.0-to-0.13.0.postgres.sql
Modified:
    hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
    hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java

Modified: hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
URL: http://svn.apache.org/viewvc/hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java?rev=1631893&r1=1631892&r2=1631893&view=diff
==============================================================================
--- hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java (original)
+++ hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java Tue Oct 14 21:36:32 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/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
URL: http://svn.apache.org/viewvc/hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java?rev=1631893&r1=1631892&r2=1631893&view=diff
==============================================================================
--- hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java (original)
+++ hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java Tue Oct 14 21:36:32 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/trunk/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java?rev=1631893&r1=1631892&r2=1631893&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java Tue Oct 14 21:36:32 2014
@@ -19,9 +19,12 @@
 package org.apache.hive.beeline;
 
 import java.io.BufferedWriter;
+import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
 import java.util.Random;
 
 import junit.framework.TestCase;
@@ -31,14 +34,14 @@ import org.apache.commons.lang.StringUti
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaException;
 import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfo;
-import org.apache.hive.beeline.HiveSchemaHelper;
 import org.apache.hive.beeline.HiveSchemaHelper.NestedScriptParser;
-import org.apache.hive.beeline.HiveSchemaTool;
 
 public class TestSchemaTool extends TestCase {
   private HiveSchemaTool schemaTool;
   private HiveConf hiveConf;
   private String testMetastoreDB;
+  private PrintStream errStream;
+  private PrintStream outStream;
 
   @Override
   protected void setUp() throws Exception {
@@ -48,8 +51,11 @@ public class TestSchemaTool extends Test
     System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname,
         "jdbc:derby:" + testMetastoreDB + ";create=true");
     hiveConf = new HiveConf(this.getClass());
-    schemaTool = new HiveSchemaTool(System.getProperty("test.tmp.dir"), hiveConf, "derby");
+    schemaTool = new HiveSchemaTool(
+        System.getProperty("test.tmp.dir", "target/tmp"), hiveConf, "derby");
     System.setProperty("beeLine.system.exit", "true");
+    errStream = System.err;
+    outStream = System.out;
   }
 
   @Override
@@ -58,6 +64,8 @@ public class TestSchemaTool extends Test
     if (metaStoreDir.exists()) {
       FileUtils.forceDeleteOnExit(metaStoreDir);
     }
+    System.setOut(outStream);
+    System.setErr(errStream);
   }
 
   /**
@@ -121,12 +129,42 @@ public class TestSchemaTool extends Test
       foundException = true;
     }
     if (!foundException) {
-      throw new Exception("Hive operations shouldn't pass with older version schema");
+      throw new Exception(
+          "Hive operations shouldn't pass with older version schema");
     }
 
-    // upgrade schema from 0.7.0 to latest
+    // Generate dummy pre-upgrade script with errors
+    String invalidPreUpgradeScript = writeDummyPreUpgradeScript(
+        0, "upgrade-0.11.0-to-0.12.0.derby.sql", "foo bar;");
+    // Generate dummy pre-upgrade scripts with valid SQL
+    String validPreUpgradeScript0 = writeDummyPreUpgradeScript(
+        0, "upgrade-0.12.0-to-0.13.0.derby.sql",
+        "CREATE TABLE schema_test0 (id integer);");
+    String validPreUpgradeScript1 = writeDummyPreUpgradeScript(
+        1, "upgrade-0.12.0-to-0.13.0.derby.sql",
+        "CREATE TABLE schema_test1 (id integer);");
+
+    // Capture system out and err
+    schemaTool.setVerbose(true);
+    OutputStream stderr = new ByteArrayOutputStream();
+    PrintStream errPrintStream = new PrintStream(stderr);
+    System.setErr(errPrintStream);
+    OutputStream stdout = new ByteArrayOutputStream();
+    PrintStream outPrintStream = new PrintStream(stdout);
+    System.setOut(outPrintStream);
+
+    // Upgrade schema from 0.7.0 to latest
     schemaTool.doUpgrade("0.7.0");
-    // verify that driver works fine with latest schema
+
+    // Verify that the schemaTool ran pre-upgrade scripts and ignored errors
+    assertTrue(stderr.toString().contains(invalidPreUpgradeScript));
+    assertTrue(stderr.toString().contains("foo"));
+    assertFalse(stderr.toString().contains(validPreUpgradeScript0));
+    assertFalse(stderr.toString().contains(validPreUpgradeScript1));
+    assertTrue(stdout.toString().contains(validPreUpgradeScript0));
+    assertTrue(stdout.toString().contains(validPreUpgradeScript1));
+
+    // Verify that driver works fine with latest schema
     schemaTool.verifySchemaVersion();
   }
 
@@ -152,9 +190,9 @@ public class TestSchemaTool extends Test
     String expectedSQL = StringUtils.join(resultScript, System.getProperty("line.separator")) +
         System.getProperty("line.separator");
     File testScriptFile = generateTestScript(testScript);
-    String flattenedSql = HiveSchemaTool.buildCommand(
-        HiveSchemaHelper.getDbCommandParser("derby"),
-        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    String flattenedSql = HiveSchemaHelper.getDbCommandParser("derby")
+        .buildCommand(testScriptFile.getParentFile().getPath(),
+            testScriptFile.getName());
 
     assertEquals(expectedSQL, flattenedSql);
   }
@@ -194,9 +232,9 @@ public class TestSchemaTool extends Test
       };
 
     File testScriptFile = generateTestScript(parentTestScript);
-    String flattenedSql = HiveSchemaTool.buildCommand(
-        HiveSchemaHelper.getDbCommandParser("derby"),
-        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    String flattenedSql = HiveSchemaHelper.getDbCommandParser("derby")
+        .buildCommand(testScriptFile.getParentFile().getPath(),
+            testScriptFile.getName());
     assertFalse(flattenedSql.contains("RUN"));
     assertFalse(flattenedSql.contains("comment"));
     assertTrue(flattenedSql.contains(childTab1));
@@ -239,9 +277,9 @@ public class TestSchemaTool extends Test
       };
 
     File testScriptFile = generateTestScript(parentTestScript);
-    String flattenedSql = HiveSchemaTool.buildCommand(
-        HiveSchemaHelper.getDbCommandParser("mysql"),
-        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    String flattenedSql = HiveSchemaHelper.getDbCommandParser("mysql")
+        .buildCommand(testScriptFile.getParentFile().getPath(),
+            testScriptFile.getName());
     assertFalse(flattenedSql.contains("RUN"));
     assertFalse(flattenedSql.contains("comment"));
     assertTrue(flattenedSql.contains(childTab1));
@@ -282,8 +320,8 @@ public class TestSchemaTool extends Test
         System.getProperty("line.separator");
     File testScriptFile = generateTestScript(testScript);
     NestedScriptParser testDbParser = HiveSchemaHelper.getDbCommandParser("mysql");
-    String flattenedSql = HiveSchemaTool.buildCommand(testDbParser,
-        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    String flattenedSql = testDbParser.buildCommand(testScriptFile.getParentFile().getPath(),
+        testScriptFile.getName());
 
     assertEquals(expectedSQL, flattenedSql);
   }
@@ -317,8 +355,8 @@ public class TestSchemaTool extends Test
         System.getProperty("line.separator");
     File testScriptFile = generateTestScript(testScript);
     NestedScriptParser testDbParser = HiveSchemaHelper.getDbCommandParser("mysql");
-    String flattenedSql = HiveSchemaTool.buildCommand(testDbParser,
-        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    String flattenedSql = testDbParser.buildCommand(testScriptFile.getParentFile().getPath(),
+        testScriptFile.getName());
 
     assertEquals(expectedSQL, flattenedSql);
   }
@@ -358,9 +396,9 @@ public class TestSchemaTool extends Test
       };
 
     File testScriptFile = generateTestScript(parentTestScript);
-    String flattenedSql = HiveSchemaTool.buildCommand(
-        HiveSchemaHelper.getDbCommandParser("oracle"),
-        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    String flattenedSql = HiveSchemaHelper.getDbCommandParser("oracle")
+        .buildCommand(testScriptFile.getParentFile().getPath(),
+            testScriptFile.getName());
     assertFalse(flattenedSql.contains("@"));
     assertFalse(flattenedSql.contains("comment"));
     assertTrue(flattenedSql.contains(childTab1));
@@ -380,4 +418,21 @@ public class TestSchemaTool extends Test
     out.close();
     return testScriptFile;
   }
-}
+
+  /**
+   * Write out a dummy pre-upgrade script with given SQL statement.
+   */
+  private String writeDummyPreUpgradeScript(int index, String upgradeScriptName,
+      String sql) throws Exception {
+    String preUpgradeScript = "pre-" + index + "-" + upgradeScriptName;
+    String dummyPreScriptPath = System.getProperty("test.tmp.dir", "target/tmp") +
+        File.separatorChar + "scripts" + File.separatorChar + "metastore" +
+        File.separatorChar + "upgrade" + File.separatorChar + "derby" +
+        File.separatorChar + preUpgradeScript;
+    FileWriter fstream = new FileWriter(dummyPreScriptPath);
+    BufferedWriter out = new BufferedWriter(fstream);
+    out.write(sql + System.getProperty("line.separator") + ";");
+    out.close();
+    return preUpgradeScript;
+  }
+}
\ No newline at end of file

Added: hive/trunk/metastore/scripts/upgrade/postgres/pre-0-upgrade-0.12.0-to-0.13.0.postgres.sql
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/scripts/upgrade/postgres/pre-0-upgrade-0.12.0-to-0.13.0.postgres.sql?rev=1631893&view=auto
==============================================================================
--- hive/trunk/metastore/scripts/upgrade/postgres/pre-0-upgrade-0.12.0-to-0.13.0.postgres.sql (added)
+++ hive/trunk/metastore/scripts/upgrade/postgres/pre-0-upgrade-0.12.0-to-0.13.0.postgres.sql Tue Oct 14 21:36:32 2014
@@ -0,0 +1 @@
+CREATE LANGUAGE plpgsql;

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java?rev=1631893&r1=1631892&r2=1631893&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java Tue Oct 14 21:36:32 2014
@@ -37,6 +37,7 @@ public class MetaStoreSchemaInfo {
   private static String UPGRADE_FILE_PREFIX="upgrade-";
   private static String INIT_FILE_PREFIX="hive-schema-";
   private static String VERSION_UPGRADE_LIST = "upgrade.order";
+  private static String PRE_UPGRADE_PREFIX = "pre-";
   private final String dbType;
   private final String hiveSchemaVersions[];
   private final HiveConf hiveConf;
@@ -138,6 +139,10 @@ public class MetaStoreSchemaInfo {
     return UPGRADE_FILE_PREFIX +  fileVersion + "." + dbType + SQL_FILE_EXTENSION;
   }
 
+  public static String getPreUpgradeScriptName(int index, String upgradeScriptName) {
+    return PRE_UPGRADE_PREFIX + index + "-" + upgradeScriptName;
+  }
+
   public static String getHiveSchemaVersion() {
     String hiveVersion = HiveVersionInfo.getShortVersion();
     // if there is an equivalent version, return that, else return this version
@@ -149,4 +154,4 @@ public class MetaStoreSchemaInfo {
     }
   }
 
-}
+}
\ No newline at end of file