You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2013/09/26 05:01:50 UTC

svn commit: r1526363 - in /hive/branches/branch-0.12: ./ beeline/src/java/org/apache/hive/beeline/ beeline/src/test/org/apache/hive/beeline/src/test/ bin/ bin/ext/ metastore/scripts/upgrade/derby/ metastore/scripts/upgrade/mysql/ metastore/scripts/upgr...

Author: thejas
Date: Thu Sep 26 03:01:50 2013
New Revision: 1526363

URL: http://svn.apache.org/r1526363
Log:
HIVE-5301 : Add a schema tool for offline metastore schema upgrade (Prasad Mujumdar via Ashutosh Chauhan)

Added:
    hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
    hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
    hive/branches/branch-0.12/beeline/src/test/org/apache/hive/beeline/src/test/TestSchemaTool.java
    hive/branches/branch-0.12/bin/ext/schemaTool.sh
    hive/branches/branch-0.12/bin/schematool
Modified:
    hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/BeeLine.java
    hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
    hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/Commands.java
    hive/branches/branch-0.12/build.xml
    hive/branches/branch-0.12/metastore/scripts/upgrade/derby/014-HIVE-3764.derby.sql
    hive/branches/branch-0.12/metastore/scripts/upgrade/mysql/014-HIVE-3764.mysql.sql
    hive/branches/branch-0.12/metastore/scripts/upgrade/oracle/014-HIVE-3764.oracle.sql
    hive/branches/branch-0.12/metastore/scripts/upgrade/postgres/014-HIVE-3764.postgres.sql

