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:16 UTC

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

Repository: hive
Updated Branches:
  refs/heads/master 02a038e23 -> eae5225f4


http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskInfo.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskInfo.java
new file mode 100644
index 0000000..cd1d57b
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskInfo.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
+
+/**
+ * Print Hive version and schema version.
+ */
+class SchemaToolTaskInfo extends SchemaToolTask {
+  @Override
+  void setCommandLineArguments(SchemaToolCommandLine 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/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskInit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskInit.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskInit.java
new file mode 100644
index 0000000..e3fa495
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskInit.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+
+/**
+ * Initialize the metastore schema.
+ */
+class SchemaToolTaskInit extends SchemaToolTask {
+  private boolean validate = true;
+  private String toVersion;
+
+  @Override
+  void setCommandLineArguments(SchemaToolCommandLine 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.execSql(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/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskMoveDatabase.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskMoveDatabase.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskMoveDatabase.java
new file mode 100644
index 0000000..8a9b9d1
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskMoveDatabase.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+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 SchemaToolTaskMoveDatabase extends SchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(SchemaToolTaskMoveDatabase.class.getName());
+
+  private String fromCatName;
+  private String toCatName;
+  private String dbName;
+
+  @Override
+  void setCommandLineArguments(SchemaToolCommandLine 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/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskMoveTable.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskMoveTable.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskMoveTable.java
new file mode 100644
index 0000000..a8f9228
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskMoveTable.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+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 SchemaToolTaskMoveTable extends SchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(SchemaToolTaskMoveTable.class.getName());
+
+  private String fromCat;
+  private String toCat;
+  private String fromDb;
+  private String toDb;
+  private String tableName;
+
+  @Override
+  void setCommandLineArguments(SchemaToolCommandLine 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/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskUpgrade.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskUpgrade.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskUpgrade.java
new file mode 100644
index 0000000..5e71609
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskUpgrade.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+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 SchemaToolTaskUpgrade extends SchemaToolTask {
+  private String fromVersion;
+
+  @Override
+  void setCommandLineArguments(SchemaToolCommandLine 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.execSql(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.execSql(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/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskValidate.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskValidate.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskValidate.java
new file mode 100644
index 0000000..d86b457
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskValidate.java
@@ -0,0 +1,630 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+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.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 SchemaToolTaskValidate extends SchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(SchemaToolTaskValidate.class.getName());
+
+  @Override
+  void setCommandLineArguments(SchemaToolCommandLine 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<>();
+    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<>();
+    List<String> subScripts   = new ArrayList<>();
+
+    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<>();
+    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);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/dbinstall/DbInstallBase.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/dbinstall/DbInstallBase.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/dbinstall/DbInstallBase.java
index 4722a56..2915720 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/dbinstall/DbInstallBase.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/dbinstall/DbInstallBase.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hive.metastore.dbinstall;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfoFactory;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.tools.MetastoreSchemaTool;
 import org.junit.After;
 import org.junit.Assert;
@@ -46,8 +44,6 @@ public abstract class DbInstallBase {
   private static final String FIRST_VERSION = "1.2.0";
   private static final int MAX_STARTUP_WAIT = 5 * 60 * 1000;
 
-  private String metastoreHome;
-
   protected abstract String getDockerContainerName();
   protected abstract String getDockerImageName();
   protected abstract String[] getDockerAdditionalArgs();
@@ -86,7 +82,7 @@ public abstract class DbInstallBase {
       throw new RuntimeException("Container failed to be ready in " + MAX_STARTUP_WAIT/1000 +
           " seconds");
     }
-    MetastoreSchemaTool.homeDir = metastoreHome = System.getProperty("test.tmp.dir", "target/tmp");
+    MetastoreSchemaTool.setHomeDirForTesting();
   }
 
   @After
@@ -145,7 +141,7 @@ public abstract class DbInstallBase {
   }
 
   private int createUser() {
-    return MetastoreSchemaTool.run(buildArray(
+    return new MetastoreSchemaTool().run(buildArray(
         "-createUser",
         "-dbType",
         getDbType(),
@@ -167,7 +163,7 @@ public abstract class DbInstallBase {
   }
 
   private int installLatest() {
-    return MetastoreSchemaTool.run(buildArray(
+    return new MetastoreSchemaTool().run(buildArray(
         "-initSchema",
         "-dbType",
         getDbType(),
@@ -183,7 +179,7 @@ public abstract class DbInstallBase {
   }
 
   private int installAVersion(String version) {
-    return MetastoreSchemaTool.run(buildArray(
+    return new MetastoreSchemaTool().run(buildArray(
         "-initSchemaTo",
         version,
         "-dbType",
@@ -200,7 +196,7 @@ public abstract class DbInstallBase {
   }
 
   private int upgradeToLatest() {
-    return MetastoreSchemaTool.run(buildArray(
+    return new MetastoreSchemaTool().run(buildArray(
         "-upgradeSchema",
         "-dbType",
         getDbType(),
@@ -219,17 +215,6 @@ public abstract class DbInstallBase {
     return strs;
   }
 
-  private String getCurrentVersionMinusOne() throws HiveMetaException {
-    List<String> scripts = MetaStoreSchemaInfoFactory.get(
-        MetastoreConf.newMetastoreConf(), metastoreHome, getDbType()
-    ).getUpgradeScripts(FIRST_VERSION);
-    Assert.assertTrue(scripts.size() > 0);
-    String lastUpgradePath = scripts.get(scripts.size() - 1);
-    String version = lastUpgradePath.split("-")[1];
-    LOG.info("Current version minus 1 is " + version);
-    return version;
-  }
-
   @Test
   public void install() {
     Assert.assertEquals(0, createUser());

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolForMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolForMetastore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolForMetastore.java
index 9e425cf..0657ae5 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolForMetastore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolForMetastore.java
@@ -27,10 +27,13 @@ import java.io.OutputStream;
 import java.io.PrintStream;
 import java.net.URI;
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.SQLException;
 import java.util.Random;
 
+import org.apache.commons.dbcp.DelegatingConnection;
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.text.StrTokenizer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.HiveMetaException;
 import org.apache.hadoop.hive.metastore.IMetaStoreSchemaInfo;
@@ -56,6 +59,8 @@ public class TestSchemaToolForMetastore {
   private String testMetastoreDB;
   private PrintStream errStream;
   private PrintStream outStream;
+  private String argsBase;
+  private SchemaToolTaskValidate validator;
 
   @Before
   public void setUp() throws HiveMetaException, IOException {
@@ -64,14 +69,21 @@ public class TestSchemaToolForMetastore {
     System.setProperty(ConfVars.CONNECT_URL_KEY.toString(),
         "jdbc:derby:" + testMetastoreDB + ";create=true");
     conf = MetastoreConf.newMetastoreConf();
-    schemaTool = new MetastoreSchemaTool(
-        System.getProperty("test.tmp.dir", "target/tmp"), conf, "derby");
-    schemaTool.setUserName(MetastoreConf.getVar(schemaTool.getConf(), ConfVars.CONNECTION_USER_NAME));
-    schemaTool.setPassWord(MetastoreConf.getPassword(schemaTool.getConf(), ConfVars.PWD));
+    schemaTool = new MetastoreSchemaTool();
+    schemaTool.init(System.getProperty("test.tmp.dir", "target/tmp"),
+        new String[]{"-dbType", "derby", "--info"}, null, conf);
+    String userName = MetastoreConf.getVar(schemaTool.getConf(), ConfVars.CONNECTION_USER_NAME);
+    String passWord = MetastoreConf.getPassword(schemaTool.getConf(), ConfVars.PWD);
+    schemaTool.setUserName(userName);
+    schemaTool.setPassWord(passWord);
+    argsBase = "-dbType derby -userName " + userName + " -passWord " + passWord + " ";
     System.setProperty("beeLine.system.exit", "true");
     errStream = System.err;
     outStream = System.out;
     conn = schemaTool.getConnectionToMetastore(false);
+
+    validator = new SchemaToolTaskValidate();
+    validator.setHiveSchemaTool(schemaTool);
   }
 
   @After
@@ -87,24 +99,26 @@ public class TestSchemaToolForMetastore {
     }
   }
 
-  // Test the sequence validation functionality
+  /*
+   * Test the sequence validation functionality
+   */
   @Test
   public void testValidateSequences() throws Exception {
-    schemaTool.doInit();
+    execute(new SchemaToolTaskInit(), "-initSchema");
 
     // Test empty database
-    boolean isValid = schemaTool.validateSequences(conn);
+    boolean isValid = validator.validateSequences(conn);
     Assert.assertTrue(isValid);
 
     // Test valid case
     String[] scripts = new String[] {
+        "insert into CTLGS values(99, 'test_cat_1', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb');",
         "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100);",
-        "insert into CTLGS values(37, 'mycat', 'my description', 'hdfs://tmp');",
-        "insert into DBS values(99, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'mycat');"
+        "insert into DBS values(99, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'test_cat_1');"
     };
     File scriptFile = generateTestScript(scripts);
-    schemaTool.runSqlLine(scriptFile.getPath());
-    isValid = schemaTool.validateSequences(conn);
+    schemaTool.execSql(scriptFile.getPath());
+    isValid = validator.validateSequences(conn);
     Assert.assertTrue(isValid);
 
     // Test invalid case
@@ -112,25 +126,27 @@ public class TestSchemaToolForMetastore {
         "delete from SEQUENCE_TABLE;",
         "delete from DBS;",
         "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100);",
-        "insert into DBS values(102, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'mycat');"
+        "insert into DBS values(102, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'test_cat_1');"
     };
     scriptFile = generateTestScript(scripts);
-    schemaTool.runSqlLine(scriptFile.getPath());
-    isValid = schemaTool.validateSequences(conn);
+    schemaTool.execSql(scriptFile.getPath());
+    isValid = validator.validateSequences(conn);
     Assert.assertFalse(isValid);
   }
 
-  // Test to validate that all tables exist in the HMS metastore.
+  /*
+   * Test to validate that all tables exist in the HMS metastore.
+   */
   @Test
   public void testValidateSchemaTables() throws Exception {
-    schemaTool.doInit("1.2.0");
+    execute(new SchemaToolTaskInit(), "-initSchemaTo 1.2.0");
 
-    boolean isValid = schemaTool.validateSchemaTables(conn);
+    boolean isValid = validator.validateSchemaTables(conn);
     Assert.assertTrue(isValid);
 
-    // upgrade from 2.0.0 schema and re-validate
-    schemaTool.doUpgrade("1.2.0");
-    isValid = schemaTool.validateSchemaTables(conn);
+    // upgrade from 1.2.0 schema and re-validate
+    execute(new SchemaToolTaskUpgrade(), "-upgradeSchemaFrom 1.2.0");
+    isValid = validator.validateSchemaTables(conn);
     Assert.assertTrue(isValid);
 
     // Simulate a missing table scenario by renaming a couple of tables
@@ -140,8 +156,8 @@ public class TestSchemaToolForMetastore {
     };
 
     File scriptFile = generateTestScript(scripts);
-    schemaTool.runSqlLine(scriptFile.getPath());
-    isValid = schemaTool.validateSchemaTables(conn);
+    schemaTool.execSql(scriptFile.getPath());
+    isValid = validator.validateSchemaTables(conn);
     Assert.assertFalse(isValid);
 
     // Restored the renamed tables
@@ -151,31 +167,49 @@ public class TestSchemaToolForMetastore {
     };
 
     scriptFile = generateTestScript(scripts);
-    schemaTool.runSqlLine(scriptFile.getPath());
-    isValid = schemaTool.validateSchemaTables(conn);
+    schemaTool.execSql(scriptFile.getPath());
+    isValid = validator.validateSchemaTables(conn);
     Assert.assertTrue(isValid);
-   }
+
+    // Check that an exception from getMetaData() is reported correctly
+    try {
+      // Make a Connection object that will throw an exception
+      BadMetaDataConnection bad = new BadMetaDataConnection(conn);
+      validator.validateSchemaTables(bad);
+      Assert.fail("did not get expected exception");
+    } catch (HiveMetaException hme) {
+      String message = hme.getMessage();
+      Assert.assertTrue("Bad HiveMetaException message :" + message,
+          message.contains("Failed to retrieve schema tables from Hive Metastore DB"));
+      Throwable cause = hme.getCause();
+      Assert.assertNotNull("HiveMetaException did not contain a cause", cause);
+      String causeMessage = cause.getMessage();
+      Assert.assertTrue("Bad SQLException message: " + causeMessage, causeMessage.contains(
+          BadMetaDataConnection.FAILURE_TEXT));
+    }
+  }
+
 
   // Test the validation of incorrect NULL values in the tables
   @Test
   public void testValidateNullValues() throws Exception {
-    schemaTool.doInit();
+    execute(new SchemaToolTaskInit(), "-initSchema");
 
     // Test empty database
-    boolean isValid = schemaTool.validateColumnNullValues(conn);
+    boolean isValid = validator.validateColumnNullValues(conn);
     Assert.assertTrue(isValid);
 
     // Test valid case
     createTestHiveTableSchemas();
-    isValid = schemaTool.validateColumnNullValues(conn);
+    isValid = validator.validateColumnNullValues(conn);
 
     // Test invalid case
     String[] scripts = new String[] {
         "update TBLS set SD_ID=null"
     };
     File scriptFile = generateTestScript(scripts);
-    schemaTool.runSqlLine(scriptFile.getPath());
-    isValid = schemaTool.validateColumnNullValues(conn);
+    schemaTool.execSql(scriptFile.getPath());
+    isValid = validator.validateColumnNullValues(conn);
     Assert.assertFalse(isValid);
   }
 
@@ -183,7 +217,7 @@ public class TestSchemaToolForMetastore {
   @Test
   public void testSchemaInitDryRun() throws Exception {
     schemaTool.setDryRun(true);
-    schemaTool.doInit("3.0.0");
+    execute(new SchemaToolTaskInit(), "-initSchemaTo 1.2.0");
     schemaTool.setDryRun(false);
     try {
       schemaTool.verifySchemaVersion();
@@ -197,10 +231,10 @@ public class TestSchemaToolForMetastore {
   // Test dryrun of schema upgrade
   @Test
   public void testSchemaUpgradeDryRun() throws Exception {
-    schemaTool.doInit("1.2.0");
+    execute(new SchemaToolTaskInit(), "-initSchemaTo 1.2.0");
 
     schemaTool.setDryRun(true);
-    schemaTool.doUpgrade("1.2.0");
+    execute(new SchemaToolTaskUpgrade(), "-upgradeSchemaFrom 1.2.0");
     schemaTool.setDryRun(false);
     try {
       schemaTool.verifySchemaVersion();
@@ -218,8 +252,7 @@ public class TestSchemaToolForMetastore {
   public void testSchemaInit() throws Exception {
     IMetaStoreSchemaInfo metastoreSchemaInfo = MetaStoreSchemaInfoFactory.get(conf,
         System.getProperty("test.tmp.dir", "target/tmp"), "derby");
-    LOG.info("Starting testSchemaInit");
-    schemaTool.doInit(metastoreSchemaInfo.getHiveSchemaVersion());
+    execute(new SchemaToolTaskInit(), "-initSchemaTo " + metastoreSchemaInfo.getHiveSchemaVersion());
     schemaTool.verifySchemaVersion();
   }
 
@@ -227,35 +260,35 @@ public class TestSchemaToolForMetastore {
   * Test validation for schema versions
   */
   @Test
- public void testValidateSchemaVersions() throws Exception {
-   schemaTool.doInit();
-   boolean isValid = schemaTool.validateSchemaVersions();
-   // Test an invalid case with multiple versions
-   String[] scripts = new String[] {
-       "insert into VERSION values(100, '2.2.0', 'Hive release version 2.2.0')"
-   };
-   File scriptFile = generateTestScript(scripts);
-   schemaTool.runSqlLine(scriptFile.getPath());
-   isValid = schemaTool.validateSchemaVersions();
-   Assert.assertFalse(isValid);
-
-   scripts = new String[] {
-       "delete from VERSION where VER_ID = 100"
-   };
-   scriptFile = generateTestScript(scripts);
-   schemaTool.runSqlLine(scriptFile.getPath());
-   isValid = schemaTool.validateSchemaVersions();
-   Assert.assertTrue(isValid);
-
-   // Test an invalid case without version
-   scripts = new String[] {
-       "delete from VERSION"
-   };
-   scriptFile = generateTestScript(scripts);
-   schemaTool.runSqlLine(scriptFile.getPath());
-   isValid = schemaTool.validateSchemaVersions();
-   Assert.assertFalse(isValid);
- }
+  public void testValidateSchemaVersions() throws Exception {
+    execute(new SchemaToolTaskInit(), "-initSchema");
+    boolean isValid = validator.validateSchemaVersions();
+    // Test an invalid case with multiple versions
+    String[] scripts = new String[] {
+        "insert into VERSION values(100, '2.2.0', 'Hive release version 2.2.0')"
+    };
+    File scriptFile = generateTestScript(scripts);
+    schemaTool.execSql(scriptFile.getPath());
+    isValid = validator.validateSchemaVersions();
+    Assert.assertFalse(isValid);
+
+    scripts = new String[] {
+        "delete from VERSION where VER_ID = 100"
+    };
+    scriptFile = generateTestScript(scripts);
+    schemaTool.execSql(scriptFile.getPath());
+    isValid = validator.validateSchemaVersions();
+    Assert.assertTrue(isValid);
+
+    // Test an invalid case without version
+    scripts = new String[] {
+        "delete from VERSION"
+    };
+    scriptFile = generateTestScript(scripts);
+    schemaTool.execSql(scriptFile.getPath());
+    isValid = validator.validateSchemaVersions();
+    Assert.assertFalse(isValid);
+  }
 
   /**
    * Test schema upgrade
@@ -264,7 +297,7 @@ public class TestSchemaToolForMetastore {
   public void testSchemaUpgrade() throws Exception {
     boolean foundException = false;
     // Initialize 1.2.0 schema
-    schemaTool.doInit("1.2.0");
+    execute(new SchemaToolTaskInit(), "-initSchemaTo 1.2.0");
     // verify that driver fails due to older version schema
     try {
       schemaTool.verifySchemaVersion();
@@ -298,17 +331,7 @@ public class TestSchemaToolForMetastore {
     System.setOut(outPrintStream);
 
     // Upgrade schema from 0.7.0 to latest
-    Exception caught = null;
-    try {
-      schemaTool.doUpgrade("1.2.0");
-    } catch (Exception e) {
-      caught = e;
-    }
-
-    LOG.info("stdout is " + stdout.toString());
-    LOG.info("stderr is " + stderr.toString());
-
-    if (caught != null) Assert.fail(caught.getMessage());
+    execute(new SchemaToolTaskUpgrade(), "-upgradeSchemaFrom 1.2.0");
 
     // Verify that the schemaTool ran pre-upgrade scripts and ignored errors
     Assert.assertTrue(stderr.toString().contains(invalidPreUpgradeScript));
@@ -327,38 +350,38 @@ public class TestSchemaToolForMetastore {
    */
   @Test
   public void testValidateLocations() throws Exception {
-    schemaTool.doInit();
+    execute(new SchemaToolTaskInit(), "-initSchema");
     URI defaultRoot = new URI("hdfs://myhost.com:8020");
     URI defaultRoot2 = new URI("s3://myhost2.com:8888");
     //check empty DB
-    boolean isValid = schemaTool.validateLocations(conn, null);
+    boolean isValid = validator.validateLocations(conn, null);
     Assert.assertTrue(isValid);
-    isValid = schemaTool.validateLocations(conn, new URI[] {defaultRoot,defaultRoot2});
+    isValid = validator.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
     Assert.assertTrue(isValid);
 
- // Test valid case
+    // Test valid case
     String[] scripts = new String[] {
-         "insert into CTLGS values (1, 'mycat', 'mydescription', 'hdfs://myhost.com:8020/user/hive/warehouse');",
-         "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'mycat');",
-         "insert into DBS values(7, 'db with bad port', 'hdfs://myhost.com:8020/', 'haDB', 'public', 'role', 'mycat');",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3,null,'org.apache.hadoop.mapred.TextInputFormat','N','N',null,-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n');",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3 ,1435255431,2,0 ,'hive',0,3,'myView','VIRTUAL_VIEW','select a.col1,a.col2 from foo','select * from foo','n');",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4012 ,1435255431,7,0 ,'hive',0,4000,'mytal4012','MANAGED_TABLE',NULL,NULL,'n');",
-         "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2);",
-         "insert into SKEWED_STRING_LIST values(1);",
-         "insert into SKEWED_STRING_LIST values(2);",
-         "insert into SKEWED_COL_VALUE_LOC_MAP values(1,1,'hdfs://myhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/');",
-         "insert into SKEWED_COL_VALUE_LOC_MAP values(2,2,'s3://myhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/');"
-       };
+        "insert into CTLGS values(3, 'test_cat_2', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb');",
+        "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'test_cat_2');",
+        "insert into DBS values(7, 'db with bad port', 'hdfs://myhost.com:8020/', 'haDB', 'public', 'role', 'test_cat_2');",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3,null,'org.apache.hadoop.mapred.TextInputFormat','N','N',null,-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n');",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3 ,1435255431,2,0 ,'hive',0,3,'myView','VIRTUAL_VIEW','select a.col1,a.col2 from foo','select * from foo','n');",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4012 ,1435255431,7,0 ,'hive',0,4000,'mytal4012','MANAGED_TABLE',NULL,NULL,'n');",
+        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2);",
+        "insert into SKEWED_STRING_LIST values(1);",
+        "insert into SKEWED_STRING_LIST values(2);",
+        "insert into SKEWED_COL_VALUE_LOC_MAP values(1,1,'hdfs://myhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/');",
+        "insert into SKEWED_COL_VALUE_LOC_MAP values(2,2,'s3://myhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/');"
+    };
     File scriptFile = generateTestScript(scripts);
-    schemaTool.runSqlLine(scriptFile.getPath());
-    isValid = schemaTool.validateLocations(conn, null);
+    schemaTool.execSql(scriptFile.getPath());
+    isValid = validator.validateLocations(conn, null);
     Assert.assertTrue(isValid);
-    isValid = schemaTool.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
+    isValid = validator.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
     Assert.assertTrue(isValid);
     scripts = new String[] {
         "delete from SKEWED_COL_VALUE_LOC_MAP;",
@@ -367,10 +390,10 @@ public class TestSchemaToolForMetastore {
         "delete from TBLS;",
         "delete from SDS;",
         "delete from DBS;",
-        "insert into DBS values(2, 'my db', '/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'mycat');",
-        "insert into DBS values(4, 'my db2', 'hdfs://myhost.com:8020', '', 'public', 'role', 'mycat');",
-        "insert into DBS values(6, 'db with bad port', 'hdfs://myhost.com:8020:', 'zDB', 'public', 'role', 'mycat');",
-        "insert into DBS values(7, 'db with bad port', 'hdfs://mynameservice.com/', 'haDB', 'public', 'role', 'mycat');",
+        "insert into DBS values(2, 'my db', '/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'test_cat_2');",
+        "insert into DBS values(4, 'my db2', 'hdfs://myhost.com:8020', '', 'public', 'role', 'test_cat_2');",
+        "insert into DBS values(6, 'db with bad port', 'hdfs://myhost.com:8020:', 'zDB', 'public', 'role', 'test_cat_2');",
+        "insert into DBS values(7, 'db with bad port', 'hdfs://mynameservice.com/', 'haDB', 'public', 'role', 'test_cat_2');",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n');",
@@ -398,23 +421,23 @@ public class TestSchemaToolForMetastore {
         "insert into SKEWED_COL_VALUE_LOC_MAP values(2,2,'file:///user/admin/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/');"
     };
     scriptFile = generateTestScript(scripts);
-    schemaTool.runSqlLine(scriptFile.getPath());
-    isValid = schemaTool.validateLocations(conn, null);
+    schemaTool.execSql(scriptFile.getPath());
+    isValid = validator.validateLocations(conn, null);
     Assert.assertFalse(isValid);
-    isValid = schemaTool.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
+    isValid = validator.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
     Assert.assertFalse(isValid);
   }
 
   @Test
   public void testHiveMetastoreDbPropertiesTable() throws HiveMetaException, IOException {
-    schemaTool.doInit("3.0.0");
+    execute(new SchemaToolTaskInit(), "-initSchemaTo 3.0.0");
     validateMetastoreDbPropertiesTable();
   }
 
   @Test
   public void testMetastoreDbPropertiesAfterUpgrade() throws HiveMetaException, IOException {
-    schemaTool.doInit("1.2.0");
-    schemaTool.doUpgrade();
+    execute(new SchemaToolTaskInit(), "-initSchemaTo 1.2.0");
+    execute(new SchemaToolTaskUpgrade(), "-upgradeSchema");
     validateMetastoreDbPropertiesTable();
   }
 
@@ -432,7 +455,7 @@ public class TestSchemaToolForMetastore {
   }
 
   private void validateMetastoreDbPropertiesTable() throws HiveMetaException, IOException {
-    boolean isValid = schemaTool.validateSchemaTables(conn);
+    boolean isValid = (boolean) validator.validateSchemaTables(conn);
     Assert.assertTrue(isValid);
     // adding same property key twice should throw unique key constraint violation exception
     String[] scripts = new String[] {
@@ -441,12 +464,13 @@ public class TestSchemaToolForMetastore {
     File scriptFile = generateTestScript(scripts);
     Exception ex = null;
     try {
-      schemaTool.runSqlLine(scriptFile.getPath());
+      schemaTool.execSql(scriptFile.getPath());
     } catch (Exception iox) {
       ex = iox;
     }
     Assert.assertTrue(ex != null && ex instanceof IOException);
   }
+
   /**
    * Write out a dummy pre-upgrade script with given SQL statement.
    */
@@ -476,6 +500,35 @@ public class TestSchemaToolForMetastore {
           "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2);"
         };
      File scriptFile = generateTestScript(scripts);
-     schemaTool.runSqlLine(scriptFile.getPath());
+     schemaTool.execSql(scriptFile.getPath());
+  }
+
+  /**
+   * A mock Connection class that throws an exception out of getMetaData().
+   */
+  class BadMetaDataConnection extends DelegatingConnection {
+    static final String FAILURE_TEXT = "fault injected";
+
+    BadMetaDataConnection(Connection connection) {
+      super(connection);
+    }
+
+    @Override
+    public DatabaseMetaData getMetaData() throws SQLException {
+      throw new SQLException(FAILURE_TEXT);
+    }
+  }
+
+  private void execute(SchemaToolTask task, String taskArgs) throws HiveMetaException {
+    try {
+      StrTokenizer tokenizer = new StrTokenizer(argsBase + taskArgs, ' ', '\"');
+      SchemaToolCommandLine cl = new SchemaToolCommandLine(tokenizer.getTokenArray(), null);
+      task.setCommandLineArguments(cl);
+    } catch (Exception e) {
+      throw new IllegalStateException("Could not parse comman line \n" + argsBase + taskArgs, e);
+    }
+
+    task.setHiveSchemaTool(schemaTool);
+    task.execute();
   }
 }


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/beeline/src/test/org/apache/hive/beeline/schematool/TestHiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/test/org/apache/hive/beeline/schematool/TestHiveSchemaTool.java b/beeline/src/test/org/apache/hive/beeline/schematool/TestHiveSchemaTool.java
index 8b477bd..8514dc8 100644
--- a/beeline/src/test/org/apache/hive/beeline/schematool/TestHiveSchemaTool.java
+++ b/beeline/src/test/org/apache/hive/beeline/schematool/TestHiveSchemaTool.java
@@ -43,13 +43,13 @@ import static org.powermock.api.mockito.PowerMockito.verifyStatic;
 
 @RunWith(PowerMockRunner.class)
 @PowerMockIgnore("javax.management.*")
-@PrepareForTest({ HiveSchemaHelper.class, HiveSchemaTool.CommandBuilder.class })
+@PrepareForTest({ HiveSchemaHelper.class, HiveSchemaTool.HiveSchemaToolCommandBuilder.class })
 public class TestHiveSchemaTool {
 
   String scriptFile = System.getProperty("java.io.tmpdir") + File.separator + "someScript.sql";
   @Mock
   private HiveConf hiveConf;
-  private HiveSchemaTool.CommandBuilder builder;
+  private HiveSchemaTool.HiveSchemaToolCommandBuilder builder;
   private String pasword = "reallySimplePassword";
 
   @Before
@@ -66,7 +66,7 @@ public class TestHiveSchemaTool {
     if (!file.exists()) {
       file.createNewFile();
     }
-    builder = new HiveSchemaTool.CommandBuilder(hiveConf, null, null, "testUser", pasword, scriptFile);
+    builder = new HiveSchemaTool.HiveSchemaToolCommandBuilder(hiveConf, null, null, "testUser", pasword, scriptFile);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolCatalogOps.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolCatalogOps.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolCatalogOps.java
new file mode 100644
index 0000000..42eb979
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolCatalogOps.java
@@ -0,0 +1,479 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.text.StrTokenizer;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.FunctionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.thrift.TException;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+public class TestSchemaToolCatalogOps {
+  private static MetastoreSchemaTool schemaTool;
+  private static HiveConf conf;
+  private IMetaStoreClient client;
+  private static String testMetastoreDB;
+  private static String argsBase;
+
+  @BeforeClass
+  public static void initDb() throws HiveMetaException, IOException {
+    conf = new HiveConf();
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.AUTO_CREATE_ALL, false);
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.HMS_HANDLER_ATTEMPTS, 1);
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.THRIFT_CONNECTION_RETRIES, 1);
+    testMetastoreDB = System.getProperty("java.io.tmpdir") +
+        File.separator + "testschematoolcatopsdb";
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CONNECT_URL_KEY,
+        "jdbc:derby:" + testMetastoreDB + ";create=true");
+    schemaTool = new MetastoreSchemaTool();
+    schemaTool.init(System.getProperty("test.tmp.dir", "target/tmp"),
+        new String[]{"-dbType", "derby", "--info"}, null, conf);
+
+    String userName = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CONNECTION_USER_NAME);
+    String passWord = MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.PWD);
+    schemaTool.setUserName(userName);
+    schemaTool.setPassWord(passWord);
+
+    argsBase = "-dbType derby -userName " + userName + " -passWord " + passWord + " ";
+    execute(new SchemaToolTaskInit(), "-initSchema"); // Pre-install the database so all the tables are there.
+  }
+
+  @AfterClass
+  public static void removeDb() throws Exception {
+    File metaStoreDir = new File(testMetastoreDB);
+    if (metaStoreDir.exists()) {
+      FileUtils.forceDeleteOnExit(metaStoreDir);
+    }
+  }
+
+  @Before
+  public void createClient() throws MetaException {
+    client = new HiveMetaStoreClient(conf);
+  }
+
+  @Test
+  public void createCatalog() throws HiveMetaException, TException {
+    String catName = "my_test_catalog";
+    String location = "file:///tmp/my_test_catalog";
+    String description = "very descriptive";
+    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new SchemaToolTaskCreateCatalog(), argsCreate);
+
+    Catalog cat = client.getCatalog(catName);
+    Assert.assertEquals(location, cat.getLocationUri());
+    Assert.assertEquals(description, cat.getDescription());
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void createExistingCatalog() throws HiveMetaException {
+    String catName = "hive";
+    String location = "somewhere";
+    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\"",
+        catName, location);
+    execute(new SchemaToolTaskCreateCatalog(), argsCreate);
+  }
+
+  @Test
+  public void createExistingCatalogWithIfNotExists() throws HiveMetaException {
+    String catName = "my_existing_test_catalog";
+    String location = "file:///tmp/my_test_catalog";
+    String description = "very descriptive";
+    String argsCreate1 = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new SchemaToolTaskCreateCatalog(), argsCreate1);
+
+    String argsCreate2 =
+        String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\" -ifNotExists",
+        catName, location, description);
+    execute(new SchemaToolTaskCreateCatalog(), argsCreate2);
+  }
+
+  @Test
+  public void alterCatalog() throws HiveMetaException, TException {
+    String catName = "an_alterable_catalog";
+    String location = "file:///tmp/an_alterable_catalog";
+    String description = "description";
+    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new SchemaToolTaskCreateCatalog(), argsCreate);
+
+    location = "file:///tmp/somewhere_else";
+    String argsAlter1 = String.format("-alterCatalog %s -catalogLocation \"%s\"",
+        catName, location);
+    execute(new SchemaToolTaskAlterCatalog(), argsAlter1);
+    Catalog cat = client.getCatalog(catName);
+    Assert.assertEquals(location, cat.getLocationUri());
+    Assert.assertEquals(description, cat.getDescription());
+
+    description = "a better description";
+    String argsAlter2 = String.format("-alterCatalog %s -catalogDescription \"%s\"",
+        catName, description);
+    execute(new SchemaToolTaskAlterCatalog(), argsAlter2);
+    cat = client.getCatalog(catName);
+    Assert.assertEquals(location, cat.getLocationUri());
+    Assert.assertEquals(description, cat.getDescription());
+
+    location = "file:///tmp/a_third_location";
+    description = "best description yet";
+    String argsAlter3 = String.format("-alterCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new SchemaToolTaskAlterCatalog(), argsAlter3);
+    cat = client.getCatalog(catName);
+    Assert.assertEquals(location, cat.getLocationUri());
+    Assert.assertEquals(description, cat.getDescription());
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void alterBogusCatalog() throws HiveMetaException {
+    String catName = "nosuch";
+    String location = "file:///tmp/somewhere";
+    String description = "whatever";
+    String argsAlter = String.format("-alterCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new SchemaToolTaskAlterCatalog(), argsAlter);
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void alterCatalogNoChange() throws HiveMetaException {
+    String catName = "alter_cat_no_change";
+    String location = "file:///tmp/alter_cat_no_change";
+    String description = "description";
+    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new SchemaToolTaskCreateCatalog(), argsCreate);
+
+    String argsAlter = String.format("-alterCatalog %s", catName);
+    execute(new SchemaToolTaskAlterCatalog(), argsAlter);
+  }
+
+  @Test
+  public void moveDatabase() throws HiveMetaException, TException {
+    String toCatName = "moveDbCat";
+    String dbName = "moveDbDb";
+    String tableName = "moveDbTable";
+    String funcName = "movedbfunc";
+    String partVal = "moveDbKey";
+
+    new CatalogBuilder()
+        .setName(toCatName)
+        .setLocation("file:///tmp")
+        .create(client);
+
+    Database db = new DatabaseBuilder()
+        .setCatalogName(DEFAULT_CATALOG_NAME)
+        .setName(dbName)
+        .create(client, conf);
+
+    new FunctionBuilder()
+        .inDb(db)
+        .setName(funcName)
+        .setClass("org.apache.hive.myudf")
+        .create(client, conf);
+
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .addPartCol("p", "string")
+        .create(client, conf);
+
+    new PartitionBuilder()
+        .inTable(table)
+        .addValue(partVal)
+        .addToTable(client, conf);
+
+    String argsMoveDB = String.format("-moveDatabase %s -fromCatalog %s -toCatalog %s", dbName,
+        DEFAULT_CATALOG_NAME, toCatName);
+    execute(new SchemaToolTaskMoveDatabase(), argsMoveDB);
+
+    Database fetchedDb = client.getDatabase(toCatName, dbName);
+    Assert.assertNotNull(fetchedDb);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedDb.getCatalogName());
+
+    Function fetchedFunction = client.getFunction(toCatName, dbName, funcName);
+    Assert.assertNotNull(fetchedFunction);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedFunction.getCatName());
+    Assert.assertEquals(dbName.toLowerCase(), fetchedFunction.getDbName());
+
+    Table fetchedTable = client.getTable(toCatName, dbName, tableName);
+    Assert.assertNotNull(fetchedTable);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedTable.getCatName());
+    Assert.assertEquals(dbName.toLowerCase(), fetchedTable.getDbName());
+
+    Partition fetchedPart =
+        client.getPartition(toCatName, dbName, tableName, Collections.singletonList(partVal));
+    Assert.assertNotNull(fetchedPart);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedPart.getCatName());
+    Assert.assertEquals(dbName.toLowerCase(), fetchedPart.getDbName());
+    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
+  }
+
+  @Test
+  public void moveDatabaseWithExistingDbOfSameNameAlreadyInTargetCatalog()
+      throws TException, HiveMetaException {
+    String catName = "clobberCatalog";
+    new CatalogBuilder()
+        .setName(catName)
+        .setLocation("file:///tmp")
+        .create(client);
+    try {
+      String argsMoveDB = String.format("-moveDatabase %s -fromCatalog %s -toCatalog %s",
+          DEFAULT_DATABASE_NAME, catName, DEFAULT_CATALOG_NAME);
+      execute(new SchemaToolTaskMoveDatabase(), argsMoveDB);
+      Assert.fail("Attempt to move default database should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> dbNames = new HashSet<>(client.getAllDatabases(DEFAULT_CATALOG_NAME));
+    Assert.assertTrue(dbNames.contains(DEFAULT_DATABASE_NAME));
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void moveNonExistentDatabase() throws TException, HiveMetaException {
+    String catName = "moveNonExistentDb";
+    new CatalogBuilder()
+        .setName(catName)
+        .setLocation("file:///tmp")
+        .create(client);
+    String argsMoveDB = String.format("-moveDatabase nosuch -fromCatalog %s -toCatalog %s",
+        catName, DEFAULT_CATALOG_NAME);
+    execute(new SchemaToolTaskMoveDatabase(), argsMoveDB);
+  }
+
+  @Test
+  public void moveDbToNonExistentCatalog() throws TException, HiveMetaException {
+    String dbName = "doomedToHomelessness";
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(client, conf);
+    try {
+      String argsMoveDB = String.format("-moveDatabase %s -fromCatalog %s -toCatalog nosuch",
+          dbName, DEFAULT_CATALOG_NAME);
+      execute(new SchemaToolTaskMoveDatabase(), argsMoveDB);
+      Assert.fail("Attempt to move database to non-existent catalog should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> dbNames = new HashSet<>(client.getAllDatabases(DEFAULT_CATALOG_NAME));
+    Assert.assertTrue(dbNames.contains(dbName.toLowerCase()));
+  }
+
+  @Test
+  public void moveTable() throws TException, HiveMetaException {
+    String toCatName = "moveTableCat";
+    String toDbName = "moveTableDb";
+    String tableName = "moveTableTable";
+    String partVal = "moveTableKey";
+
+    new CatalogBuilder()
+        .setName(toCatName)
+        .setLocation("file:///tmp")
+        .create(client);
+
+    new DatabaseBuilder()
+        .setCatalogName(toCatName)
+        .setName(toDbName)
+        .create(client, conf);
+
+    Table table = new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .addPartCol("p", "string")
+        .create(client, conf);
+
+    new PartitionBuilder()
+        .inTable(table)
+        .addValue(partVal)
+        .addToTable(client, conf);
+
+    String argsMoveTable = String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
+        tableName, DEFAULT_CATALOG_NAME, toCatName, DEFAULT_DATABASE_NAME, toDbName);
+    execute(new SchemaToolTaskMoveTable(), argsMoveTable);
+
+    Table fetchedTable = client.getTable(toCatName, toDbName, tableName);
+    Assert.assertNotNull(fetchedTable);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedTable.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedTable.getDbName());
+
+    Partition fetchedPart =
+        client.getPartition(toCatName, toDbName, tableName, Collections.singletonList(partVal));
+    Assert.assertNotNull(fetchedPart);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedPart.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedPart.getDbName());
+    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
+  }
+
+  @Test
+  public void moveTableWithinCatalog() throws TException, HiveMetaException {
+    String toDbName = "moveTableWithinCatalogDb";
+    String tableName = "moveTableWithinCatalogTable";
+    String partVal = "moveTableWithinCatalogKey";
+
+    new DatabaseBuilder()
+        .setName(toDbName)
+        .create(client, conf);
+
+    Table table = new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .addPartCol("p", "string")
+        .create(client, conf);
+
+    new PartitionBuilder()
+        .inTable(table)
+        .addValue(partVal)
+        .addToTable(client, conf);
+
+    String argsMoveTable = String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
+        tableName, DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName);
+    execute(new SchemaToolTaskMoveTable(), argsMoveTable);
+
+    Table fetchedTable = client.getTable(DEFAULT_CATALOG_NAME, toDbName, tableName);
+    Assert.assertNotNull(fetchedTable);
+    Assert.assertEquals(DEFAULT_CATALOG_NAME, fetchedTable.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedTable.getDbName());
+
+    Partition fetchedPart =
+        client.getPartition(DEFAULT_CATALOG_NAME, toDbName, tableName, Collections.singletonList(partVal));
+    Assert.assertNotNull(fetchedPart);
+    Assert.assertEquals(DEFAULT_CATALOG_NAME, fetchedPart.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedPart.getDbName());
+    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
+  }
+
+  @Test
+  public void moveTableWithExistingTableOfSameNameAlreadyInTargetDatabase()
+      throws TException, HiveMetaException {
+    String toDbName = "clobberTableDb";
+    String tableName = "clobberTableTable";
+
+    Database toDb = new DatabaseBuilder()
+        .setName(toDbName)
+        .create(client, conf);
+
+    new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .create(client, conf);
+
+    new TableBuilder()
+        .inDb(toDb)
+        .setTableName(tableName)
+        .addCol("b", "varchar(32)")
+        .create(client, conf);
+
+    try {
+      String argsMoveTable =
+          String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
+          tableName, DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName);
+      execute(new SchemaToolTaskMoveTable(), argsMoveTable);
+      Assert.fail("Attempt to move table should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> tableNames = new HashSet<>(client.getAllTables(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME));
+    Assert.assertTrue(tableNames.contains(tableName.toLowerCase()));
+
+    // Make sure the table in the target database didn't get clobbered
+    Table fetchedTable =  client.getTable(DEFAULT_CATALOG_NAME, toDbName, tableName);
+    Assert.assertEquals("b", fetchedTable.getSd().getCols().get(0).getName());
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void moveNonExistentTable() throws TException, HiveMetaException {
+    String toDbName = "moveNonExistentTable";
+    new DatabaseBuilder()
+        .setName(toDbName)
+        .create(client, conf);
+    String argsMoveTable =
+        String.format("-moveTable nosuch -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
+        DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName);
+    execute(new SchemaToolTaskMoveTable(), argsMoveTable);
+  }
+
+  @Test
+  public void moveTableToNonExistentDb() throws TException, HiveMetaException {
+    String tableName = "doomedToWander";
+    new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .create(client, conf);
+
+    try {
+      String argsMoveTable =
+          String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase nosuch",
+          tableName, DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME);
+      execute(new SchemaToolTaskMoveTable(), argsMoveTable);
+      Assert.fail("Attempt to move table to non-existent table should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> tableNames = new HashSet<>(client.getAllTables(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME));
+    Assert.assertTrue(tableNames.contains(tableName.toLowerCase()));
+  }
+
+  private static void execute(SchemaToolTask task, String taskArgs) throws HiveMetaException {
+    try {
+      StrTokenizer tokenizer = new StrTokenizer(argsBase + taskArgs, ' ', '\"');
+      SchemaToolCommandLine cl = new SchemaToolCommandLine(tokenizer.getTokenArray(), null);
+      task.setCommandLineArguments(cl);
+    } catch (Exception e) {
+      throw new IllegalStateException("Could not parse comman line \n" + argsBase + taskArgs, e);
+    }
+
+    task.setHiveSchemaTool(schemaTool);
+    task.execute();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaTool.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaTool.java
index 314aa60..0132a0b 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaTool.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaTool.java
@@ -19,340 +19,24 @@
 package org.apache.hive.beeline.schematool;
 
 import java.io.BufferedWriter;
-import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.net.URI;
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.SQLException;
-import java.util.Random;
 
-import junit.framework.TestCase;
 
-import org.apache.commons.dbcp.DelegatingConnection;
-import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang3.text.StrTokenizer;
-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.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
 import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.NestedScriptParser;
 import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.PostgresCommandParser;
-import org.apache.hadoop.hive.shims.ShimLoader;
+import org.junit.Assert;
+import org.junit.Test;
 
-public class TestSchemaTool extends TestCase {
-  private static HiveSchemaTool schemaTool;
-  private Connection conn;
-  private HiveConf hiveConf;
-  private String testMetastoreDB;
-  private PrintStream errStream;
-  private PrintStream outStream;
-  private String argsBase;
-  private HiveSchemaToolTaskValidate validator;
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    testMetastoreDB = System.getProperty("java.io.tmpdir") +
-        File.separator + "test_metastore-" + new Random().nextInt();
-    System.setProperty(MetastoreConf.ConfVars.CONNECT_URL_KEY.getVarname(),
-        "jdbc:derby:" + testMetastoreDB + ";create=true");
-    hiveConf = new HiveConf(this.getClass());
-    schemaTool = new HiveSchemaTool(
-        System.getProperty("test.tmp.dir", "target/tmp"), hiveConf, "derby", null);
-
-    String userName = hiveConf.get(MetastoreConf.ConfVars.CONNECTION_USER_NAME.getVarname());
-    String passWord = ShimLoader.getHadoopShims().getPassword(schemaTool.getHiveConf(),
-        MetastoreConf.ConfVars.PWD.getVarname());
-    schemaTool.setUserName(userName);
-    schemaTool.setPassWord(passWord);
-
-    argsBase = "-dbType derby -userName " + userName + " -passWord " + passWord + " ";
-
-    System.setProperty("beeLine.system.exit", "true");
-    errStream = System.err;
-    outStream = System.out;
-    conn = schemaTool.getConnectionToMetastore(false);
-
-    validator = new HiveSchemaToolTaskValidate();
-    validator.setHiveSchemaTool(schemaTool);
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    File metaStoreDir = new File(testMetastoreDB);
-    if (metaStoreDir.exists()) {
-      FileUtils.forceDeleteOnExit(metaStoreDir);
-    }
-    System.setOut(outStream);
-    System.setErr(errStream);
-    if (conn != null) {
-      conn.close();
-    }
-  }
-
-  /**
-   * Test the sequence validation functionality
-   * @throws Exception
-   */
-  public void testValidateSequences() throws Exception {
-    execute(new HiveSchemaToolTaskInit(), "-initSchema");
-
-    // Test empty database
-    boolean isValid = validator.validateSequences(conn);
-    assertTrue(isValid);
-
-    // Test valid case
-    String[] scripts = new String[] {
-        "insert into CTLGS values(99, 'test_cat_1', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb')",
-        "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100)",
-        "insert into DBS values(99, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'test_cat_1')"
-    };
-    File scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = validator.validateSequences(conn);
-    assertTrue(isValid);
-
-    // Test invalid case
-    scripts = new String[] {
-        "delete from SEQUENCE_TABLE",
-        "delete from DBS",
-        "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100)",
-        "insert into DBS values(102, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'test_cat_1')"
-    };
-    scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = validator.validateSequences(conn);
-    assertFalse(isValid);
-  }
-
-  /**
-   * Test to validate that all tables exist in the HMS metastore.
-   * @throws Exception
-   */
-  public void testValidateSchemaTables() throws Exception {
-    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 2.0.0");
-
-    boolean isValid = (boolean)validator.validateSchemaTables(conn);
-    assertTrue(isValid);
-
-    // upgrade from 2.0.0 schema and re-validate
-    execute(new HiveSchemaToolTaskUpgrade(), "-upgradeSchemaFrom 2.0.0");
-    isValid = (boolean)validator.validateSchemaTables(conn);
-    assertTrue(isValid);
-
-    // Simulate a missing table scenario by renaming a couple of tables
-    String[] scripts = new String[] {
-        "RENAME TABLE SEQUENCE_TABLE to SEQUENCE_TABLE_RENAMED",
-        "RENAME TABLE NUCLEUS_TABLES to NUCLEUS_TABLES_RENAMED"
-    };
-
-    File scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = validator.validateSchemaTables(conn);
-    assertFalse(isValid);
-
-    // Restored the renamed tables
-    scripts = new String[] {
-        "RENAME TABLE SEQUENCE_TABLE_RENAMED to SEQUENCE_TABLE",
-        "RENAME TABLE NUCLEUS_TABLES_RENAMED to NUCLEUS_TABLES"
-    };
-
-    scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = validator.validateSchemaTables(conn);
-    assertTrue(isValid);
-
-    // Check that an exception from getMetaData() is reported correctly
-    try {
-      // Make a Connection object that will throw an exception
-      BadMetaDataConnection bad = new BadMetaDataConnection(conn);
-      validator.validateSchemaTables(bad);
-      fail("did not get expected exception");
-    } catch (HiveMetaException hme) {
-      String message = hme.getMessage();
-      assertTrue("Bad HiveMetaException message :" + message,
-          message.contains("Failed to retrieve schema tables from Hive Metastore DB"));
-      Throwable cause = hme.getCause();
-      assertNotNull("HiveMetaException did not contain a cause", cause);
-      String causeMessage = cause.getMessage();
-      assertTrue("Bad SQLException message: " + causeMessage, causeMessage.contains(
-          BadMetaDataConnection.FAILURE_TEXT));
-    }
-  }
-
-  /*
-   * Test the validation of incorrect NULL values in the tables
-   * @throws Exception
-   */
-  public void testValidateNullValues() throws Exception {
-    execute(new HiveSchemaToolTaskInit(), "-initSchema");
-
-    // Test empty database
-    boolean isValid = validator.validateColumnNullValues(conn);
-    assertTrue(isValid);
-
-    // Test valid case
-    createTestHiveTableSchemas();
-    isValid = validator.validateColumnNullValues(conn);
-
-    // Test invalid case
-    String[] scripts = new String[] {
-        "update TBLS set SD_ID=null"
-    };
-    File scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = validator.validateColumnNullValues(conn);
-    assertFalse(isValid);
-  }
-
-  /**
-   * Test dryrun of schema initialization
-   * @throws Exception
-   */
-  public void testSchemaInitDryRun() throws Exception {
-    schemaTool.setDryRun(true);
-    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 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 {
-    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 0.7.0");
-
-    schemaTool.setDryRun(true);
-    execute(new HiveSchemaToolTaskUpgrade(), "-upgradeSchemaFrom 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 {
-    IMetaStoreSchemaInfo metastoreSchemaInfo = MetaStoreSchemaInfoFactory.get(hiveConf,
-        System.getProperty("test.tmp.dir", "target/tmp"), "derby");
-    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo " + metastoreSchemaInfo.getHiveSchemaVersion());
-    schemaTool.verifySchemaVersion();
-  }
-
-  /**
-  * Test validation for schema versions
-  * @throws Exception
-  */
-  public void testValidateSchemaVersions() throws Exception {
-    execute(new HiveSchemaToolTaskInit(), "-initSchema");
-    boolean isValid = validator.validateSchemaVersions();
-    // Test an invalid case with multiple versions
-    String[] scripts = new String[] {
-        "insert into VERSION values(100, '2.2.0', 'Hive release version 2.2.0')"
-    };
-    File scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = validator.validateSchemaVersions();
-    assertFalse(isValid);
-
-    scripts = new String[] {
-        "delete from VERSION where VER_ID = 100"
-    };
-    scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = validator.validateSchemaVersions();
-    assertTrue(isValid);
-
-    // Test an invalid case without version
-    scripts = new String[] {
-        "delete from VERSION"
-    };
-    scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = validator.validateSchemaVersions();
-    assertFalse(isValid);
-  }
-
-  /**
-   * Test schema upgrade
-   * @throws Exception
-   */
-  public void testSchemaUpgrade() throws Exception {
-    boolean foundException = false;
-    // Initialize 0.7.0 schema
-    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 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");
-    }
-
-    // Generate dummy pre-upgrade script with errors
-    String invalidPreUpgradeScript = writeDummyPreUpgradeScript(
-        0, "upgrade-0.11.0-to-0.12.0.derby.sql", "foo bar;");
-    // Generate dummy pre-upgrade scripts with valid SQL
-    String validPreUpgradeScript0 = writeDummyPreUpgradeScript(
-        0, "upgrade-0.12.0-to-0.13.0.derby.sql",
-        "CREATE TABLE schema_test0 (id integer);");
-    String validPreUpgradeScript1 = writeDummyPreUpgradeScript(
-        1, "upgrade-0.12.0-to-0.13.0.derby.sql",
-        "CREATE TABLE schema_test1 (id integer);");
-
-    // Capture system out and err
-    schemaTool.setVerbose(true);
-    OutputStream stderr = new ByteArrayOutputStream();
-    PrintStream errPrintStream = new PrintStream(stderr);
-    System.setErr(errPrintStream);
-    OutputStream stdout = new ByteArrayOutputStream();
-    PrintStream outPrintStream = new PrintStream(stdout);
-    System.setOut(outPrintStream);
-
-    // Upgrade schema from 0.7.0 to latest
-    execute(new HiveSchemaToolTaskUpgrade(), "-upgradeSchemaFrom 0.7.0");
-
-    // Verify that the schemaTool ran pre-upgrade scripts and ignored errors
-    assertTrue(stderr.toString().contains(invalidPreUpgradeScript));
-    assertTrue(stderr.toString().contains("foo"));
-    assertFalse(stderr.toString().contains(validPreUpgradeScript0));
-    assertFalse(stderr.toString().contains(validPreUpgradeScript1));
-    assertTrue(stdout.toString().contains(validPreUpgradeScript0));
-    assertTrue(stdout.toString().contains(validPreUpgradeScript1));
-
-    // Verify that driver works fine with latest schema
-    schemaTool.verifySchemaVersion();
-  }
+public class TestSchemaTool {
 
   /**
    * Test script formatting
-   * @throws Exception
    */
+  @Test
   public void testScripts() throws Exception {
     String testScript[] = {
         "-- this is a comment",
@@ -375,13 +59,13 @@ public class TestSchemaTool extends TestCase {
         .buildCommand(testScriptFile.getParentFile().getPath(),
             testScriptFile.getName());
 
-    assertEquals(expectedSQL, flattenedSql);
+    Assert.assertEquals(expectedSQL, flattenedSql);
   }
 
   /**
    * Test nested script formatting
-   * @throws Exception
    */
+  @Test
   public void testNestedScriptsForDerby() throws Exception {
     String childTab1 = "childTab1";
     String childTab2 = "childTab2";
@@ -416,17 +100,17 @@ public class TestSchemaTool extends TestCase {
     String flattenedSql = HiveSchemaHelper.getDbCommandParser("derby", false)
         .buildCommand(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));
+    Assert.assertFalse(flattenedSql.contains("RUN"));
+    Assert.assertFalse(flattenedSql.contains("comment"));
+    Assert.assertTrue(flattenedSql.contains(childTab1));
+    Assert.assertTrue(flattenedSql.contains(childTab2));
+    Assert.assertTrue(flattenedSql.contains(parentTab));
   }
 
   /**
    * Test nested script formatting
-   * @throws Exception
    */
+  @Test
   public void testNestedScriptsForMySQL() throws Exception {
     String childTab1 = "childTab1";
     String childTab2 = "childTab2";
@@ -461,17 +145,17 @@ public class TestSchemaTool extends TestCase {
     String flattenedSql = HiveSchemaHelper.getDbCommandParser("mysql", false)
         .buildCommand(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));
+    Assert.assertFalse(flattenedSql.contains("RUN"));
+    Assert.assertFalse(flattenedSql.contains("comment"));
+    Assert.assertTrue(flattenedSql.contains(childTab1));
+    Assert.assertTrue(flattenedSql.contains(childTab2));
+    Assert.assertTrue(flattenedSql.contains(parentTab));
   }
 
   /**
    * Test script formatting
-   * @throws Exception
    */
+  @Test
   public void testScriptWithDelimiter() throws Exception {
     String testScript[] = {
         "-- this is a comment",
@@ -504,13 +188,13 @@ public class TestSchemaTool extends TestCase {
     String flattenedSql = testDbParser.buildCommand(testScriptFile.getParentFile().getPath(),
         testScriptFile.getName());
 
-    assertEquals(expectedSQL, flattenedSql);
+    Assert.assertEquals(expectedSQL, flattenedSql);
   }
 
   /**
    * Test script formatting
-   * @throws Exception
    */
+  @Test
   public void testScriptMultiRowComment() throws Exception {
     String testScript[] = {
         "-- this is a comment",
@@ -539,13 +223,13 @@ public class TestSchemaTool extends TestCase {
     String flattenedSql = testDbParser.buildCommand(testScriptFile.getParentFile().getPath(),
         testScriptFile.getName());
 
-    assertEquals(expectedSQL, flattenedSql);
+    Assert.assertEquals(expectedSQL, flattenedSql);
   }
 
   /**
    * Test nested script formatting
-   * @throws Exception
    */
+  @Test
   public void testNestedScriptsForOracle() throws Exception {
     String childTab1 = "childTab1";
     String childTab2 = "childTab2";
@@ -580,17 +264,17 @@ public class TestSchemaTool extends TestCase {
     String flattenedSql = HiveSchemaHelper.getDbCommandParser("oracle", false)
         .buildCommand(testScriptFile.getParentFile().getPath(),
             testScriptFile.getName());
-    assertFalse(flattenedSql.contains("@"));
-    assertFalse(flattenedSql.contains("comment"));
-    assertTrue(flattenedSql.contains(childTab1));
-    assertTrue(flattenedSql.contains(childTab2));
-    assertTrue(flattenedSql.contains(parentTab));
+    Assert.assertFalse(flattenedSql.contains("@"));
+    Assert.assertFalse(flattenedSql.contains("comment"));
+    Assert.assertTrue(flattenedSql.contains(childTab1));
+    Assert.assertTrue(flattenedSql.contains(childTab2));
+    Assert.assertTrue(flattenedSql.contains(parentTab));
   }
 
   /**
    * Test script formatting
-   * @throws Exception
    */
+  @Test
   public void testPostgresFilter() throws Exception {
     String testScript[] = {
         "-- this is a comment",
@@ -616,7 +300,7 @@ public class TestSchemaTool extends TestCase {
     File testScriptFile = generateTestScript(testScript);
     String flattenedSql = noDbOptParser.buildCommand(
         testScriptFile.getParentFile().getPath(), testScriptFile.getName());
-    assertEquals(expectedSQL, flattenedSql);
+    Assert.assertEquals(expectedSQL, flattenedSql);
 
     String expectedScriptWithOptionAbsent[] = {
         "DROP TABLE IF EXISTS fooTab",
@@ -634,101 +318,7 @@ public class TestSchemaTool extends TestCase {
     testScriptFile = generateTestScript(testScript);
     flattenedSql = dbOptParser.buildCommand(
         testScriptFile.getParentFile().getPath(), testScriptFile.getName());
-    assertEquals(expectedSQL, flattenedSql);
-  }
-
-  /**
-   * Test validate uri of locations
-   * @throws Exception
-   */
-  public void testValidateLocations() throws Exception {
-    execute(new HiveSchemaToolTaskInit(), "-initSchema");
-    URI defaultRoot = new URI("hdfs://myhost.com:8020");
-    URI defaultRoot2 = new URI("s3://myhost2.com:8888");
-    //check empty DB
-    boolean isValid = validator.validateLocations(conn, null);
-    assertTrue(isValid);
-    isValid = validator.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
-    assertTrue(isValid);
-
- // Test valid case
-    String[] scripts = new String[] {
-         "insert into CTLGS values(3, 'test_cat_2', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb')",
-         "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'test_cat_2')",
-         "insert into DBS values(7, 'db with bad port', 'hdfs://myhost.com:8020/', 'haDB', 'public', 'role', 'test_cat_2')",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3,null,'org.apache.hadoop.mapred.TextInputFormat','N','N',null,-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3 ,1435255431,2,0 ,'hive',0,3,'myView','VIRTUAL_VIEW','select a.col1,a.col2 from foo','select * from foo','n')",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4012 ,1435255431,7,0 ,'hive',0,4000,'mytal4012','MANAGED_TABLE',NULL,NULL,'n')",
-         "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)",
-         "insert into SKEWED_STRING_LIST values(1)",
-         "insert into SKEWED_STRING_LIST values(2)",
-         "insert into SKEWED_COL_VALUE_LOC_MAP values(1,1,'hdfs://myhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')",
-         "insert into SKEWED_COL_VALUE_LOC_MAP values(2,2,'s3://myhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')"
-       };
-    File scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = validator.validateLocations(conn, null);
-    assertTrue(isValid);
-    isValid = validator.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
-    assertTrue(isValid);
-    scripts = new String[] {
-        "delete from SKEWED_COL_VALUE_LOC_MAP",
-        "delete from SKEWED_STRING_LIST",
-        "delete from PARTITIONS",
-        "delete from TBLS",
-        "delete from SDS",
-        "delete from DBS",
-        "insert into DBS values(2, 'my db', '/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'test_cat_2')",
-        "insert into DBS values(4, 'my db2', 'hdfs://myhost.com:8020', '', 'public', 'role', 'test_cat_2')",
-        "insert into DBS values(6, 'db with bad port', 'hdfs://myhost.com:8020:', 'zDB', 'public', 'role', 'test_cat_2')",
-        "insert into DBS values(7, 'db with bad port', 'hdfs://mynameservice.com/', 'haDB', 'public', 'role', 'test_cat_2')",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4001,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4003,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4004,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4002,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (5000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2016_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3000 ,1435255431,2,0 ,'hive',0,3000,'mytal3000','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4011 ,1435255431,4,0 ,'hive',0,4001,'mytal4011','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4012 ,1435255431,4,0 ,'hive',0,4002,'','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4013 ,1435255431,4,0 ,'hive',0,4003,'mytal4013','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4014 ,1435255431,2,0 ,'hive',0,4003,'','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4001, 1441402388,0, 'd1=1/d2=4001',4001,4011)",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4002, 1441402388,0, 'd1=1/d2=4002',4002,4012)",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4003, 1441402388,0, 'd1=1/d2=4003',4003,4013)",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4004, 1441402388,0, 'd1=1/d2=4004',4004,4014)",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(5000, 1441402388,0, 'd1=1/d2=5000',5000,2)",
-        "insert into SKEWED_STRING_LIST values(1)",
-        "insert into SKEWED_STRING_LIST values(2)",
-        "insert into SKEWED_COL_VALUE_LOC_MAP values(1,1,'hdfs://yourhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')",
-        "insert into SKEWED_COL_VALUE_LOC_MAP values(2,2,'file:///user/admin/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')"
-    };
-    scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = validator.validateLocations(conn, null);
-    assertFalse(isValid);
-    isValid = validator.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
-    assertFalse(isValid);
-  }
-
-  public void testHiveMetastoreDbPropertiesTable() throws HiveMetaException, IOException {
-    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 3.0.0");
-    validateMetastoreDbPropertiesTable();
-  }
-
-  public void testMetastoreDbPropertiesAfterUpgrade() throws HiveMetaException, IOException {
-    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 2.0.0");
-    execute(new HiveSchemaToolTaskUpgrade(), "-upgradeSchema");
-    validateMetastoreDbPropertiesTable();
+    Assert.assertEquals(expectedSQL, flattenedSql);
   }
 
   private File generateTestScript(String [] stmts) throws IOException {
@@ -743,84 +333,4 @@ public class TestSchemaTool extends TestCase {
     out.close();
     return testScriptFile;
   }
-
-  private void validateMetastoreDbPropertiesTable() throws HiveMetaException, IOException {
-    boolean isValid = (boolean) validator.validateSchemaTables(conn);
-    assertTrue(isValid);
-    // adding same property key twice should throw unique key constraint violation exception
-    String[] scripts = new String[] {
-        "insert into METASTORE_DB_PROPERTIES values ('guid', 'test-uuid-1', 'dummy uuid 1')",
-        "insert into METASTORE_DB_PROPERTIES values ('guid', 'test-uuid-2', 'dummy uuid 2')", };
-    File scriptFile = generateTestScript(scripts);
-    Exception ex = null;
-    try {
-      schemaTool.runBeeLine(scriptFile.getPath());
-    } catch (Exception iox) {
-      ex = iox;
-    }
-    assertTrue(ex != null && ex instanceof IOException);
-  }
-  /**
-   * Write out a dummy pre-upgrade script with given SQL statement.
-   */
-  private String writeDummyPreUpgradeScript(int index, String upgradeScriptName,
-      String sql) throws Exception {
-    String preUpgradeScript = "pre-" + index + "-" + upgradeScriptName;
-    String dummyPreScriptPath = System.getProperty("test.tmp.dir", "target/tmp") +
-        File.separatorChar + "scripts" + File.separatorChar + "metastore" +
-        File.separatorChar + "upgrade" + File.separatorChar + "derby" +
-        File.separatorChar + preUpgradeScript;
-    FileWriter fstream = new FileWriter(dummyPreScriptPath);
-    BufferedWriter out = new BufferedWriter(fstream);
-    out.write(sql + System.getProperty("line.separator") + ";");
-    out.close();
-    return preUpgradeScript;
-  }
-
-  /**
-   * Insert the records in DB to simulate a hive table
-   * @throws IOException
-   */
-  private void createTestHiveTableSchemas() throws IOException {
-     String[] scripts = new String[] {
-          "insert into CTLGS values(2, 'my_catalog', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb')",
-          "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8021/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'my_catalog')",
-          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-          "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
-          "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3 ,1435255431,2,0 ,'hive',0,2,'aTable','MANAGED_TABLE',NULL,NULL,'n')",
-          "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)"
-        };
-     File scriptFile = generateTestScript(scripts);
-     schemaTool.runBeeLine(scriptFile.getPath());
-  }
-
-  /**
-   * A mock Connection class that throws an exception out of getMetaData().
-   */
-  class BadMetaDataConnection extends DelegatingConnection {
-    static final String FAILURE_TEXT = "fault injected";
-
-    BadMetaDataConnection(Connection connection) {
-      super(connection);
-    }
-
-    @Override
-    public DatabaseMetaData getMetaData() throws SQLException {
-      throw new SQLException(FAILURE_TEXT);
-    }
-  }
-
-  private void execute(HiveSchemaToolTask task, String taskArgs) throws HiveMetaException {
-    try {
-      StrTokenizer tokenizer = new StrTokenizer(argsBase + taskArgs, ' ', '\"');
-      HiveSchemaToolCommandLine cl = new HiveSchemaToolCommandLine(tokenizer.getTokenArray());
-      task.setCommandLineArguments(cl);
-    } catch (Exception e) {
-      throw new IllegalStateException("Could not parse comman line \n" + argsBase + taskArgs, e);
-    }
-
-    task.setHiveSchemaTool(schemaTool);
-    task.execute();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaToolCatalogOps.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaToolCatalogOps.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaToolCatalogOps.java
deleted file mode 100644
index a13603d..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaToolCatalogOps.java
+++ /dev/null
@@ -1,478 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.schematool;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.text.StrTokenizer;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
-import org.apache.hadoop.hive.metastore.api.Catalog;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.FunctionBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.thrift.TException;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
-
-public class TestSchemaToolCatalogOps {
-  private static HiveSchemaTool schemaTool;
-  private static HiveConf conf;
-  private IMetaStoreClient client;
-  private static String testMetastoreDB;
-  private static String argsBase;
-
-  @BeforeClass
-  public static void initDb() throws HiveMetaException, IOException {
-    conf = new HiveConf();
-    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.AUTO_CREATE_ALL, false);
-    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.HMS_HANDLER_ATTEMPTS, 1);
-    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.THRIFT_CONNECTION_RETRIES, 1);
-    testMetastoreDB = System.getProperty("java.io.tmpdir") +
-        File.separator + "testschematoolcatopsdb";
-    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CONNECT_URL_KEY,
-        "jdbc:derby:" + testMetastoreDB + ";create=true");
-    schemaTool = new HiveSchemaTool(
-        System.getProperty("test.tmp.dir", "target/tmp"), conf, "derby", null);
-
-    String userName = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CONNECTION_USER_NAME);
-    String passWord = MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.PWD);
-    schemaTool.setUserName(userName);
-    schemaTool.setPassWord(passWord);
-
-    argsBase = "-dbType derby -userName " + userName + " -passWord " + passWord + " ";
-    execute(new HiveSchemaToolTaskInit(), "-initSchema"); // Pre-install the database so all the tables are there.
-  }
-
-  @AfterClass
-  public static void removeDb() throws Exception {
-    File metaStoreDir = new File(testMetastoreDB);
-    if (metaStoreDir.exists()) {
-      FileUtils.forceDeleteOnExit(metaStoreDir);
-    }
-  }
-
-  @Before
-  public void createClient() throws MetaException {
-    client = new HiveMetaStoreClient(conf);
-  }
-
-  @Test
-  public void createCatalog() throws HiveMetaException, TException {
-    String catName = "my_test_catalog";
-    String location = "file:///tmp/my_test_catalog";
-    String description = "very descriptive";
-    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
-        catName, location, description);
-    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate);
-
-    Catalog cat = client.getCatalog(catName);
-    Assert.assertEquals(location, cat.getLocationUri());
-    Assert.assertEquals(description, cat.getDescription());
-  }
-
-  @Test(expected = HiveMetaException.class)
-  public void createExistingCatalog() throws HiveMetaException {
-    String catName = "hive";
-    String location = "somewhere";
-    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\"",
-        catName, location);
-    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate);
-  }
-
-  @Test
-  public void createExistingCatalogWithIfNotExists() throws HiveMetaException {
-    String catName = "my_existing_test_catalog";
-    String location = "file:///tmp/my_test_catalog";
-    String description = "very descriptive";
-    String argsCreate1 = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
-        catName, location, description);
-    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate1);
-
-    String argsCreate2 =
-        String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\" -ifNotExists",
-        catName, location, description);
-    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate2);
-  }
-
-  @Test
-  public void alterCatalog() throws HiveMetaException, TException {
-    String catName = "an_alterable_catalog";
-    String location = "file:///tmp/an_alterable_catalog";
-    String description = "description";
-    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
-        catName, location, description);
-    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate);
-
-    location = "file:///tmp/somewhere_else";
-    String argsAlter1 = String.format("-alterCatalog %s -catalogLocation \"%s\"",
-        catName, location);
-    execute(new HiveSchemaToolTaskAlterCatalog(), argsAlter1);
-    Catalog cat = client.getCatalog(catName);
-    Assert.assertEquals(location, cat.getLocationUri());
-    Assert.assertEquals(description, cat.getDescription());
-
-    description = "a better description";
-    String argsAlter2 = String.format("-alterCatalog %s -catalogDescription \"%s\"",
-        catName, description);
-    execute(new HiveSchemaToolTaskAlterCatalog(), argsAlter2);
-    cat = client.getCatalog(catName);
-    Assert.assertEquals(location, cat.getLocationUri());
-    Assert.assertEquals(description, cat.getDescription());
-
-    location = "file:///tmp/a_third_location";
-    description = "best description yet";
-    String argsAlter3 = String.format("-alterCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
-        catName, location, description);
-    execute(new HiveSchemaToolTaskAlterCatalog(), argsAlter3);
-    cat = client.getCatalog(catName);
-    Assert.assertEquals(location, cat.getLocationUri());
-    Assert.assertEquals(description, cat.getDescription());
-  }
-
-  @Test(expected = HiveMetaException.class)
-  public void alterBogusCatalog() throws HiveMetaException {
-    String catName = "nosuch";
-    String location = "file:///tmp/somewhere";
-    String description = "whatever";
-    String argsAlter = String.format("-alterCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
-        catName, location, description);
-    execute(new HiveSchemaToolTaskAlterCatalog(), argsAlter);
-  }
-
-  @Test(expected = HiveMetaException.class)
-  public void alterCatalogNoChange() throws HiveMetaException {
-    String catName = "alter_cat_no_change";
-    String location = "file:///tmp/alter_cat_no_change";
-    String description = "description";
-    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
-        catName, location, description);
-    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate);
-
-    String argsAlter = String.format("-alterCatalog %s", catName);
-    execute(new HiveSchemaToolTaskAlterCatalog(), argsAlter);
-  }
-
-  @Test
-  public void moveDatabase() throws HiveMetaException, TException {
-    String toCatName = "moveDbCat";
-    String dbName = "moveDbDb";
-    String tableName = "moveDbTable";
-    String funcName = "movedbfunc";
-    String partVal = "moveDbKey";
-
-    new CatalogBuilder()
-        .setName(toCatName)
-        .setLocation("file:///tmp")
-        .create(client);
-
-    Database db = new DatabaseBuilder()
-        .setCatalogName(DEFAULT_CATALOG_NAME)
-        .setName(dbName)
-        .create(client, conf);
-
-    new FunctionBuilder()
-        .inDb(db)
-        .setName(funcName)
-        .setClass("org.apache.hive.myudf")
-        .create(client, conf);
-
-    Table table = new TableBuilder()
-        .inDb(db)
-        .setTableName(tableName)
-        .addCol("a", "int")
-        .addPartCol("p", "string")
-        .create(client, conf);
-
-    new PartitionBuilder()
-        .inTable(table)
-        .addValue(partVal)
-        .addToTable(client, conf);
-
-    String argsMoveDB = String.format("-moveDatabase %s -fromCatalog %s -toCatalog %s", dbName,
-        DEFAULT_CATALOG_NAME, toCatName);
-    execute(new HiveSchemaToolTaskMoveDatabase(), argsMoveDB);
-
-    Database fetchedDb = client.getDatabase(toCatName, dbName);
-    Assert.assertNotNull(fetchedDb);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedDb.getCatalogName());
-
-    Function fetchedFunction = client.getFunction(toCatName, dbName, funcName);
-    Assert.assertNotNull(fetchedFunction);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedFunction.getCatName());
-    Assert.assertEquals(dbName.toLowerCase(), fetchedFunction.getDbName());
-
-    Table fetchedTable = client.getTable(toCatName, dbName, tableName);
-    Assert.assertNotNull(fetchedTable);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedTable.getCatName());
-    Assert.assertEquals(dbName.toLowerCase(), fetchedTable.getDbName());
-
-    Partition fetchedPart =
-        client.getPartition(toCatName, dbName, tableName, Collections.singletonList(partVal));
-    Assert.assertNotNull(fetchedPart);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedPart.getCatName());
-    Assert.assertEquals(dbName.toLowerCase(), fetchedPart.getDbName());
-    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
-  }
-
-  @Test
-  public void moveDatabaseWithExistingDbOfSameNameAlreadyInTargetCatalog()
-      throws TException, HiveMetaException {
-    String catName = "clobberCatalog";
-    new CatalogBuilder()
-        .setName(catName)
-        .setLocation("file:///tmp")
-        .create(client);
-    try {
-      String argsMoveDB = String.format("-moveDatabase %s -fromCatalog %s -toCatalog %s",
-          DEFAULT_DATABASE_NAME, catName, DEFAULT_CATALOG_NAME);
-      execute(new HiveSchemaToolTaskMoveDatabase(), argsMoveDB);
-      Assert.fail("Attempt to move default database should have failed.");
-    } catch (HiveMetaException e) {
-      // good
-    }
-
-    // Make sure nothing really moved
-    Set<String> dbNames = new HashSet<>(client.getAllDatabases(DEFAULT_CATALOG_NAME));
-    Assert.assertTrue(dbNames.contains(DEFAULT_DATABASE_NAME));
-  }
-
-  @Test(expected = HiveMetaException.class)
-  public void moveNonExistentDatabase() throws TException, HiveMetaException {
-    String catName = "moveNonExistentDb";
-    new CatalogBuilder()
-        .setName(catName)
-        .setLocation("file:///tmp")
-        .create(client);
-    String argsMoveDB = String.format("-moveDatabase nosuch -fromCatalog %s -toCatalog %s",
-        catName, DEFAULT_CATALOG_NAME);
-    execute(new HiveSchemaToolTaskMoveDatabase(), argsMoveDB);
-  }
-
-  @Test
-  public void moveDbToNonExistentCatalog() throws TException, HiveMetaException {
-    String dbName = "doomedToHomelessness";
-    new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-    try {
-      String argsMoveDB = String.format("-moveDatabase %s -fromCatalog %s -toCatalog nosuch",
-          dbName, DEFAULT_CATALOG_NAME);
-      execute(new HiveSchemaToolTaskMoveDatabase(), argsMoveDB);
-      Assert.fail("Attempt to move database to non-existent catalog should have failed.");
-    } catch (HiveMetaException e) {
-      // good
-    }
-
-    // Make sure nothing really moved
-    Set<String> dbNames = new HashSet<>(client.getAllDatabases(DEFAULT_CATALOG_NAME));
-    Assert.assertTrue(dbNames.contains(dbName.toLowerCase()));
-  }
-
-  @Test
-  public void moveTable() throws TException, HiveMetaException {
-    String toCatName = "moveTableCat";
-    String toDbName = "moveTableDb";
-    String tableName = "moveTableTable";
-    String partVal = "moveTableKey";
-
-    new CatalogBuilder()
-        .setName(toCatName)
-        .setLocation("file:///tmp")
-        .create(client);
-
-    new DatabaseBuilder()
-        .setCatalogName(toCatName)
-        .setName(toDbName)
-        .create(client, conf);
-
-    Table table = new TableBuilder()
-        .setTableName(tableName)
-        .addCol("a", "int")
-        .addPartCol("p", "string")
-        .create(client, conf);
-
-    new PartitionBuilder()
-        .inTable(table)
-        .addValue(partVal)
-        .addToTable(client, conf);
-
-    String argsMoveTable = String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
-        tableName, DEFAULT_CATALOG_NAME, toCatName, DEFAULT_DATABASE_NAME, toDbName);
-    execute(new HiveSchemaToolTaskMoveTable(), argsMoveTable);
-
-    Table fetchedTable = client.getTable(toCatName, toDbName, tableName);
-    Assert.assertNotNull(fetchedTable);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedTable.getCatName());
-    Assert.assertEquals(toDbName.toLowerCase(), fetchedTable.getDbName());
-
-    Partition fetchedPart =
-        client.getPartition(toCatName, toDbName, tableName, Collections.singletonList(partVal));
-    Assert.assertNotNull(fetchedPart);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedPart.getCatName());
-    Assert.assertEquals(toDbName.toLowerCase(), fetchedPart.getDbName());
-    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
-  }
-
-  @Test
-  public void moveTableWithinCatalog() throws TException, HiveMetaException {
-    String toDbName = "moveTableWithinCatalogDb";
-    String tableName = "moveTableWithinCatalogTable";
-    String partVal = "moveTableWithinCatalogKey";
-
-    new DatabaseBuilder()
-        .setName(toDbName)
-        .create(client, conf);
-
-    Table table = new TableBuilder()
-        .setTableName(tableName)
-        .addCol("a", "int")
-        .addPartCol("p", "string")
-        .create(client, conf);
-
-    new PartitionBuilder()
-        .inTable(table)
-        .addValue(partVal)
-        .addToTable(client, conf);
-
-    String argsMoveTable = String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
-        tableName, DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName);
-    execute(new HiveSchemaToolTaskMoveTable(), argsMoveTable);
-
-    Table fetchedTable = client.getTable(DEFAULT_CATALOG_NAME, toDbName, tableName);
-    Assert.assertNotNull(fetchedTable);
-    Assert.assertEquals(DEFAULT_CATALOG_NAME, fetchedTable.getCatName());
-    Assert.assertEquals(toDbName.toLowerCase(), fetchedTable.getDbName());
-
-    Partition fetchedPart =
-        client.getPartition(DEFAULT_CATALOG_NAME, toDbName, tableName, Collections.singletonList(partVal));
-    Assert.assertNotNull(fetchedPart);
-    Assert.assertEquals(DEFAULT_CATALOG_NAME, fetchedPart.getCatName());
-    Assert.assertEquals(toDbName.toLowerCase(), fetchedPart.getDbName());
-    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
-  }
-
-  @Test
-  public void moveTableWithExistingTableOfSameNameAlreadyInTargetDatabase()
-      throws TException, HiveMetaException {
-    String toDbName = "clobberTableDb";
-    String tableName = "clobberTableTable";
-
-    Database toDb = new DatabaseBuilder()
-        .setName(toDbName)
-        .create(client, conf);
-
-    new TableBuilder()
-        .setTableName(tableName)
-        .addCol("a", "int")
-        .create(client, conf);
-
-    new TableBuilder()
-        .inDb(toDb)
-        .setTableName(tableName)
-        .addCol("b", "varchar(32)")
-        .create(client, conf);
-
-    try {
-      String argsMoveTable =
-          String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
-          tableName, DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName);
-      execute(new HiveSchemaToolTaskMoveTable(), argsMoveTable);
-      Assert.fail("Attempt to move table should have failed.");
-    } catch (HiveMetaException e) {
-      // good
-    }
-
-    // Make sure nothing really moved
-    Set<String> tableNames = new HashSet<>(client.getAllTables(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME));
-    Assert.assertTrue(tableNames.contains(tableName.toLowerCase()));
-
-    // Make sure the table in the target database didn't get clobbered
-    Table fetchedTable =  client.getTable(DEFAULT_CATALOG_NAME, toDbName, tableName);
-    Assert.assertEquals("b", fetchedTable.getSd().getCols().get(0).getName());
-  }
-
-  @Test(expected = HiveMetaException.class)
-  public void moveNonExistentTable() throws TException, HiveMetaException {
-    String toDbName = "moveNonExistentTable";
-    new DatabaseBuilder()
-        .setName(toDbName)
-        .create(client, conf);
-    String argsMoveTable =
-        String.format("-moveTable nosuch -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
-        DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName);
-    execute(new HiveSchemaToolTaskMoveTable(), argsMoveTable);
-  }
-
-  @Test
-  public void moveTableToNonExistentDb() throws TException, HiveMetaException {
-    String tableName = "doomedToWander";
-    new TableBuilder()
-        .setTableName(tableName)
-        .addCol("a", "int")
-        .create(client, conf);
-
-    try {
-      String argsMoveTable =
-          String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase nosuch",
-          tableName, DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME);
-      execute(new HiveSchemaToolTaskMoveTable(), argsMoveTable);
-      Assert.fail("Attempt to move table to non-existent table should have failed.");
-    } catch (HiveMetaException e) {
-      // good
-    }
-
-    // Make sure nothing really moved
-    Set<String> tableNames = new HashSet<>(client.getAllTables(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME));
-    Assert.assertTrue(tableNames.contains(tableName.toLowerCase()));
-  }
-
-  private static void execute(HiveSchemaToolTask task, String taskArgs) throws HiveMetaException {
-    try {
-      StrTokenizer tokenizer = new StrTokenizer(argsBase + taskArgs, ' ', '\"');
-      HiveSchemaToolCommandLine cl = new HiveSchemaToolCommandLine(tokenizer.getTokenArray());
-      task.setCommandLineArguments(cl);
-    } catch (Exception e) {
-      throw new IllegalStateException("Could not parse comman line \n" + argsBase + taskArgs, e);
-    }
-
-    task.setHiveSchemaTool(schemaTool);
-    task.execute();
-  }
-}


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/MetastoreSchemaTool.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/MetastoreSchemaTool.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/MetastoreSchemaTool.java
index 7c8087f..c2018f4 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/MetastoreSchemaTool.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/MetastoreSchemaTool.java
@@ -18,25 +18,15 @@
 package org.apache.hadoop.hive.metastore.tools;
 
 import com.google.common.annotations.VisibleForTesting;
-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.commons.lang.ArrayUtils;
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 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.TableType;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
@@ -44,55 +34,44 @@ import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.NestedScriptParse
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.ImmutableMap;
 import sqlline.SqlLine;
 
 import java.io.BufferedReader;
-import java.io.BufferedWriter;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileReader;
-import java.io.FileWriter;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.PrintStream;
 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.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;
 
 public class MetastoreSchemaTool {
   private static final Logger LOG = LoggerFactory.getLogger(MetastoreSchemaTool.class);
   private static final String PASSWD_MASK = "[passwd stripped]";
 
-  @VisibleForTesting
-  public static String homeDir;
-
-  private String userName = null;
-  private String passWord = null;
-  private boolean dryRun = false;
-  private boolean verbose = false;
-  private String dbOpts = null;
-  private String url = null;
-  private String driver = null;
-  private URI[] validationServers = null; // The list of servers the database/partition/table can locate on
-  private String hiveUser; // Hive username, for use when creating the user, not for connecting
-  private String hivePasswd; // Hive password, for use when creating the user, not for connecting
-  private String hiveDb; // Hive database, for use when creating the user, not for connecting
-  private final Configuration conf;
-  private final String dbType;
-  private final IMetaStoreSchemaInfo metaStoreSchemaInfo;
-  private boolean needsQuotedIdentifier;
+  protected Configuration conf;
+
+  protected String dbOpts = null;
+  protected String dbType;
+  protected String driver = null;
+  protected boolean dryRun = false;
+  protected String hiveDb; // Hive database, for use when creating the user, not for connecting
+  protected String hivePasswd; // Hive password, for use when creating the user, not for connecting
+  protected String hiveUser; // Hive username, for use when creating the user, not for connecting
+  protected String metaDbType;
+  protected IMetaStoreSchemaInfo metaStoreSchemaInfo;
+  protected boolean needsQuotedIdentifier;
+  protected String quoteCharacter;
+  protected String passWord = null;
+  protected String url = null;
+  protected String userName = null;
+  protected URI[] validationServers = null; // The list of servers the database/partition/table can locate on
+  protected boolean verbose = false;
+  protected SchemaToolCommandLine cmdLine;
+
+  private static String homeDir;
 
   private static String findHomeDir() {
     // If METASTORE_HOME is set, use it, else use HIVE_HOME for backwards compatibility.
@@ -100,54 +79,129 @@ public class MetastoreSchemaTool {
     return homeDir == null ? System.getenv("HIVE_HOME") : homeDir;
   }
 
-  private MetastoreSchemaTool(String dbType) throws HiveMetaException {
-    this(findHomeDir(), MetastoreConf.newMetastoreConf(), dbType);
+  @VisibleForTesting
+  public static void setHomeDirForTesting() {
+    homeDir = System.getProperty("test.tmp.dir", "target/tmp");
   }
 
-  MetastoreSchemaTool(String metastoreHome, Configuration conf, String dbType)
-      throws HiveMetaException {
+  @VisibleForTesting
+  public MetastoreSchemaTool() {
+
+  }
+
+  @VisibleForTesting
+  public void init(String metastoreHome, String[] args, OptionGroup additionalOptions,
+                   Configuration conf) throws HiveMetaException {
+    try {
+      cmdLine = new SchemaToolCommandLine(args, additionalOptions);
+    } catch (ParseException e) {
+      System.err.println("Failed to parse command line. ");
+      throw new HiveMetaException(e);
+    }
+
     if (metastoreHome == null || metastoreHome.isEmpty()) {
       throw new HiveMetaException("No Metastore home directory provided");
     }
     this.conf = conf;
-    this.dbType = dbType;
-    this.needsQuotedIdentifier = getDbCommandParser(dbType).needsQuotedIdentifier();
+    this.dbType = cmdLine.getDbType();
+    this.metaDbType = cmdLine.getMetaDbType();
+    NestedScriptParser parser = getDbCommandParser(dbType, metaDbType);
+    this.needsQuotedIdentifier = parser.needsQuotedIdentifier();
+    this.quoteCharacter = parser.getQuoteCharacter();
     this.metaStoreSchemaInfo = MetaStoreSchemaInfoFactory.get(conf, metastoreHome, dbType);
+
+    if (cmdLine.hasOption("userName")) {
+      setUserName(cmdLine.getOptionValue("userName"));
+    } else {
+      setUserName(getConf().get(MetastoreConf.ConfVars.CONNECTION_USER_NAME.getVarname()));
+    }
+    if (cmdLine.hasOption("passWord")) {
+      setPassWord(cmdLine.getOptionValue("passWord"));
+    } else {
+      try {
+        setPassWord(MetastoreConf.getPassword(getConf(), ConfVars.PWD));
+      } catch (IOException err) {
+        throw new HiveMetaException("Error getting metastore password", err);
+      }
+    }
+    if (cmdLine.hasOption("url")) {
+      setUrl(cmdLine.getOptionValue("url"));
+    }
+    if (cmdLine.hasOption("driver")) {
+      setDriver(cmdLine.getOptionValue("driver"));
+    }
+    if (cmdLine.hasOption("dryRun")) {
+      setDryRun(true);
+    }
+    if (cmdLine.hasOption("verbose")) {
+      setVerbose(true);
+    }
+    if (cmdLine.hasOption("dbOpts")) {
+      setDbOpts(cmdLine.getOptionValue("dbOpts"));
+    }
+    if (cmdLine.hasOption("validate") && cmdLine.hasOption("servers")) {
+      setValidationServers(cmdLine.getOptionValue("servers"));
+    }
+    if (cmdLine.hasOption("hiveUser")) {
+      setHiveUser(cmdLine.getOptionValue("hiveUser"));
+    }
+    if (cmdLine.hasOption("hivePassword")) {
+      setHivePasswd(cmdLine.getOptionValue("hivePassword"));
+    }
+    if (cmdLine.hasOption("hiveDb")) {
+      setHiveDb(cmdLine.getOptionValue("hiveDb"));
+    }
   }
 
-  Configuration getConf() {
+  public Configuration getConf() {
     return conf;
   }
 
-  void setUrl(String url) {
+  protected String getDbType() {
+    return dbType;
+  }
+
+  protected void setUrl(String url) {
     this.url = url;
   }
 
-  void setDriver(String driver) {
+  protected void setDriver(String driver) {
     this.driver = driver;
   }
 
-  void setUserName(String userName) {
+  public void setUserName(String userName) {
     this.userName = userName;
   }
 
-  void setPassWord(String passWord) {
+  public void setPassWord(String passWord) {
     this.passWord = passWord;
   }
 
-  void setDryRun(boolean dryRun) {
+  protected boolean isDryRun() {
+    return dryRun;
+  }
+
+  protected void setDryRun(boolean dryRun) {
     this.dryRun = dryRun;
   }
 
-  void setVerbose(boolean verbose) {
+  protected boolean isVerbose() {
+    return verbose;
+  }
+
+  protected void setVerbose(boolean verbose) {
     this.verbose = verbose;
   }
 
-  private void setDbOpts(String dbOpts) {
+  protected void setDbOpts(String dbOpts) {
     this.dbOpts = dbOpts;
   }
 
-  private void setValidationServers(String servers) {
+  protected URI[] getValidationServers() {
+    return validationServers;
+  }
+
+  protected void setValidationServers(String servers) {
     if(StringUtils.isNotEmpty(servers)) {
       String[] strServers = servers.split(",");
       this.validationServers = new URI[strServers.length];
@@ -157,340 +211,53 @@ public class MetastoreSchemaTool {
     }
   }
 
-  private void setHiveUser(String hiveUser) {
-    this.hiveUser = hiveUser;
-  }
-
-  private void setHivePasswd(String hivePasswd) {
-    this.hivePasswd = hivePasswd;
+  protected String getHiveUser() {
+    return hiveUser;
   }
 
-  private void setHiveDb(String hiveDb) {
-    this.hiveDb = hiveDb;
+  protected void setHiveUser(String hiveUser) {
+    this.hiveUser = hiveUser;
   }
 
-  private static int usage(Options cmdLineOptions) {
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp("schemaTool", cmdLineOptions);
-    return 1;
+  protected String getHivePasswd() {
+    return hivePasswd;
   }
 
-  Connection getConnectionToMetastore(boolean printInfo)
-      throws HiveMetaException {
-    return HiveSchemaHelper.getConnectionToMetastore(userName,
-        passWord, url, driver, printInfo, conf, null);
+  protected void setHivePasswd(String hivePasswd) {
+    this.hivePasswd = hivePasswd;
   }
 
-  private NestedScriptParser getDbCommandParser(String dbType) {
-    return HiveSchemaHelper.getDbCommandParser(dbType, dbOpts, userName,
-	passWord, conf, null, true);
+  protected String getHiveDb() {
+    return hiveDb;
   }
 
-  /***
-   * Print Hive version and schema version
-   */
-  private void showInfo() throws HiveMetaException {
-    String hiveVersion = metaStoreSchemaInfo.getHiveSchemaVersion();
-    String dbVersion = metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(true));
-    System.out.println("Hive distribution version:\t " + hiveVersion);
-    System.out.println("Metastore schema version:\t " + dbVersion);
-    assertCompatibleVersion(hiveVersion, dbVersion);
-  }
-
-  boolean validateLocations(Connection conn, URI[] defaultServers) throws HiveMetaException {
-    System.out.println("Validating DFS locations");
-    boolean rtn;
-    rtn = checkMetaStoreDBLocation(conn, defaultServers);
-    rtn = checkMetaStoreTableLocation(conn, defaultServers) && rtn;
-    rtn = checkMetaStorePartitionLocation(conn, defaultServers) && rtn;
-    rtn = checkMetaStoreSkewedColumnsLocation(conn, defaultServers) && rtn;
-    System.out.println((rtn ? "Succeeded" : "Failed") + " in DFS location validation.");
-    return rtn;
+  protected void setHiveDb(String hiveDb) {
+    this.hiveDb = hiveDb;
   }
 
-  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;
+  protected SchemaToolCommandLine getCmdLine() {
+    return cmdLine;
   }
 
-  private boolean checkMetaStoreDBLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String dbLoc;
-    boolean isValid = true;
-    int numOfInvalid = 0;
-    if (needsQuotedIdentifier) {
-      dbLoc = "select dbt.\"DB_ID\", dbt.\"NAME\", dbt.\"DB_LOCATION_URI\" from \"DBS\" dbt order by dbt.\"DB_ID\" ";
-    } else {
-      dbLoc = "select dbt.DB_ID, dbt.NAME, dbt.DB_LOCATION_URI from DBS dbt order by dbt.DB_ID";
-    }
-
-    try(Statement stmt = conn.createStatement();
-        ResultSet res = stmt.executeQuery(dbLoc)) {
-      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);
-    }
-    if (numOfInvalid > 0) {
-      isValid = false;
-    }
-    return isValid;
-  }
-
-  private boolean checkMetaStoreTableLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String tabLoc, tabIDRange;
-    boolean isValid = true;
-    int numOfInvalid = 0;
-    if (needsQuotedIdentifier) {
-      tabIDRange = "select max(\"TBL_ID\"), min(\"TBL_ID\") from \"TBLS\" ";
-    } else {
-      tabIDRange = "select max(TBL_ID), min(TBL_ID) from TBLS";
-    }
-
-    if (needsQuotedIdentifier) {
-      tabLoc = "select tbl.\"TBL_ID\", tbl.\"TBL_NAME\", sd.\"LOCATION\", dbt.\"DB_ID\", dbt.\"NAME\" from \"TBLS\" tbl inner join " +
-    "\"SDS\" sd on tbl.\"SD_ID\" = sd.\"SD_ID\" and tbl.\"TBL_TYPE\" != '" + TableType.VIRTUAL_VIEW +
-    "' and tbl.\"TBL_ID\" >= ? and tbl.\"TBL_ID\"<= ? " + "inner join \"DBS\" dbt on tbl.\"DB_ID\" = dbt.\"DB_ID\" order by tbl.\"TBL_ID\" ";
-    } else {
-      tabLoc = "select tbl.TBL_ID, tbl.TBL_NAME, sd.LOCATION, dbt.DB_ID, dbt.NAME from TBLS tbl join SDS sd on tbl.SD_ID = sd.SD_ID and tbl.TBL_TYPE !='"
-      + TableType.VIRTUAL_VIEW + "' and tbl.TBL_ID >= ? and tbl.TBL_ID <= ?  inner join DBS dbt on tbl.DB_ID = dbt.DB_ID order by tbl.TBL_ID";
-    }
-
-    long maxID = 0, minID = 0;
-    long rtnSize = 2000;
-
-    try {
-      Statement stmt = conn.createStatement();
-      ResultSet res = stmt.executeQuery(tabIDRange);
-      if (res.next()) {
-        maxID = res.getLong(1);
-        minID = res.getLong(2);
-      }
-      res.close();
-      stmt.close();
-      PreparedStatement pStmt = conn.prepareStatement(tabLoc);
-      while (minID <= maxID) {
-        pStmt.setLong(1, minID);
-        pStmt.setLong(2, minID + rtnSize);
-        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++;
-          }
-        }
-        res.close();
-        minID += rtnSize + 1;
-
-      }
-      pStmt.close();
-
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get Table Location Info.", e);
-    }
-    if (numOfInvalid > 0) {
-      isValid = false;
-    }
-    return isValid;
-  }
-
-  private boolean checkMetaStorePartitionLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String partLoc, partIDRange;
-    boolean isValid = true;
-    int numOfInvalid = 0;
-    if (needsQuotedIdentifier) {
-      partIDRange = "select max(\"PART_ID\"), min(\"PART_ID\") from \"PARTITIONS\" ";
-    } else {
-      partIDRange = "select max(PART_ID), min(PART_ID) from PARTITIONS";
-    }
-
-    if (needsQuotedIdentifier) {
-      partLoc = "select pt.\"PART_ID\", pt.\"PART_NAME\", sd.\"LOCATION\", tbl.\"TBL_ID\", tbl.\"TBL_NAME\",dbt.\"DB_ID\", dbt.\"NAME\" from \"PARTITIONS\" pt "
-           + "inner join \"SDS\" sd on pt.\"SD_ID\" = sd.\"SD_ID\" and pt.\"PART_ID\" >= ? and pt.\"PART_ID\"<= ? "
-           + " inner join \"TBLS\" tbl on pt.\"TBL_ID\" = tbl.\"TBL_ID\" inner join "
-           + "\"DBS\" dbt on tbl.\"DB_ID\" = dbt.\"DB_ID\" order by tbl.\"TBL_ID\" ";
-    } else {
-      partLoc = "select pt.PART_ID, pt.PART_NAME, sd.LOCATION, tbl.TBL_ID, tbl.TBL_NAME, dbt.DB_ID, dbt.NAME from PARTITIONS pt "
-          + "inner join SDS sd on pt.SD_ID = sd.SD_ID and pt.PART_ID >= ? and pt.PART_ID <= ?  "
-          + "inner join TBLS tbl on tbl.TBL_ID = pt.TBL_ID inner join DBS dbt on tbl.DB_ID = dbt.DB_ID order by tbl.TBL_ID ";
-    }
-
-    long maxID = 0, minID = 0;
-    long rtnSize = 2000;
-
-    try {
-      Statement stmt = conn.createStatement();
-      ResultSet res = stmt.executeQuery(partIDRange);
-      if (res.next()) {
-        maxID = res.getLong(1);
-        minID = res.getLong(2);
-      }
-      res.close();
-      stmt.close();
-      PreparedStatement pStmt = conn.prepareStatement(partLoc);
-      while (minID <= maxID) {
-        pStmt.setLong(1, minID);
-        pStmt.setLong(2, minID + rtnSize);
-        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++;
-          }
-        }
-        res.close();
-        minID += rtnSize + 1;
-      }
-      pStmt.close();
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get Partiton Location Info.", e);
-    }
-    if (numOfInvalid > 0) {
-      isValid = false;
-    }
-    return isValid;
+  public Connection getConnectionToMetastore(boolean printInfo) throws HiveMetaException {
+    return HiveSchemaHelper.getConnectionToMetastore(userName,
+        passWord, url, driver, printInfo, conf, null);
   }
 
-  private boolean checkMetaStoreSkewedColumnsLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String skewedColLoc, skewedColIDRange;
-    boolean isValid = true;
-    int numOfInvalid = 0;
-    if (needsQuotedIdentifier) {
-      skewedColIDRange = "select max(\"STRING_LIST_ID_KID\"), min(\"STRING_LIST_ID_KID\") from \"SKEWED_COL_VALUE_LOC_MAP\" ";
-    } else {
-      skewedColIDRange = "select max(STRING_LIST_ID_KID), min(STRING_LIST_ID_KID) from SKEWED_COL_VALUE_LOC_MAP";
-    }
-
-    if (needsQuotedIdentifier) {
-      skewedColLoc = "select t.\"TBL_NAME\", t.\"TBL_ID\", sk.\"STRING_LIST_ID_KID\", sk.\"LOCATION\", db.\"NAME\", db.\"DB_ID\" "
-           + " from \"TBLS\" t, \"SDS\" s, \"DBS\" db, \"SKEWED_COL_VALUE_LOC_MAP\" sk "
-           + "where sk.\"SD_ID\" = s.\"SD_ID\" and s.\"SD_ID\" = t.\"SD_ID\" and t.\"DB_ID\" = db.\"DB_ID\" and "
-           + "sk.\"STRING_LIST_ID_KID\" >= ? and sk.\"STRING_LIST_ID_KID\" <= ? order by t.\"TBL_ID\" ";
-    } else {
-      skewedColLoc = "select t.TBL_NAME, t.TBL_ID, sk.STRING_LIST_ID_KID, sk.LOCATION, db.NAME, db.DB_ID from TBLS t, SDS s, DBS db, SKEWED_COL_VALUE_LOC_MAP sk "
-           + "where sk.SD_ID = s.SD_ID and s.SD_ID = t.SD_ID and t.DB_ID = db.DB_ID and sk.STRING_LIST_ID_KID >= ? and sk.STRING_LIST_ID_KID <= ? order by t.TBL_ID ";
-    }
-
-    long maxID = 0, minID = 0;
-    long rtnSize = 2000;
-
-    try {
-      Statement stmt = conn.createStatement();
-      ResultSet res = stmt.executeQuery(skewedColIDRange);
-      if (res.next()) {
-        maxID = res.getLong(1);
-        minID = res.getLong(2);
-      }
-      res.close();
-      stmt.close();
-      PreparedStatement pStmt = conn.prepareStatement(skewedColLoc);
-      while (minID <= maxID) {
-        pStmt.setLong(1, minID);
-        pStmt.setLong(2, minID + rtnSize);
-        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++;
-          }
-        }
-        res.close();
-        minID += rtnSize + 1;
-      }
-      pStmt.close();
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get skewed columns location info.", e);
-    }
-    if (numOfInvalid > 0) {
-      isValid = false;
-    }
-    return isValid;
+  protected NestedScriptParser getDbCommandParser(String dbType, String metaDbType) {
+    return HiveSchemaHelper.getDbCommandParser(dbType, dbOpts, userName,
+        passWord, conf, null, true);
   }
 
-  /**
-   * 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) {
-      logAndPrintToError(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)) {
-          logAndPrintToError(entity + ", Location: "+ entityLocation + ", Error: missing location scheme.");
-          isValid = false;
-        } else if (StringUtils.isEmpty(path)) {
-          logAndPrintToError(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) {
-            logAndPrintToError(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, "/")) {
-          logAndPrintToError(entity + ", Location: "+ entityLocation + ", Warn: location set to root, not a recommended config.");
-        }
-      } catch (Exception pe) {
-        logAndPrintToError(entity + ", Error: invalid location - " + pe.getMessage());
-        isValid =false;
-      }
-    }
-
-    return isValid;
+  protected MetaStoreConnectionInfo getConnectionInfo(boolean printInfo) {
+    return new MetaStoreConnectionInfo(userName, passWord, url, driver, printInfo, conf,
+        dbType, hiveDb);
   }
 
-  // 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);
-    }
+  protected IMetaStoreSchemaInfo getMetaStoreSchemaInfo() {
+    return metaStoreSchemaInfo;
   }
 
-
   /**
    * check if the current schema version in metastore matches the Hive version
    */
@@ -505,7 +272,7 @@ public class MetastoreSchemaTool {
     assertCompatibleVersion(metaStoreSchemaInfo.getHiveSchemaVersion(), newSchemaVersion);
   }
 
-  private void assertCompatibleVersion(String hiveSchemaVersion, String dbSchemaVersion)
+  protected void assertCompatibleVersion(String hiveSchemaVersion, String dbSchemaVersion)
       throws HiveMetaException {
     if (!metaStoreSchemaInfo.isVersionCompatible(hiveSchemaVersion, dbSchemaVersion)) {
       throw new HiveMetaException("Metastore schema version is not compatible. Hive Version: "
@@ -513,504 +280,22 @@ public class MetastoreSchemaTool {
     }
   }
 
-  /**
-   * Perform metastore schema upgrade. extract the current schema version from metastore
-   */
-  void doUpgrade() throws HiveMetaException {
-    String fromVersion =
-      metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(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);
-  }
-
-  private MetaStoreConnectionInfo getConnectionInfo(boolean printInfo) {
-    return new MetaStoreConnectionInfo(userName, passWord, url, driver, printInfo, conf,
-        dbType, hiveDb);
-  }
-  /**
-   * Perform metastore schema upgrade
-   *
-   * @param fromSchemaVer
-   *          Existing version of the metastore. If null, then read from the metastore
-   */
-  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) {
-          runPreUpgrade(scriptDir, scriptFile);
-          runSqlLine(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
-   *
-   */
-  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
-   */
-  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) {
-        runSqlLine(initScriptDir, initScriptFile);
-        System.out.println("Initialization script completed");
-      }
-    } catch (IOException e) {
-      throw new HiveMetaException("Schema initialization FAILED!" +
-          " Metastore state would be inconsistent !!", e);
-    }
-  }
-
-  private void doCreateUser() throws HiveMetaException {
-    testConnectionToMetastore();
-    System.out.println("Starting user creation");
-
-    String scriptDir = metaStoreSchemaInfo.getMetaStoreScriptDir();
-    String protoCreateFile = metaStoreSchemaInfo.getCreateUserScript();
-
-    try {
-      File createFile = subUserAndPassword(scriptDir, protoCreateFile);
-      System.out.println("Creation script " + createFile.getAbsolutePath());
-      if (!dryRun) {
-        if ("oracle".equals(dbType)) oracleCreateUserHack(createFile);
-        else runSqlLine(createFile.getParent(), createFile.getName());
-        System.out.println("User creation completed");
-      }
-    } catch (IOException e) {
-      throw new HiveMetaException("User creation FAILED!" +
-          " Metastore unusable !!", e);
-    }
-  }
-
-  private File subUserAndPassword(String parent, String filename) throws IOException {
-    File createFile = File.createTempFile("create-hive-user-" + dbType, ".sql");
-    BufferedWriter writer = new BufferedWriter(new FileWriter(createFile));
-    File proto = new File(parent, filename);
-    BufferedReader reader = new BufferedReader(new FileReader(proto));
-    reader.lines()
-        .map(s -> s.replace("_REPLACE_WITH_USER_", hiveUser)
-            .replace("_REPLACE_WITH_PASSWD_", hivePasswd)
-            .replace("_REPLACE_WITH_DB_", hiveDb))
-        .forEach(s -> {
-            try {
-              writer.write(s);
-              writer.newLine();
-            } catch (IOException e) {
-              throw new RuntimeException("Unable to write to tmp file ", e);
-            }
-          });
-    reader.close();
-    writer.close();
-    return createFile;
-  }
-
-  private void oracleCreateUserHack(File createFile) throws HiveMetaException {
-    LOG.debug("Found oracle, hacking our way through it rather than using SqlLine");
-    try (BufferedReader reader = new BufferedReader(new FileReader(createFile))) {
-      try (Connection conn = getConnectionToMetastore(false)) {
-        try (Statement stmt = conn.createStatement()) {
-          reader.lines()
-              .forEach(s -> {
-                assert s.charAt(s.length() - 1) == ';';
-                try {
-                  stmt.execute(s.substring(0, s.length() - 1));
-                } catch (SQLException e) {
-                  LOG.error("statement <" + s.substring(0, s.length() - 2) + "> failed", e);
-                  throw new RuntimeException(e);
-                }
-              });
-        }
-      }
-    } catch (IOException e) {
-      LOG.error("Caught IOException trying to read modified create user script " +
-          createFile.getAbsolutePath(), e);
-      throw new HiveMetaException(e);
-    } catch (HiveMetaException e) {
-      LOG.error("Failed to connect to RDBMS", e);
-      throw e;
-    } catch (SQLException e) {
-      LOG.error("Got SQLException", e);
-    }
-  }
-
-  private int doValidate() throws HiveMetaException {
-    System.out.println("Starting metastore validation\n");
-    Connection conn = getConnectionToMetastore(false);
-    boolean success = true;
-    try {
-      if (validateSchemaVersions()) {
-        System.out.println("[SUCCESS]\n");
-      } else {
-        success = false;
-        System.out.println("[FAIL]\n");
-      }
-      if (validateSequences(conn)) {
-        System.out.println("[SUCCESS]\n");
-      } else {
-        success = false;
-        System.out.println("[FAIL]\n");
-      }
-      if (validateSchemaTables(conn)) {
-        System.out.println("[SUCCESS]\n");
-      } else {
-        success = false;
-        System.out.println("[FAIL]\n");
-      }
-      if (validateLocations(conn, this.validationServers)) {
-        System.out.println("[SUCCESS]\n");
-      } else {
-        System.out.println("[WARN]\n");
-      }
-      if (validateColumnNullValues(conn)) {
-        System.out.println("[SUCCESS]\n");
-      } else {
-        System.out.println("[WARN]\n");
-      }
-    } finally {
-      if (conn != null) {
-        try {
-          conn.close();
-        } catch (SQLException e) {
-          // Not a lot you can do here.
-        }
-      }
-    }
-
-    System.out.print("Done with metastore validation: ");
-    if (!success) {
-      System.out.println("[FAIL]");
-      return 1;
-    } else {
-      System.out.println("[SUCCESS]");
-      return 0;
-    }
-  }
-
-  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 (String seqName : seqNameToTable.keySet()) {
-        String tableName = seqNameToTable.get(seqName).getLeft();
-        String tableKey = seqNameToTable.get(seqName).getRight();
-        String fullSequenceName = "org.apache.hadoop.hive.metastore.model." + seqName;
-        String seqQuery = needsQuotedIdentifier ?
-            ("select t.\"NEXT_VAL\" from \"SEQUENCE_TABLE\" t WHERE t.\"SEQUENCE_NAME\"=? order by t.\"SEQUENCE_NAME\" ")
-            : ("select t.NEXT_VAL from SEQUENCE_TABLE t WHERE t.SEQUENCE_NAME=? order by t.SEQUENCE_NAME ");
-        String maxIdQuery = needsQuotedIdentifier ?
-            ("select max(\"" + tableKey + "\") from \"" + tableName + "\"")
-            : ("select max(" + tableKey + ") from " + tableName);
-
-        ResultSet res = stmt.executeQuery(maxIdQuery);
-        if (res.next()) {
-          long maxId = res.getLong(1);
-          if (maxId > 0) {
-            PreparedStatement pStmt = conn.prepareStatement(seqQuery);
-            pStmt.setString(1, fullSequenceName);
-            ResultSet resSeq = pStmt.executeQuery();
-            if (!resSeq.next()) {
-              isValid = false;
-              logAndPrintToError("Missing SEQUENCE_NAME " + seqName + " from SEQUENCE_TABLE");
-            } else if (resSeq.getLong(1) < maxId) {
-              isValid = false;
-              logAndPrintToError("NEXT_VAL for " + seqName + " in SEQUENCE_TABLE < max(" +
-                  tableKey + ") in " + tableName);
-            }
-          }
-        }
-      }
-
-      System.out.println((isValid ? "Succeeded" :"Failed") + " in sequence number validation for SEQUENCE_TABLE.");
-      return isValid;
-    } catch(SQLException e) {
-        throw new HiveMetaException("Failed to validate sequence number for SEQUENCE_TABLE", e);
-    }
-  }
-
-  boolean validateSchemaVersions() throws HiveMetaException {
-    System.out.println("Validating schema version");
-    try {
-      String newSchemaVersion = metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(false));
-      assertCompatibleVersion(metaStoreSchemaInfo.getHiveSchemaVersion(), 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")) {
-        logAndPrintToError(hme.getMessage());
-        System.out.println("Failed in schema version validation.");
-        return false;
-      } else {
-        throw hme;
-      }
-    }
-    System.out.println("Succeeded in schema version validation.");
-    return true;
-  }
-
-  boolean validateSchemaTables(Connection conn) throws HiveMetaException {
-    String version;
-    ResultSet rs              = null;
-    DatabaseMetaData metadata;
-    List<String> dbTables     = new ArrayList<>();
-    List<String> schemaTables = new ArrayList<>();
-    List<String> subScripts   = new ArrayList<>();
-    Connection hmsConn;
-
-    System.out.println("Validating metastore schema tables");
-    try {
-      version = metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(false));
-    } catch (HiveMetaException he) {
-      logAndPrintToError("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());
-      return false;
-    }
-
-    // re-open the hms connection
-    hmsConn = getConnectionToMetastore(false);
-
-    LOG.debug("Validating tables in the schema for version " + version);
-    try {
-      metadata       = conn.getMetaData();
-      String[] types = {"TABLE"};
-      rs             = metadata.getTables(null, hmsConn.getSchema(), "%", types);
-      String table;
-
-      while (rs.next()) {
-        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());
-    } finally {
-      if (rs != null) {
-        try {
-          rs.close();
-        } catch (SQLException e) {
-          // Not a lot you can do here.
-        }
-      }
-    }
-
-    // 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
-    String baseDir    = new File(metaStoreSchemaInfo.getMetaStoreScriptDir()).getParent();
-    String schemaFile = new File(metaStoreSchemaInfo.getMetaStoreScriptDir(),
-        metaStoreSchemaInfo.generateInitFileName(version)).getPath();
-    try {
-      LOG.debug("Parsing schema script " + schemaFile);
-      subScripts.addAll(findCreateTable(schemaFile, schemaTables));
-      while (subScripts.size() > 0) {
-        schemaFile = baseDir + "/" + dbType + "/" + subScripts.remove(0);
-        LOG.debug("Parsing subscript " + schemaFile);
-        subScripts.addAll(findCreateTable(schemaFile, schemaTables));
-      }
-    } catch (Exception e) {
-      logAndPrintToError("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);
-      logAndPrintToError("Table(s) [ " + Arrays.toString(schemaTables.toArray())
-          + " ] are missing from the metastore database schema.");
-      System.out.println("Failed in schema table validation.");
-      return false;
-    } else {
-      System.out.println("Succeeded in schema table validation.");
-      return true;
-    }
-  }
-
-  private List<String> findCreateTable(String path, List<String> tableList)
-      throws Exception {
-    NestedScriptParser sp           = HiveSchemaHelper.getDbCommandParser(dbType, true);
-    Matcher matcher;
-    Pattern regexp;
-    List<String> subs               = new ArrayList<>();
-    int groupNo                     = 2;
-
-    regexp = Pattern.compile("CREATE TABLE(\\s+IF NOT EXISTS)?\\s+(\\S+).*");
-
-    if (!(new File(path)).exists()) {
-      throw new Exception(path + " does not exist. Potentially incorrect version in the metastore VERSION table");
-    }
-
-    try (
-      BufferedReader reader = new BufferedReader(new FileReader(path))
-    ){
-      String line;
-      while ((line = reader.readLine()) != null) {
-        if (sp.isNestedScript(line)) {
-          String subScript;
-          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 = regexp.matcher(line);
-
-        if (matcher.find()) {
-          String table = matcher.group(groupNo);
-          if (dbType.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;
-  }
-
-  boolean validateColumnNullValues(Connection conn) throws HiveMetaException {
-    System.out.println("Validating columns for incorrect NULL values.");
-    boolean isValid = true;
-    try {
-      Statement stmt = conn.createStatement();
-      String tblQuery = needsQuotedIdentifier ?
-          ("select t.* from \"TBLS\" t WHERE t.\"SD_ID\" IS NULL and (t.\"TBL_TYPE\"='" + TableType.EXTERNAL_TABLE + "' or t.\"TBL_TYPE\"='" + TableType.MANAGED_TABLE + "') order by t.\"TBL_ID\" ")
-          : ("select t.* from TBLS t WHERE t.SD_ID IS NULL and (t.TBL_TYPE='" + TableType.EXTERNAL_TABLE + "' or t.TBL_TYPE='" + TableType.MANAGED_TABLE + "') order by t.TBL_ID ");
-
-      ResultSet res = stmt.executeQuery(tblQuery);
-      while (res.next()) {
-         long tableId = res.getLong("TBL_ID");
-         String tableName = res.getString("TBL_NAME");
-         String tableType = res.getString("TBL_TYPE");
-         isValid = false;
-         logAndPrintToError("SD_ID in TBLS should not be NULL for Table Name=" + tableName + ", Table ID=" + tableId + ", Table Type=" + tableType);
-      }
-
-      System.out.println((isValid ? "Succeeded" : "Failed") + " in column validation for incorrect NULL values.");
-      return isValid;
-    } catch(SQLException e) {
-        throw new HiveMetaException("Failed to validate columns for incorrect NULL values", e);
-    }
-  }
-
-  /**
-   *  Run pre-upgrade scripts corresponding to a given upgrade script,
-   *  if any exist. The errors from pre-upgrade are ignored.
-   *  Pre-upgrade scripts typically contain setup statements which
-   *  may fail on some database versions and failure is ignorable.
-   *
-   *  @param scriptDir upgrade script directory name
-   *  @param scriptFile upgrade script file name
-   */
-  private void runPreUpgrade(String scriptDir, String scriptFile) {
-    for (int i = 0;; i++) {
-      String preUpgradeScript =
-          metaStoreSchemaInfo.getPreUpgradeScriptName(i, scriptFile);
-      File preUpgradeScriptFile = new File(scriptDir, preUpgradeScript);
-      if (!preUpgradeScriptFile.isFile()) {
-        break;
-      }
-
-      try {
-        runSqlLine(scriptDir, preUpgradeScript);
-        System.out.println("Completed " + preUpgradeScript);
-      } catch (Exception e) {
-        // Ignore the pre-upgrade script errors
-        logAndPrintToError("Warning in pre-upgrade script " + preUpgradeScript + ": "
-            + e.getMessage());
-        if (verbose) {
-          e.printStackTrace();
-        }
-      }
-    }
-  }
-
   /***
-   * Run beeline with the given metastore script. Flatten the nested scripts
-   * into single file.
+   * Execute a given metastore script. This default version uses sqlline to execute the files,
+   * which requires only running one file.  Subclasses can use other executors.
+   * @param scriptDir directory script is in
+   * @param scriptFile file in the directory to run
+   * @throws IOException if it cannot read the file or directory
+   * @throws HiveMetaException default implementation never throws this
    */
-  private void runSqlLine(String scriptDir, String scriptFile)
-      throws IOException, HiveMetaException {
+  protected void execSql(String scriptDir, String scriptFile) throws IOException, HiveMetaException {
 
-    // This no longer does expansions of run commands in the files as it used to.  Instead it
-    // depends on the developers to have already unrolled those in the files.
-    runSqlLine(scriptDir + File.separatorChar + scriptFile);
+    execSql(scriptDir + File.separatorChar + scriptFile);
   }
 
   // Generate the beeline args per hive conf and execute the given script
-  void runSqlLine(String sqlScriptFile) throws IOException {
-    CommandBuilder builder = new CommandBuilder(conf, url, driver,
-        userName, passWord, sqlScriptFile);
+  protected void execSql(String sqlScriptFile) throws IOException {
+    CommandBuilder builder = new CommandBuilder(conf, url, driver, userName, passWord, sqlScriptFile);
 
     // run the script using SqlLine
     SqlLine sqlLine = new SqlLine();
@@ -1025,11 +310,6 @@ public class MetastoreSchemaTool {
       sqlLine.setOutputStream(new PrintStream(out));
       System.setProperty("sqlline.silent", "true");
     }
-    //sqlLine.getOpts().setAllowMultiLineCommand(false);
-    //System.setProperty("sqlline.isolation","TRANSACTION_READ_COMMITTED");
-    // We can be pretty sure that an entire line can be processed as a single command since
-    // we always add a line separator at the end while calling dbCommandParser.buildCommand.
-    //sqlLine.getOpts().setEntireLineAsCommand(true);
     LOG.info("Going to run command <" + builder.buildToLog() + ">");
     SqlLine.Status status = sqlLine.begin(builder.buildToRun(), null, false);
     if (LOG.isDebugEnabled() && outputForLog != null) {
@@ -1041,38 +321,55 @@ public class MetastoreSchemaTool {
     }
   }
 
-  static class CommandBuilder {
-    private final Configuration conf;
-    private final String userName;
-    private final String password;
-    private final String sqlScriptFile;
-    private final String driver;
-    private final String url;
+  // test the connection metastore using the config property
+  protected void testConnectionToMetastore() throws HiveMetaException {
+    Connection conn = getConnectionToMetastore(true);
+    try {
+      conn.close();
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to close metastore connection", e);
+    }
+  }
+
+  // Quote if the database requires it
+  protected String quote(String stmt) {
+    stmt = stmt.replace("<q>", needsQuotedIdentifier ? quoteCharacter : "");
+    stmt = stmt.replace("<qa>", quoteCharacter);
+    return stmt;
+  }
+
+  protected static class CommandBuilder {
+    protected final String userName;
+    protected final String password;
+    protected final String sqlScriptFile;
+    protected final String driver;
+    protected final String url;
 
-    CommandBuilder(Configuration conf, String url, String driver,
-                   String userName, String password, String sqlScriptFile) {
-      this.conf = conf;
+    protected CommandBuilder(Configuration conf, String url, String driver, String userName,
+                             String password, String sqlScriptFile) throws IOException {
       this.userName = userName;
       this.password = password;
-      this.url = url;
-      this.driver = driver;
+      this.url = url == null ?
+          HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECT_URL_KEY, conf) : url;
+      this.driver = driver == null ?
+          HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECTION_DRIVER, conf) : driver;
       this.sqlScriptFile = sqlScriptFile;
     }
 
-    String[] buildToRun() throws IOException {
+    public String[] buildToRun() throws IOException {
       return argsWith(password);
     }
 
-    String buildToLog() throws IOException {
+    public String buildToLog() throws IOException {
       logScript();
       return StringUtils.join(argsWith(PASSWD_MASK), " ");
     }
 
-    private String[] argsWith(String password) throws IOException {
+    protected String[] argsWith(String password) throws IOException {
       return new String[]
         {
-          "-u", url == null ? MetastoreConf.getVar(conf, ConfVars.CONNECT_URL_KEY) : url,
-          "-d", driver == null ? MetastoreConf.getVar(conf, ConfVars.CONNECTION_DRIVER) : driver,
+          "-u", url,
+          "-d", driver,
           "-n", userName,
           "-p", password,
           "--isolation=TRANSACTION_READ_COMMITTED",
@@ -1094,194 +391,51 @@ public class MetastoreSchemaTool {
   }
 
   // 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");
-    Option validateOpt = new Option("validate", "Validate the database");
-    Option createUserOpt = new Option("createUser", "Create the Hive user, use admin user and " +
-        "password with this");
-
-    OptionGroup optGroup = new OptionGroup();
-    optGroup.addOption(upgradeOpt).addOption(initOpt).
-                addOption(help).addOption(upgradeFromOpt).addOption(createUserOpt)
-                .addOption(initToOpt).addOption(infoOpt).addOption(validateOpt);
-    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 hiveUserOpt = OptionBuilder
-        .hasArg()
-        .withDescription("Hive user (for use with createUser)")
-        .create("hiveUser");
-    Option hivePasswdOpt = OptionBuilder
-        .hasArg()
-        .withDescription("Hive password (for use with createUser)")
-        .create("hivePassword");
-    Option hiveDbOpt = OptionBuilder
-        .hasArg()
-        .withDescription("Hive database (for use with createUser)")
-        .create("hiveDb");
-    Option dbTypeOpt = OptionBuilder.withArgName("databaseType")
-                .hasArgs().withDescription("Metastore database type")
-                .create("dbType");
-    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");
-    cmdLineOptions.addOption(help);
-    cmdLineOptions.addOption(dryRunOpt);
-    cmdLineOptions.addOption(userNameOpt);
-    cmdLineOptions.addOption(passwdOpt);
-    cmdLineOptions.addOption(dbTypeOpt);
-    cmdLineOptions.addOption(verboseOpt);
-    cmdLineOptions.addOption(urlOpt);
-    cmdLineOptions.addOption(driverOpt);
-    cmdLineOptions.addOption(dbOpts);
-    cmdLineOptions.addOption(serversOpt);
-    cmdLineOptions.addOption(hiveUserOpt);
-    cmdLineOptions.addOption(hivePasswdOpt);
-    cmdLineOptions.addOption(hiveDbOpt);
-    cmdLineOptions.addOptionGroup(optGroup);
-  }
-
   private static void logAndPrintToError(String errmsg) {
     LOG.error(errmsg);
     System.err.println(errmsg);
   }
 
   public static void main(String[] args) {
-    System.exit(run(args));
+    MetastoreSchemaTool tool = new MetastoreSchemaTool();
+    System.exit(tool.run(args));
   }
 
-  public static int run(String[] args) {
-    LOG.debug("Going to run command: " + StringUtils.join(args, " "));
-    CommandLineParser parser = new GnuParser();
-    CommandLine line;
-    String dbType;
-    String schemaVer;
-    Options cmdLineOptions = new Options();
-
-    // Argument handling
-    initOptions(cmdLineOptions);
-    try {
-      line = parser.parse(cmdLineOptions, args);
-    } catch (ParseException e) {
-      logAndPrintToError("HiveSchemaTool:Parsing failed.  Reason: " + e.getLocalizedMessage());
-      return usage(cmdLineOptions);
-    }
-
-    assert line != null;
-    if (line.hasOption("help")) {
-      HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp("schemaTool", cmdLineOptions);
-      return 1;
-    }
-
-    if (line.hasOption("dbType")) {
-      dbType = line.getOptionValue("dbType");
-      if ((!dbType.equalsIgnoreCase(HiveSchemaHelper.DB_DERBY) &&
-          !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_MSSQL) &&
-          !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_MYSQL) &&
-          !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_POSTGRACE) && !dbType
-          .equalsIgnoreCase(HiveSchemaHelper.DB_ORACLE))) {
-        logAndPrintToError("Unsupported dbType " + dbType);
-        return usage(cmdLineOptions);
-      }
-    } else {
-      logAndPrintToError("no dbType supplied");
-      return usage(cmdLineOptions);
-    }
+  public int run(String[] args) {
+    return run(findHomeDir(), args, null, MetastoreConf.newMetastoreConf());
+  }
 
-    System.setProperty(ConfVars.SCHEMA_VERIFICATION.toString(), "true");
+  public int run(String metastoreHome, String[] args, OptionGroup additionalOptions,
+                 Configuration conf) {
     try {
-      MetastoreSchemaTool schemaTool = new MetastoreSchemaTool(dbType);
-
-      if (line.hasOption("userName")) {
-        schemaTool.setUserName(line.getOptionValue("userName"));
-      } else {
-        schemaTool.setUserName(MetastoreConf.getVar(schemaTool.getConf(), ConfVars.CONNECTION_USER_NAME));
-      }
-      if (line.hasOption("passWord")) {
-        schemaTool.setPassWord(line.getOptionValue("passWord"));
-      } else {
-        try {
-          schemaTool.setPassWord(MetastoreConf.getPassword(schemaTool.getConf(), ConfVars.PWD));
-        } catch (IOException err) {
-          throw new HiveMetaException("Error getting metastore password", err);
-        }
-      }
-      if (line.hasOption("hiveUser")) {
-        schemaTool.setHiveUser(line.getOptionValue("hiveUser"));
-      }
-      if (line.hasOption("hivePassword")) {
-        schemaTool.setHivePasswd(line.getOptionValue("hivePassword"));
-      }
-      if (line.hasOption("hiveDb")) {
-        schemaTool.setHiveDb(line.getOptionValue("hiveDb"));
-      }
-      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"));
-      }
-      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 if (line.hasOption("validate")) {
-        return schemaTool.doValidate();
-      } else if (line.hasOption("createUser")) {
-        schemaTool.doCreateUser();
+      init(metastoreHome, args, additionalOptions, conf);
+      SchemaToolTask task;
+      if (cmdLine.hasOption("info")) {
+        task = new SchemaToolTaskInfo();
+      } else if (cmdLine.hasOption("upgradeSchema") || cmdLine.hasOption("upgradeSchemaFrom")) {
+        task = new SchemaToolTaskUpgrade();
+      } else if (cmdLine.hasOption("initSchema") || cmdLine.hasOption("initSchemaTo")) {
+        task = new SchemaToolTaskInit();
+      } else if (cmdLine.hasOption("validate")) {
+        task = new SchemaToolTaskValidate();
+      } else if (cmdLine.hasOption("createCatalog")) {
+        task = new SchemaToolTaskCreateCatalog();
+      } else if (cmdLine.hasOption("alterCatalog")) {
+        task = new SchemaToolTaskAlterCatalog();
+      } else if (cmdLine.hasOption("moveDatabase")) {
+        task = new SchemaToolTaskMoveDatabase();
+      } else if (cmdLine.hasOption("moveTable")) {
+        task = new SchemaToolTaskMoveTable();
+      } else if (cmdLine.hasOption("createUser")) {
+        task = new SchemaToolTaskCreateUser();
       } else {
-        logAndPrintToError("no valid option supplied");
-        return usage(cmdLineOptions);
+        throw new HiveMetaException("No task defined!");
       }
+
+      task.setHiveSchemaTool(this);
+      task.setCommandLineArguments(cmdLine);
+      task.execute();
+      return 0;
     } catch (HiveMetaException e) {
       logAndPrintToError(e.getMessage());
       if (e.getCause() != null) {
@@ -1290,19 +444,17 @@ public class MetastoreSchemaTool {
             + t.getClass().getName() + " : "
             + t.getMessage());
         if (e.getCause() instanceof SQLException) {
-          logAndPrintToError("SQL Error code: " + ((SQLException)t).getErrorCode());
+          logAndPrintToError("SQL Error code: " + ((SQLException) t).getErrorCode());
         }
       }
-      if (line.hasOption("verbose")) {
+      if (cmdLine.hasOption("verbose")) {
         e.printStackTrace();
       } else {
         logAndPrintToError("Use --verbose for detailed stacktrace.");
       }
       logAndPrintToError("*** schemaTool failed ***");
       return 1;
-    }
-    System.out.println("schemaTool completed");
-    return 0;
 
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolCommandLine.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolCommandLine.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolCommandLine.java
new file mode 100644
index 0000000..7eba2b7
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolCommandLine.java
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+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;
+
+public class SchemaToolCommandLine {
+  private final Options cmdLineOptions;
+
+  @SuppressWarnings("static-access")
+  private Options createOptions(OptionGroup additionalOptions) {
+    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");
+    Option createUserOpt = new Option("createUser", "Create the Hive user, set hiveUser to the db" +
+        " admin user and the hive password to the db admin password with this");
+
+    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)
+      .addOption(createUserOpt);
+    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 hiveUserOpt = OptionBuilder
+        .hasArg()
+        .withDescription("Hive user (for use with createUser)")
+        .create("hiveUser");
+    Option hivePasswdOpt = OptionBuilder
+        .hasArg()
+        .withDescription("Hive password (for use with createUser)")
+        .create("hivePassword");
+    Option hiveDbOpt = OptionBuilder
+        .hasArg()
+        .withDescription("Hive database (for use with createUser)")
+        .create("hiveDb");
+    /*
+    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);
+    options.addOption(hiveUserOpt);
+    options.addOption(hivePasswdOpt);
+    options.addOption(hiveDbOpt);
+    if (additionalOptions != null) options.addOptionGroup(additionalOptions);
+
+    return options;
+  }
+
+  private final CommandLine cl;
+  private final String dbType;
+  private final String metaDbType;
+
+  public SchemaToolCommandLine(String[] args, OptionGroup additionalOptions) throws ParseException {
+    cmdLineOptions = createOptions(additionalOptions);
+    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);
+    }
+  }
+
+  public String getDbType() {
+    return dbType;
+  }
+
+  public 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/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTask.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTask.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTask.java
new file mode 100644
index 0000000..87ef6b5
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTask.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+
+public abstract class SchemaToolTask {
+  protected MetastoreSchemaTool schemaTool;
+
+  void setHiveSchemaTool(MetastoreSchemaTool schemaTool) {
+    this.schemaTool = schemaTool;
+  }
+
+  abstract void setCommandLineArguments(SchemaToolCommandLine cl);
+
+  abstract void execute() throws HiveMetaException;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/eae5225f/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskAlterCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskAlterCatalog.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskAlterCatalog.java
new file mode 100644
index 0000000..4e3b3d1
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskAlterCatalog.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+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 SchemaToolTaskAlterCatalog extends SchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(SchemaToolTaskAlterCatalog.class.getName());
+
+  private String catName;
+  private String location;
+  private String description;
+
+  @Override
+  void setCommandLineArguments(SchemaToolCommandLine 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/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskCreateCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskCreateCatalog.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskCreateCatalog.java
new file mode 100644
index 0000000..7857970
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskCreateCatalog.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+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 SchemaToolTaskCreateCatalog extends SchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(SchemaToolTaskCreateCatalog.class.getName());
+
+  private String catName;
+  private String location;
+  private String description;
+  private boolean ifNotExists;
+
+  @Override
+  void setCommandLineArguments(SchemaToolCommandLine 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/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskCreateUser.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskCreateUser.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskCreateUser.java
new file mode 100644
index 0000000..a9ad005
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SchemaToolTaskCreateUser.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.tools;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+public class SchemaToolTaskCreateUser extends SchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(SchemaToolTaskCreateUser.class);
+
+  @Override
+  void setCommandLineArguments(SchemaToolCommandLine cl) {
+
+  }
+
+  @Override
+  void execute() throws HiveMetaException {
+    schemaTool.testConnectionToMetastore();
+    System.out.println("Starting user creation");
+
+    String scriptDir = schemaTool.getMetaStoreSchemaInfo().getMetaStoreScriptDir();
+    String protoCreateFile = schemaTool.getMetaStoreSchemaInfo().getCreateUserScript();
+
+    try {
+      File createFile = subUserAndPassword(scriptDir, protoCreateFile);
+      System.out.println("Creation script " + createFile.getAbsolutePath());
+      if (!schemaTool.isDryRun()) {
+        if ("oracle".equals(schemaTool.getDbType())) oracleCreateUserHack(createFile);
+        else schemaTool.execSql(createFile.getParent(), createFile.getName());
+        System.out.println("User creation completed");
+      }
+    } catch (IOException e) {
+      throw new HiveMetaException("User creation FAILED!" +
+          " Metastore unusable !!", e);
+    }
+
+  }
+
+  private File subUserAndPassword(String parent, String filename) throws IOException {
+    File createFile = File.createTempFile("create-hive-user-" + schemaTool.getDbType(), ".sql");
+    BufferedWriter writer = new BufferedWriter(new FileWriter(createFile));
+    File proto = new File(parent, filename);
+    BufferedReader reader = new BufferedReader(new FileReader(proto));
+    reader.lines()
+        .map(s -> s.replace("_REPLACE_WITH_USER_", schemaTool.getHiveUser())
+            .replace("_REPLACE_WITH_PASSWD_", schemaTool.getHivePasswd())
+            .replace("_REPLACE_WITH_DB_", schemaTool.getHiveDb()))
+        .forEach(s -> {
+          try {
+            writer.write(s);
+            writer.newLine();
+          } catch (IOException e) {
+            throw new RuntimeException("Unable to write to tmp file ", e);
+          }
+        });
+    reader.close();
+    writer.close();
+    return createFile;
+  }
+
+  private void oracleCreateUserHack(File createFile) throws HiveMetaException {
+    LOG.debug("Found oracle, hacking our way through it rather than using SqlLine");
+    try (BufferedReader reader = new BufferedReader(new FileReader(createFile))) {
+      try (Connection conn = schemaTool.getConnectionToMetastore(false)) {
+        try (Statement stmt = conn.createStatement()) {
+          reader.lines()
+              .forEach(s -> {
+                assert s.charAt(s.length() - 1) == ';';
+                try {
+                  stmt.execute(s.substring(0, s.length() - 1));
+                } catch (SQLException e) {
+                  LOG.error("statement <" + s.substring(0, s.length() - 2) + "> failed", e);
+                  throw new RuntimeException(e);
+                }
+              });
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Caught IOException trying to read modified create user script " +
+          createFile.getAbsolutePath(), e);
+      throw new HiveMetaException(e);
+    } catch (HiveMetaException e) {
+      LOG.error("Failed to connect to RDBMS", e);
+      throw e;
+    } catch (SQLException e) {
+      LOG.error("Got SQLException", e);
+    }
+  }
+}


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

Posted by ga...@apache.org.
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);
-    }
-  }
-}