You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2018/07/06 16:27:19 UTC

[4/4] hive git commit: HIVE-20060 Refactor HiveSchemaTool and MetastoreSchemaTool (Alan Gates, reviewed by Daniel Dai)

HIVE-20060 Refactor HiveSchemaTool and MetastoreSchemaTool (Alan Gates, reviewed by Daniel Dai)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/eae5225f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/eae5225f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/eae5225f

Branch: refs/heads/master
Commit: eae5225f4301254cd8c5ad127bc242890bd441a8
Parents: 02a038e
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Jul 6 09:26:17 2018 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Fri Jul 6 09:26:17 2018 -0700

----------------------------------------------------------------------
 .../hive/beeline/schematool/HiveSchemaTool.java |  338 +----
 .../schematool/HiveSchemaToolCommandLine.java   |  286 ----
 .../beeline/schematool/HiveSchemaToolTask.java  |   32 -
 .../HiveSchemaToolTaskAlterCatalog.java         |   90 --
 .../HiveSchemaToolTaskCreateCatalog.java        |  132 --
 .../schematool/HiveSchemaToolTaskInfo.java      |   43 -
 .../schematool/HiveSchemaToolTaskInit.java      |   73 -
 .../HiveSchemaToolTaskMoveDatabase.java         |   96 --
 .../schematool/HiveSchemaToolTaskMoveTable.java |  142 --
 .../schematool/HiveSchemaToolTaskUpgrade.java   |  116 --
 .../schematool/HiveSchemaToolTaskValidate.java  |  631 ---------
 .../beeline/schematool/TestHiveSchemaTool.java  |    6 +-
 .../tools/TestSchemaToolCatalogOps.java         |  479 +++++++
 .../hive/beeline/schematool/TestSchemaTool.java |  550 +-------
 .../schematool/TestSchemaToolCatalogOps.java    |  478 -------
 .../metastore/tools/MetastoreSchemaTool.java    | 1282 +++---------------
 .../metastore/tools/SchemaToolCommandLine.java  |  308 +++++
 .../hive/metastore/tools/SchemaToolTask.java    |   32 +
 .../tools/SchemaToolTaskAlterCatalog.java       |   90 ++
 .../tools/SchemaToolTaskCreateCatalog.java      |  132 ++
 .../tools/SchemaToolTaskCreateUser.java         |  115 ++
 .../metastore/tools/SchemaToolTaskInfo.java     |   43 +
 .../metastore/tools/SchemaToolTaskInit.java     |   73 +
 .../tools/SchemaToolTaskMoveDatabase.java       |   96 ++
 .../tools/SchemaToolTaskMoveTable.java          |  142 ++
 .../metastore/tools/SchemaToolTaskUpgrade.java  |  116 ++
 .../metastore/tools/SchemaToolTaskValidate.java |  630 +++++++++
 .../hive/metastore/dbinstall/DbInstallBase.java |   25 +-
 .../tools/TestSchemaToolForMetastore.java       |  277 ++--
 29 files changed, 2708 insertions(+), 4145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaTool.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaTool.java
index 9c3f30b..69514e5 100644
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaTool.java
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaTool.java
@@ -17,207 +17,50 @@
  */
 package org.apache.hive.beeline.schematool;
 
-import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.io.output.NullOutputStream;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.apache.hadoop.hive.metastore.IMetaStoreSchemaInfo;
-import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfoFactory;
-import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
 import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
 import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.NestedScriptParser;
-import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.hive.metastore.tools.MetastoreSchemaTool;
 import org.apache.hive.beeline.BeeLine;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
-
-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.net.URI;
-import java.sql.Connection;
-import java.sql.SQLException;
 