Modified: hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/BeeLine.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/BeeLine.java?rev=1526363&r1=1526362&r2=1526363&view=diff
==============================================================================
--- hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/BeeLine.java (original)
+++ hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/BeeLine.java Thu Sep 26 03:01:50 2013
@@ -653,7 +653,7 @@ public class BeeLine {
   public ConsoleReader getConsoleReader(InputStream inputStream) throws IOException {
     if (inputStream != null) {
       // ### NOTE: fix for sf.net bug 879425.
-      consoleReader = new ConsoleReader(inputStream, new PrintWriter(System.out, true));
+      consoleReader = new ConsoleReader(inputStream, new PrintWriter(getOutputStream(), true));
     } else {
       consoleReader = new ConsoleReader();
     }
@@ -790,6 +790,11 @@ public class BeeLine {
     if (trimmed.length() == 0) {
       return false;
     }
+
+    if (!getOpts().isAllowMultiLineCommand()) {
+      return false;
+    }
+
     return !trimmed.endsWith(";");
   }
 
@@ -1614,7 +1619,7 @@ public class BeeLine {
     }
   }
 
-  BeeLineOpts getOpts() {
+  public BeeLineOpts getOpts() {
     return opts;
   }
 

Modified: hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java?rev=1526363&r1=1526362&r2=1526363&view=diff
==============================================================================
--- hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java (original)
+++ hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java Thu Sep 26 03:01:50 2013
@@ -74,6 +74,7 @@ class BeeLineOpts implements Completor {
   private String isolation = DEFAULT_ISOLATION_LEVEL;
   private String outputFormat = "table";
   private boolean trimScripts = true;
+  private boolean allowMultiLineCommand = true;
 
   private final File rcFile = new File(saveDir(), "beeline.properties");
   private String historyFile = new File(saveDir(), "history").getAbsolutePath();
@@ -421,4 +422,14 @@ class BeeLineOpts implements Completor {
   public File getPropertiesFile() {
     return rcFile;
   }
+
+  public boolean isAllowMultiLineCommand() {
+    return allowMultiLineCommand;
+  }
+
+  public void setAllowMultiLineCommand(boolean allowMultiLineCommand) {
+    this.allowMultiLineCommand = allowMultiLineCommand;
+  }
+
+
 }

Modified: hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/Commands.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/Commands.java?rev=1526363&r1=1526362&r2=1526363&view=diff
==============================================================================
--- hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/Commands.java (original)
+++ hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/Commands.java Thu Sep 26 03:01:50 2013
@@ -674,7 +674,7 @@ public class Commands {
 
     // use multiple lines for statements not terminated by ";"
     try {
-      while (!(line.trim().endsWith(";"))) {
+      while (!(line.trim().endsWith(";")) && beeLine.getOpts().isAllowMultiLineCommand()) {
         StringBuilder prompt = new StringBuilder(beeLine.getPrompt());
         for (int i = 0; i < prompt.length() - 1; i++) {
           if (prompt.charAt(i) != '>') {

Added: hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java?rev=1526363&view=auto
==============================================================================
--- hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java (added)
+++ hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java Thu Sep 26 03:01:50 2013
@@ -0,0 +1,251 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.beeline;
+
+import java.util.IllegalFormatException;
+
+public class HiveSchemaHelper {
+  public static final String DB_DERBY = "derby";
+  public static final String DB_MYSQL = "mysql";
+  public static final String DB_POSTGRACE = "postgrace";
+  public static final String DB_ORACLE = "oracle";
+
+  public interface NestedScriptParser {
+
+    public enum CommandType {
+      PARTIAL_STATEMENT,
+      TERMINATED_STATEMENT,
+      COMMENT
+    }
+
+    static final String DEFAUTL_DELIMITER = ";";
+    /***
+     * Find the type of given command
+     * @param dbCommand
+     * @return
+     */
+    public boolean isPartialCommand(String dbCommand) throws IllegalArgumentException;
+
+    /** Parse the DB specific nesting format and extract the inner script name if any
+     * @param dbCommand command from parent script
+     * @return
+     * @throws IllegalFormatException
+     */
+    public String getScriptName(String dbCommand) throws IllegalArgumentException;
+
+    /***
+     * Find if the given command is a nested script execution
+     * @param dbCommand
+     * @return
+     */
+    public boolean isNestedScript(String dbCommand);
+
+    /***
+     * Find if the given command is should be passed to DB
+     * @param dbCommand
+     * @return
+     */
+    public boolean isNonExecCommand(String dbCommand);
+
+    /***
+     * Get the SQL statement delimiter
+     * @return
+     */
+    public String getDelimiter();
+
+    /***
+     * Clear any client specific tags
+     * @return
+     */
+    public String cleanseCommand(String dbCommand);
+  }
+
+
+  /***
+   * Base implemenation of NestedScriptParser
+   * abstractCommandParser.
+   *
+   */
+  private static abstract class AbstractCommandParser implements NestedScriptParser {
+
+    @Override
+    public boolean isPartialCommand(String dbCommand) throws IllegalArgumentException{
+      if (dbCommand == null || dbCommand.isEmpty()) {
+        throw new IllegalArgumentException("invalid command line " + dbCommand);
+      }
+      dbCommand = dbCommand.trim();
+      if (dbCommand.endsWith(getDelimiter()) || isNonExecCommand(dbCommand)) {
+        return false;
+      } else {
+        return true;
+      }
+    }
+
+    @Override
+    public boolean isNonExecCommand(String dbCommand) {
+      return (dbCommand.startsWith("--") || dbCommand.startsWith("#"));
+    }
+
+    @Override
+    public String getDelimiter() {
+      return DEFAUTL_DELIMITER;
+    }
+
+    @Override
+    public String cleanseCommand(String dbCommand) {
+      // strip off the delimiter
+      if (dbCommand.endsWith(getDelimiter())) {
+        dbCommand = dbCommand.substring(0,
+            dbCommand.length() - getDelimiter().length());
+      }
+      return dbCommand;
+    }
+  }
+
+
+  // Derby commandline parser
+  public static class DerbyCommandParser extends AbstractCommandParser {
+    private static String DERBY_NESTING_TOKEN = "RUN";
+
+    @Override
+    public String getScriptName(String dbCommand) throws IllegalArgumentException {
+
+      if (!isNestedScript(dbCommand)) {
+        throw new IllegalArgumentException("Not a script format " + dbCommand);
+      }
+      String[] tokens = dbCommand.split(" ");
+      if (tokens.length != 2) {
+        throw new IllegalArgumentException("Couldn't parse line " + dbCommand);
+      }
+      return tokens[1].replace(";", "").replaceAll("'", "");
+    }
+
+    @Override
+    public boolean isNestedScript(String dbCommand) {
+      // Derby script format is RUN '<file>'
+     return dbCommand.startsWith(DERBY_NESTING_TOKEN);
+    }
+  }
+
+
+  // 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;
+
+    @Override
+    public boolean isPartialCommand(String dbCommand) throws IllegalArgumentException{
+      boolean isPartial = super.isPartialCommand(dbCommand);
+      // if this is a delimiter directive, reset our delimiter
+      if (dbCommand.startsWith(DELIMITER_TOKEN)) {
+        String[] tokens = dbCommand.split(" ");
+        if (tokens.length != 2) {
+          throw new IllegalArgumentException("Couldn't parse line " + dbCommand);
+        }
+        delimiter = tokens[1];
+      }
+      return isPartial;
+    }
+
+    @Override
+    public String getScriptName(String dbCommand) throws IllegalArgumentException {
+      String[] tokens = dbCommand.split(" ");
+      if (tokens.length != 2) {
+        throw new IllegalArgumentException("Couldn't parse line " + dbCommand);
+      }
+      // remove ending ';'
+      return tokens[1].replace(";", "");
+    }
+
+    @Override
+    public boolean isNestedScript(String dbCommand) {
+      return dbCommand.startsWith(MYSQL_NESTING_TOKEN);
+    }
+
+    @Override
+    public String getDelimiter() {
+      return delimiter;
+    }
+
+    @Override
+    public boolean isNonExecCommand(String dbCommand) {
+      return super.isNonExecCommand(dbCommand) ||
+          (dbCommand.startsWith("/*") && dbCommand.endsWith("*/")) ||
+          dbCommand.startsWith(DELIMITER_TOKEN);
+    }
+
+    @Override
+    public String cleanseCommand(String dbCommand) {
+      return super.cleanseCommand(dbCommand).replaceAll("/\\*.*?\\*/[^;]", "");
+    }
+
+  }
+
+  // Postgres specific parser
+  public static class PostgresCommandParser extends AbstractCommandParser {
+    private static String POSTGRES_NESTING_TOKEN = "\\i";
+
+    @Override
+    public String getScriptName(String dbCommand) throws IllegalArgumentException {
+      String[] tokens = dbCommand.split(" ");
+      if (tokens.length != 2) {
+        throw new IllegalArgumentException("Couldn't parse line " + dbCommand);
+      }
+      // remove ending ';'
+      return tokens[1].replace(";", "");
+    }
+
+    @Override
+    public boolean isNestedScript(String dbCommand) {
+      return dbCommand.startsWith(POSTGRES_NESTING_TOKEN);
+    }
+  }
+
+  //Oracle specific parser
+  public static class OracleCommandParser extends AbstractCommandParser {
+    private static String ORACLE_NESTING_TOKEN = "@";
+    @Override
+    public String getScriptName(String dbCommand) throws IllegalArgumentException {
+      if (!isNestedScript(dbCommand)) {
+        throw new IllegalArgumentException("Not a nested script format " + dbCommand);
+      }
+      // remove ending ';'
+      return dbCommand.replace(";", "");
+    }
+
+    @Override
+    public boolean isNestedScript(String dbCommand) {
+      return dbCommand.startsWith(ORACLE_NESTING_TOKEN);
+    }
+  }
+
+  public static NestedScriptParser getDbCommandParser(String dbName) {
+    if (dbName.equalsIgnoreCase(DB_DERBY)) {
+      return new DerbyCommandParser();
+    } else if (dbName.equalsIgnoreCase(DB_MYSQL)) {
+      return new MySqlCommandParser();
+    } else if (dbName.equalsIgnoreCase(DB_POSTGRACE)) {
+      return new PostgresCommandParser();
+    } else if (dbName.equalsIgnoreCase(DB_ORACLE)) {
+      return new OracleCommandParser();
+    } else {
+      throw new IllegalArgumentException("Unknown dbType " + dbName);
+    }
+  }
+}

Added: hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java?rev=1526363&view=auto
==============================================================================
--- hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java (added)
+++ hive/branches/branch-0.12/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java Thu Sep 26 03:01:50 2013
@@ -0,0 +1,506 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.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;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.OptionGroup;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.io.output.NullOutputStream;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfo;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hive.beeline.HiveSchemaHelper.NestedScriptParser;
+
+public class HiveSchemaTool {
+  private String userName = null;
+  private String passWord = null;
+  private boolean dryRun = false;
+  private boolean verbose = false;
+  private final HiveConf hiveConf;
+  private final String dbType;
+  private final MetaStoreSchemaInfo metaStoreSchemaInfo;
+
+  public HiveSchemaTool(String dbType) throws HiveMetaException {
+    this(System.getenv("HIVE_HOME"), new HiveConf(HiveSchemaTool.class), dbType);
+  }
+
+  public HiveSchemaTool(String hiveHome, HiveConf hiveConf, String dbType)
+        throws HiveMetaException {
+    if (hiveHome == null || hiveHome.isEmpty()) {
+      throw new HiveMetaException("No Hive home directory provided");
+    }
+    this.hiveConf = hiveConf;
+    this.dbType = dbType;
+    this.metaStoreSchemaInfo = new MetaStoreSchemaInfo(hiveHome, hiveConf, dbType);
+    userName = hiveConf.get(ConfVars.METASTORE_CONNECTION_USER_NAME.varname);
+    passWord = hiveConf.get(HiveConf.ConfVars.METASTOREPWD.varname);
+  }
+
+  public HiveConf getHiveConf() {
+    return hiveConf;
+  }
+
+  public void setUserName(String userName) {
+    this.userName = userName;
+  }
+
+  public void setPassWord(String passWord) {
+    this.passWord = passWord;
+  }
+
+  public void setDryRun(boolean dryRun) {
+    this.dryRun = dryRun;
+  }
+
+  public void setVerbose(boolean verbose) {
+    this.verbose = verbose;
+  }
+
+  private static void printAndExit(Options cmdLineOptions) {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("schemaTool", cmdLineOptions);
+    System.exit(1);
+  }
+
+  /***
+   * Print Hive version and schema version
+   * @throws MetaException
+   */
+  public void showInfo() throws HiveMetaException {
+    Connection metastoreConn = getConnectionToMetastore(true);
+    System.out.println("Hive distribution version:\t " +
+        MetaStoreSchemaInfo.getHiveSchemaVersion());
+    System.out.println("Metastore schema version:\t " +
+        getMetaStoreSchemaVersion(metastoreConn));
+  }
+
+  // read schema version from metastore
+  private String getMetaStoreSchemaVersion(Connection metastoreConn)
+        throws HiveMetaException {
+    String versionQuery = "select t.SCHEMA_VERSION from VERSION t";
+    try {
+      Statement stmt = metastoreConn.createStatement();
+      ResultSet res = stmt.executeQuery(versionQuery);
+      if (!res.next()) {
+        throw new HiveMetaException("Didn't find version data in metastore");
+      }
+      String currentSchemaVersion = res.getString(1);
+      metastoreConn.close();
+      return currentSchemaVersion;
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to get schema version.", e);
+    }
+  }
+
+  // test the connection metastore using the config property
+  private void testConnectionToMetastore() throws HiveMetaException {
+    Connection conn = getConnectionToMetastore(true);
+    try {
+      conn.close();
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to close metastore connection", e);
+    }
+  }
+
+  /***
+   * 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
+   * @throws MetaException
+   */
+  public void verifySchemaVersion() throws HiveMetaException {
+    // don't check version if its a dry run
+    if (dryRun) {
+      return;
+    }
+    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);
+    }
+  }
+
+  /**
+   * Perform metastore schema upgrade. extract the current schema version from metastore
+   * @throws MetaException
+   */
+  public void doUpgrade() throws HiveMetaException {
+    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");
+    }
+    doUpgrade(fromVersion);
+  }
+
+  /**
+   * Perform metastore schema upgrade
+   *
+   * @param fromSchemaVer
+   *          Existing version of the metastore. If null, then read from the metastore
+   * @throws MetaException
+   */
+  public void doUpgrade(String fromSchemaVer) throws HiveMetaException {
+    if (MetaStoreSchemaInfo.getHiveSchemaVersion().equals(fromSchemaVer)) {
+      System.out.println("No schema upgrade required from version " + fromSchemaVer);
+      return;
+    }
+    // Find the list of scripts to execute for this upgrade
+    List<String> upgradeScripts =
+        metaStoreSchemaInfo.getUpgradeScripts(fromSchemaVer);
+    testConnectionToMetastore();
+    System.out.println("Starting upgrade metastore schema from version " +
+        fromSchemaVer + " to " + MetaStoreSchemaInfo.getHiveSchemaVersion());
+    String scriptDir = metaStoreSchemaInfo.getMetaStoreScriptDir();
+    try {
+      for (String scriptFile : upgradeScripts) {
+        System.out.println("Upgrade script " + scriptFile);
+        if (!dryRun) {
+          runBeeLine(scriptDir, scriptFile);
+          System.out.println("Completed " + scriptFile);
+        }
+      }
+    } catch (IOException eIO) {
+      throw new HiveMetaException(
+          "Upgrade FAILED! Metastore state would be inconsistent !!", eIO);
+    }
+
+    // Revalidated the new version after upgrade
+    verifySchemaVersion();
+  }
+
+  /**
+   * Initialize the metastore schema to current version
+   *
+   * @throws MetaException
+   */
+  public void doInit() throws HiveMetaException {
+    doInit(MetaStoreSchemaInfo.getHiveSchemaVersion());
+
+    // Revalidated the new version after upgrade
+    verifySchemaVersion();
+  }
+
+  /**
+   * Initialize the metastore schema
+   *
+   * @param toVersion
+   *          If null then current hive version is used
+   * @throws MetaException
+   */
+  public void doInit(String toVersion) throws HiveMetaException {
+    testConnectionToMetastore();
+    System.out.println("Starting metastore schema initialization to " + toVersion);
+
+    String initScriptDir = metaStoreSchemaInfo.getMetaStoreScriptDir();
+    String initScriptFile = metaStoreSchemaInfo.generateInitFileName(toVersion);
+
+    try {
+      System.out.println("Initialization script " + initScriptFile);
+      if (!dryRun) {
+        runBeeLine(initScriptDir, initScriptFile);
+        System.out.println("Initialization script completed");
+      }
+    } catch (IOException e) {
+      throw new HiveMetaException("Schema initialization FAILED!" +
+      		" Metastore state would be inconsistent !!", e);
+    }
+  }
+
+  // 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"));
+        }
+      }
+      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);
+    // expand the nested script
+    String sqlCommands = buildCommand(dbCommandParser, scriptDir, scriptFile);
+    File tmpFile = File.createTempFile("schematool", ".sql");
+    tmpFile.deleteOnExit();
+
+    // write out the buffer into a file. Add beeline commands for autocommit and close
+    FileWriter fstream = new FileWriter(tmpFile.getPath());
+    BufferedWriter out = new BufferedWriter(fstream);
+    out.write("!autocommit on" + System.getProperty("line.separator"));
+    out.write(sqlCommands);
+    out.write("!closeall" + System.getProperty("line.separator"));
+    out.close();
+    runBeeLine(tmpFile.getPath());
+  }
+
+  // Generate the beeline args per hive conf and execute the given script
+  public void runBeeLine(String sqlScriptFile) throws IOException {
+    List<String> argList = new ArrayList<String>();
+    argList.add("-u");
+    argList.add(getValidConfVar(ConfVars.METASTORECONNECTURLKEY));
+    argList.add("-d");
+    argList.add(getValidConfVar(ConfVars.METASTORE_CONNECTION_DRIVER));
+    argList.add("-n");
+    argList.add(userName);
+    argList.add("-p");
+    argList.add(passWord);
+    argList.add("-f");
+    argList.add(sqlScriptFile);
+
+    // run the script using Beeline
+    BeeLine beeLine = new BeeLine();
+    if (!verbose) {
+      beeLine.setOutputStream(new PrintStream(new NullOutputStream()));
+      beeLine.getOpts().setSilent(true);
+    }
+    beeLine.getOpts().setAllowMultiLineCommand(false);
+    int status = beeLine.begin(argList.toArray(new String[0]), null);
+    if (status != 0) {
+      throw new IOException("Schema script failed, errorcode " + status);
+    }
+  }
+
+  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) {
+    Option help = new Option("help", "print this message");
+    Option upgradeOpt = new Option("upgradeSchema", "Schema upgrade");
+    Option upgradeFromOpt = OptionBuilder.withArgName("upgradeFrom").hasArg().
+                withDescription("Schema upgrade from a version").
+                create("upgradeSchemaFrom");
+    Option initOpt = new Option("initSchema", "Schema initialization");
+    Option initToOpt = OptionBuilder.withArgName("initTo").hasArg().
+                withDescription("Schema initialization to a version").
+                create("initSchemaTo");
+    Option infoOpt = new Option("info", "Show config and schema details");
+
+    OptionGroup optGroup = new OptionGroup();
+    optGroup.addOption(upgradeOpt).addOption(initOpt).
+                addOption(help).addOption(upgradeFromOpt).
+                addOption(initToOpt).addOption(infoOpt);
+    optGroup.setRequired(true);
+
+    Option userNameOpt = OptionBuilder.withArgName("user")
+                .hasArgs()
+                .withDescription("Override config file user name")
+                .create("userName");
+    Option passwdOpt = OptionBuilder.withArgName("password")
+                .hasArgs()
+                 .withDescription("Override config file password")
+                 .create("passWord");
+    Option dbTypeOpt = OptionBuilder.withArgName("databaseType")
+                .hasArgs().withDescription("Metastore database type")
+                .create("dbType");
+    Option dryRunOpt = new Option("dryRun", "list SQL scripts (no execute)");
+    Option verboseOpt = new Option("verbose", "only print SQL statements");
+
+    cmdLineOptions.addOption(help);
+    cmdLineOptions.addOption(dryRunOpt);
+    cmdLineOptions.addOption(userNameOpt);
+    cmdLineOptions.addOption(passwdOpt);
+    cmdLineOptions.addOption(dbTypeOpt);
+    cmdLineOptions.addOption(verboseOpt);
+    cmdLineOptions.addOptionGroup(optGroup);
+  }
+
+  public static void main(String[] args) {
+    CommandLineParser parser = new GnuParser();
+    CommandLine line = null;
+    String dbType = null;
+    String schemaVer = null;
+    Options cmdLineOptions = new Options();
+
+    // Argument handling
+    initOptions(cmdLineOptions);
+    try {
+      line = parser.parse(cmdLineOptions, args);
+    } catch (ParseException e) {
+      System.err.println("HiveSchemaTool:Parsing failed.  Reason: " + e.getLocalizedMessage());
+      printAndExit(cmdLineOptions);
+    }
+
+    if (line.hasOption("help")) {
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp("schemaTool", cmdLineOptions);
+      return;
+    }
+
+    if (line.hasOption("dbType")) {
+      dbType = line.getOptionValue("dbType");
+      if ((!dbType.equalsIgnoreCase(HiveSchemaHelper.DB_DERBY) &&
+          !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_MYSQL) &&
+          !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_POSTGRACE) && !dbType
+          .equalsIgnoreCase(HiveSchemaHelper.DB_ORACLE))) {
+        System.err.println("Unsupported dbType " + dbType);
+        printAndExit(cmdLineOptions);
+      }
+    } else {
+      System.err.println("no dbType supplied");
+      printAndExit(cmdLineOptions);
+    }
+
+    System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.varname, "true");
+    try {
+      HiveSchemaTool schemaTool = new HiveSchemaTool(dbType);
+
+      if (line.hasOption("userName")) {
+        schemaTool.setUserName(line.getOptionValue("userName"));
+      }
+      if (line.hasOption("passWord")) {
+        schemaTool.setPassWord(line.getOptionValue("passWord"));
+      }
+      if (line.hasOption("dryRun")) {
+        schemaTool.setDryRun(true);
+      }
+      if (line.hasOption("verbose")) {
+        schemaTool.setVerbose(true);
+      }
+
+      if (line.hasOption("info")) {
+        schemaTool.showInfo();
+      } else if (line.hasOption("upgradeSchema")) {
+        schemaTool.doUpgrade();
+      } else if (line.hasOption("upgradeSchemaFrom")) {
+        schemaVer = line.getOptionValue("upgradeSchemaFrom");
+        schemaTool.doUpgrade(schemaVer);
+      } else if (line.hasOption("initSchema")) {
+        schemaTool.doInit();
+      } else if (line.hasOption("initSchemaTo")) {
+        schemaVer = line.getOptionValue("initSchemaTo");
+        schemaTool.doInit(schemaVer);
+      } else {
+        System.err.println("no valid option supplied");
+        printAndExit(cmdLineOptions);
+      }
+    } catch (HiveMetaException e) {
+      System.err.println(e);
+      if (line.hasOption("verbose")) {
+        e.printStackTrace();
+      }
+      System.err.println("*** schemaTool failed ***");
+      System.exit(1);
+    }
+    System.out.println("schemaTool completeted");
+
+  }
+}