-public class HiveSchemaTool {
+public class HiveSchemaTool extends MetastoreSchemaTool {
   private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaTool.class.getName());
 
-  private final HiveConf hiveConf;
-  private final String dbType;
-  private final String metaDbType;
-  private final IMetaStoreSchemaInfo metaStoreSchemaInfo;
-  private final boolean needsQuotedIdentifier;
-  private String quoteCharacter;
-
-  private String url = null;
-  private String driver = null;
-  private String userName = null;
-  private String passWord = null;
-  private boolean dryRun = false;
-  private boolean verbose = false;
-  private String dbOpts = null;
-  private URI[] validationServers = null; // The list of servers the database/partition/table can locate on
-
-  private HiveSchemaTool(String dbType, String metaDbType) throws HiveMetaException {
-    this(System.getenv("HIVE_HOME"), new HiveConf(HiveSchemaTool.class), dbType, metaDbType);
-  }
-
-  @VisibleForTesting
-  public HiveSchemaTool(String hiveHome, HiveConf hiveConf, String dbType, String metaDbType)
-      throws HiveMetaException {
-    if (hiveHome == null || hiveHome.isEmpty()) {
-      throw new HiveMetaException("No Hive home directory provided");
-    }
-    this.hiveConf = hiveConf;
-    this.dbType = dbType;
-    this.metaDbType = metaDbType;
-    NestedScriptParser parser = getDbCommandParser(dbType, metaDbType);
-    this.needsQuotedIdentifier = parser.needsQuotedIdentifier();
-    this.quoteCharacter = parser.getQuoteCharacter();
-    this.metaStoreSchemaInfo = MetaStoreSchemaInfoFactory.get(hiveConf, hiveHome, dbType);
-    // If the dbType is "hive", this is setting up the information schema in Hive.
-    // We will set the default jdbc url and driver.
-    // It is overriden by command line options if passed (-url and -driver
-    if (dbType.equalsIgnoreCase(HiveSchemaHelper.DB_HIVE)) {
-      url = HiveSchemaHelper.EMBEDDED_HS2_URL;
-      driver = HiveSchemaHelper.HIVE_JDBC_DRIVER;
-    }
-  }
-
-  HiveConf getHiveConf() {
-    return hiveConf;
-  }
-
-  String getDbType() {
-    return dbType;
-  }
-
-  IMetaStoreSchemaInfo getMetaStoreSchemaInfo() {
-    return metaStoreSchemaInfo;
-  }
-
-  private void setUrl(String url) {
-    this.url = url;
-  }
-
-  private void setDriver(String driver) {
-    this.driver = driver;
-  }
-
-  @VisibleForTesting
-  public void setUserName(String userName) {
-    this.userName = userName;
-  }
-
-  @VisibleForTesting
-  public void setPassWord(String passWord) {
-    this.passWord = passWord;
-  }
-
-  @VisibleForTesting
-  public void setDryRun(boolean dryRun) {
-    this.dryRun = dryRun;
-  }
-
-  boolean isDryRun() {
-    return dryRun;
-  }
-
-  @VisibleForTesting
-  public void setVerbose(boolean verbose) {
-    this.verbose = verbose;
-  }
-
-  boolean isVerbose() {
-    return verbose;
-  }
-
-  private void setDbOpts(String dbOpts) {
-    this.dbOpts = dbOpts;
-  }
-
-  private void setValidationServers(String servers) {
-    if(StringUtils.isNotEmpty(servers)) {
-      String[] strServers = servers.split(",");
-      this.validationServers = new URI[strServers.length];
-      for (int i = 0; i < validationServers.length; i++) {
-        validationServers[i] = new Path(strServers[i]).toUri();
-      }
-    }
-  }
-
-  URI[] getValidationServers() {
-    return validationServers;
-  }
-
-  Connection getConnectionToMetastore(boolean printInfo) throws HiveMetaException {
-    return HiveSchemaHelper.getConnectionToMetastore(userName, passWord, url, driver, printInfo, hiveConf,
-        null);
-  }
 
-  private NestedScriptParser getDbCommandParser(String dbType, String metaDbType) {
-    return HiveSchemaHelper.getDbCommandParser(dbType, dbOpts, userName, passWord, hiveConf,
+  @Override
+  protected NestedScriptParser getDbCommandParser(String dbType, String metaDbType) {
+    return HiveSchemaHelper.getDbCommandParser(dbType, dbOpts, userName, passWord, conf,
         metaDbType, false);
   }
 
-  // test the connection metastore using the config property
-  void testConnectionToMetastore() throws HiveMetaException {
-    Connection conn = getConnectionToMetastore(true);
-    try {
-      conn.close();
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to close metastore connection", e);
-    }
-  }
-
-  /**
-   * check if the current schema version in metastore matches the Hive version
-   * @throws MetaException
-   */
-  void verifySchemaVersion() throws HiveMetaException {
-    // don't check version if its a dry run
-    if (dryRun) {
-      return;
-    }
-    String newSchemaVersion = metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(false));
-    // verify that the new version is added to schema
-    assertCompatibleVersion(metaStoreSchemaInfo.getHiveSchemaVersion(), newSchemaVersion);
-  }
-
-  void assertCompatibleVersion(String hiveSchemaVersion, String dbSchemaVersion)
-      throws HiveMetaException {
-    if (!metaStoreSchemaInfo.isVersionCompatible(hiveSchemaVersion, dbSchemaVersion)) {
-      throw new HiveMetaException("Metastore schema version is not compatible. Hive Version: "
-          + hiveSchemaVersion + ", Database Schema Version: " + dbSchemaVersion);
-    }
-  }
-
-  MetaStoreConnectionInfo getConnectionInfo(boolean printInfo) {
-    return new MetaStoreConnectionInfo(userName, passWord, url, driver, printInfo, hiveConf,
+  @Override
+  protected MetaStoreConnectionInfo getConnectionInfo(boolean printInfo) {
+    return new MetaStoreConnectionInfo(userName, passWord, url, driver, printInfo, conf,
         dbType, metaDbType);
   }
 
-  // Quote if the database requires it
-  String quote(String stmt) {
-    stmt = stmt.replace("<q>", needsQuotedIdentifier ? quoteCharacter : "");
-    stmt = stmt.replace("<qa>", quoteCharacter);
-    return stmt;
-  }
-
   /***
    * Run beeline with the given metastore script. Flatten the nested scripts
    * into single file.
    */
-  void runBeeLine(String scriptDir, String scriptFile)
+  @Override
+  protected void execSql(String scriptDir, String scriptFile)
       throws IOException, HiveMetaException {
     NestedScriptParser dbCommandParser = getDbCommandParser(dbType, metaDbType);
 
@@ -237,12 +80,12 @@ public class HiveSchemaTool {
     out.write(sqlCommands);
     out.write("!closeall" + System.getProperty("line.separator"));
     out.close();
-    runBeeLine(tmpFile.getPath());
+    execSql(tmpFile.getPath());
   }
 
-  // Generate the beeline args per hive conf and execute the given script
-  void runBeeLine(String sqlScriptFile) throws IOException {
-    CommandBuilder builder = new CommandBuilder(hiveConf, url, driver,
+  @Override
+  protected void execSql(String sqlScriptFile) throws IOException {
+    CommandBuilder builder = new HiveSchemaToolCommandBuilder(conf, url, driver,
         userName, passWord, sqlScriptFile);
 
     // run the script using Beeline
@@ -264,34 +107,15 @@ public class HiveSchemaTool {
     }
   }
 
-  static class CommandBuilder {
-    private final String userName;
-    private final String password;
-    private final String sqlScriptFile;
-    private final String driver;
-    private final String url;
-
-    CommandBuilder(HiveConf hiveConf, String url, String driver, String userName, String password,
-        String sqlScriptFile) throws IOException {
-      this.userName = userName;
-      this.password = password;
-      this.url = url == null ?
-          HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECT_URL_KEY, hiveConf) : url;
-      this.driver = driver == null ?
-          HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECTION_DRIVER, hiveConf) : driver;
-      this.sqlScriptFile = sqlScriptFile;
-    }
-
-    String[] buildToRun() {
-      return argsWith(password);
-    }
+  static class HiveSchemaToolCommandBuilder extends MetastoreSchemaTool.CommandBuilder {
 
-    String buildToLog() throws IOException {
-      logScript();
-      return StringUtils.join(argsWith(BeeLine.PASSWD_MASK), " ");
+    HiveSchemaToolCommandBuilder(Configuration conf, String url, String driver, String userName,
+                                 String password, String sqlScriptFile) throws IOException {
+      super(conf, url, driver, userName, password, sqlScriptFile);
     }
 
-    private String[] argsWith(String password) {
+    @Override
+    protected String[] argsWith(String password) {
       return new String[]
           {
             "-u", url,
@@ -301,115 +125,17 @@ public class HiveSchemaTool {
             "-f", sqlScriptFile
           };
     }
-
-    private void logScript() throws IOException {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Going to invoke file that contains:");
-        try (BufferedReader reader = new BufferedReader(new FileReader(sqlScriptFile))) {
-          String line;
-          while ((line = reader.readLine()) != null) {
-            LOG.debug("script: " + line);
-          }
-        }
-      }
-    }
   }
 
   public static void main(String[] args) {
-    HiveSchemaToolCommandLine line = null;
-    try {
-      line = new HiveSchemaToolCommandLine(args);
-    } catch (ParseException e) {
-      System.exit(1);
-    }
-
+    MetastoreSchemaTool tool = new HiveSchemaTool();
+    OptionGroup additionalGroup = new OptionGroup();
+    Option metaDbTypeOpt = OptionBuilder.withArgName("metaDatabaseType")
+        .hasArgs().withDescription("Used only if upgrading the system catalog for hive")
+        .create("metaDbType");
+    additionalGroup.addOption(metaDbTypeOpt);
     System.setProperty(MetastoreConf.ConfVars.SCHEMA_VERIFICATION.getVarname(), "true");
-    try {
-      HiveSchemaTool schemaTool = createSchemaTool(line);
-
-      HiveSchemaToolTask task = null;
-      if (line.hasOption("info")) {
-        task = new HiveSchemaToolTaskInfo();
-      } else if (line.hasOption("upgradeSchema") || line.hasOption("upgradeSchemaFrom")) {
-        task = new HiveSchemaToolTaskUpgrade();
-      } else if (line.hasOption("initSchema") || line.hasOption("initSchemaTo")) {
-        task = new HiveSchemaToolTaskInit();
-      } else if (line.hasOption("validate")) {
-        task = new HiveSchemaToolTaskValidate();
-      } else if (line.hasOption("createCatalog")) {
-        task = new HiveSchemaToolTaskCreateCatalog();
-      } else if (line.hasOption("alterCatalog")) {
-        task = new HiveSchemaToolTaskAlterCatalog();
-      } else if (line.hasOption("moveDatabase")) {
-        task = new HiveSchemaToolTaskMoveDatabase();
-      } else if (line.hasOption("moveTable")) {
-        task = new HiveSchemaToolTaskMoveTable();
-      } else {
-        throw new HiveMetaException("No task defined!");
-      }
-
-      task.setHiveSchemaTool(schemaTool);
-      task.setCommandLineArguments(line);
-      task.execute();
-
-    } catch (HiveMetaException e) {
-      System.err.println(e);
-      if (e.getCause() != null) {
-        Throwable t = e.getCause();
-        System.err.println("Underlying cause: " + t.getClass().getName() + " : " + t.getMessage());
-        if (e.getCause() instanceof SQLException) {
-          System.err.println("SQL Error code: " + ((SQLException)t).getErrorCode());
-        }
-      }
-      if (line.hasOption("verbose")) {
-        e.printStackTrace();
-      } else {
-        System.err.println("Use --verbose for detailed stacktrace.");
-      }
-      System.err.println("*** schemaTool failed ***");
-      System.exit(1);
-    }
-    System.out.println("schemaTool completed");
-    System.exit(0);
-  }
-
-  private static HiveSchemaTool createSchemaTool(HiveSchemaToolCommandLine line) throws HiveMetaException {
-    HiveSchemaTool schemaTool = new HiveSchemaTool(line.getDbType(), line.getMetaDbType());
-
-    if (line.hasOption("userName")) {
-      schemaTool.setUserName(line.getOptionValue("userName"));
-    } else {
-      schemaTool.setUserName(
-          schemaTool.getHiveConf().get(MetastoreConf.ConfVars.CONNECTION_USER_NAME.getVarname()));
-    }
-    if (line.hasOption("passWord")) {
-      schemaTool.setPassWord(line.getOptionValue("passWord"));
-    } else {
-      try {
-        schemaTool.setPassWord(ShimLoader.getHadoopShims().getPassword(schemaTool.getHiveConf(),
-            MetastoreConf.ConfVars.PWD.getVarname()));
-      } catch (IOException err) {
-        throw new HiveMetaException("Error getting metastore password", err);
-      }
-    }
-    if (line.hasOption("url")) {
-      schemaTool.setUrl(line.getOptionValue("url"));
-    }
-    if (line.hasOption("driver")) {
-      schemaTool.setDriver(line.getOptionValue("driver"));
-    }
-    if (line.hasOption("dryRun")) {
-      schemaTool.setDryRun(true);
-    }
-    if (line.hasOption("verbose")) {
-      schemaTool.setVerbose(true);
-    }
-    if (line.hasOption("dbOpts")) {
-      schemaTool.setDbOpts(line.getOptionValue("dbOpts"));
-    }
-    if (line.hasOption("validate") && line.hasOption("servers")) {
-      schemaTool.setValidationServers(line.getOptionValue("servers"));
-    }
-    return schemaTool;
+    System.exit(tool.run(System.getenv("HIVE_HOME"), args, additionalGroup,
+        new HiveConf(HiveSchemaTool.class)));
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolCommandLine.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolCommandLine.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolCommandLine.java
deleted file mode 100644
index 8ca8343..0000000
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolCommandLine.java
+++ /dev/null
@@ -1,286 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-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.hadoop.hive.metastore.tools.HiveSchemaHelper;
-
-import com.google.common.collect.ImmutableSet;
-
-import java.util.Set;
-
-class HiveSchemaToolCommandLine {
-  private final Options cmdLineOptions = createOptions();
-
-  @SuppressWarnings("static-access")
-  private Options createOptions() {
-    Option help = new Option("help", "print this message");
-    Option infoOpt = new Option("info", "Show config and schema details");
-    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 validateOpt = new Option("validate", "Validate the database");
-    Option createCatalog = OptionBuilder
-        .hasArg()
-        .withDescription("Create a catalog, requires --catalogLocation parameter as well")
-        .create("createCatalog");
-    Option alterCatalog = OptionBuilder
-        .hasArg()
-        .withDescription("Alter a catalog, requires --catalogLocation and/or --catalogDescription parameter as well")
-        .create("alterCatalog");
-    Option moveDatabase = OptionBuilder
-        .hasArg()
-        .withDescription("Move a database between catalogs.  Argument is the database name. " +
-            "Requires --fromCatalog and --toCatalog parameters as well")
-        .create("moveDatabase");
-    Option moveTable = OptionBuilder
-        .hasArg()
-        .withDescription("Move a table to a different database.  Argument is the table name. " +
-            "Requires --fromCatalog, --toCatalog, --fromDatabase, and --toDatabase " +
-            " parameters as well.")
-        .create("moveTable");
-
-    OptionGroup optGroup = new OptionGroup();
-    optGroup
-      .addOption(help)
-      .addOption(infoOpt)
-      .addOption(upgradeOpt)
-      .addOption(upgradeFromOpt)
-      .addOption(initOpt)
-      .addOption(initToOpt)
-      .addOption(validateOpt)
-      .addOption(createCatalog)
-      .addOption(alterCatalog)
-      .addOption(moveDatabase)
-      .addOption(moveTable);
-    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").isRequired()
-        .create("dbType");
-    Option metaDbTypeOpt = OptionBuilder.withArgName("metaDatabaseType")
-        .hasArgs().withDescription("Used only if upgrading the system catalog for hive")
-        .create("metaDbType");
-    Option urlOpt = OptionBuilder.withArgName("url")
-        .hasArgs().withDescription("connection url to the database")
-        .create("url");
-    Option driverOpt = OptionBuilder.withArgName("driver")
-        .hasArgs().withDescription("driver name for connection")
-        .create("driver");
-    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");
-    Option serversOpt = OptionBuilder.withArgName("serverList")
-        .hasArgs().withDescription("a comma-separated list of servers used in location validation in the format of " +
-            "scheme://authority (e.g. hdfs://localhost:8000)")
-        .create("servers");
-    Option catalogLocation = OptionBuilder
-        .hasArg()
-        .withDescription("Location of new catalog, required when adding a catalog")
-        .create("catalogLocation");
-    Option catalogDescription = OptionBuilder
-        .hasArg()
-        .withDescription("Description of new catalog")
-        .create("catalogDescription");
-    Option ifNotExists = OptionBuilder
-        .withDescription("If passed then it is not an error to create an existing catalog")
-        .create("ifNotExists");
-    Option fromCatalog = OptionBuilder
-        .hasArg()
-        .withDescription("Catalog a moving database or table is coming from.  This is " +
-            "required if you are moving a database or table.")
-        .create("fromCatalog");
-    Option toCatalog = OptionBuilder
-        .hasArg()
-        .withDescription("Catalog a moving database or table is going to.  This is " +
-            "required if you are moving a database or table.")
-        .create("toCatalog");
-    Option fromDatabase = OptionBuilder
-        .hasArg()
-        .withDescription("Database a moving table is coming from.  This is " +
-            "required if you are moving a table.")
-        .create("fromDatabase");
-    Option toDatabase = OptionBuilder
-        .hasArg()
-        .withDescription("Database a moving table is going to.  This is " +
-            "required if you are moving a table.")
-        .create("toDatabase");
-
-    Options options = new Options();
-    options.addOption(help);
-    options.addOptionGroup(optGroup);
-    options.addOption(dbTypeOpt);
-    options.addOption(metaDbTypeOpt);
-    options.addOption(userNameOpt);
-    options.addOption(passwdOpt);
-    options.addOption(urlOpt);
-    options.addOption(driverOpt);
-    options.addOption(dbOpts);
-    options.addOption(dryRunOpt);
-    options.addOption(verboseOpt);
-    options.addOption(serversOpt);
-    options.addOption(catalogLocation);
-    options.addOption(catalogDescription);
-    options.addOption(ifNotExists);
-    options.addOption(fromCatalog);
-    options.addOption(toCatalog);
-    options.addOption(fromDatabase);
-    options.addOption(toDatabase);
-
-    return options;
-  }
-
-  private final CommandLine cl;
-  private final String dbType;
-  private final String metaDbType;
-
-  HiveSchemaToolCommandLine(String[] args) throws ParseException {
-    cl = getCommandLine(args);
-    if (cl.hasOption("help")) {
-      printAndExit(null);
-    }
-
-    dbType = cl.getOptionValue("dbType");
-    metaDbType = cl.getOptionValue("metaDbType");
-
-    validate();
-  }
-
-  private CommandLine getCommandLine(String[] args)  throws ParseException {
-    try {
-      CommandLineParser parser = new GnuParser();
-      return parser.parse(cmdLineOptions, args);
-    } catch (ParseException e) {
-      printAndExit("HiveSchemaTool:Parsing failed. Reason: " + e.getLocalizedMessage());
-      return null;
-    }
-  }
-
-  private static final Set<String> VALID_DB_TYPES = ImmutableSet.of(HiveSchemaHelper.DB_DERBY,
-      HiveSchemaHelper.DB_HIVE, HiveSchemaHelper.DB_MSSQL, HiveSchemaHelper.DB_MYSQL,
-      HiveSchemaHelper.DB_POSTGRACE, HiveSchemaHelper.DB_ORACLE);
-
-  private static final Set<String> VALID_META_DB_TYPES = ImmutableSet.of(HiveSchemaHelper.DB_DERBY,
-      HiveSchemaHelper.DB_MSSQL, HiveSchemaHelper.DB_MYSQL, HiveSchemaHelper.DB_POSTGRACE,
-      HiveSchemaHelper.DB_ORACLE);
-
-  private void validate() throws ParseException {
-    if (!VALID_DB_TYPES.contains(dbType)) {
-      printAndExit("Unsupported dbType " + dbType);
-    }
-
-    if (metaDbType != null) {
-      if (!dbType.equals(HiveSchemaHelper.DB_HIVE)) {
-        printAndExit("metaDbType may only be set if dbType is hive");
-      }
-      if (!VALID_META_DB_TYPES.contains(metaDbType)) {
-        printAndExit("Unsupported metaDbType " + metaDbType);
-      }
-    } else if (dbType.equalsIgnoreCase(HiveSchemaHelper.DB_HIVE)) {
-      System.err.println();
-      printAndExit("metaDbType must be set if dbType is hive");
-    }
-
-    if ((cl.hasOption("createCatalog")) && !cl.hasOption("catalogLocation")) {
-      System.err.println();
-      printAndExit("catalogLocation must be set for createCatalog");
-    }
-
-    if (!cl.hasOption("createCatalog") && !cl.hasOption("alterCatalog") &&
-        (cl.hasOption("catalogLocation") || cl.hasOption("catalogDescription"))) {
-      printAndExit("catalogLocation and catalogDescription may be set only for createCatalog and alterCatalog");
-    }
-
-    if (!cl.hasOption("createCatalog") && cl.hasOption("ifNotExists")) {
-      printAndExit("ifNotExists may be set only for createCatalog");
-    }
-
-    if (cl.hasOption("moveDatabase") &&
-        (!cl.hasOption("fromCatalog") || !cl.hasOption("toCatalog"))) {
-      printAndExit("fromCatalog and toCatalog must be set for moveDatabase");
-    }
-
-    if (cl.hasOption("moveTable") &&
-        (!cl.hasOption("fromCatalog") || !cl.hasOption("toCatalog") ||
-         !cl.hasOption("fromDatabase") || !cl.hasOption("toDatabase"))) {
-      printAndExit("fromCatalog, toCatalog, fromDatabase and toDatabase must be set for moveTable");
-    }
-
-    if ((!cl.hasOption("moveDatabase") && !cl.hasOption("moveTable")) &&
-        (cl.hasOption("fromCatalog") || cl.hasOption("toCatalog"))) {
-      printAndExit("fromCatalog and toCatalog may be set only for moveDatabase and moveTable");
-    }
-
-    if (!cl.hasOption("moveTable") &&
-        (cl.hasOption("fromDatabase") || cl.hasOption("toDatabase"))) {
-      printAndExit("fromDatabase and toDatabase may be set only for moveTable");
-    }
-  }
-
-  private void printAndExit(String reason) throws ParseException {
-    if (reason != null) {
-      System.err.println(reason);
-    }
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp("schemaTool", cmdLineOptions);
-    if (reason != null) {
-      throw new ParseException(reason);
-    } else {
-      System.exit(0);
-    }
-  }
-
-  String getDbType() {
-    return dbType;
-  }
-
-  String getMetaDbType() {
-    return metaDbType;
-  }
-
-  boolean hasOption(String opt) {
-    return cl.hasOption(opt);
-  }
-
-  String getOptionValue(String opt) {
-    return cl.getOptionValue(opt);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTask.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTask.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTask.java
deleted file mode 100644
index 3172c2f..0000000
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTask.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-
-abstract class HiveSchemaToolTask {
-  protected HiveSchemaTool schemaTool;
-
-  void setHiveSchemaTool(HiveSchemaTool schemaTool) {
-    this.schemaTool = schemaTool;
-  }
-
-  abstract void setCommandLineArguments(HiveSchemaToolCommandLine cl);
-
-  abstract void execute() throws HiveMetaException;
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskAlterCatalog.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskAlterCatalog.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskAlterCatalog.java
deleted file mode 100644
index 802fe3a..0000000
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskAlterCatalog.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
-
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Print Hive version and schema version.
- */
-class HiveSchemaToolTaskAlterCatalog extends HiveSchemaToolTask {
-  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaToolTaskAlterCatalog.class.getName());
-
-  private String catName;
-  private String location;
-  private String description;
-
-  @Override
-  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
-    catName = normalizeIdentifier(cl.getOptionValue("alterCatalog"));
-    location = cl.getOptionValue("catalogLocation");
-    description = cl.getOptionValue("catalogDescription");
-  }
-
-  private static final String UPDATE_CATALOG_STMT =
-      "update <q>CTLGS<q> " +
-      "   set <q>LOCATION_URI<q> = %s, " +
-      "       <qa>DESC<qa> = %s " +
-      " where <q>NAME<q> = '%s'";
-
-  @Override
-  void execute() throws HiveMetaException {
-    if (location == null && description == null) {
-      throw new HiveMetaException("Asked to update catalog " + catName + " but not given any changes to update");
-    }
-    System.out.println("Updating catalog " + catName);
-
-    Connection conn = schemaTool.getConnectionToMetastore(true);
-    boolean success = false;
-    try {
-      conn.setAutoCommit(false);
-      try (Statement stmt = conn.createStatement()) {
-        Object updateLocation = location == null ? schemaTool.quote("<q>LOCATION_URI<q>") : "'" + location + "'";
-        Object updateDescription = description == null ? schemaTool.quote("<qa>DESC<qa>") : "'" + description + "'";
-        String update = String.format(schemaTool.quote(UPDATE_CATALOG_STMT), updateLocation, updateDescription,
-            catName);
-        LOG.debug("Going to run " + update);
-        int count = stmt.executeUpdate(update);
-        if (count != 1) {
-          throw new HiveMetaException("Failed to find catalog " + catName + " to update");
-        }
-        conn.commit();
-        success = true;
-      }
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to update catalog", e);
-    } finally {
-      try {
-        if (!success) {
-          conn.rollback();
-        }
-      } catch (SQLException e) {
-        // Not really much we can do here.
-        LOG.error("Failed to rollback, everything will probably go bad from here.", e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskCreateCatalog.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskCreateCatalog.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskCreateCatalog.java
deleted file mode 100644
index 810dfea..0000000
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskCreateCatalog.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
-
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Print Hive version and schema version.
- */
-class HiveSchemaToolTaskCreateCatalog extends HiveSchemaToolTask {
-  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaToolTaskCreateCatalog.class.getName());
-
-  private String catName;
-  private String location;
-  private String description;
-  private boolean ifNotExists;
-
-  @Override
-  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
-    catName = normalizeIdentifier(cl.getOptionValue("createCatalog"));
-    location = cl.getOptionValue("catalogLocation");
-    description = cl.getOptionValue("catalogDescription");
-    ifNotExists = cl.hasOption("ifNotExists");
-  }
-
-  @Override
-  void execute() throws HiveMetaException {
-    System.out.println("Create catalog " + catName + " at location " + location);
-
-    Connection conn = schemaTool.getConnectionToMetastore(true);
-    boolean success = false;
-    try {
-      conn.setAutoCommit(false);
-      try (Statement stmt = conn.createStatement()) {
-        // If they set ifNotExists check for existence first, and bail if it exists.  This is
-        // more reliable then attempting to parse the error message from the SQLException.
-        if (ifNotExists && catalogExists(stmt)) {
-          return;
-        }
-
-        int catNum = getNextCatalogId(stmt);
-        addCatalog(conn, stmt, catNum);
-        success = true;
-      }
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to add catalog", e);
-    } finally {
-      try {
-        if (!success) {
-          conn.rollback();
-        }
-      } catch (SQLException e) {
-        // Not really much we can do here.
-        LOG.error("Failed to rollback, everything will probably go bad from here.", e);
-      }
-    }
-  }
-
-  private static final String CATALOG_EXISTS_QUERY =
-      "select <q>NAME<q> " +
-      "  from <q>CTLGS<q> " +
-      " where <q>NAME<q> = '%s'";
-
-  private boolean catalogExists(Statement stmt) throws SQLException {
-    String query = String.format(schemaTool.quote(CATALOG_EXISTS_QUERY), catName);
-    LOG.debug("Going to run " + query);
-    try (ResultSet rs = stmt.executeQuery(query)) {
-      if (rs.next()) {
-        System.out.println("Catalog " + catName + " already exists");
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-  private static final String NEXT_CATALOG_ID_QUERY =
-      "select max(<q>CTLG_ID<q>) " +
-      "  from <q>CTLGS<q>";
-
-  private int getNextCatalogId(Statement stmt) throws SQLException, HiveMetaException {
-    String query = schemaTool.quote(NEXT_CATALOG_ID_QUERY);
-    LOG.debug("Going to run " + query);
-    try (ResultSet rs = stmt.executeQuery(query)) {
-      if (!rs.next()) {
-        throw new HiveMetaException("No catalogs found, have you upgraded the database?");
-      }
-      int nextId = rs.getInt(1) + 1;
-      // We need to stay out of the way of any sequences used by the underlying database.
-      // Otherwise the next time the client tries to add a catalog we'll get an error.
-      // There should never be billions of catalogs, so we'll shift our sequence number up
-      // there to avoid clashes.
-      int floor = 1 << 30;
-      return Math.max(nextId, floor);
-    }
-  }
-
-  private static final String ADD_CATALOG_STMT =
-      "insert into <q>CTLGS<q> (<q>CTLG_ID<q>, <q>NAME<q>, <qa>DESC<qa>, <q>LOCATION_URI<q>) " +
-      "     values (%d, '%s', '%s', '%s')";
-
-  private void addCatalog(Connection conn, Statement stmt, int catNum) throws SQLException {
-    String update = String.format(schemaTool.quote(ADD_CATALOG_STMT), catNum, catName, description, location);
-    LOG.debug("Going to run " + update);
-    stmt.execute(update);
-    conn.commit();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInfo.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInfo.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInfo.java
deleted file mode 100644
index b70ea87..0000000
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInfo.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
-
-/**
- * Print Hive version and schema version.
- */
-class HiveSchemaToolTaskInfo extends HiveSchemaToolTask {
-  @Override
-  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
-    // do nothing
-  }
-
-  @Override
-  void execute() throws HiveMetaException {
-    String hiveVersion = schemaTool.getMetaStoreSchemaInfo().getHiveSchemaVersion();
-    MetaStoreConnectionInfo connectionInfo = schemaTool.getConnectionInfo(true);
-    String dbVersion = schemaTool.getMetaStoreSchemaInfo().getMetaStoreSchemaVersion(connectionInfo);
-
-    System.out.println("Hive distribution version:\t " + hiveVersion);
-    System.out.println("Metastore schema version:\t " + dbVersion);
-
-    schemaTool.assertCompatibleVersion(hiveVersion, dbVersion);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInit.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInit.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInit.java
deleted file mode 100644
index 40fd1e7..0000000
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInit.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-
-/**
- * Initialize the metastore schema.
- */
-class HiveSchemaToolTaskInit extends HiveSchemaToolTask {
-  private boolean validate = true;
-  private String toVersion;
-
-  @Override
-  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
-    if (cl.hasOption("initSchemaTo")) {
-      this.toVersion = cl.getOptionValue("initSchemaTo");
-      this.validate = false;
-    }
-  }
-
-  private void ensureToVersion() throws HiveMetaException {
-    if (toVersion != null) {
-      return;
-    }
-
-    // If null then current hive version is used
-    toVersion = schemaTool.getMetaStoreSchemaInfo().getHiveSchemaVersion();
-    System.out.println("Initializing the schema to: " + toVersion);
-  }
-
-  @Override
-  void execute() throws HiveMetaException {
-    ensureToVersion();
-
-    schemaTool.testConnectionToMetastore();
-    System.out.println("Starting metastore schema initialization to " + toVersion);
-
-    String initScriptDir = schemaTool.getMetaStoreSchemaInfo().getMetaStoreScriptDir();
-    String initScriptFile = schemaTool.getMetaStoreSchemaInfo().generateInitFileName(toVersion);
-
-    try {
-      System.out.println("Initialization script " + initScriptFile);
-      if (!schemaTool.isDryRun()) {
-        schemaTool.runBeeLine(initScriptDir, initScriptFile);
-        System.out.println("Initialization script completed");
-      }
-    } catch (IOException e) {
-      throw new HiveMetaException("Schema initialization FAILED! Metastore state would be inconsistent!", e);
-    }
-
-    if (validate) {
-      schemaTool.verifySchemaVersion();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveDatabase.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveDatabase.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveDatabase.java
deleted file mode 100644
index ee7c0f8..0000000
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveDatabase.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
-
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Print Hive version and schema version.
- */
-class HiveSchemaToolTaskMoveDatabase extends HiveSchemaToolTask {
-  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaToolTaskMoveDatabase.class.getName());
-
-  private String fromCatName;
-  private String toCatName;
-  private String dbName;
-
-  @Override
-  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
-    fromCatName = normalizeIdentifier(cl.getOptionValue("fromCatalog"));
-    toCatName = normalizeIdentifier(cl.getOptionValue("toCatalog"));
-    dbName = normalizeIdentifier(cl.getOptionValue("moveDatabase"));
-  }
-
-  @Override
-  void execute() throws HiveMetaException {
-    System.out.println(String.format("Moving database %s from catalog %s to catalog %s",
-        dbName, fromCatName, toCatName));
-    Connection conn = schemaTool.getConnectionToMetastore(true);
-    boolean success = false;
-    try {
-      conn.setAutoCommit(false);
-      try (Statement stmt = conn.createStatement()) {
-        updateCatalogNameInTable(stmt, "DBS", "CTLG_NAME", "NAME", fromCatName, toCatName, dbName, false);
-        updateCatalogNameInTable(stmt, "TAB_COL_STATS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
-        updateCatalogNameInTable(stmt, "PART_COL_STATS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
-        updateCatalogNameInTable(stmt, "PARTITION_EVENTS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
-        updateCatalogNameInTable(stmt, "NOTIFICATION_LOG", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
-        conn.commit();
-        success = true;
-      }
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to move database", e);
-    } finally {
-      try {
-        if (!success) {
-          conn.rollback();
-        }
-      } catch (SQLException e) {
-        // Not really much we can do here.
-        LOG.error("Failed to rollback, everything will probably go bad from here.");
-      }
-    }
-  }
-
-  private static final String UPDATE_CATALOG_NAME_STMT =
-      "update <q>%s<q> " +
-      "   set <q>%s<q> = '%s' " +
-      " where <q>%s<q> = '%s' " +
-      "   and <q>%s<q> = '%s'";
-
-  private void updateCatalogNameInTable(Statement stmt, String tableName, String catColName, String dbColName,
-      String fromCatName, String toCatName, String dbName, boolean zeroUpdatesOk)
-      throws HiveMetaException, SQLException {
-    String update = String.format(schemaTool.quote(UPDATE_CATALOG_NAME_STMT), tableName, catColName, toCatName,
-        catColName, fromCatName, dbColName, dbName);
-    LOG.debug("Going to run " + update);
-    int numUpdated = stmt.executeUpdate(update);
-    if (numUpdated != 1 && !(zeroUpdatesOk && numUpdated == 0)) {
-      throw new HiveMetaException("Failed to properly update the " + tableName +
-          " table.  Expected to update 1 row but instead updated " + numUpdated);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveTable.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveTable.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveTable.java
deleted file mode 100644
index fcefef8..0000000
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveTable.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
-
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Print Hive version and schema version.
- */
-class HiveSchemaToolTaskMoveTable extends HiveSchemaToolTask {
-  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaToolTaskMoveTable.class.getName());
-
-  private String fromCat;
-  private String toCat;
-  private String fromDb;
-  private String toDb;
-  private String tableName;
-
-  @Override
-  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
-    fromCat = normalizeIdentifier(cl.getOptionValue("fromCatalog"));
-    toCat = normalizeIdentifier(cl.getOptionValue("toCatalog"));
-    fromDb = normalizeIdentifier(cl.getOptionValue("fromDatabase"));
-    toDb = normalizeIdentifier(cl.getOptionValue("toDatabase"));
-    tableName = normalizeIdentifier(cl.getOptionValue("moveTable"));
-  }
-
-  @Override
-  void execute() throws HiveMetaException {
-    Connection conn = schemaTool.getConnectionToMetastore(true);
-    boolean success = false;
-    try {
-      conn.setAutoCommit(false);
-      try (Statement stmt = conn.createStatement()) {
-        updateTableId(stmt);
-        updateDbNameForTable(stmt, "TAB_COL_STATS", "TABLE_NAME", fromCat, toCat, fromDb, toDb, tableName);
-        updateDbNameForTable(stmt, "PART_COL_STATS", "TABLE_NAME", fromCat, toCat, fromDb, toDb, tableName);
-        updateDbNameForTable(stmt, "PARTITION_EVENTS", "TBL_NAME", fromCat, toCat, fromDb, toDb, tableName);
-        updateDbNameForTable(stmt, "NOTIFICATION_LOG", "TBL_NAME", fromCat, toCat, fromDb, toDb, tableName);
-        conn.commit();
-        success = true;
-      }
-    } catch (SQLException se) {
-      throw new HiveMetaException("Failed to move table", se);
-    } finally {
-      try {
-        if (!success) {
-          conn.rollback();
-        }
-      } catch (SQLException e) {
-        // Not really much we can do here.
-        LOG.error("Failed to rollback, everything will probably go bad from here.");
-      }
-
-    }
-  }
-
-  private static final String UPDATE_TABLE_ID_STMT =
-      "update <q>TBLS<q> " +
-      "   set <q>DB_ID<q> = %d " +
-      " where <q>DB_ID<q> = %d " +
-      "   and <q>TBL_NAME<q> = '%s'";
-
-  private void updateTableId(Statement stmt) throws SQLException, HiveMetaException {
-    // Find the old database id
-    long oldDbId = getDbId(stmt, fromDb, fromCat);
-
-    // Find the new database id
-    long newDbId = getDbId(stmt, toDb, toCat);
-
-    String update = String.format(schemaTool.quote(UPDATE_TABLE_ID_STMT), newDbId, oldDbId, tableName);
-    LOG.debug("Going to run " + update);
-    int numUpdated = stmt.executeUpdate(update);
-    if (numUpdated != 1) {
-      throw new HiveMetaException(
-          "Failed to properly update TBLS table.  Expected to update " +
-              "1 row but instead updated " + numUpdated);
-    }
-  }
-
-  private static final String DB_ID_QUERY =
-      "select <q>DB_ID<q> " +
-      "  from <q>DBS<q> " +
-      " where <q>NAME<q> = '%s' " +
-      "   and <q>CTLG_NAME<q> = '%s'";
-
-  private long getDbId(Statement stmt, String db, String catalog) throws SQLException, HiveMetaException {
-    String query = String.format(schemaTool.quote(DB_ID_QUERY), db, catalog);
-    LOG.debug("Going to run " + query);
-    try (ResultSet rs = stmt.executeQuery(query)) {
-      if (!rs.next()) {
-        throw new HiveMetaException("Unable to find database " + fromDb);
-      }
-      return rs.getLong(1);
-    }
-  }
-
-  private static final String UPDATE_DB_NAME_STMT =
-      "update <q>%s<q> " +
-      "   set <q>CAT_NAME<q> = '%s', " +
-      "       <q>DB_NAME<q> = '%s' " +
-      " where <q>CAT_NAME<q> = '%s' " +
-      "   and <q>DB_NAME<q> = '%s' " +
-      "   and <q>%s<q> = '%s'";
-
-  private void updateDbNameForTable(Statement stmt, String tableName, String tableColumnName, String fromCat,
-      String toCat, String fromDb, String toDb, String hiveTblName) throws HiveMetaException, SQLException {
-    String update = String.format(schemaTool.quote(UPDATE_DB_NAME_STMT), tableName, toCat, toDb, fromCat, fromDb,
-        tableColumnName, hiveTblName);
-
-    LOG.debug("Going to run " + update);
-    int numUpdated = stmt.executeUpdate(update);
-    if (numUpdated > 1 || numUpdated < 0) {
-      throw new HiveMetaException("Failed to properly update the " + tableName +
-          " table.  Expected to update 1 row but instead updated " + numUpdated);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskUpgrade.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskUpgrade.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskUpgrade.java
deleted file mode 100644
index fa4742f..0000000
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskUpgrade.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
-
-/**
- * Perform metastore schema upgrade.
- */
-class HiveSchemaToolTaskUpgrade extends HiveSchemaToolTask {
-  private String fromVersion;
-
-  @Override
-  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
-    if (cl.hasOption("upgradeSchemaFrom")) {
-      this.fromVersion = cl.getOptionValue("upgradeSchemaFrom");
-    }
-  }
-
-  private void ensureFromVersion() throws HiveMetaException {
-    if (fromVersion != null) {
-      return;
-    }
-
-    // If null, then read from the metastore
-    MetaStoreConnectionInfo connectionInfo = schemaTool.getConnectionInfo(false);
-    fromVersion = schemaTool.getMetaStoreSchemaInfo().getMetaStoreSchemaVersion(connectionInfo);
-    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");
-    }
-    System.out.println("Upgrading from the version " + fromVersion);
-  }
-
-  @Override
-  void execute() throws HiveMetaException {
-    ensureFromVersion();
-
-    if (schemaTool.getMetaStoreSchemaInfo().getHiveSchemaVersion().equals(fromVersion)) {
-      System.out.println("No schema upgrade required from version " + fromVersion);
-      return;
-    }
-
-    // Find the list of scripts to execute for this upgrade
-    List<String> upgradeScripts = schemaTool.getMetaStoreSchemaInfo().getUpgradeScripts(fromVersion);
-    schemaTool.testConnectionToMetastore();
-    System.out.println("Starting upgrade metastore schema from version " + fromVersion + " to " +
-        schemaTool.getMetaStoreSchemaInfo().getHiveSchemaVersion());
-    String scriptDir = schemaTool.getMetaStoreSchemaInfo().getMetaStoreScriptDir();
-    try {
-      for (String scriptFile : upgradeScripts) {
-        System.out.println("Upgrade script " + scriptFile);
-        if (!schemaTool.isDryRun()) {
-          runPreUpgrade(scriptDir, scriptFile);
-          schemaTool.runBeeLine(scriptDir, scriptFile);
-          System.out.println("Completed " + scriptFile);
-        }
-      }
-    } catch (IOException e) {
-      throw new HiveMetaException("Upgrade FAILED! Metastore state would be inconsistent !!", e);
-    }
-
-    // Revalidated the new version after upgrade
-    schemaTool.verifySchemaVersion();
-  }
-
-  /**
-   *  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 = schemaTool.getMetaStoreSchemaInfo().getPreUpgradeScriptName(i, scriptFile);
-      File preUpgradeScriptFile = new File(scriptDir, preUpgradeScript);
-      if (!preUpgradeScriptFile.isFile()) {
-        break;
-      }
-
-      try {
-        schemaTool.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 (schemaTool.isVerbose()) {
-          e.printStackTrace();
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskValidate.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskValidate.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskValidate.java
deleted file mode 100644
index c4f6d19..0000000
--- a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskValidate.java
+++ /dev/null
@@ -1,631 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.net.URI;
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.SQLFeatureNotSupportedException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.apache.hadoop.hive.metastore.TableType;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.NestedScriptParser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableMap;
-
-/**
- * Print Hive version and schema version.
- */
-class HiveSchemaToolTaskValidate extends HiveSchemaToolTask {
-  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaToolTaskValidate.class.getName());
-
-  @Override
-  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
-    // do nothing
-  }
-
-  @Override
-  void execute() throws HiveMetaException {
-    System.out.println("Starting metastore validation\n");
-    Connection conn = schemaTool.getConnectionToMetastore(false);
-    boolean success = true;
-    try {
-      success &= validateSchemaVersions();
-      success &= validateSequences(conn);
-      success &= validateSchemaTables(conn);
-      success &= validateLocations(conn, schemaTool.getValidationServers());
-      success &= validateColumnNullValues(conn);
-    } finally {
-      if (conn != null) {
-        try {
-          conn.close();
-        } catch (SQLException e) {
-          throw new HiveMetaException("Failed to close metastore connection", e);
-        }
-      }
-    }
-
-    System.out.print("Done with metastore validation: ");
-    if (!success) {
-      System.out.println("[FAIL]");
-      throw new HiveMetaException("Validation failed");
-    } else {
-      System.out.println("[SUCCESS]");
-    }
-  }
-
-  boolean validateSchemaVersions() throws HiveMetaException {
-    System.out.println("Validating schema version");
-    try {
-      String hiveSchemaVersion = schemaTool.getMetaStoreSchemaInfo().getHiveSchemaVersion();
-      MetaStoreConnectionInfo connectionInfo = schemaTool.getConnectionInfo(false);
-      String newSchemaVersion = schemaTool.getMetaStoreSchemaInfo().getMetaStoreSchemaVersion(connectionInfo);
-      schemaTool.assertCompatibleVersion(hiveSchemaVersion, newSchemaVersion);
-    } catch (HiveMetaException hme) {
-      if (hme.getMessage().contains("Metastore schema version is not compatible") ||
-          hme.getMessage().contains("Multiple versions were found in metastore") ||
-          hme.getMessage().contains("Could not find version info in metastore VERSION table")) {
-        System.err.println(hme.getMessage());
-        System.out.println("[FAIL]\n");
-        return false;
-      } else {
-        throw hme;
-      }
-    }
-    System.out.println("[SUCCESS]\n");
-    return true;
-  }
-
-  private static final String QUERY_SEQ =
-      "  select t.<q>NEXT_VAL<q>" +
-      "    from <q>SEQUENCE_TABLE<q> t " +
-      "   where t.<q>SEQUENCE_NAME<q> = ? " +
-      "order by t.<q>SEQUENCE_NAME<q>";
-
-  private static final String QUERY_MAX_ID =
-      "select max(<q>%s<q>)" +
-      "  from <q>%s<q>";
-
-  @VisibleForTesting
-  boolean validateSequences(Connection conn) throws HiveMetaException {
-    Map<String, Pair<String, String>> seqNameToTable =
-        new ImmutableMap.Builder<String, Pair<String, String>>()
-        .put("MDatabase", Pair.of("DBS", "DB_ID"))
-        .put("MRole", Pair.of("ROLES", "ROLE_ID"))
-        .put("MGlobalPrivilege", Pair.of("GLOBAL_PRIVS", "USER_GRANT_ID"))
-        .put("MTable", Pair.of("TBLS","TBL_ID"))
-        .put("MStorageDescriptor", Pair.of("SDS", "SD_ID"))
-        .put("MSerDeInfo", Pair.of("SERDES", "SERDE_ID"))
-        .put("MColumnDescriptor", Pair.of("CDS", "CD_ID"))
-        .put("MTablePrivilege", Pair.of("TBL_PRIVS", "TBL_GRANT_ID"))
-        .put("MTableColumnStatistics", Pair.of("TAB_COL_STATS", "CS_ID"))
-        .put("MPartition", Pair.of("PARTITIONS", "PART_ID"))
-        .put("MPartitionColumnStatistics", Pair.of("PART_COL_STATS", "CS_ID"))
-        .put("MFunction", Pair.of("FUNCS", "FUNC_ID"))
-        .put("MIndex", Pair.of("IDXS", "INDEX_ID"))
-        .put("MStringList", Pair.of("SKEWED_STRING_LIST", "STRING_LIST_ID"))
-        .build();
-
-    System.out.println("Validating sequence number for SEQUENCE_TABLE");
-
-    boolean isValid = true;
-    try {
-      Statement stmt = conn.createStatement();
-      for (Map.Entry<String, Pair<String, String>> e : seqNameToTable.entrySet()) {
-        String tableName = e.getValue().getLeft();
-        String tableKey = e.getValue().getRight();
-        String fullSequenceName = "org.apache.hadoop.hive.metastore.model." + e.getKey();
-        String seqQuery = schemaTool.quote(QUERY_SEQ);
-        String maxIdQuery = String.format(schemaTool.quote(QUERY_MAX_ID), tableKey, tableName);
-
-        ResultSet res = stmt.executeQuery(maxIdQuery);
-        if (res.next()) {
-          long maxId = res.getLong(1);
-          if (maxId > 0) {
-            PreparedStatement stmtSeq = conn.prepareStatement(seqQuery);
-            stmtSeq.setString(1, fullSequenceName);
-            ResultSet resSeq = stmtSeq.executeQuery();
-            if (!resSeq.next()) {
-              isValid = false;
-              System.err.println("Missing SEQUENCE_NAME " + e.getKey() + " from SEQUENCE_TABLE");
-            } else if (resSeq.getLong(1) < maxId) {
-              isValid = false;
-              System.err.println("NEXT_VAL for " + e.getKey() + " in SEQUENCE_TABLE < max(" + tableKey +
-                  ") in " + tableName);
-            }
-          }
-        }
-      }
-
-      System.out.println(isValid ? "[SUCCESS]\n" :"[FAIL]\n");
-      return isValid;
-    } catch (SQLException e) {
-        throw new HiveMetaException("Failed to validate sequence number for SEQUENCE_TABLE", e);
-    }
-  }
-
-  @VisibleForTesting
-  boolean validateSchemaTables(Connection conn) throws HiveMetaException {
-    System.out.println("Validating metastore schema tables");
-    String version = null;
-    try {
-      MetaStoreConnectionInfo connectionInfo = schemaTool.getConnectionInfo(false);
-      version = schemaTool.getMetaStoreSchemaInfo().getMetaStoreSchemaVersion(connectionInfo);
-    } catch (HiveMetaException he) {
-      System.err.println("Failed to determine schema version from Hive Metastore DB. " + he.getMessage());
-      System.out.println("Failed in schema table validation.");
-      LOG.debug("Failed to determine schema version from Hive Metastore DB," + he.getMessage(), he);
-      return false;
-    }
-
-    Connection hmsConn = schemaTool.getConnectionToMetastore(false);
-
-    LOG.debug("Validating tables in the schema for version " + version);
-    List<String> dbTables = new ArrayList<String>();
-    ResultSet rs = null;
-    try {
-      String schema = null;
-      try {
-        schema = hmsConn.getSchema();
-      } catch (SQLFeatureNotSupportedException e) {
-        LOG.debug("schema is not supported");
-      }
-
-      DatabaseMetaData metadata = conn.getMetaData();
-      rs = metadata.getTables(null, schema, "%", new String[] {"TABLE"});
-
-      while (rs.next()) {
-        String table = rs.getString("TABLE_NAME");
-        dbTables.add(table.toLowerCase());
-        LOG.debug("Found table " + table + " in HMS dbstore");
-      }
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to retrieve schema tables from Hive Metastore DB," +
-          e.getMessage(), e);
-    } finally {
-      if (rs != null) {
-        try {
-          rs.close();
-        } catch (SQLException e) {
-          throw new HiveMetaException("Failed to close resultset", e);
-        }
-      }
-    }
-
-    // parse the schema file to determine the tables that are expected to exist
-    // we are using oracle schema because it is simpler to parse, no quotes or backticks etc
-    List<String> schemaTables = new ArrayList<String>();
-    List<String> subScripts   = new ArrayList<String>();
-
-    String baseDir    = new File(schemaTool.getMetaStoreSchemaInfo().getMetaStoreScriptDir()).getParent();
-    String schemaFile = new File(schemaTool.getMetaStoreSchemaInfo().getMetaStoreScriptDir(),
-        schemaTool.getMetaStoreSchemaInfo().generateInitFileName(version)).getPath();
-    try {
-      LOG.debug("Parsing schema script " + schemaFile);
-      subScripts.addAll(findCreateTable(schemaFile, schemaTables));
-      while (subScripts.size() > 0) {
-        schemaFile = baseDir + "/" + schemaTool.getDbType() + "/" + subScripts.remove(0);
-        LOG.debug("Parsing subscript " + schemaFile);
-        subScripts.addAll(findCreateTable(schemaFile, schemaTables));
-      }
-    } catch (Exception e) {
-      System.err.println("Exception in parsing schema file. Cause:" + e.getMessage());
-      System.out.println("Failed in schema table validation.");
-      return false;
-    }
-
-    LOG.debug("Schema tables:[ " + Arrays.toString(schemaTables.toArray()) + " ]");
-    LOG.debug("DB tables:[ " + Arrays.toString(dbTables.toArray()) + " ]");
-
-    // now diff the lists
-    schemaTables.removeAll(dbTables);
-    if (schemaTables.size() > 0) {
-      Collections.sort(schemaTables);
-      System.err.println("Table(s) [ " + Arrays.toString(schemaTables.toArray()) + " ] " +
-          "are missing from the metastore database schema.");
-      System.out.println("[FAIL]\n");
-      return false;
-    } else {
-      System.out.println("[SUCCESS]\n");
-      return true;
-    }
-  }
-
-  @VisibleForTesting
-  List<String> findCreateTable(String path, List<String> tableList) throws Exception {
-    if (!(new File(path)).exists()) {
-      throw new Exception(path + " does not exist. Potentially incorrect version in the metastore VERSION table");
-    }
-
-    List<String> subs = new ArrayList<String>();
-    NestedScriptParser sp = HiveSchemaHelper.getDbCommandParser(schemaTool.getDbType(), false);
-    Pattern regexp = Pattern.compile("CREATE TABLE(\\s+IF NOT EXISTS)?\\s+(\\S+).*");
-
-    try (BufferedReader reader = new BufferedReader(new FileReader(path));) {
-      String line = null;
-      while ((line = reader.readLine()) != null) {
-        if (sp.isNestedScript(line)) {
-          String subScript = sp.getScriptName(line);
-          LOG.debug("Schema subscript " + subScript + " found");
-          subs.add(subScript);
-          continue;
-        }
-        line = line.replaceAll("( )+", " "); //suppress multi-spaces
-        line = line.replaceAll("\\(", " ");
-        line = line.replaceAll("IF NOT EXISTS ", "");
-        line = line.replaceAll("`", "");
-        line = line.replaceAll("'", "");
-        line = line.replaceAll("\"", "");
-        Matcher matcher = regexp.matcher(line);
-
-        if (matcher.find()) {
-          String table = matcher.group(2);
-          if (schemaTool.getDbType().equals("derby")) {
-            table = table.replaceAll("APP\\.", "");
-          }
-          tableList.add(table.toLowerCase());
-          LOG.debug("Found table " + table + " in the schema");
-        }
-      }
-    } catch (IOException ex){
-      throw new Exception(ex.getMessage());
-    }
-
-    return subs;
-  }
-
-  @VisibleForTesting
-  boolean validateLocations(Connection conn, URI[] defaultServers) throws HiveMetaException {
-    System.out.println("Validating DFS locations");
-    boolean rtn = true;
-    rtn &= checkMetaStoreDBLocation(conn, defaultServers);
-    rtn &= checkMetaStoreTableLocation(conn, defaultServers);
-    rtn &= checkMetaStorePartitionLocation(conn, defaultServers);
-    rtn &= checkMetaStoreSkewedColumnsLocation(conn, defaultServers);
-    System.out.println(rtn ? "[SUCCESS]\n" : "[FAIL]\n");
-    return rtn;
-  }
-
-  private static final String QUERY_DB_LOCATION =
-      "  select dbt.<q>DB_ID<q>, " +
-      "         dbt.<q>NAME<q>, " +
-      "         dbt.<q>DB_LOCATION_URI<q> " +
-      "    from <q>DBS<q> dbt " +
-      "order by dbt.<q>DB_ID<q> ";
-
-  private boolean checkMetaStoreDBLocation(Connection conn, URI[] defaultServers) throws HiveMetaException {
-    String dbLocQuery = schemaTool.quote(QUERY_DB_LOCATION);
-
-    int numOfInvalid = 0;
-    try (Statement stmt = conn.createStatement();
-         ResultSet res = stmt.executeQuery(dbLocQuery)) {
-      while (res.next()) {
-        String locValue = res.getString(3);
-        String dbName = getNameOrID(res, 2, 1);
-        if (!checkLocation("Database " + dbName, locValue, defaultServers)) {
-          numOfInvalid++;
-        }
-      }
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get DB Location Info.", e);
-    }
-    return numOfInvalid == 0;
-  }
-
-  private static final String TAB_ID_RANGE_QUERY =
-      "select max(<q>TBL_ID<q>), " +
-      "       min(<q>TBL_ID<q>) " +
-      "  from <q>TBLS<q> ";
-
-  private static final String TAB_LOC_QUERY =
-      "    select tbl.<q>TBL_ID<q>, " +
-      "           tbl.<q>TBL_NAME<q>, " +
-      "           sd.<q>LOCATION<q>, " +
-      "           dbt.<q>DB_ID<q>, " +
-      "           dbt.<q>NAME<q> " +
-      "      from <q>TBLS<q> tbl " +
-      "inner join <q>SDS<q> sd on sd.<q>SD_ID<q> = tbl.<q>SD_ID<q> " +
-      "inner join <q>DBS<q> dbt on tbl.<q>DB_ID<q> = dbt.<q>DB_ID<q> " +
-      "     where tbl.<q>TBL_TYPE<q> != '%s' " +
-      "       and tbl.<q>TBL_ID<q> >= ? " +
-      "       and tbl.<q>TBL_ID<q> <= ? " +
-      "  order by tbl.<q>TBL_ID<q> ";
-
-  private static final int TAB_LOC_CHECK_SIZE = 2000;
-
-  private boolean checkMetaStoreTableLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String tabIDRangeQuery = schemaTool.quote(TAB_ID_RANGE_QUERY);
-    String tabLocQuery = String.format(schemaTool.quote(TAB_LOC_QUERY), TableType.VIRTUAL_VIEW);
-
-    try {
-      long maxID = 0, minID = 0;
-      try (Statement stmt = conn.createStatement();
-           ResultSet res = stmt.executeQuery(tabIDRangeQuery)) {
-        if (res.next()) {
-          maxID = res.getLong(1);
-          minID = res.getLong(2);
-        }
-      }
-
-      int numOfInvalid = 0;
-      try (PreparedStatement pStmt = conn.prepareStatement(tabLocQuery)) {
-        while (minID <= maxID) {
-          pStmt.setLong(1, minID);
-          pStmt.setLong(2, minID + TAB_LOC_CHECK_SIZE);
-          try (ResultSet res = pStmt.executeQuery()) {
-            while (res.next()) {
-              String locValue = res.getString(3);
-              String entity = "Database " + getNameOrID(res, 5, 4) + ", Table "  + getNameOrID(res, 2, 1);
-              if (!checkLocation(entity, locValue, defaultServers)) {
-                numOfInvalid++;
-              }
-            }
-          }
-          minID += TAB_LOC_CHECK_SIZE + 1;
-        }
-      }
-
-      return numOfInvalid == 0;
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get Table Location Info.", e);
-    }
-  }
-
-  private static final String QUERY_PART_ID_RANGE =
-      "select max(<q>PART_ID<q>)," +
-      "       min(<q>PART_ID<q>)" +
-      "  from <q>PARTITIONS<q> ";
-
-  private static final String QUERY_PART_LOC =
-      "    select pt.<q>PART_ID<q>, " +
-      "           pt.<q>PART_NAME<q>, " +
-      "           sd.<q>LOCATION<q>, " +
-      "           tbl.<q>TBL_ID<q>, " +
-      "           tbl.<q>TBL_NAME<q>, " +
-      "           dbt.<q>DB_ID<q>, " +
-      "           dbt.<q>NAME<q> " +
-      "      from <q>PARTITIONS<q> pt " +
-      "inner join <q>SDS<q> sd on sd.<q>SD_ID<q> = pt.<q>SD_ID<q> " +
-      "inner join <q>TBLS<q> tbl on tbl.<q>TBL_ID<q> = pt.<q>TBL_ID<q> " +
-      "inner join <q>DBS<q> dbt on dbt.<q>DB_ID<q> = tbl.<q>DB_ID<q> " +
-      "     where pt.<q>PART_ID<q> >= ? " +
-      "       and pt.<q>PART_ID<q> <= ? " +
-      "  order by tbl.<q>TBL_ID<q> ";
-
-  private static final int PART_LOC_CHECK_SIZE = 2000;
-
-  private boolean checkMetaStorePartitionLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String queryPartIDRange = schemaTool.quote(QUERY_PART_ID_RANGE);
-    String queryPartLoc = schemaTool.quote(QUERY_PART_LOC);
-
-    try {
-      long maxID = 0, minID = 0;
-      try (Statement stmt = conn.createStatement();
-           ResultSet res = stmt.executeQuery(queryPartIDRange)) {
-        if (res.next()) {
-          maxID = res.getLong(1);
-          minID = res.getLong(2);
-        }
-      }
-
-      int numOfInvalid = 0;
-      try (PreparedStatement pStmt = conn.prepareStatement(queryPartLoc)) {
-        while (minID <= maxID) {
-          pStmt.setLong(1, minID);
-          pStmt.setLong(2, minID + PART_LOC_CHECK_SIZE);
-          try (ResultSet res = pStmt.executeQuery()) {
-            while (res.next()) {
-              String locValue = res.getString(3);
-              String entity = "Database " + getNameOrID(res, 7, 6) + ", Table "  + getNameOrID(res, 5, 4) +
-                  ", Partition " + getNameOrID(res, 2, 1);
-              if (!checkLocation(entity, locValue, defaultServers)) {
-                numOfInvalid++;
-              }
-            }
-          }
-          minID += PART_LOC_CHECK_SIZE + 1;
-        }
-      }
-
-      return numOfInvalid == 0;
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get Partition Location Info.", e);
-    }
-  }
-
-  private static final String QUERY_SKEWED_COL_ID_RANGE =
-      "select max(<q>STRING_LIST_ID_KID<q>), " +
-      "       min(<q>STRING_LIST_ID_KID<q>) " +
-      "  from <q>SKEWED_COL_VALUE_LOC_MAP<q> ";
-
-  private static final String QUERY_SKEWED_COL_LOC =
-      "  select t.<q>TBL_NAME<q>, " +
-      "         t.<q>TBL_ID<q>, " +
-      "         sk.<q>STRING_LIST_ID_KID<q>, " +
-      "         sk.<q>LOCATION<q>, " +
-      "         db.<q>NAME<q>, " +
-      "         db.<q>DB_ID<q> " +
-      "    from <q>TBLS<q> t " +
-      "    join <q>SDS<q> s on s.<q>SD_ID<q> = t.<q>SD_ID<q> " +
-      "    join <q>DBS<q> db on db.<q>DB_ID<q> = t.<q>DB_ID<q> " +
-      "    join <q>SKEWED_COL_VALUE_LOC_MAP<q> sk on sk.<q>SD_ID<q> = s.<q>SD_ID<q> " +
-      "   where sk.<q>STRING_LIST_ID_KID<q> >= ? " +
-      "     and sk.<q>STRING_LIST_ID_KID<q> <= ? " +
-      "order by t.<q>TBL_ID<q> ";
-
-  private static final int SKEWED_COL_LOC_CHECK_SIZE = 2000;
-
-  private boolean checkMetaStoreSkewedColumnsLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String querySkewedColIDRange = schemaTool.quote(QUERY_SKEWED_COL_ID_RANGE);
-    String querySkewedColLoc = schemaTool.quote(QUERY_SKEWED_COL_LOC);
-
-    try {
-      long maxID = 0, minID = 0;
-      try (Statement stmt = conn.createStatement();
-           ResultSet res = stmt.executeQuery(querySkewedColIDRange)) {
-        if (res.next()) {
-          maxID = res.getLong(1);
-          minID = res.getLong(2);
-        }
-      }
-
-      int numOfInvalid = 0;
-      try (PreparedStatement pStmt = conn.prepareStatement(querySkewedColLoc)) {
-        while (minID <= maxID) {
-          pStmt.setLong(1, minID);
-          pStmt.setLong(2, minID + SKEWED_COL_LOC_CHECK_SIZE);
-          try (ResultSet res = pStmt.executeQuery()) {
-            while (res.next()) {
-              String locValue = res.getString(4);
-              String entity = "Database " + getNameOrID(res, 5, 6) + ", Table " + getNameOrID(res, 1, 2) +
-                  ", String list " + res.getString(3);
-              if (!checkLocation(entity, locValue, defaultServers)) {
-                numOfInvalid++;
-              }
-            }
-          }
-          minID += SKEWED_COL_LOC_CHECK_SIZE + 1;
-        }
-      }
-
-      return numOfInvalid == 0;
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get skewed columns location info.", e);
-    }
-  }
-
-  /**
-   * Check if the location is valid for the given entity.
-   * @param entity          the entity to represent a database, partition or table
-   * @param entityLocation  the location
-   * @param defaultServers  a list of the servers that the location needs to match.
-   *                        The location host needs to match one of the given servers.
-   *                        If empty, then no check against such list.
-   * @return true if the location is valid
-   */
-  private boolean checkLocation(String entity, String entityLocation, URI[] defaultServers) {
-    boolean isValid = true;
-
-    if (entityLocation == null) {
-      System.err.println(entity + ", Error: empty location");
-      isValid = false;
-    } else {
-      try {
-        URI currentUri = new Path(entityLocation).toUri();
-        String scheme = currentUri.getScheme();
-        String path   = currentUri.getPath();
-        if (StringUtils.isEmpty(scheme)) {
-          System.err.println(entity + ", Location: "+ entityLocation + ", Error: missing location scheme.");
-          isValid = false;
-        } else if (StringUtils.isEmpty(path)) {
-          System.err.println(entity + ", Location: "+ entityLocation + ", Error: missing location path.");
-          isValid = false;
-        } else if (ArrayUtils.isNotEmpty(defaultServers) && currentUri.getAuthority() != null) {
-          String authority = currentUri.getAuthority();
-          boolean matchServer = false;
-          for(URI server : defaultServers) {
-            if (StringUtils.equalsIgnoreCase(server.getScheme(), scheme) &&
-                StringUtils.equalsIgnoreCase(server.getAuthority(), authority)) {
-              matchServer = true;
-              break;
-            }
-          }
-          if (!matchServer) {
-            System.err.println(entity + ", Location: " + entityLocation + ", Error: mismatched server.");
-            isValid = false;
-          }
-        }
-
-        // if there is no path element other than "/", report it but not fail
-        if (isValid && StringUtils.containsOnly(path, "/")) {
-          System.err.println(entity + ", Location: "+ entityLocation + ", Warn: location set to root, " +
-              "not a recommended config.");
-        }
-      } catch (Exception pe) {
-        System.err.println(entity + ", Error: invalid location - " + pe.getMessage());
-        isValid =false;
-      }
-    }
-
-    return isValid;
-  }
-
-  private String getNameOrID(ResultSet res, int nameInx, int idInx) throws SQLException {
-    String itemName = res.getString(nameInx);
-    return  (itemName == null || itemName.isEmpty()) ? "ID: " + res.getString(idInx) : "Name: " + itemName;
-  }
-
-  private static final String QUERY_COLUMN_NULL_VALUES =
-      "  select t.*" +
-      "    from <q>TBLS<q> t" +
-      "   where t.<q>SD_ID<q> IS NULL" +
-      "     and (t.<q>TBL_TYPE<q> = '" + TableType.EXTERNAL_TABLE + "' or" +
-      "          t.<q>TBL_TYPE<q> = '" + TableType.MANAGED_TABLE + "') " +
-      "order by t.<q>TBL_ID<q> ";
-
-  @VisibleForTesting
-  boolean validateColumnNullValues(Connection conn) throws HiveMetaException {
-    System.out.println("Validating columns for incorrect NULL values.");
-
-    boolean isValid = true;
-    String queryColumnNullValues = schemaTool.quote(QUERY_COLUMN_NULL_VALUES);
-
-    try (Statement stmt = conn.createStatement();
-         ResultSet res = stmt.executeQuery(queryColumnNullValues)) {
-      while (res.next()) {
-         long tableId = res.getLong("TBL_ID");
-         String tableName = res.getString("TBL_NAME");
-         String tableType = res.getString("TBL_TYPE");
-         isValid = false;
-         System.err.println("SD_ID in TBLS should not be NULL for Table Name=" + tableName + ", Table ID=" + tableId + ", Table Type=" + tableType);
-      }
-
-      System.out.println(isValid ? "[SUCCESS]\n" : "[FAIL]\n");
-      return isValid;
-    } catch(SQLException e) {
-        throw new HiveMetaException("Failed to validate columns for incorrect NULL values", e);
-    }
-  }
-}