Added: hive/branches/branch-0.12/beeline/src/test/org/apache/hive/beeline/src/test/TestSchemaTool.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/beeline/src/test/org/apache/hive/beeline/src/test/TestSchemaTool.java?rev=1526363&view=auto
==============================================================================
--- hive/branches/branch-0.12/beeline/src/test/org/apache/hive/beeline/src/test/TestSchemaTool.java (added)
+++ hive/branches/branch-0.12/beeline/src/test/org/apache/hive/beeline/src/test/TestSchemaTool.java Thu Sep 26 03:01:50 2013
@@ -0,0 +1,336 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.beeline.src.test;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+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;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    testMetastoreDB = System.getProperty("java.io.tmpdir") +
+        File.separator + "test_metastore-" + new Random().nextInt();
+    System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname,
+        "jdbc:derby:" + testMetastoreDB + ";create=true");
+    hiveConf = new HiveConf(this.getClass());
+    schemaTool = new HiveSchemaTool(System.getProperty("hive.home"), hiveConf, "derby");
+    System.setProperty("beeLine.system.exit", "true");
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    File metaStoreDir = new File(testMetastoreDB);
+    if (metaStoreDir.exists()) {
+      FileUtils.deleteDirectory(metaStoreDir);
+    }
+  }
+
+  /**
+   * Test dryrun of schema initialization
+   * @throws Exception
+   */
+  public void testSchemaInitDryRun() throws Exception {
+    schemaTool.setDryRun(true);
+    schemaTool.doInit("0.7.0");
+    schemaTool.setDryRun(false);
+    try {
+      schemaTool.verifySchemaVersion();
+    } catch (HiveMetaException e) {
+      // The connection should fail since it the dry run
+      return;
+    }
+    fail("Dry run shouldn't create actual metastore");
+  }
+
+  /**
+   * Test dryrun of schema upgrade
+   * @throws Exception
+   */
+  public void testSchemaUpgradeDryRun() throws Exception {
+    schemaTool.doInit("0.7.0");
+
+    schemaTool.setDryRun(true);
+    schemaTool.doUpgrade("0.7.0");
+    schemaTool.setDryRun(false);
+    try {
+      schemaTool.verifySchemaVersion();
+    } catch (HiveMetaException e) {
+      // The connection should fail since it the dry run
+      return;
+    }
+    fail("Dry run shouldn't upgrade metastore schema");
+  }
+
+  /**
+   * Test schema initialization
+   * @throws Exception
+   */
+  public void testSchemaInit() throws Exception {
+    schemaTool.doInit("0.12.0");
+    }
+
+  /**
+   * Test schema upgrade
+   * @throws Exception
+   */
+  public void testSchemaUpgrade() throws Exception {
+    boolean foundException = false;
+    // Initialize 0.7.0 schema
+    schemaTool.doInit("0.7.0");
+    // verify that driver fails due to older version schema
+    try {
+      schemaTool.verifySchemaVersion();
+    } catch (HiveMetaException e) {
+      // Expected to fail due to old schema
+      foundException = true;
+    }
+    if (!foundException) {
+      throw new Exception("Hive operations shouldn't pass with older version schema");
+    }
+
+    // upgrade schema from 0.7.0 to latest
+    schemaTool.doUpgrade("0.7.0");
+    // verify that driver works fine with latest schema
+    schemaTool.verifySchemaVersion();
+  }
+
+  /**
+   * Test script formatting
+   * @throws Exception
+   */
+  public void testScripts() throws Exception {
+    String testScript[] = {
+        "-- this is a comment",
+      "DROP TABLE IF EXISTS fooTab;",
+      "/*!1234 this is comment code like mysql */;",
+      "CREATE TABLE fooTab(id INTEGER);",
+      "DROP TABLE footab;",
+      "-- ending comment"
+    };
+    String resultScript[] = {
+      "DROP TABLE IF EXISTS fooTab",
+      "/*!1234 this is comment code like mysql */",
+      "CREATE TABLE fooTab(id INTEGER)",
+      "DROP TABLE footab",
+    };
+    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());
+
+    assertEquals(expectedSQL, flattenedSql);
+  }
+
+  /**
+   * Test nested script formatting
+   * @throws Exception
+   */
+  public void testNestedScriptsForDerby() throws Exception {
+    String childTab1 = "childTab1";
+    String childTab2 = "childTab2";
+    String parentTab = "fooTab";
+
+    String childTestScript1[] = {
+      "-- this is a comment ",
+      "DROP TABLE IF EXISTS " + childTab1 + ";",
+      "CREATE TABLE " + childTab1 + "(id INTEGER);",
+      "DROP TABLE " + childTab1 + ";"
+    };
+    String childTestScript2[] = {
+        "-- this is a comment",
+        "DROP TABLE IF EXISTS " + childTab2 + ";",
+        "CREATE TABLE " + childTab2 + "(id INTEGER);",
+        "-- this is also a comment",
+        "DROP TABLE " + childTab2 + ";"
+    };
+
+    String parentTestScript[] = {
+        " -- this is a comment",
+        "DROP TABLE IF EXISTS " + parentTab + ";",
+        " -- this is another comment ",
+        "CREATE TABLE " + parentTab + "(id INTEGER);",
+        "RUN '" + generateTestScript(childTestScript1).getName() + "';",
+        "DROP TABLE " + parentTab + ";",
+        "RUN '" + generateTestScript(childTestScript2).getName() + "';",
+        "--ending comment ",
+      };
+
+    File testScriptFile = generateTestScript(parentTestScript);
+    String flattenedSql = HiveSchemaTool.buildCommand(
+        HiveSchemaHelper.getDbCommandParser("derby"),
+        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    assertFalse(flattenedSql.contains("RUN"));
+    assertFalse(flattenedSql.contains("comment"));
+    assertTrue(flattenedSql.contains(childTab1));
+    assertTrue(flattenedSql.contains(childTab2));
+    assertTrue(flattenedSql.contains(parentTab));
+  }
+
+  /**
+   * Test nested script formatting
+   * @throws Exception
+   */
+  public void testNestedScriptsForMySQL() throws Exception {
+    String childTab1 = "childTab1";
+    String childTab2 = "childTab2";
+    String parentTab = "fooTab";
+
+    String childTestScript1[] = {
+      "/* this is a comment code */",
+      "DROP TABLE IF EXISTS " + childTab1 + ";",
+      "CREATE TABLE " + childTab1 + "(id INTEGER);",
+      "DROP TABLE " + childTab1 + ";"
+    };
+    String childTestScript2[] = {
+        "/* this is a special exec code */;",
+        "DROP TABLE IF EXISTS " + childTab2 + ";",
+        "CREATE TABLE " + childTab2 + "(id INTEGER);",
+        "-- this is a comment",
+        "DROP TABLE " + childTab2 + ";"
+    };
+
+    String parentTestScript[] = {
+        " -- this is a comment",
+        "DROP TABLE IF EXISTS " + parentTab + ";",
+        " /* this is special exec code */;",
+        "CREATE TABLE " + parentTab + "(id INTEGER);",
+        "SOURCE " + generateTestScript(childTestScript1).getName() + ";",
+        "DROP TABLE " + parentTab + ";",
+        "SOURCE " + generateTestScript(childTestScript2).getName() + ";",
+        "--ending comment ",
+      };
+
+    File testScriptFile = generateTestScript(parentTestScript);
+    String flattenedSql = HiveSchemaTool.buildCommand(
+        HiveSchemaHelper.getDbCommandParser("mysql"),
+        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    assertFalse(flattenedSql.contains("RUN"));
+    assertFalse(flattenedSql.contains("comment"));
+    assertTrue(flattenedSql.contains(childTab1));
+    assertTrue(flattenedSql.contains(childTab2));
+    assertTrue(flattenedSql.contains(parentTab));
+  }
+
+  /**
+   * Test script formatting
+   * @throws Exception
+   */
+  public void testScriptWithDelimiter() throws Exception {
+    String testScript[] = {
+        "-- this is a comment",
+      "DROP TABLE IF EXISTS fooTab;",
+      "DELIMITER $$",
+      "/*!1234 this is comment code like mysql */$$",
+      "CREATE TABLE fooTab(id INTEGER)$$",
+      "CREATE PROCEDURE fooProc()",
+      "SELECT * FROM fooTab;",
+      "CALL barProc();",
+      "END PROCEDURE$$",
+      "DELIMITER ;",
+      "DROP TABLE footab;",
+      "-- ending comment"
+    };
+    String resultScript[] = {
+      "DROP TABLE IF EXISTS fooTab",
+      "/*!1234 this is comment code like mysql */",
+      "CREATE TABLE fooTab(id INTEGER)",
+      "CREATE PROCEDURE fooProc()" + " " +
+      "SELECT * FROM fooTab;" + " " +
+      "CALL barProc();" + " " +
+      "END PROCEDURE",
+      "DROP TABLE footab",
+    };
+    String expectedSQL = StringUtils.join(resultScript, System.getProperty("line.separator")) +
+        System.getProperty("line.separator");
+    File testScriptFile = generateTestScript(testScript);
+    NestedScriptParser testDbParser = HiveSchemaHelper.getDbCommandParser("mysql");
+    String flattenedSql = HiveSchemaTool.buildCommand(testDbParser,
+        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+
+    assertEquals(expectedSQL, flattenedSql);
+  }
+
+  /**
+   * Test script formatting
+   * @throws Exception
+   */
+  public void testScriptMultiRowComment() throws Exception {
+    String testScript[] = {
+        "-- this is a comment",
+      "DROP TABLE IF EXISTS fooTab;",
+      "DELIMITER $$",
+      "/*!1234 this is comment code like mysql */$$",
+      "CREATE TABLE fooTab(id INTEGER)$$",
+      "DELIMITER ;",
+      "/* multiline comment started ",
+      " * multiline comment continue",
+      " * multiline comment ended */",
+      "DROP TABLE footab;",
+      "-- ending comment"
+    };
+    String parsedScript[] = {
+      "DROP TABLE IF EXISTS fooTab",
+      "/*!1234 this is comment code like mysql */",
+      "CREATE TABLE fooTab(id INTEGER)",
+      "DROP TABLE footab",
+    };
+
+    String expectedSQL = StringUtils.join(parsedScript, System.getProperty("line.separator")) +
+        System.getProperty("line.separator");
+    File testScriptFile = generateTestScript(testScript);
+    NestedScriptParser testDbParser = HiveSchemaHelper.getDbCommandParser("mysql");
+    String flattenedSql = HiveSchemaTool.buildCommand(testDbParser,
+        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+
+    assertEquals(expectedSQL, flattenedSql);
+  }
+
+  private File generateTestScript(String [] stmts) throws IOException {
+    File testScriptFile = File.createTempFile("schematest", ".sql");
+    testScriptFile.deleteOnExit();
+    FileWriter fstream = new FileWriter(testScriptFile.getPath());
+    BufferedWriter out = new BufferedWriter(fstream);
+    for (String line: stmts) {
+      out.write(line);
+      out.newLine();
+    }
+    out.close();
+    return testScriptFile;
+  }
+}

Added: hive/branches/branch-0.12/bin/ext/schemaTool.sh
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/bin/ext/schemaTool.sh?rev=1526363&view=auto
==============================================================================
--- hive/branches/branch-0.12/bin/ext/schemaTool.sh (added)
+++ hive/branches/branch-0.12/bin/ext/schemaTool.sh Thu Sep 26 03:01:50 2013
@@ -0,0 +1,28 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+THISSERVICE=schemaTool
+export SERVICE_LIST="${SERVICE_LIST}${THISSERVICE} "
+
+schemaTool() {
+
+  CLASS=org.apache.hive.beeline.HiveSchemaTool
+  execHiveCmd $CLASS "$@"
+}
+
+schemaTool_help () {
+  CLASS=org.apache.hive.beeline.HiveSchemaTool
+  execHiveCmd $CLASS "--help"
+}

Added: hive/branches/branch-0.12/bin/schematool
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/bin/schematool?rev=1526363&view=auto
==============================================================================
--- hive/branches/branch-0.12/bin/schematool (added)
+++ hive/branches/branch-0.12/bin/schematool Thu Sep 26 03:01:50 2013
@@ -0,0 +1,21 @@
+#!/usr/bin/env bash
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/hive --service schemaTool "$@"

Modified: hive/branches/branch-0.12/build.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/build.xml?rev=1526363&r1=1526362&r2=1526363&view=diff
==============================================================================
--- hive/branches/branch-0.12/build.xml (original)
+++ hive/branches/branch-0.12/build.xml Thu Sep 26 03:01:50 2013
@@ -449,6 +449,7 @@
     <mkdir dir="${target.lib.dir}/php"/>
     <copy file="${hive.root}/bin/hive" todir="${target.bin.dir}"/>
     <copy file="${hive.root}/bin/metatool" todir="${target.bin.dir}"/>
+    <copy file="${hive.root}/bin/schematool" todir="${target.bin.dir}"/>
     <copy file="${hive.root}/bin/beeline" todir="${target.bin.dir}"/>
     <copy file="${hive.root}/bin/hiveserver2" todir="${target.bin.dir}"/>
     <copy file="${hive.root}/bin/hive-config.sh" todir="${target.bin.dir}"/>

Modified: hive/branches/branch-0.12/metastore/scripts/upgrade/derby/014-HIVE-3764.derby.sql
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/metastore/scripts/upgrade/derby/014-HIVE-3764.derby.sql?rev=1526363&r1=1526362&r2=1526363&view=diff
==============================================================================
--- hive/branches/branch-0.12/metastore/scripts/upgrade/derby/014-HIVE-3764.derby.sql (original)
+++ hive/branches/branch-0.12/metastore/scripts/upgrade/derby/014-HIVE-3764.derby.sql Thu Sep 26 03:01:50 2013
@@ -2,7 +2,7 @@
 -- Hive HIVE-3764
 -- Support metastore version consistency check
 
-CREATE TABLE "APP"."VERSION" ("VER_ID" BIGINT NOT NULL, "SCHEMA_VERSION" VARCHAR(127) NOT NULL, "COMMENT" VARCHAR(255));
+CREATE TABLE "APP"."VERSION" ("VER_ID" BIGINT NOT NULL, "SCHEMA_VERSION" VARCHAR(127) NOT NULL, "VERSION_COMMENT" VARCHAR(255));
 ALTER TABLE "APP"."VERSION" ADD CONSTRAINT "VERSION_PK" PRIMARY KEY ("VER_ID");
 
-INSERT INTO "APP"."VERSION" (VER_ID, SCHEMA_VERSION, COMMENT) VALUES (1, '', 'Initial value');
+INSERT INTO "APP"."VERSION" (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '', 'Initial value');

Modified: hive/branches/branch-0.12/metastore/scripts/upgrade/mysql/014-HIVE-3764.mysql.sql
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/metastore/scripts/upgrade/mysql/014-HIVE-3764.mysql.sql?rev=1526363&r1=1526362&r2=1526363&view=diff
==============================================================================
--- hive/branches/branch-0.12/metastore/scripts/upgrade/mysql/014-HIVE-3764.mysql.sql (original)
+++ hive/branches/branch-0.12/metastore/scripts/upgrade/mysql/014-HIVE-3764.mysql.sql Thu Sep 26 03:01:50 2013
@@ -2,8 +2,8 @@
 CREATE TABLE IF NOT EXISTS `VERSION` (
   `VER_ID` BIGINT NOT NULL,
   `SCHEMA_VERSION` VARCHAR(127) NOT NULL,
-  `COMMENT` VARCHAR(255),
+  `VERSION_COMMENT` VARCHAR(255),
   PRIMARY KEY (`VER_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, COMMENT) VALUES (1, '', 'Initial value');
+INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '', 'Initial value');

Modified: hive/branches/branch-0.12/metastore/scripts/upgrade/oracle/014-HIVE-3764.oracle.sql
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/metastore/scripts/upgrade/oracle/014-HIVE-3764.oracle.sql?rev=1526363&r1=1526362&r2=1526363&view=diff
==============================================================================
--- hive/branches/branch-0.12/metastore/scripts/upgrade/oracle/014-HIVE-3764.oracle.sql (original)
+++ hive/branches/branch-0.12/metastore/scripts/upgrade/oracle/014-HIVE-3764.oracle.sql Thu Sep 26 03:01:50 2013
@@ -3,8 +3,8 @@
 CREATE TABLE IF NOT EXISTS VERSION (
   VER_ID NUMBER NOT NULL,
   SCHEMA_VERSION VARCHAR(127) NOT NULL,
-  COMMENT VARCHAR(255)
+  VERSION_COMMENT VARCHAR(255)
 )
 ALTER TABLE VERSION ADD CONSTRAINT VERSION_PK PRIMARY KEY (VER_ID);
 
-INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, COMMENT) VALUES (1, '', 'Initial value');
+INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '', 'Initial value');

Modified: hive/branches/branch-0.12/metastore/scripts/upgrade/postgres/014-HIVE-3764.postgres.sql
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/metastore/scripts/upgrade/postgres/014-HIVE-3764.postgres.sql?rev=1526363&r1=1526362&r2=1526363&view=diff
==============================================================================
--- hive/branches/branch-0.12/metastore/scripts/upgrade/postgres/014-HIVE-3764.postgres.sql (original)
+++ hive/branches/branch-0.12/metastore/scripts/upgrade/postgres/014-HIVE-3764.postgres.sql Thu Sep 26 03:01:50 2013
@@ -4,9 +4,9 @@
 CREATE TABLE "VERSION" (
   "VER_ID" bigint,
   "SCHEMA_VERSION" character varying(127) NOT NULL,
-  "COMMENT" character varying(255) NOT NULL,
+  "VERSION_COMMENT" character varying(255) NOT NULL,
   PRIMARY KEY ("VER_ID")
 );
 ALTER TABLE ONLY "VERSION" ADD CONSTRAINT "VERSION_pkey" PRIMARY KEY ("VER_ID");
 
-INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, COMMENT) VALUES (1, '', 'Initial value');
+INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '', 'Initial value');