You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by rv...@apache.org on 2017/02/24 19:52:04 UTC

[01/50] [abbrv] bigtop git commit: Beginnings of Hive JDBC tests. Tests written, not yet run.

Repository: bigtop
Updated Branches:
  refs/heads/BIGTOP-2666 [created] 1df0c5d55


Beginnings of Hive JDBC tests.  Tests written, not yet run.


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

Branch: refs/heads/BIGTOP-2666
Commit: ffd069428a6ae52d9abd3eea5aa9ac4207253167
Parents: ae81761
Author: Alan Gates <ga...@hortonworks.com>
Authored: Mon Sep 26 14:40:00 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:45:54 2017 -0800

----------------------------------------------------------------------
 bigtop-tests/spec-tests/runtime/build.gradle    |   3 +
 .../org/odpi/specs/runtime/hive/TestJdbc.java   | 604 +++++++++++++++++++
 2 files changed, 607 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/ffd06942/bigtop-tests/spec-tests/runtime/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/build.gradle b/bigtop-tests/spec-tests/runtime/build.gradle
index 3042ecd..1285a19 100644
--- a/bigtop-tests/spec-tests/runtime/build.gradle
+++ b/bigtop-tests/spec-tests/runtime/build.gradle
@@ -18,6 +18,9 @@
 def junitVersion = '4.11'
 dependencies {
   compile group: 'junit', name: 'junit', version: junitVersion, transitive: 'true'
+  compile group: 'commons-logging', name: 'commons-logging', version: '1.1.3'
+  compile group: 'org.apache.hive', name: 'hive-jdbc', version: '1.2.1'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-common', version: '2.7.2'
   if (System.env.HADOOP_CONF_DIR) testRuntime files(System.env.HADOOP_CONF_DIR)
 }
 

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ffd06942/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestJdbc.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestJdbc.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestJdbc.java
new file mode 100644
index 0000000..c518302
--- /dev/null
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestJdbc.java
@@ -0,0 +1,604 @@
+package org.odpi.specs.runtime.hive;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.Statement;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.Properties;
+
+/**
+ * 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.
+ */
+public class TestJdbc {
+  public static final Log LOG = LogFactory.getLog(TestJdbc.class.getName());
+
+  private static final String URL = "odpiHiveTestJdbcUrl";
+  private static final String USER = "odpiHiveTestJdbcUser";
+  private static final String PASSWD = "odpiHiveTestJdbcPassword";
+
+  protected static Connection conn;
+
+  @BeforeClass
+  public static void connectToJdbc() throws SQLException {
+    // Assume they've put the URL for the JDBC driver in an environment variable.
+    String jdbcUrl = System.getProperty(URL);
+    String jdbcUser = System.getProperty(USER);
+    String jdbcPasswd = System.getProperty(PASSWD);
+
+    LOG.info("URL is " + jdbcUrl);
+    LOG.info("User is " + jdbcUser);
+    LOG.info("Passwd is " + jdbcPasswd);
+    LOG.info("Passwd is null " + (jdbcPasswd == null));
+
+    if (jdbcUrl == null || jdbcUser == null ) {
+      String msg = "You must set the URL, user, and password for the JDBC connection before\n" +
+          "running these tests.  Each is set as a Java property:\n" +
+          URL + " = the URL " +
+          USER + " = the user " +
+          PASSWD + " = the password ";
+      throw new RuntimeException(msg);
+    }
+
+    Properties props = new Properties();
+    props.put("user", jdbcUser);
+    if (jdbcPasswd != null && jdbcPasswd != "") props.put("password", jdbcPasswd);
+    conn = DriverManager.getConnection(jdbcUrl, props);
+  }
+
+  @AfterClass
+  public static void closeJdbc() throws SQLException {
+    if (conn != null) conn.close();
+  }
+
+  /**
+   * Test simple non-statement related class.  setSchema is tested elsewhere because there's work
+   * to do for that one.  Similarly with getMetadata.
+   * @throws SQLException
+   */
+  @Test
+  public void nonStatementCalls() throws SQLException {
+    conn.clearWarnings();
+
+    boolean isAutoCommit = conn.getAutoCommit();
+    LOG.debug("Auto commit is " + isAutoCommit);
+
+    String catalog = conn.getCatalog();
+    LOG.debug("Catalog is " + catalog);
+
+    String schema = conn.getSchema();
+    LOG.debug("Schema is " + schema);
+
+    int txnIsolation = conn.getTransactionIsolation();
+    LOG.debug("Transaction Isolation is " + txnIsolation);
+
+    SQLWarning warning = conn.getWarnings();
+    while (warning != null) {
+      LOG.debug("Found a warning: " + warning.getMessage());
+      warning = warning.getNextWarning();
+    }
+
+    boolean closed = conn.isClosed();
+    LOG.debug("Is closed? " + closed);
+
+    boolean readOnly = conn.isReadOnly();
+    LOG.debug("Is read only?" + readOnly);
+
+    // Hive doesn't support catalogs, so setting this to whatever should be fine.  If we have
+    // non-Hive systems trying to pass this setting it to a non-valid catalog name may cause
+    // issues, so we may need to make this value configurable or something.
+    conn.setCatalog("fred");
+  }
+
+  /**
+   * Test simple DatabaseMetaData calls.  getColumns is tested elsewhere, as we need to call
+   * that on a valid table.  Same with getFunctions.
+   * @throws SQLException
+   */
+  @Test
+  public void databaseMetaDataCalls() throws SQLException {
+    DatabaseMetaData md = conn.getMetaData();
+
+    boolean boolrc = md.allTablesAreSelectable();
+    LOG.debug("All tables are selectable? " + boolrc);
+
+    String strrc = md.getCatalogSeparator();
+    LOG.debug("Catalog separator " + strrc);
+
+    strrc = md.getCatalogTerm();
+    LOG.debug("Catalog term " + strrc);
+
+    ResultSet rs = md.getCatalogs();
+    while (rs.next()) {
+      strrc = rs.getString(1);
+      LOG.debug("Found catalog " + strrc);
+    }
+
+    Connection c = md.getConnection();
+
+    int intrc = md.getDatabaseMajorVersion();
+    LOG.debug("DB major version is " + intrc);
+
+    intrc = md.getDatabaseMinorVersion();
+    LOG.debug("DB minor version is " + intrc);
+
+    strrc = md.getDatabaseProductName();
+    LOG.debug("DB product name is " + strrc);
+
+    strrc = md.getDatabaseProductVersion();
+    LOG.debug("DB product version is " + strrc);
+
+    intrc = md.getDefaultTransactionIsolation();
+    LOG.debug("Default transaction isolation is " + intrc);
+
+    intrc = md.getDriverMajorVersion();
+    LOG.debug("Driver major version is " + intrc);
+
+    intrc = md.getDriverMinorVersion();
+    LOG.debug("Driver minor version is " + intrc);
+
+    strrc = md.getDriverName();
+    LOG.debug("Driver name is " + strrc);
+
+    strrc = md.getDriverVersion();
+    LOG.debug("Driver version is " + strrc);
+
+    strrc = md.getExtraNameCharacters();
+    LOG.debug("Extra name characters is " + strrc);
+
+    strrc = md.getIdentifierQuoteString();
+    LOG.debug("Identifier quote string is " + strrc);
+
+    // In Hive 1.2 this always returns an empty RS
+    rs = md.getImportedKeys("a", "b", "d");
+
+    // In Hive 1.2 this always returns an empty RS
+    rs = md.getIndexInfo("a", "b", "d", true, true);
+
+    intrc = md.getJDBCMajorVersion();
+    LOG.debug("JDBC major version is " + intrc);
+
+    intrc = md.getJDBCMinorVersion();
+    LOG.debug("JDBC minor version is " + intrc);
+
+    intrc = md.getMaxColumnNameLength();
+    LOG.debug("Maximum column name length is " + intrc);
+
+    strrc = md.getNumericFunctions();
+    LOG.debug("Numeric functions are " + strrc);
+
+    // In Hive 1.2 this always returns an empty RS
+    rs = md.getPrimaryKeys("a", "b", "d");
+
+    // In Hive 1.2 this always returns an empty RS
+    rs = md.getProcedureColumns("a", "b", "d", "e");
+
+    strrc = md.getProcedureTerm();
+    LOG.debug("Procedures are called " + strrc);
+
+    // In Hive 1.2 this always returns an empty RS
+    rs = md.getProcedures("a", "b", "d");
+
+    strrc = md.getSchemaTerm();
+    LOG.debug("Schemas are called " + strrc);
+
+    rs = md.getSchemas();
+    while (rs.next()) {
+      strrc = rs.getString(1);
+      LOG.debug("Found schema " + strrc);
+    }
+
+    strrc = md.getSearchStringEscape();
+    LOG.debug("Search string escape is " + strrc);
+
+    strrc = md.getStringFunctions();
+    LOG.debug("String functions are " + strrc);
+
+    strrc = md.getSystemFunctions();
+    LOG.debug("System functions are " + strrc);
+
+    rs = md.getTableTypes();
+    while (rs.next()) {
+      strrc = rs.getString(1);
+      LOG.debug("Found table type " + strrc);
+    }
+
+    strrc = md.getTimeDateFunctions();
+    LOG.debug("Time/date functions are " + strrc);
+
+    rs = md.getTypeInfo();
+    while (rs.next()) {
+      strrc = rs.getString(1);
+      LOG.debug("Found type " + strrc);
+    }
+
+    // In Hive 1.2 this always returns an empty RS
+    rs = md.getUDTs("a", "b", "d", null);
+
+    boolrc = md.supportsAlterTableWithAddColumn();
+    LOG.debug("Supports alter table with add column? " + boolrc);
+
+    boolrc = md.supportsAlterTableWithDropColumn();
+    LOG.debug("Supports alter table with drop column? " + boolrc);
+
+    boolrc = md.supportsBatchUpdates();
+    LOG.debug("Supports batch updates? " + boolrc);
+
+    boolrc = md.supportsCatalogsInDataManipulation();
+    LOG.debug("Supports catalogs in data manipulation? " + boolrc);
+
+    boolrc = md.supportsCatalogsInIndexDefinitions();
+    LOG.debug("Supports catalogs in index definition? " + boolrc);
+
+    boolrc = md.supportsCatalogsInPrivilegeDefinitions();
+    LOG.debug("Supports catalogs in privilege definition? " + boolrc);
+
+    boolrc = md.supportsCatalogsInProcedureCalls();
+    LOG.debug("Supports catalogs in procedure calls? " + boolrc);
+
+    boolrc = md.supportsCatalogsInTableDefinitions();
+    LOG.debug("Supports catalogs in table definition? " + boolrc);
+
+    boolrc = md.supportsColumnAliasing();
+    LOG.debug("Supports column aliasing? " + boolrc);
+
+    boolrc = md.supportsFullOuterJoins();
+    LOG.debug("Supports full outer joins? " + boolrc);
+
+    boolrc = md.supportsGroupBy();
+    LOG.debug("Supports group by? " + boolrc);
+
+    boolrc = md.supportsLimitedOuterJoins();
+    LOG.debug("Supports limited outer joins? " + boolrc);
+
+    boolrc = md.supportsMultipleResultSets();
+    LOG.debug("Supports limited outer joins? " + boolrc);
+
+    boolrc = md.supportsNonNullableColumns();
+    LOG.debug("Supports non-nullable columns? " + boolrc);
+
+    boolrc = md.supportsOuterJoins();
+    LOG.debug("Supports outer joins? " + boolrc);
+
+    boolrc = md.supportsPositionedDelete();
+    LOG.debug("Supports positioned delete? " + boolrc);
+
+    boolrc = md.supportsPositionedUpdate();
+    LOG.debug("Supports positioned update? " + boolrc);
+
+    boolrc = md.supportsResultSetHoldability(ResultSet.HOLD_CURSORS_OVER_COMMIT);
+    LOG.debug("Supports result set holdability? " + boolrc);
+
+    boolrc = md.supportsResultSetType(ResultSet.HOLD_CURSORS_OVER_COMMIT);
+    LOG.debug("Supports result set type? " + boolrc);
+
+    boolrc = md.supportsSavepoints();
+    LOG.debug("Supports savepoints? " + boolrc);
+
+    boolrc = md.supportsSchemasInDataManipulation();
+    LOG.debug("Supports schemas in data manipulation? " + boolrc);
+
+    boolrc = md.supportsSchemasInIndexDefinitions();
+    LOG.debug("Supports schemas in index definitions? " + boolrc);
+
+    boolrc = md.supportsSchemasInPrivilegeDefinitions();
+    LOG.debug("Supports schemas in privilege definitions? " + boolrc);
+
+    boolrc = md.supportsSchemasInProcedureCalls();
+    LOG.debug("Supports schemas in procedure calls? " + boolrc);
+
+    boolrc = md.supportsSchemasInTableDefinitions();
+    LOG.debug("Supports schemas in table definitions? " + boolrc);
+
+    boolrc = md.supportsSelectForUpdate();
+    LOG.debug("Supports select for update? " + boolrc);
+
+    boolrc = md.supportsStoredProcedures();
+    LOG.debug("Supports stored procedures? " + boolrc);
+
+    boolrc = md.supportsTransactions();
+    LOG.debug("Supports transactions? " + boolrc);
+
+    boolrc = md.supportsUnion();
+    LOG.debug("Supports union? " + boolrc);
+
+    boolrc = md.supportsUnionAll();
+    LOG.debug("Supports union all? " + boolrc);
+
+  }
+
+  @Test
+  public void setSchema() throws SQLException {
+    try (Statement stmt = conn.createStatement(ResultSet.TYPE_SCROLL_INSENSITIVE,
+        ResultSet.CONCUR_READ_ONLY)) {
+
+      final String dbName = "odpi_jdbc_test_db";
+
+      stmt.execute("drop database if exists " + dbName);
+      stmt.execute("create database " + dbName);
+
+      conn.setSchema(dbName);
+
+      DatabaseMetaData md = conn.getMetaData();
+
+      ResultSet rs = md.getSchemas(null, dbName);
+
+      while (rs.next()) {
+        String schemaName = rs.getString(2);
+        LOG.debug("Schema name is " + schemaName);
+      }
+
+      final String tableName = "odpi_jdbc_test_table";
+      stmt.execute("drop table if exists " + tableName);
+      stmt.execute("create table " + tableName + " (i int, s varchar(32))");
+
+      rs = md.getTables(null, dbName, tableName, null);
+      while (rs.next()) {
+        String tName = rs.getString(3);
+        LOG.debug("Schema name is " + tName);
+      }
+
+      rs = md.getColumns(null, dbName, tableName, "i");
+      while (rs.next()) {
+        String colName = rs.getString(4);
+        LOG.debug("Schema name is " + colName);
+      }
+
+      rs = md.getFunctions(null, dbName, null);
+      while (rs.next()) {
+        String funcName = rs.getString(3);
+        LOG.debug("Schema name is " + funcName);
+      }
+    }
+  }
+
+  @Test
+  public void statement() throws SQLException {
+    try (Statement stmt = conn.createStatement()) {
+      stmt.cancel();
+    }
+
+    try (Statement stmt = conn.createStatement()) {
+      stmt.clearWarnings();
+
+      final String tableName = "odpi_jdbc_statement_test_table";
+
+      stmt.execute("drop table if exists " + tableName);
+      stmt.execute("create table " + tableName + " (a int, b varchar(32))");
+
+      stmt.executeUpdate("insert into " + tableName + " values (1, 'abc'), (2, 'def')");
+
+      int intrc = stmt.getUpdateCount();
+      LOG.debug("Update count is " + intrc);
+
+      ResultSet rs = stmt.executeQuery("select * from " + tableName);
+      while (rs.next()) {
+        LOG.debug("Fetched " + rs.getInt(1) + "," + rs.getString(2));
+      }
+
+      Connection localConn = stmt.getConnection();
+
+      intrc = stmt.getFetchDirection();
+      LOG.debug("Fetch direction is " + intrc);
+
+      intrc = stmt.getFetchSize();
+      LOG.debug("Fetch size is " + intrc);
+
+      intrc = stmt.getMaxRows();
+      LOG.debug("max rows is " + intrc);
+
+      boolean boolrc = stmt.getMoreResults();
+      LOG.debug("more results is " + boolrc);
+
+      intrc = stmt.getQueryTimeout();
+      LOG.debug("query timeout is " + intrc);
+
+      stmt.execute("select * from " + tableName);
+      rs = stmt.getResultSet();
+      while (rs.next()) {
+        LOG.debug("Fetched " + rs.getInt(1) + "," + rs.getString(2));
+      }
+
+      intrc = stmt.getResultSetType();
+      LOG.debug("result set type is " + intrc);
+
+      SQLWarning warning = stmt.getWarnings();
+      while (warning != null) {
+        LOG.debug("Found a warning: " + warning.getMessage());
+        warning = warning.getNextWarning();
+      }
+
+      boolrc = stmt.isClosed();
+      LOG.debug("is closed " + boolrc);
+
+      boolrc = stmt.isCloseOnCompletion();
+      LOG.debug("is close on completion " + boolrc);
+
+      boolrc = stmt.isPoolable();
+      LOG.debug("is poolable " + boolrc);
+
+      stmt.setFetchDirection(ResultSet.FETCH_FORWARD);
+      stmt.setFetchSize(500);
+      stmt.setMaxRows(500);
+      stmt.setQueryTimeout(30);
+    }
+  }
+
+  @Test
+  public void preparedStmtAndResultSet() throws SQLException {
+    final String tableName = "odpi_jdbc_psars_test_table";
+    try (Statement stmt = conn.createStatement()) {
+      stmt.execute("drop table if exists " + tableName);
+      stmt.execute("create table " + tableName + " (bd decimal(15,3), by binary, bo boolean, " +
+          "ti tinyint, da date, db double, fl float, i int, lo long, sh short, st string," +
+          " tm timestamp");
+    }
+
+    try (PreparedStatement ps = conn.prepareStatement("insert into " + tableName +
+        " values (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?")) {
+      ps.setBigDecimal(1, new BigDecimal("1234231421343.123"));
+      ps.setBinaryStream(2, new ByteArrayInputStream("abcdef".getBytes()));
+      ps.setBoolean(3, true);
+      ps.setByte(4, (byte)1);
+      ps.setDate(5, new Date(1999, 12, 1));
+      ps.setDouble(6, 3.141592654);
+      ps.setFloat(7, 3.14f);
+      ps.setInt(8, 3);
+      ps.setLong(9, 10L);
+      ps.setShort(10, (short)20);
+      ps.setString(11, "abc");
+      ps.setTimestamp(12, new Timestamp(1999, 12, 1, 13, 23, 40, 0));
+      ps.executeUpdate();
+    }
+
+    try (PreparedStatement ps = conn.prepareStatement("insert into " + tableName + " (i, st) " +
+        "values(?, ?)", ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_READ_ONLY)) {
+      ps.setNull(1, Types.INTEGER);
+      ps.setObject(2, "mary had a little lamb");
+      ps.executeUpdate();
+      ps.setNull(1, Types.INTEGER, null);
+      ps.setString(2, "its fleece was white as snow");
+      ps.clearParameters();
+      ps.setNull(1, Types.INTEGER, null);
+      ps.setString(2, "its fleece was white as snow");
+      ps.execute();
+
+    }
+
+    try (Statement stmt = conn.createStatement()) {
+
+      ResultSet rs = stmt.executeQuery("select * from " + tableName);
+
+      ResultSetMetaData md = rs.getMetaData();
+
+      int colCnt = md.getColumnCount();
+      LOG.debug("Column count is " + colCnt);
+
+      for (int i = 1; i <= colCnt; i++) {
+        LOG.debug("Looking at column " + i);
+        String strrc = md.getColumnClassName(i);
+        LOG.debug("Column class name is " + strrc);
+
+        int intrc = md.getColumnDisplaySize(i);
+        LOG.debug("Column display size is " + intrc);
+
+        strrc = md.getColumnLabel(i);
+        LOG.debug("Column label is " + strrc);
+
+        strrc = md.getColumnName(i);
+        LOG.debug("Column name is " + strrc);
+
+        intrc = md.getColumnType(i);
+        LOG.debug("Column type is " + intrc);
+
+        strrc = md.getColumnTypeName(i);
+        LOG.debug("Column type name is " + strrc);
+
+        intrc = md.getPrecision(i);
+        LOG.debug("Precision is " + intrc);
+
+        intrc = md.getScale(i);
+        LOG.debug("Scale is " + intrc);
+
+        boolean boolrc = md.isAutoIncrement(i);
+        LOG.debug("Is auto increment? " + boolrc);
+
+        boolrc = md.isCaseSensitive(i);
+        LOG.debug("Is case sensitive? " + boolrc);
+
+        boolrc = md.isCurrency(i);
+        LOG.debug("Is currency? " + boolrc);
+
+        intrc = md.getScale(i);
+        LOG.debug("Scale is " + intrc);
+
+        intrc = md.isNullable(i);
+        LOG.debug("Is nullable? " + intrc);
+
+        boolrc = md.isReadOnly(i);
+        LOG.debug("Is read only? " + boolrc);
+
+      }
+
+      while (rs.next()) {
+        LOG.debug("bd = " + rs.getBigDecimal(1));
+        LOG.debug("bd = " + rs.getBigDecimal("bd"));
+        LOG.debug("bd = " + rs.getBigDecimal(1, 2));
+        LOG.debug("bd = " + rs.getBigDecimal("bd", 2));
+        LOG.debug("by = " + rs.getBinaryStream(2));
+        LOG.debug("by = " + rs.getBinaryStream("by"));
+        LOG.debug("bo = " + rs.getBoolean(3));
+        LOG.debug("bo = " + rs.getBoolean("bo"));
+        LOG.debug("ti = " + rs.getByte(4));
+        LOG.debug("ti = " + rs.getByte("ti"));
+        LOG.debug("da = " + rs.getDate(5));
+        LOG.debug("da = " + rs.getDate("da"));
+        LOG.debug("db = " + rs.getDouble(6));
+        LOG.debug("db = " + rs.getDouble("db"));
+        LOG.debug("fl = " + rs.getFloat(7));
+        LOG.debug("fl = " + rs.getFloat("fl"));
+        LOG.debug("i = " + rs.getInt(8));
+        LOG.debug("i = " + rs.getInt("i"));
+        LOG.debug("lo = " + rs.getLong(9));
+        LOG.debug("lo = " + rs.getLong("lo"));
+        LOG.debug("sh = " + rs.getShort(10));
+        LOG.debug("sh = " + rs.getShort("sh"));
+        LOG.debug("st = " + rs.getString(11));
+        LOG.debug("st = " + rs.getString("st"));
+        LOG.debug("tm = " + rs.getTimestamp(12));
+        LOG.debug("tm = " + rs.getTimestamp("tm"));
+        LOG.debug("tm = " + rs.getObject(12));
+        LOG.debug("tm = " + rs.getObject("tm"));
+        LOG.debug("tm was null " + rs.wasNull());
+      }
+      LOG.debug("bd is column " + rs.findColumn("bd"));
+
+      int intrc = rs.getConcurrency();
+      LOG.debug("concurrency " + intrc);
+
+      intrc = rs.getFetchDirection();
+      LOG.debug("fetch direction " + intrc);
+
+      intrc = rs.getType();
+      LOG.debug("type " + intrc);
+
+      Statement copy = rs.getStatement();
+
+      SQLWarning warning = rs.getWarnings();
+      while (warning != null) {
+        LOG.debug("Found a warning: " + warning.getMessage());
+        warning = warning.getNextWarning();
+      }
+      rs.clearWarnings();
+    }
+  }
+}


[42/50] [abbrv] bigtop git commit: Added TODOs to outline remaining work.

Posted by rv...@apache.org.
Added TODOs to outline remaining work.


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

Branch: refs/heads/BIGTOP-2666
Commit: e5518cdb461165b0db045f05f81b436343f2f4a0
Parents: 1b4f286
Author: Alan Gates <ga...@hortonworks.com>
Authored: Tue Nov 8 07:42:28 2016 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:24 2017 -0800

----------------------------------------------------------------------
 .../src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java | 6 ++++++
 .../test/java/org/odpi/specs/runtime/hive/TestHCatalog.java   | 7 ++++++-
 2 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/e5518cdb/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java b/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
index 6fcfe37..a6ff375 100644
--- a/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
+++ b/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
@@ -72,6 +72,12 @@ public class HCatalogMR extends Configured implements Tool {
     HCatOutputFormat.setSchema(job, HCatSchemaUtils.getHCatSchema(outputSchemaStr));
     job.setOutputFormatClass(HCatOutputFormat.class);
 
+    // TODO All four of these jars need to be in the distributed cache of the job for the job to
+    // succeed.  I loaded them into a known location in HDFS to get them in the cache.  There may
+    // be a way to load them from a file on the gateway machine.  We could also put in a hdfs dfs
+    // -put operation into a gradle step as part of the build so that the jars are picked up from
+    // the distribution and put in a known location in HDFS from when they can be picked up in
+    // the distributed cache.
     job.addCacheArchive(new URI("hdfs:/user/gates/hive-hcatalog-core-1.2.1.jar"));
     job.addCacheArchive(new URI("hdfs:/user/gates/hive-metastore-1.2.1.jar"));
     job.addCacheArchive(new URI("hdfs:/user/gates/hive-exec-1.2.1.jar"));

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e5518cdb/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
index bb237d8..87e3eb0 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
@@ -118,7 +118,7 @@ public class TestHCatalog {
         new HCatFieldSchema("word", HCatFieldSchema.Type.STRING, ""),
         new HCatFieldSchema("count", HCatFieldSchema.Type.INT, "")));
 
-    // TODO Could I use HCatWriter hear and the reader to read it?
+    // LATER Could I use HCatWriter here and the reader to read it?
     // Write some stuff into a file in the location of the table
     table = client.getTable("default", inputTable);
     String inputFile = table.getSd().getLocation() + "/input";
@@ -132,12 +132,17 @@ public class TestHCatalog {
     out.close();
 
     Map<String, String> env = new HashMap<>();
+    // TODO These need to be set from the environment rather than hard wired
     env.put("HADOOP_HOME","/Users/gates/grid/odpi-testing/hadoop-2.7.3");
     env.put("HADOOP_CLASSPATH", "/Users/gates/grid/odpi-testing/apache-hive-1.2.1-bin/hcatalog/share/hcatalog/hive-hcatalog-core-1.2.1.jar");
     env.put("HIVE_HOME", "/Users/gates/grid/odpi-testing/apache-hive-1.2.1-bin");
     Map<String, String> results = HiveHelper.execCommand(new CommandLine("/Users/gates/grid/odpi-testing/apache-hive-1.2.1-bin/bin/hive")
         .addArgument("--service")
         .addArgument("jar")
+        // TODO This is the jar built by gradle, but I didn't know how to take the jar built in
+        // the build phase and reference it in the test phase.  Perhaps a move operation could be
+        // put in the middle so the jar is moved to a known location that can be referenced here,
+        // or maybe gradle can pass in its working directory so that we can reference it from there.
         .addArgument("/Users/gates/git/bigtop/runtime-1.2.0-SNAPSHOT.jar")
         .addArgument(HCatalogMR.class.getName())
         .addArgument(inputTable)


[13/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/metainfo.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/metainfo.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/metainfo.xml
new file mode 100755
index 0000000..d6ecbed
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/metainfo.xml
@@ -0,0 +1,518 @@
+<metainfo>
+  <schemaVersion>2.0</schemaVersion>
+  <services><service>
+    <name>HIVE</name>
+    <displayName>Hive</displayName>
+    <version>1.2.1</version>
+    <comment>Data warehouse system for ad-hoc queries &amp; analysis of large datasets and table &amp; storage management service</comment>
+    <components>
+        <component>
+            <name>HCAT</name>
+            <displayName>HCat Client</displayName>
+            <category>CLIENT</category>
+            <deleted>false</deleted>
+            <cardinality>0+</cardinality>
+            <versionAdvertised>true</versionAdvertised>
+            <versionAdvertisedInternal>false</versionAdvertisedInternal>
+            <commandScript>
+                <script>scripts/hcat_client.py</script>
+                <scriptType>PYTHON</scriptType>
+                <timeout>0</timeout>
+            </commandScript>
+            <configFiles>
+                <configFile>
+                    <type>env</type>
+                    <fileName>hcat-env.sh</fileName>
+                    <dictionaryName>hcat-env</dictionaryName>
+                </configFile>
+            </configFiles>
+            <configuration-dependencies>
+                <config-type>hive-site</config-type>
+            </configuration-dependencies>
+            <recovery_enabled>false</recovery_enabled>
+        </component>
+        <component>
+            <name>HIVE_SERVER</name>
+            <displayName>HiveServer2</displayName>
+            <category>MASTER</category>
+            <deleted>false</deleted>
+            <cardinality>1+</cardinality>
+            <versionAdvertisedInternal>true</versionAdvertisedInternal>
+            <commandScript>
+                <script>scripts/hive_server.py</script>
+                <scriptType>PYTHON</scriptType>
+                <timeout>0</timeout>
+            </commandScript>
+            <logs>
+                <log>
+                    <logId>hive_hiveserver2</logId>
+                    <primary>true</primary>
+                </log>
+            </logs>
+            <dependencies>
+                <dependency>
+                    <auto-deploy>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>TEZ/TEZ_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <co-locate>HIVE/HIVE_SERVER</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER</coLocate>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <co-locate>HIVE/HIVE_SERVER</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER</coLocate>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>ZOOKEEPER/ZOOKEEPER_SERVER</name>
+                    <scope>cluster</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>YARN/YARN_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>MAPREDUCE2/MAPREDUCE2_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+            </dependencies>
+            <configuration-dependencies>
+                <config-type>hiveserver2-site</config-type>
+                <config-type>hive-site</config-type>
+            </configuration-dependencies>
+            <recovery_enabled>false</recovery_enabled>
+            <reassignAllowed>true</reassignAllowed>
+        </component>
+        <component>
+            <name>HIVE_CLIENT</name>
+            <displayName>Hive Client</displayName>
+            <category>CLIENT</category>
+            <deleted>false</deleted>
+            <cardinality>1+</cardinality>
+            <versionAdvertised>true</versionAdvertised>
+            <versionAdvertisedInternal>false</versionAdvertisedInternal>
+            <commandScript>
+                <script>scripts/hive_client.py</script>
+                <scriptType>PYTHON</scriptType>
+                <timeout>0</timeout>
+            </commandScript>
+            <configFiles>
+                <configFile>
+                    <type>xml</type>
+                    <fileName>hive-site.xml</fileName>
+                    <dictionaryName>hive-site</dictionaryName>
+                </configFile>
+                <configFile>
+                    <type>env</type>
+                    <fileName>hive-env.sh</fileName>
+                    <dictionaryName>hive-env</dictionaryName>
+                </configFile>
+                <configFile>
+                    <type>env</type>
+                    <fileName>hive-log4j.properties</fileName>
+                    <dictionaryName>hive-log4j</dictionaryName>
+                </configFile>
+                <configFile>
+                    <type>env</type>
+                    <fileName>hive-exec-log4j.properties</fileName>
+                    <dictionaryName>hive-exec-log4j</dictionaryName>
+                </configFile>
+            </configFiles>
+            <configuration-dependencies>
+                <config-type>hive-site</config-type>
+            </configuration-dependencies>
+            <recovery_enabled>false</recovery_enabled>
+        </component>
+        <component>
+            <name>WEBHCAT_SERVER</name>
+            <displayName>WebHCat Server</displayName>
+            <category>MASTER</category>
+            <deleted>false</deleted>
+            <cardinality>1+</cardinality>
+            <versionAdvertisedInternal>true</versionAdvertisedInternal>
+            <commandScript>
+                <script>scripts/webhcat_server.py</script>
+                <scriptType>PYTHON</scriptType>
+                <timeout>1200</timeout>
+            </commandScript>
+            <clientsToUpdateConfigs>
+                <client>HCAT</client>
+            </clientsToUpdateConfigs>
+            <dependencies>
+                <dependency>
+                    <auto-deploy>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>HDFS/HDFS_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>HIVE/HIVE_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>MAPREDUCE2/MAPREDUCE2_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <co-locate>HIVE/WEBHCAT_SERVER</co-locate>
+                        <coLocate>HIVE/WEBHCAT_SERVER</coLocate>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <co-locate>HIVE/WEBHCAT_SERVER</co-locate>
+                        <coLocate>HIVE/WEBHCAT_SERVER</coLocate>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>ZOOKEEPER/ZOOKEEPER_SERVER</name>
+                    <scope>cluster</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>ZOOKEEPER/ZOOKEEPER_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>YARN/YARN_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>PIG/PIG</name>
+                    <scope>host</scope>
+                </dependency>
+            </dependencies>
+            <configuration-dependencies>
+                <config-type>hive-site</config-type>
+            </configuration-dependencies>
+            <recovery_enabled>false</recovery_enabled>
+            <reassignAllowed>true</reassignAllowed>
+        </component>
+        <component>
+            <name>MYSQL_SERVER</name>
+            <displayName>MySQL Server</displayName>
+            <category>MASTER</category>
+            <deleted>false</deleted>
+            <cardinality>0-1</cardinality>
+            <versionAdvertised>false</versionAdvertised>
+            <versionAdvertisedInternal>false</versionAdvertisedInternal>
+            <commandScript>
+                <script>scripts/mysql_server.py</script>
+                <scriptType>PYTHON</scriptType>
+                <timeout>0</timeout>
+            </commandScript>
+            <customCommands>
+                <customCommand>
+                    <name>CLEAN</name>
+                    <commandScript>
+                        <script>scripts/mysql_server.py</script>
+                        <scriptType>PYTHON</scriptType>
+                        <timeout>600</timeout>
+                    </commandScript>
+                    <background>false</background>
+                </customCommand>
+            </customCommands>
+            <recovery_enabled>false</recovery_enabled>
+            <reassignAllowed>true</reassignAllowed>
+        </component>
+        <component>
+            <name>HIVE_METASTORE</name>
+            <displayName>Hive Metastore</displayName>
+            <category>MASTER</category>
+            <deleted>false</deleted>
+            <cardinality>1+</cardinality>
+            <versionAdvertisedInternal>true</versionAdvertisedInternal>
+            <commandScript>
+                <script>scripts/hive_metastore.py</script>
+                <scriptType>PYTHON</scriptType>
+                <timeout>1200</timeout>
+            </commandScript>
+            <logs>
+                <log>
+                    <logId>hive_metastore</logId>
+                    <primary>true</primary>
+                </log>
+            </logs>
+            <configuration-dependencies>
+                <config-type>hive-site</config-type>
+            </configuration-dependencies>
+            <auto-deploy>
+                <co-locate>HIVE/HIVE_SERVER</co-locate>
+                <coLocate>HIVE/HIVE_SERVER</coLocate>
+                <enabled>true</enabled>
+            </auto-deploy>
+            <recovery_enabled>false</recovery_enabled>
+            <reassignAllowed>true</reassignAllowed>
+        </component>
+        <component>
+            <name>HIVE_SERVER_INTERACTIVE</name>
+            <displayName>HiveServer2 Interactive</displayName>
+            <category>MASTER</category>
+            <deleted>false</deleted>
+            <cardinality>0-1</cardinality>
+            <versionAdvertised>true</versionAdvertised>
+            <versionAdvertisedInternal>false</versionAdvertisedInternal>
+            <commandScript>
+                <script>scripts/hive_server_interactive.py</script>
+                <scriptType>PYTHON</scriptType>
+                <timeout>0</timeout>
+            </commandScript>
+            <customCommands>
+                <customCommand>
+                    <name>RESTART_LLAP</name>
+                    <commandScript>
+                        <script>scripts/hive_server_interactive.py</script>
+                        <scriptType>PYTHON</scriptType>
+                        <timeout>600</timeout>
+                    </commandScript>
+                    <background>false</background>
+                </customCommand>
+            </customCommands>
+            <dependencies>
+                <dependency>
+                    <auto-deploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>ZOOKEEPER/ZOOKEEPER_SERVER</name>
+                    <scope>cluster</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>YARN/YARN_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>HDFS/HDFS_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>MAPREDUCE2/MAPREDUCE2_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>TEZ/TEZ_CLIENT</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>PIG/PIG</name>
+                    <scope>host</scope>
+                </dependency>
+                <dependency>
+                    <auto-deploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </auto-deploy>
+                    <autoDeploy>
+                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
+                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
+                        <enabled>true</enabled>
+                    </autoDeploy>
+                    <name>SLIDER/SLIDER</name>
+                    <scope>host</scope>
+                </dependency>
+            </dependencies>
+            <configuration-dependencies>
+                <config-type>beeline-log4j2</config-type>
+                <config-type>hive-exec-log4j2</config-type>
+                <config-type>hive-log4j2</config-type>
+                <config-type>hive-site</config-type>
+                <config-type>hive-interactive-site</config-type>
+                <config-type>tez-interactive-site</config-type>
+                <config-type>hiveserver2-interactive-site</config-type>
+                <config-type>hive-interactive-env</config-type>
+                <config-type>llap-cli-log4j2</config-type>
+                <config-type>llap-daemon-log4j</config-type>
+            </configuration-dependencies>
+            <recovery_enabled>false</recovery_enabled>
+        </component>
+    </components>
+    <deleted>false</deleted>
+    <configuration-dependencies>
+        <config-type>application-properties</config-type>
+        <config-type>hive-atlas-application.properties</config-type>
+        <config-type>hive-log4j</config-type>
+        <config-type>hive-exec-log4j</config-type>
+        <config-type>tez-site</config-type>
+        <config-type>hive-env</config-type>
+        <config-type>hivemetastore-site.xml</config-type>
+        <config-type>webhcat-site</config-type>
+        <config-type>webhcat-env</config-type>
+        <config-type>ranger-hive-plugin-properties</config-type>
+        <config-type>ranger-hive-audit</config-type>
+        <config-type>ranger-hive-policymgr-ssl</config-type>
+        <config-type>ranger-hive-security</config-type>
+        <config-type>mapred-site</config-type>
+        <config-type>application.properties</config-type>
+    </configuration-dependencies>
+    <widgetsFileName>widgets.json</widgetsFileName>
+    <metricsFileName>metrics.json</metricsFileName>
+    <osSpecifics>
+        <osSpecific>
+            <osFamily>any</osFamily>
+            <packages>
+                <package>
+                    <name>mysql-connector-java</name>
+                    <condition>should_install_mysl_connector</condition>
+                    <skipUpgrade>true</skipUpgrade>
+                </package>
+            </packages>
+        </osSpecific>
+        <osSpecific>
+            <osFamily>redhat7,amazon2015,redhat6,suse11,suse12</osFamily>
+            <packages>
+                <package>
+                    <name>hive</name>
+                    <skipUpgrade>false</skipUpgrade>
+                </package>
+                <package>
+                    <name>hive-hcatalog</name>
+                    <skipUpgrade>false</skipUpgrade>
+                </package>
+                <package>
+                    <name>hive-webhcat</name>
+                    <skipUpgrade>false</skipUpgrade>
+                </package>
+            </packages>
+        </osSpecific>
+        <osSpecific>
+            <osFamily>amazon2015,redhat6,suse11,suse12</osFamily>
+            <packages>
+                <package>
+                    <name>mysql</name>
+                    <condition>should_install_mysql</condition>
+                    <skipUpgrade>true</skipUpgrade>
+                </package>
+            </packages>
+        </osSpecific>
+        <osSpecific>
+            <osFamily>amazon2015,redhat6,debian7,ubuntu12,ubuntu14,ubuntu16</osFamily>
+            <packages>
+                <package>
+                    <name>mysql-server</name>
+                    <condition>should_install_mysql</condition>
+                    <skipUpgrade>true</skipUpgrade>
+                </package>
+            </packages>
+        </osSpecific>
+    </osSpecifics>
+    <configuration-dir>configuration</configuration-dir>
+    <commandScript>
+        <script>scripts/service_check.py</script>
+        <scriptType>PYTHON</scriptType>
+        <timeout>300</timeout>
+    </commandScript>
+    <requiredServices>
+        <service>ZOOKEEPER</service>
+        <service>HDFS</service>
+        <service>YARN</service>
+    </requiredServices>
+</service></services>
+</metainfo>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_interactive_thrift_port.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_interactive_thrift_port.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_interactive_thrift_port.py
new file mode 100755
index 0000000..6917160
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_interactive_thrift_port.py
@@ -0,0 +1,216 @@
+#!/usr/bin/env python
+
+"""
+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.
+"""
+
+import os
+import socket
+import time
+import logging
+import traceback
+from resource_management.libraries.functions import hive_check
+from resource_management.libraries.functions import format
+from resource_management.libraries.functions import get_kinit_path
+from ambari_commons.os_check import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+
+OK_MESSAGE = "TCP OK - {0:.3f}s response on port {1}"
+CRITICAL_MESSAGE = "Connection failed on host {0}:{1} ({2})"
+
+HIVE_SERVER_INTERACTIVE_THRIFT_PORT_KEY = '{{hive-interactive-site/hive.server2.thrift.port}}'
+HIVE_SERVER_INTERACTIVE_THRIFT_HTTP_PORT_KEY = '{{hive-interactive-site/hive.server2.thrift.http.port}}'
+HIVE_SERVER_INTERACTIVE_TRANSPORT_MODE_KEY = '{{hive-site/hive.server2.transport.mode}}'
+SECURITY_ENABLED_KEY = '{{cluster-env/security_enabled}}'
+HIVE_SERVER2_INTERACTIVE_AUTHENTICATION_KEY = '{{hive-interactive-site/hive.server2.authentication}}'
+HIVE_SERVER2_AUTHENTICATION_KEY = '{{hive-site/hive.server2.authentication}}'
+HIVE_SERVER_INTERACTIVE_PRINCIPAL_KEY = '{{hive-site/hive.server2.authentication.kerberos.principal}}'
+SMOKEUSER_KEYTAB_KEY = '{{cluster-env/smokeuser_keytab}}'
+SMOKEUSER_PRINCIPAL_KEY = '{{cluster-env/smokeuser_principal_name}}'
+SMOKEUSER_KEY = '{{cluster-env/smokeuser}}'
+HIVE_SSL = '{{hive-site/hive.server2.use.SSL}}'
+HIVE_SSL_KEYSTORE_PATH = '{{hive-site/hive.server2.keystore.path}}'
+HIVE_SSL_KEYSTORE_PASSWORD = '{{hive-site/hive.server2.keystore.password}}'
+
+# The configured Kerberos executable search paths, if any
+KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY = '{{kerberos-env/executable_search_paths}}'
+
+THRIFT_PORT_DEFAULT = 10500
+HIVE_SERVER_INTERACTIVE_TRANSPORT_MODE_DEFAULT = 'binary'
+HIVE_SERVER_INTERACTIVE_PRINCIPAL_DEFAULT = 'hive/_HOST@EXAMPLE.COM'
+HIVE_SERVER2_INTERACTIVE_AUTHENTICATION_DEFAULT = 'NOSASL'
+
+# default keytab location
+SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY = 'default.smoke.keytab'
+SMOKEUSER_KEYTAB_DEFAULT = '/etc/security/keytabs/smokeuser.headless.keytab'
+
+# default smoke principal
+SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY = 'default.smoke.principal'
+SMOKEUSER_PRINCIPAL_DEFAULT = 'ambari-qa@EXAMPLE.COM'
+
+# default smoke user
+SMOKEUSER_SCRIPT_PARAM_KEY = 'default.smoke.user'
+SMOKEUSER_DEFAULT = 'ambari-qa'
+
+HADOOPUSER_KEY = '{{cluster-env/hadoop.user.name}}'
+HADOOPUSER_DEFAULT = 'hadoop'
+
+CHECK_COMMAND_TIMEOUT_KEY = 'check.command.timeout'
+CHECK_COMMAND_TIMEOUT_DEFAULT = 60.0
+
+logger = logging.getLogger('ambari_alerts')
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def get_tokens():
+  """
+  Returns a tuple of tokens in the format {{site/property}} that will be used
+  to build the dictionary passed into execute
+  """
+  return (HIVE_SERVER_INTERACTIVE_THRIFT_PORT_KEY, SECURITY_ENABLED_KEY, SMOKEUSER_KEY,
+          HIVE_SERVER2_INTERACTIVE_AUTHENTICATION_KEY, HIVE_SERVER2_AUTHENTICATION_KEY,
+          HIVE_SERVER_INTERACTIVE_PRINCIPAL_KEY, SMOKEUSER_KEYTAB_KEY, SMOKEUSER_PRINCIPAL_KEY,
+          HIVE_SERVER_INTERACTIVE_THRIFT_HTTP_PORT_KEY, HIVE_SERVER_INTERACTIVE_TRANSPORT_MODE_KEY,
+          KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY, HIVE_SSL, HIVE_SSL_KEYSTORE_PATH, HIVE_SSL_KEYSTORE_PASSWORD)
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def get_tokens():
+  pass
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def execute(configurations={}, parameters={}, host_name=None):
+  """
+  Returns a tuple containing the result code and a pre-formatted result label
+
+  Keyword arguments:
+  configurations (dictionary): a mapping of configuration key to value
+  parameters (dictionary): a mapping of script parameter key to value
+  host_name (string): the name of this host where the alert is running
+  """
+
+  if configurations is None:
+    return ('UNKNOWN', ['There were no configurations supplied to the script.'])
+
+  transport_mode = HIVE_SERVER_INTERACTIVE_TRANSPORT_MODE_DEFAULT
+  if HIVE_SERVER_INTERACTIVE_TRANSPORT_MODE_KEY in configurations:
+    transport_mode = configurations[HIVE_SERVER_INTERACTIVE_TRANSPORT_MODE_KEY]
+
+  port = THRIFT_PORT_DEFAULT
+  if transport_mode.lower() == 'binary' and HIVE_SERVER_INTERACTIVE_THRIFT_PORT_KEY in configurations:
+    port = int(configurations[HIVE_SERVER_INTERACTIVE_THRIFT_PORT_KEY])
+  elif transport_mode.lower() == 'http' and HIVE_SERVER_INTERACTIVE_THRIFT_HTTP_PORT_KEY in configurations:
+    port = int(configurations[HIVE_SERVER_INTERACTIVE_THRIFT_HTTP_PORT_KEY])
+
+  security_enabled = False
+  if SECURITY_ENABLED_KEY in configurations:
+    security_enabled = str(configurations[SECURITY_ENABLED_KEY]).upper() == 'TRUE'
+
+  check_command_timeout = CHECK_COMMAND_TIMEOUT_DEFAULT
+  if CHECK_COMMAND_TIMEOUT_KEY in parameters:
+    check_command_timeout = float(parameters[CHECK_COMMAND_TIMEOUT_KEY])
+
+  hive_server2_authentication = HIVE_SERVER2_INTERACTIVE_AUTHENTICATION_DEFAULT
+  if HIVE_SERVER2_INTERACTIVE_AUTHENTICATION_KEY in configurations:
+    hive_server2_authentication = configurations[HIVE_SERVER2_INTERACTIVE_AUTHENTICATION_KEY]
+  elif HIVE_SERVER2_AUTHENTICATION_KEY in configurations:
+    hive_server2_authentication = configurations[HIVE_SERVER2_AUTHENTICATION_KEY]
+
+  hive_ssl = False
+  if HIVE_SSL in configurations:
+    hive_ssl = configurations[HIVE_SSL]
+
+  hive_ssl_keystore_path = None
+  if HIVE_SSL_KEYSTORE_PATH in configurations:
+    hive_ssl_keystore_path = configurations[HIVE_SSL_KEYSTORE_PATH]
+
+  hive_ssl_keystore_password = None
+  if HIVE_SSL_KEYSTORE_PASSWORD in configurations:
+    hive_ssl_keystore_password = configurations[HIVE_SSL_KEYSTORE_PASSWORD]
+
+  # defaults
+  smokeuser_keytab = SMOKEUSER_KEYTAB_DEFAULT
+  smokeuser_principal = SMOKEUSER_PRINCIPAL_DEFAULT
+  smokeuser = SMOKEUSER_DEFAULT
+
+  # check script params
+  if SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY in parameters:
+    smokeuser_principal = parameters[SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY]
+
+  if SMOKEUSER_SCRIPT_PARAM_KEY in parameters:
+    smokeuser = parameters[SMOKEUSER_SCRIPT_PARAM_KEY]
+
+  if SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY in parameters:
+    smokeuser_keytab = parameters[SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY]
+
+
+  # check configurations last as they should always take precedence
+  if SMOKEUSER_PRINCIPAL_KEY in configurations:
+    smokeuser_principal = configurations[SMOKEUSER_PRINCIPAL_KEY]
+
+  if SMOKEUSER_KEY in configurations:
+    smokeuser = configurations[SMOKEUSER_KEY]
+
+  result_code = None
+
+  if security_enabled:
+    hive_server_principal = HIVE_SERVER_INTERACTIVE_PRINCIPAL_DEFAULT
+    if HIVE_SERVER_INTERACTIVE_PRINCIPAL_KEY in configurations:
+      hive_server_principal = configurations[HIVE_SERVER_INTERACTIVE_PRINCIPAL_KEY]
+
+    if SMOKEUSER_KEYTAB_KEY in configurations:
+      smokeuser_keytab = configurations[SMOKEUSER_KEYTAB_KEY]
+
+    # Get the configured Kerberos executable search paths, if any
+    if KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY in configurations:
+      kerberos_executable_search_paths = configurations[KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY]
+    else:
+      kerberos_executable_search_paths = None
+
+    kinit_path_local = get_kinit_path(kerberos_executable_search_paths)
+    kinitcmd=format("{kinit_path_local} -kt {smokeuser_keytab} {smokeuser_principal}; ")
+  else:
+    hive_server_principal = None
+    kinitcmd=None
+
+  try:
+    if host_name is None:
+      host_name = socket.getfqdn()
+
+    start_time = time.time()
+
+    try:
+      hive_check.check_thrift_port_sasl(host_name, port, hive_server2_authentication, hive_server_principal,
+                                        kinitcmd, smokeuser, transport_mode=transport_mode, ssl=hive_ssl,
+                                        ssl_keystore=hive_ssl_keystore_path, ssl_password=hive_ssl_keystore_password,
+                                        check_command_timeout=int(check_command_timeout))
+      result_code = 'OK'
+      total_time = time.time() - start_time
+      label = OK_MESSAGE.format(total_time, port)
+    except:
+      result_code = 'CRITICAL'
+      label = CRITICAL_MESSAGE.format(host_name, port, traceback.format_exc())
+
+  except:
+    label = traceback.format_exc()
+    result_code = 'UNKNOWN'
+
+  return (result_code, [label])
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def execute(configurations={}, parameters={}, host_name=None):
+  pass
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_metastore.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_metastore.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_metastore.py
new file mode 100755
index 0000000..e02ed5a
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_metastore.py
@@ -0,0 +1,270 @@
+#!/usr/bin/env python
+
+"""
+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.
+"""
+
+import os
+import socket
+import time
+import traceback
+import logging
+
+from resource_management.core import global_lock
+from resource_management.libraries.functions import format
+from resource_management.libraries.functions import get_kinit_path
+from resource_management.core.resources import Execute
+from ambari_commons.os_check import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+
+
+OK_MESSAGE = "Metastore OK - Hive command took {0:.3f}s"
+CRITICAL_MESSAGE = "Metastore on {0} failed ({1})"
+SECURITY_ENABLED_KEY = '{{cluster-env/security_enabled}}'
+SMOKEUSER_KEYTAB_KEY = '{{cluster-env/smokeuser_keytab}}'
+SMOKEUSER_PRINCIPAL_KEY = '{{cluster-env/smokeuser_principal_name}}'
+SMOKEUSER_KEY = '{{cluster-env/smokeuser}}'
+HIVE_METASTORE_URIS_KEY = '{{hive-site/hive.metastore.uris}}'
+
+# The configured Kerberos executable search paths, if any
+KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY = '{{kerberos-env/executable_search_paths}}'
+
+# default keytab location
+SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY = 'default.smoke.keytab'
+SMOKEUSER_KEYTAB_DEFAULT = '/etc/security/keytabs/smokeuser.headless.keytab'
+
+# default smoke principal
+SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY = 'default.smoke.principal'
+SMOKEUSER_PRINCIPAL_DEFAULT = 'ambari-qa@EXAMPLE.COM'
+
+# default smoke user
+SMOKEUSER_SCRIPT_PARAM_KEY = 'default.smoke.user'
+SMOKEUSER_DEFAULT = 'ambari-qa'
+
+STACK_ROOT = '{{cluster-env/stack_root}}'
+
+HIVE_CONF_DIR_LEGACY = '/etc/hive/conf.server'
+
+HIVE_BIN_DIR_LEGACY = '/usr/lib/hive/bin'
+
+CHECK_COMMAND_TIMEOUT_KEY = 'check.command.timeout'
+CHECK_COMMAND_TIMEOUT_DEFAULT = 60.0
+
+HADOOPUSER_KEY = '{{cluster-env/hadoop.user.name}}'
+HADOOPUSER_DEFAULT = 'hadoop'
+
+logger = logging.getLogger('ambari_alerts')
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def get_tokens():
+  """
+  Returns a tuple of tokens in the format {{site/property}} that will be used
+  to build the dictionary passed into execute
+  """
+  return (SECURITY_ENABLED_KEY,SMOKEUSER_KEYTAB_KEY,SMOKEUSER_PRINCIPAL_KEY,
+    HIVE_METASTORE_URIS_KEY, SMOKEUSER_KEY, KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY,
+    STACK_ROOT)
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def get_tokens():
+  """
+  Returns a tuple of tokens in the format {{site/property}} that will be used
+  to build the dictionary passed into execute
+  """
+  return (HIVE_METASTORE_URIS_KEY, HADOOPUSER_KEY)
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def execute(configurations={}, parameters={}, host_name=None):
+  """
+  Returns a tuple containing the result code and a pre-formatted result label
+
+  Keyword arguments:
+  configurations (dictionary): a mapping of configuration key to value
+  parameters (dictionary): a mapping of script parameter key to value
+  host_name (string): the name of this host where the alert is running
+  """
+
+  if configurations is None:
+    return (('UNKNOWN', ['There were no configurations supplied to the script.']))
+
+  if not HIVE_METASTORE_URIS_KEY in configurations:
+    return (('UNKNOWN', ['Hive metastore uris were not supplied to the script.']))
+
+  metastore_uris = configurations[HIVE_METASTORE_URIS_KEY].split(',')
+
+  security_enabled = False
+  if SECURITY_ENABLED_KEY in configurations:
+    security_enabled = str(configurations[SECURITY_ENABLED_KEY]).upper() == 'TRUE'
+
+  check_command_timeout = CHECK_COMMAND_TIMEOUT_DEFAULT
+  if CHECK_COMMAND_TIMEOUT_KEY in parameters:
+    check_command_timeout = float(parameters[CHECK_COMMAND_TIMEOUT_KEY])
+
+  # defaults
+  smokeuser_keytab = SMOKEUSER_KEYTAB_DEFAULT
+  smokeuser_principal = SMOKEUSER_PRINCIPAL_DEFAULT
+  smokeuser = SMOKEUSER_DEFAULT
+
+  # check script params
+  if SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY in parameters:
+    smokeuser_principal = parameters[SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY]
+
+  if SMOKEUSER_SCRIPT_PARAM_KEY in parameters:
+    smokeuser = parameters[SMOKEUSER_SCRIPT_PARAM_KEY]
+
+  if SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY in parameters:
+    smokeuser_keytab = parameters[SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY]
+
+
+  # check configurations last as they should always take precedence
+  if SMOKEUSER_PRINCIPAL_KEY in configurations:
+    smokeuser_principal = configurations[SMOKEUSER_PRINCIPAL_KEY]
+
+  if SMOKEUSER_KEY in configurations:
+    smokeuser = configurations[SMOKEUSER_KEY]
+
+  result_code = None
+
+  try:
+    if security_enabled:
+      if SMOKEUSER_KEYTAB_KEY in configurations:
+        smokeuser_keytab = configurations[SMOKEUSER_KEYTAB_KEY]
+
+      # Get the configured Kerberos executable search paths, if any
+      if KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY in configurations:
+        kerberos_executable_search_paths = configurations[KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY]
+      else:
+        kerberos_executable_search_paths = None
+
+      kinit_path_local = get_kinit_path(kerberos_executable_search_paths)
+      kinitcmd=format("{kinit_path_local} -kt {smokeuser_keytab} {smokeuser_principal}; ")
+
+      # prevent concurrent kinit
+      kinit_lock = global_lock.get_lock(global_lock.LOCK_TYPE_KERBEROS)
+      kinit_lock.acquire()
+      try:
+        Execute(kinitcmd, user=smokeuser,
+          path=["/bin/", "/usr/bin/", "/usr/lib/hive/bin/", "/usr/sbin/"],
+          timeout=10)
+      finally:
+        kinit_lock.release()
+
+    if host_name is None:
+      host_name = socket.getfqdn()
+
+    for uri in metastore_uris:
+      if host_name in uri:
+        metastore_uri = uri
+
+    conf_dir = HIVE_CONF_DIR_LEGACY
+    bin_dir = HIVE_BIN_DIR_LEGACY
+
+
+    if STACK_ROOT in configurations:
+      hive_conf_dir = configurations[STACK_ROOT] + format("/current/hive-metastore/conf/conf.server")
+      hive_bin_dir = configurations[STACK_ROOT] + format("/current/hive-metastore/bin")
+
+      if os.path.exists(hive_conf_dir):
+        conf_dir = hive_conf_dir
+        bin_dir = hive_bin_dir
+
+    cmd = format("export HIVE_CONF_DIR='{conf_dir}' ; "
+                 "hive --hiveconf hive.metastore.uris={metastore_uri}\
+                 --hiveconf hive.metastore.client.connect.retry.delay=1\
+                 --hiveconf hive.metastore.failure.retries=1\
+                 --hiveconf hive.metastore.connect.retries=1\
+                 --hiveconf hive.metastore.client.socket.timeout=14\
+                 --hiveconf hive.execution.engine=mr -e 'show databases;'")
+
+    start_time = time.time()
+
+    try:
+      Execute(cmd, user=smokeuser,
+        path=["/bin/", "/usr/bin/", "/usr/sbin/", bin_dir],
+        timeout=int(check_command_timeout) )
+
+      total_time = time.time() - start_time
+
+      result_code = 'OK'
+      label = OK_MESSAGE.format(total_time)
+    except:
+      result_code = 'CRITICAL'
+      label = CRITICAL_MESSAGE.format(host_name, traceback.format_exc())
+
+  except:
+    label = traceback.format_exc()
+    result_code = 'UNKNOWN'
+
+  return ((result_code, [label]))
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def execute(configurations={}, parameters={}, host_name=None):
+  """
+  Returns a tuple containing the result code and a pre-formatted result label
+
+  Keyword arguments:
+  configurations (dictionary): a mapping of configuration key to value
+  parameters (dictionary): a mapping of script parameter key to value
+  host_name (string): the name of this host where the alert is running
+  """
+
+  from resource_management.libraries.functions import reload_windows_env
+  reload_windows_env()
+  hive_home = os.environ['HIVE_HOME']
+
+  if configurations is None:
+    return (('UNKNOWN', ['There were no configurations supplied to the script.']))
+  if not HIVE_METASTORE_URIS_KEY in configurations:
+    return (('UNKNOWN', ['Hive metastore uris were not supplied to the script.']))
+
+  metastore_uris = configurations[HIVE_METASTORE_URIS_KEY].split(',')
+
+  # defaults
+  hiveuser = HADOOPUSER_DEFAULT
+
+  if HADOOPUSER_KEY in configurations:
+    hiveuser = configurations[HADOOPUSER_KEY]
+
+  result_code = None
+  try:
+    if host_name is None:
+      host_name = socket.getfqdn()
+    for uri in metastore_uris:
+      if host_name in uri:
+        metastore_uri = uri
+
+    hive_cmd = os.path.join(hive_home, "bin", "hive.cmd")
+    cmd = format("cmd /c {hive_cmd} --hiveconf hive.metastore.uris={metastore_uri}\
+                 --hiveconf hive.metastore.client.connect.retry.delay=1\
+                 --hiveconf hive.metastore.failure.retries=1\
+                 --hiveconf hive.metastore.connect.retries=1\
+                 --hiveconf hive.metastore.client.socket.timeout=14\
+                 --hiveconf hive.execution.engine=mr -e 'show databases;'")
+    start_time = time.time()
+    try:
+      Execute(cmd, user=hiveuser, timeout=30)
+      total_time = time.time() - start_time
+      result_code = 'OK'
+      label = OK_MESSAGE.format(total_time)
+    except:
+      result_code = 'CRITICAL'
+      label = CRITICAL_MESSAGE.format(host_name, traceback.format_exc())
+  except:
+    label = traceback.format_exc()
+    result_code = 'UNKNOWN'
+
+  return ((result_code, [label]))

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_thrift_port.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_thrift_port.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_thrift_port.py
new file mode 100755
index 0000000..32da1cc
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_hive_thrift_port.py
@@ -0,0 +1,274 @@
+#!/usr/bin/env python
+
+"""
+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.
+"""
+
+import os
+import socket
+import time
+import logging
+import traceback
+from resource_management.libraries.functions import hive_check
+from resource_management.libraries.functions import format
+from resource_management.libraries.functions import get_kinit_path
+from ambari_commons.os_check import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+
+OK_MESSAGE = "TCP OK - {0:.3f}s response on port {1}"
+CRITICAL_MESSAGE = "Connection failed on host {0}:{1} ({2})"
+
+HIVE_SERVER_THRIFT_PORT_KEY = '{{hive-site/hive.server2.thrift.port}}'
+HIVE_SERVER_THRIFT_HTTP_PORT_KEY = '{{hive-site/hive.server2.thrift.http.port}}'
+HIVE_SERVER_TRANSPORT_MODE_KEY = '{{hive-site/hive.server2.transport.mode}}'
+SECURITY_ENABLED_KEY = '{{cluster-env/security_enabled}}'
+HIVE_SERVER2_AUTHENTICATION_KEY = '{{hive-site/hive.server2.authentication}}'
+HIVE_SERVER_PRINCIPAL_KEY = '{{hive-site/hive.server2.authentication.kerberos.principal}}'
+SMOKEUSER_KEYTAB_KEY = '{{cluster-env/smokeuser_keytab}}'
+SMOKEUSER_PRINCIPAL_KEY = '{{cluster-env/smokeuser_principal_name}}'
+SMOKEUSER_KEY = '{{cluster-env/smokeuser}}'
+HIVE_SSL = '{{hive-site/hive.server2.use.SSL}}'
+HIVE_SSL_KEYSTORE_PATH = '{{hive-site/hive.server2.keystore.path}}'
+HIVE_SSL_KEYSTORE_PASSWORD = '{{hive-site/hive.server2.keystore.password}}'
+
+# The configured Kerberos executable search paths, if any
+KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY = '{{kerberos-env/executable_search_paths}}'
+
+THRIFT_PORT_DEFAULT = 10000
+HIVE_SERVER_TRANSPORT_MODE_DEFAULT = 'binary'
+HIVE_SERVER_PRINCIPAL_DEFAULT = 'hive/_HOST@EXAMPLE.COM'
+HIVE_SERVER2_AUTHENTICATION_DEFAULT = 'NOSASL'
+
+# default keytab location
+SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY = 'default.smoke.keytab'
+SMOKEUSER_KEYTAB_DEFAULT = '/etc/security/keytabs/smokeuser.headless.keytab'
+
+# default smoke principal
+SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY = 'default.smoke.principal'
+SMOKEUSER_PRINCIPAL_DEFAULT = 'ambari-qa@EXAMPLE.COM'
+
+# default smoke user
+SMOKEUSER_SCRIPT_PARAM_KEY = 'default.smoke.user'
+SMOKEUSER_DEFAULT = 'ambari-qa'
+
+HADOOPUSER_KEY = '{{cluster-env/hadoop.user.name}}'
+HADOOPUSER_DEFAULT = 'hadoop'
+
+CHECK_COMMAND_TIMEOUT_KEY = 'check.command.timeout'
+CHECK_COMMAND_TIMEOUT_DEFAULT = 60.0
+
+logger = logging.getLogger('ambari_alerts')
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def get_tokens():
+  """
+  Returns a tuple of tokens in the format {{site/property}} that will be used
+  to build the dictionary passed into execute
+  """
+  return (HIVE_SERVER_THRIFT_PORT_KEY, SECURITY_ENABLED_KEY, SMOKEUSER_KEY,
+          HIVE_SERVER2_AUTHENTICATION_KEY, HIVE_SERVER_PRINCIPAL_KEY,
+          SMOKEUSER_KEYTAB_KEY, SMOKEUSER_PRINCIPAL_KEY, HIVE_SERVER_THRIFT_HTTP_PORT_KEY,
+          HIVE_SERVER_TRANSPORT_MODE_KEY, KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY, HIVE_SSL,
+          HIVE_SSL_KEYSTORE_PATH, HIVE_SSL_KEYSTORE_PASSWORD)
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def get_tokens():
+  """
+  Returns a tuple of tokens in the format {{site/property}} that will be used
+  to build the dictionary passed into execute
+  """
+  return (HIVE_SERVER_THRIFT_PORT_KEY, HIVE_SERVER_THRIFT_HTTP_PORT_KEY,
+          HIVE_SERVER_TRANSPORT_MODE_KEY, HADOOPUSER_KEY)
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def execute(configurations={}, parameters={}, host_name=None):
+  """
+  Returns a tuple containing the result code and a pre-formatted result label
+
+  Keyword arguments:
+  configurations (dictionary): a mapping of configuration key to value
+  parameters (dictionary): a mapping of script parameter key to value
+  host_name (string): the name of this host where the alert is running
+  """
+
+  if configurations is None:
+    return ('UNKNOWN', ['There were no configurations supplied to the script.'])
+
+  transport_mode = HIVE_SERVER_TRANSPORT_MODE_DEFAULT
+  if HIVE_SERVER_TRANSPORT_MODE_KEY in configurations:
+    transport_mode = configurations[HIVE_SERVER_TRANSPORT_MODE_KEY]
+
+  port = THRIFT_PORT_DEFAULT
+  if transport_mode.lower() == 'binary' and HIVE_SERVER_THRIFT_PORT_KEY in configurations:
+    port = int(configurations[HIVE_SERVER_THRIFT_PORT_KEY])
+  elif transport_mode.lower() == 'http' and HIVE_SERVER_THRIFT_HTTP_PORT_KEY in configurations:
+    port = int(configurations[HIVE_SERVER_THRIFT_HTTP_PORT_KEY])
+
+  security_enabled = False
+  if SECURITY_ENABLED_KEY in configurations:
+    security_enabled = str(configurations[SECURITY_ENABLED_KEY]).upper() == 'TRUE'
+
+  check_command_timeout = CHECK_COMMAND_TIMEOUT_DEFAULT
+  if CHECK_COMMAND_TIMEOUT_KEY in parameters:
+    check_command_timeout = float(parameters[CHECK_COMMAND_TIMEOUT_KEY])
+
+  hive_server2_authentication = HIVE_SERVER2_AUTHENTICATION_DEFAULT
+  if HIVE_SERVER2_AUTHENTICATION_KEY in configurations:
+    hive_server2_authentication = configurations[HIVE_SERVER2_AUTHENTICATION_KEY]
+
+  hive_ssl = False
+  if HIVE_SSL in configurations:
+    hive_ssl = configurations[HIVE_SSL]
+
+  hive_ssl_keystore_path = None
+  if HIVE_SSL_KEYSTORE_PATH in configurations:
+    hive_ssl_keystore_path = configurations[HIVE_SSL_KEYSTORE_PATH]
+
+  hive_ssl_keystore_password = None
+  if HIVE_SSL_KEYSTORE_PASSWORD in configurations:
+    hive_ssl_keystore_password = configurations[HIVE_SSL_KEYSTORE_PASSWORD]
+
+  # defaults
+  smokeuser_keytab = SMOKEUSER_KEYTAB_DEFAULT
+  smokeuser_principal = SMOKEUSER_PRINCIPAL_DEFAULT
+  smokeuser = SMOKEUSER_DEFAULT
+
+  # check script params
+  if SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY in parameters:
+    smokeuser_principal = parameters[SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY]
+
+  if SMOKEUSER_SCRIPT_PARAM_KEY in parameters:
+    smokeuser = parameters[SMOKEUSER_SCRIPT_PARAM_KEY]
+
+  if SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY in parameters:
+    smokeuser_keytab = parameters[SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY]
+
+
+  # check configurations last as they should always take precedence
+  if SMOKEUSER_PRINCIPAL_KEY in configurations:
+    smokeuser_principal = configurations[SMOKEUSER_PRINCIPAL_KEY]
+
+  if SMOKEUSER_KEY in configurations:
+    smokeuser = configurations[SMOKEUSER_KEY]
+
+  result_code = None
+
+  if security_enabled:
+    hive_server_principal = HIVE_SERVER_PRINCIPAL_DEFAULT
+    if HIVE_SERVER_PRINCIPAL_KEY in configurations:
+      hive_server_principal = configurations[HIVE_SERVER_PRINCIPAL_KEY]
+
+    if SMOKEUSER_KEYTAB_KEY in configurations:
+      smokeuser_keytab = configurations[SMOKEUSER_KEYTAB_KEY]
+
+    # Get the configured Kerberos executable search paths, if any
+    if KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY in configurations:
+      kerberos_executable_search_paths = configurations[KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY]
+    else:
+      kerberos_executable_search_paths = None
+
+    kinit_path_local = get_kinit_path(kerberos_executable_search_paths)
+    kinitcmd=format("{kinit_path_local} -kt {smokeuser_keytab} {smokeuser_principal}; ")
+  else:
+    hive_server_principal = None
+    kinitcmd=None
+
+  try:
+    if host_name is None:
+      host_name = socket.getfqdn()
+
+    start_time = time.time()
+
+    try:
+      hive_check.check_thrift_port_sasl(host_name, port, hive_server2_authentication, hive_server_principal,
+                                        kinitcmd, smokeuser, transport_mode=transport_mode, ssl=hive_ssl,
+                                        ssl_keystore=hive_ssl_keystore_path, ssl_password=hive_ssl_keystore_password,
+                                        check_command_timeout=int(check_command_timeout))
+      result_code = 'OK'
+      total_time = time.time() - start_time
+      label = OK_MESSAGE.format(total_time, port)
+    except:
+      result_code = 'CRITICAL'
+      label = CRITICAL_MESSAGE.format(host_name, port, traceback.format_exc())
+
+  except:
+    label = traceback.format_exc()
+    result_code = 'UNKNOWN'
+
+  return (result_code, [label])
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def execute(configurations={}, parameters={}, host_name=None):
+  """
+  Returns a tuple containing the result code and a pre-formatted result label
+
+  Keyword arguments:
+  configurations (dictionary): a mapping of configuration key to value
+  parameters (dictionary): a mapping of script parameter key to value
+  host_name (string): the name of this host where the alert is running
+  """
+
+  from resource_management.libraries.functions import reload_windows_env
+  from resource_management.core.resources import Execute
+  reload_windows_env()
+  hive_home = os.environ['HIVE_HOME']
+
+  if configurations is None:
+    return ('UNKNOWN', ['There were no configurations supplied to the script.'])
+
+  transport_mode = HIVE_SERVER_TRANSPORT_MODE_DEFAULT
+  if HIVE_SERVER_TRANSPORT_MODE_KEY in configurations:
+    transport_mode = configurations[HIVE_SERVER_TRANSPORT_MODE_KEY]
+
+  port = THRIFT_PORT_DEFAULT
+  if transport_mode.lower() == 'binary' and HIVE_SERVER_THRIFT_PORT_KEY in configurations:
+    port = int(configurations[HIVE_SERVER_THRIFT_PORT_KEY])
+  elif transport_mode.lower() == 'http' and HIVE_SERVER_THRIFT_HTTP_PORT_KEY in configurations:
+    port = int(configurations[HIVE_SERVER_THRIFT_HTTP_PORT_KEY])
+
+  hiveuser = HADOOPUSER_DEFAULT
+  if HADOOPUSER_KEY in configurations:
+    hiveuser = configurations[HADOOPUSER_KEY]
+
+  result_code = None
+  try:
+    if host_name is None:
+      host_name = socket.getfqdn()
+
+    beeline_url = ['jdbc:hive2://{host_name}:{port}/', "transportMode={transport_mode}"]
+    # append url according to used transport
+    if transport_mode == "http":
+      beeline_url.append('httpPath=cliservice')
+    beeline_url_string = format(";".join(beeline_url))
+    beeline_cmd = os.path.join(hive_home, "bin", "beeline.cmd")
+    cmd = format("cmd /c {beeline_cmd} -u {beeline_url_string} -e '' 2>&1 | findstr Connected")
+
+    start_time = time.time()
+    try:
+      Execute(cmd, user=hiveuser, timeout=30)
+      total_time = time.time() - start_time
+      result_code = 'OK'
+      label = OK_MESSAGE.format(total_time, port)
+    except:
+      result_code = 'CRITICAL'
+      label = CRITICAL_MESSAGE.format(host_name, port, traceback.format_exc())
+  except:
+    label = traceback.format_exc()
+    result_code = 'UNKNOWN'
+
+  return (result_code, [label])

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_llap_app_status.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_llap_app_status.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_llap_app_status.py
new file mode 100755
index 0000000..095be3f
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_llap_app_status.py
@@ -0,0 +1,299 @@
+#!/usr/bin/env python
+
+"""
+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.
+"""
+
+import time
+import logging
+import traceback
+import json
+import subprocess
+
+from resource_management.libraries.functions import format
+from resource_management.libraries.functions import get_kinit_path
+from ambari_commons.os_check import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from resource_management.core import shell
+from resource_management.core.resources import Execute
+from resource_management.core import global_lock
+from resource_management.core.exceptions import Fail
+from resource_management.libraries.script.script import Script
+
+OK_MESSAGE = "The application reported a '{0}' state in {1:.3f}s"
+MESSAGE_WITH_STATE_AND_INSTANCES = "The application reported a '{0}' state in {1:.3f}s. [Live: {2}, Desired: {3}]"
+CRITICAL_MESSAGE_WITH_STATE = "The application reported a '{0}' state. Check took {1:.3f}s"
+CRITICAL_MESSAGE = "Application information could not be retrieved"
+
+# results codes
+CRITICAL_RESULT_CODE = 'CRITICAL'
+OK_RESULT_CODE = 'OK'
+UKNOWN_STATUS_CODE = 'UNKNOWN'
+
+
+SECURITY_ENABLED_KEY = '{{cluster-env/security_enabled}}'
+
+HIVE_PRINCIPAL_KEY = '{{hive-interactive-site/hive.llap.zk.sm.principal}}'
+HIVE_PRINCIPAL_DEFAULT = 'default.hive.principal'
+
+HIVE_PRINCIPAL_KEYTAB_KEY = '{{hive-interactive-site/hive.llap.zk.sm.keytab.file}}'
+HIVE_PRINCIPAL_KEYTAB_DEFAULT = 'default.hive.keytab'
+
+HIVE_AUTHENTICATION_DEFAULT = 'NOSASL'
+
+HIVE_USER_KEY = '{{hive-env/hive_user}}'
+HIVE_USER_DEFAULT = 'default.smoke.user'
+
+STACK_ROOT = '{{cluster-env/stack_root}}'
+STACK_ROOT_DEFAULT = Script.get_stack_root()
+
+LLAP_APP_NAME_KEY = '{{hive-interactive-env/llap_app_name}}'
+LLAP_APP_NAME_DEFAULT = 'llap0'
+
+# The configured Kerberos executable search paths, if any
+KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY = '{{kerberos-env/executable_search_paths}}'
+
+
+CHECK_COMMAND_TIMEOUT_KEY = 'check.command.timeout'
+CHECK_COMMAND_TIMEOUT_DEFAULT = 120.0
+
+
+# Mapping of LLAP app states to 'user friendly' state names.
+llap_app_state_dict = {'RUNNING_ALL': 'RUNNING',
+                       'RUNNING_PARTIAL': 'RUNNING',
+                       'COMPLETE': 'NOT RUNNING',
+                       'LAUNCHING': 'LAUNCHING',
+                       'APP_NOT_FOUND': 'APP NOT FOUND'}
+
+logger = logging.getLogger('ambari_alerts')
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def get_tokens():
+  """
+  Returns a tuple of tokens in the format {{site/property}} that will be used
+  to build the dictionary passed into execute
+  """
+  return (SECURITY_ENABLED_KEY, KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY, HIVE_PRINCIPAL_KEY, HIVE_PRINCIPAL_KEYTAB_KEY,
+          HIVE_USER_KEY, STACK_ROOT, LLAP_APP_NAME_KEY)
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def execute(configurations={}, parameters={}, host_name=None):
+  """
+  Returns a tuple containing the result code and a pre-formatted result label
+
+  Keyword arguments:
+  configurations (dictionary): a mapping of configuration key to value
+  parameters (dictionary): a mapping of script parameter key to value
+  host_name (string): the name of this host where the alert is running
+  """
+
+  LLAP_APP_STATUS_CMD_TIMEOUT = 0
+
+  if configurations is None:
+    return ('UNKNOWN', ['There were no configurations supplied to the script.'])
+
+  result_code = None
+
+  try:
+    security_enabled = False
+    if SECURITY_ENABLED_KEY in configurations:
+      security_enabled = str(configurations[SECURITY_ENABLED_KEY]).upper() == 'TRUE'
+
+    check_command_timeout = CHECK_COMMAND_TIMEOUT_DEFAULT
+    if CHECK_COMMAND_TIMEOUT_KEY in configurations:
+      check_command_timeout = int(parameters[CHECK_COMMAND_TIMEOUT_KEY])
+
+    hive_user = HIVE_USER_DEFAULT
+    if HIVE_USER_KEY in configurations:
+      hive_user = configurations[HIVE_USER_KEY]
+
+    llap_app_name = LLAP_APP_NAME_DEFAULT
+    if LLAP_APP_NAME_KEY in configurations:
+      llap_app_name = configurations[LLAP_APP_NAME_KEY]
+
+    if security_enabled:
+      if HIVE_PRINCIPAL_KEY in configurations:
+        llap_principal = configurations[HIVE_PRINCIPAL_KEY]
+      else:
+        llap_principal = HIVE_PRINCIPAL_DEFAULT
+      llap_principal = llap_principal.replace('_HOST',host_name.lower())
+
+      llap_keytab = HIVE_PRINCIPAL_KEYTAB_DEFAULT
+      if HIVE_PRINCIPAL_KEYTAB_KEY in configurations:
+        llap_keytab = configurations[HIVE_PRINCIPAL_KEYTAB_KEY]
+
+      # Get the configured Kerberos executable search paths, if any
+      if KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY in configurations:
+        kerberos_executable_search_paths = configurations[KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY]
+      else:
+        kerberos_executable_search_paths = None
+
+      kinit_path_local = get_kinit_path(kerberos_executable_search_paths)
+      kinitcmd=format("{kinit_path_local} -kt {llap_keytab} {llap_principal}; ")
+
+      # prevent concurrent kinit
+      kinit_lock = global_lock.get_lock(global_lock.LOCK_TYPE_KERBEROS)
+      kinit_lock.acquire()
+      try:
+        Execute(kinitcmd, user=hive_user,
+                path=["/bin/", "/usr/bin/", "/usr/lib/hive/bin/", "/usr/sbin/"],
+                timeout=10)
+      finally:
+        kinit_lock.release()
+
+
+
+    start_time = time.time()
+    if STACK_ROOT in configurations:
+      llap_status_cmd = configurations[STACK_ROOT] + format("/current/hive-server2-hive2/bin/hive --service llapstatus --name {llap_app_name}  --findAppTimeout {LLAP_APP_STATUS_CMD_TIMEOUT}")
+    else:
+      llap_status_cmd = STACK_ROOT_DEFAULT + format("/current/hive-server2-hive2/bin/hive --service llapstatus --name {llap_app_name} --findAppTimeout {LLAP_APP_STATUS_CMD_TIMEOUT}")
+
+    code, output, error = shell.checked_call(llap_status_cmd, user=hive_user, stderr=subprocess.PIPE,
+                                             timeout=check_command_timeout,
+                                             logoutput=False)
+    # Call for getting JSON
+    llap_app_info = make_valid_json(output)
+
+    if llap_app_info is None or 'state' not in llap_app_info:
+      alert_label = traceback.format_exc()
+      result_code = UKNOWN_STATUS_CODE
+      return (result_code, [alert_label])
+
+    retrieved_llap_app_state = llap_app_info['state'].upper()
+    if retrieved_llap_app_state in ['RUNNING_ALL']:
+      result_code = OK_RESULT_CODE
+      total_time = time.time() - start_time
+      alert_label = OK_MESSAGE.format(llap_app_state_dict.get(retrieved_llap_app_state, retrieved_llap_app_state), total_time)
+    elif retrieved_llap_app_state in ['RUNNING_PARTIAL']:
+      live_instances = 0
+      desired_instances = 0
+      percentInstancesUp = 0
+      percent_desired_instances_to_be_up = 80
+      # Get 'live' and 'desired' instances
+      if 'liveInstances' not in llap_app_info or 'desiredInstances' not in llap_app_info:
+        result_code = CRITICAL_RESULT_CODE
+        total_time = time.time() - start_time
+        alert_label = CRITICAL_MESSAGE_WITH_STATE.format(llap_app_state_dict.get(retrieved_llap_app_state, retrieved_llap_app_state), total_time)
+        return (result_code, [alert_label])
+
+      live_instances = llap_app_info['liveInstances']
+      desired_instances = llap_app_info['desiredInstances']
+      if live_instances < 0 or desired_instances <= 0:
+        result_code = CRITICAL_RESULT_CODE
+        total_time = time.time() - start_time
+        alert_label = CRITICAL_MESSAGE_WITH_STATE.format(llap_app_state_dict.get(retrieved_llap_app_state, retrieved_llap_app_state), total_time)
+        return (result_code, [alert_label])
+
+      percentInstancesUp = float(live_instances) / desired_instances * 100
+      if percentInstancesUp >= percent_desired_instances_to_be_up:
+        result_code = OK_RESULT_CODE
+        total_time = time.time() - start_time
+        alert_label = MESSAGE_WITH_STATE_AND_INSTANCES.format(llap_app_state_dict.get(retrieved_llap_app_state, retrieved_llap_app_state),
+                                                              total_time,
+                                                              llap_app_info['liveInstances'],
+                                                              llap_app_info['desiredInstances'])
+      else:
+        result_code = CRITICAL_RESULT_CODE
+        total_time = time.time() - start_time
+        alert_label = MESSAGE_WITH_STATE_AND_INSTANCES.format(llap_app_state_dict.get(retrieved_llap_app_state, retrieved_llap_app_state),
+                                                              total_time,
+                                                              llap_app_info['liveInstances'],
+                                                              llap_app_info['desiredInstances'])
+    else:
+      result_code = CRITICAL_RESULT_CODE
+      total_time = time.time() - start_time
+      alert_label = CRITICAL_MESSAGE_WITH_STATE.format(llap_app_state_dict.get(retrieved_llap_app_state, retrieved_llap_app_state), total_time)
+  except:
+    alert_label = traceback.format_exc()
+    traceback.format_exc()
+    result_code = UKNOWN_STATUS_CODE
+  return (result_code, [alert_label])
+
+
+"""
+Remove extra lines from 'llapstatus' status output (eg: because of MOTD logging) so as to have a valid JSON data to be passed in
+to JSON converter.
+"""
+def make_valid_json(output):
+  '''
+
+  Note: It is assumed right now that extra lines will be only at the start and not at the end.
+
+  Sample expected JSON to be passed for 'loads' is either of the form :
+
+  Case 'A':
+  {
+      "amInfo" : {
+      "appName" : "llap0",
+      "appType" : "org-apache-slider",
+      "appId" : "APP1",
+      "containerId" : "container_1466036628595_0010_01_000001",
+      "hostname" : "hostName",
+      "amWebUrl" : "http://hostName:port/"
+    },
+    "state" : "LAUNCHING",
+    ....
+    "desiredInstances" : 1,
+    "liveInstances" : 0,
+    ....
+    ....
+  }
+
+  or
+
+  Case 'B':
+  {
+    "state" : "APP_NOT_FOUND"
+  }
+
+  '''
+  splits = output.split("\n")
+
+  len_splits = len(splits)
+  if (len_splits < 3):
+    raise Fail("Malformed JSON data received from 'llapstatus' command. Exiting ....")
+
+  marker_idx = None  # To detect where from to start reading for JSON data
+  for idx, split in enumerate(splits):
+    curr_elem = split.strip()
+    if idx + 2 > len_splits:
+      raise Fail(
+        "Iterated over the received 'llapstatus' comamnd. Couldn't validate the received output for JSON parsing.")
+    next_elem = (splits[(idx + 1)]).strip()
+    if curr_elem == "{":
+      if next_elem == "\"amInfo\" : {" and (splits[len_splits - 1]).strip() == '}':
+        # For Case 'A'
+        marker_idx = idx
+        break;
+      elif idx + 3 == len_splits and next_elem.startswith('"state" : ') and (splits[idx + 2]).strip() == '}':
+        # For Case 'B'
+        marker_idx = idx
+        break;
+
+
+  # Remove extra logging from possible JSON output
+  if marker_idx is None:
+    raise Fail("Couldn't validate the received output for JSON parsing.")
+  else:
+    if marker_idx != 0:
+      del splits[0:marker_idx]
+
+  scanned_output = '\n'.join(splits)
+  llap_app_info = json.loads(scanned_output)
+  return llap_app_info
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_webhcat_server.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_webhcat_server.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_webhcat_server.py
new file mode 100755
index 0000000..c9575c0
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/alerts/alert_webhcat_server.py
@@ -0,0 +1,228 @@
+#!/usr/bin/env python
+
+"""
+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.
+"""
+
+import ambari_simplejson as json # simplejson is much faster comparing to Python 2.6 json module and has the same functions set.
+import socket
+import time
+import urllib2
+import traceback
+import logging
+
+from resource_management.core.environment import Environment
+from resource_management.libraries.functions.curl_krb_request import curl_krb_request
+from resource_management.libraries.functions.curl_krb_request import DEFAULT_KERBEROS_KINIT_TIMER_MS
+from resource_management.libraries.functions.curl_krb_request import KERBEROS_KINIT_TIMER_PARAMETER
+
+
+RESULT_CODE_OK = "OK"
+RESULT_CODE_CRITICAL = "CRITICAL"
+RESULT_CODE_UNKNOWN = "UNKNOWN"
+
+OK_MESSAGE = "WebHCat status was OK ({0:.3f}s response from {1})"
+CRITICAL_CONNECTION_MESSAGE = "Connection failed to {0} + \n{1}"
+CRITICAL_HTTP_MESSAGE = "HTTP {0} response from {1} \n{2}"
+CRITICAL_WEBHCAT_STATUS_MESSAGE = 'WebHCat returned an unexpected status of "{0}"'
+CRITICAL_WEBHCAT_UNKNOWN_JSON_MESSAGE = "Unable to determine WebHCat health from unexpected JSON response"
+
+TEMPLETON_PORT_KEY = '{{webhcat-site/templeton.port}}'
+SECURITY_ENABLED_KEY = '{{cluster-env/security_enabled}}'
+WEBHCAT_PRINCIPAL_KEY = '{{webhcat-site/templeton.kerberos.principal}}'
+WEBHCAT_KEYTAB_KEY = '{{webhcat-site/templeton.kerberos.keytab}}'
+
+SMOKEUSER_KEYTAB_KEY = '{{cluster-env/smokeuser_keytab}}'
+SMOKEUSER_PRINCIPAL_KEY = '{{cluster-env/smokeuser_principal_name}}'
+SMOKEUSER_KEY = '{{cluster-env/smokeuser}}'
+
+# The configured Kerberos executable search paths, if any
+KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY = '{{kerberos-env/executable_search_paths}}'
+
+WEBHCAT_OK_RESPONSE = 'ok'
+WEBHCAT_PORT_DEFAULT = 50111
+
+CONNECTION_TIMEOUT_KEY = 'connection.timeout'
+CONNECTION_TIMEOUT_DEFAULT = 5.0
+CURL_CONNECTION_TIMEOUT_DEFAULT = str(int(CONNECTION_TIMEOUT_DEFAULT))
+
+# default keytab location
+SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY = 'default.smoke.keytab'
+SMOKEUSER_KEYTAB_DEFAULT = '/etc/security/keytabs/smokeuser.headless.keytab'
+
+# default smoke principal
+SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY = 'default.smoke.principal'
+SMOKEUSER_PRINCIPAL_DEFAULT = 'ambari-qa@EXAMPLE.COM'
+
+# default smoke user
+SMOKEUSER_DEFAULT = 'ambari-qa'
+logger = logging.getLogger('ambari_alerts')
+
+def get_tokens():
+  """
+  Returns a tuple of tokens in the format {{site/property}} that will be used
+  to build the dictionary passed into execute
+  """
+  return (TEMPLETON_PORT_KEY, SECURITY_ENABLED_KEY, SMOKEUSER_KEYTAB_KEY,SMOKEUSER_PRINCIPAL_KEY,
+          KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY, SMOKEUSER_KEY)
+
+
+def execute(configurations={}, parameters={}, host_name=None):
+  """
+  Returns a tuple containing the result code and a pre-formatted result label
+
+  Keyword arguments:
+  configurations (dictionary): a mapping of configuration key to value
+  parameters (dictionary): a mapping of script parameter key to value
+  host_name (string): the name of this host where the alert is running
+  """
+
+  result_code = RESULT_CODE_UNKNOWN
+
+  if configurations is None:
+    return (result_code, ['There were no configurations supplied to the script.'])
+
+  webhcat_port = WEBHCAT_PORT_DEFAULT
+  if TEMPLETON_PORT_KEY in configurations:
+    webhcat_port = int(configurations[TEMPLETON_PORT_KEY])
+
+  security_enabled = False
+  if SECURITY_ENABLED_KEY in configurations:
+    security_enabled = configurations[SECURITY_ENABLED_KEY].lower() == 'true'
+
+  # parse script arguments
+  connection_timeout = CONNECTION_TIMEOUT_DEFAULT
+  curl_connection_timeout = CURL_CONNECTION_TIMEOUT_DEFAULT
+  if CONNECTION_TIMEOUT_KEY in parameters:
+    connection_timeout = float(parameters[CONNECTION_TIMEOUT_KEY])
+    curl_connection_timeout = str(int(connection_timeout))
+
+
+  # the alert will always run on the webhcat host
+  if host_name is None:
+    host_name = socket.getfqdn()
+
+  smokeuser = SMOKEUSER_DEFAULT
+
+  if SMOKEUSER_KEY in configurations:
+    smokeuser = configurations[SMOKEUSER_KEY]
+
+  # webhcat always uses http, never SSL
+  query_url = "http://{0}:{1}/templeton/v1/status?user.name={2}".format(host_name, webhcat_port, smokeuser)
+
+  # initialize
+  total_time = 0
+  json_response = {}
+
+  if security_enabled:
+    try:
+      # defaults
+      smokeuser_keytab = SMOKEUSER_KEYTAB_DEFAULT
+      smokeuser_principal = SMOKEUSER_PRINCIPAL_DEFAULT
+
+      # check script params
+      if SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY in parameters:
+        smokeuser_principal = parameters[SMOKEUSER_PRINCIPAL_SCRIPT_PARAM_KEY]
+      if SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY in parameters:
+        smokeuser_keytab = parameters[SMOKEUSER_KEYTAB_SCRIPT_PARAM_KEY]
+
+      # check configurations last as they should always take precedence
+      if SMOKEUSER_PRINCIPAL_KEY in configurations:
+        smokeuser_principal = configurations[SMOKEUSER_PRINCIPAL_KEY]
+      if SMOKEUSER_KEYTAB_KEY in configurations:
+        smokeuser_keytab = configurations[SMOKEUSER_KEYTAB_KEY]
+
+      # Get the configured Kerberos executable search paths, if any
+      kerberos_executable_search_paths = None
+      if KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY in configurations:
+        kerberos_executable_search_paths = configurations[KERBEROS_EXECUTABLE_SEARCH_PATHS_KEY]
+
+      kinit_timer_ms = parameters.get(KERBEROS_KINIT_TIMER_PARAMETER, DEFAULT_KERBEROS_KINIT_TIMER_MS)
+
+      env = Environment.get_instance()
+      stdout, stderr, time_millis = curl_krb_request(env.tmp_dir, smokeuser_keytab, smokeuser_principal,
+        query_url, "webhcat_alert_cc_", kerberos_executable_search_paths, True,
+        "WebHCat Server Status", smokeuser, connection_timeout=curl_connection_timeout,
+        kinit_timer_ms = kinit_timer_ms)
+
+      # check the response code
+      response_code = int(stdout)
+
+      # 0 indicates no connection
+      if response_code == 0:
+        label = CRITICAL_CONNECTION_MESSAGE.format(query_url, traceback.format_exc())
+        return (RESULT_CODE_CRITICAL, [label])
+
+      # any other response aside from 200 is a problem
+      if response_code != 200:
+        label = CRITICAL_HTTP_MESSAGE.format(response_code, query_url, traceback.format_exc())
+        return (RESULT_CODE_CRITICAL, [label])
+
+      # now that we have the http status and it was 200, get the content
+      stdout, stderr, total_time = curl_krb_request(env.tmp_dir, smokeuser_keytab, smokeuser_principal,
+        query_url, "webhcat_alert_cc_", kerberos_executable_search_paths,
+        False, "WebHCat Server Status", smokeuser, connection_timeout=curl_connection_timeout,
+        kinit_timer_ms = kinit_timer_ms)
+
+      json_response = json.loads(stdout)
+    except:
+      return (RESULT_CODE_CRITICAL, [traceback.format_exc()])
+  else:
+    url_response = None
+
+    try:
+      # execute the query for the JSON that includes WebHCat status
+      start_time = time.time()
+      url_response = urllib2.urlopen(query_url, timeout=connection_timeout)
+      total_time = time.time() - start_time
+
+      json_response = json.loads(url_response.read())
+    except urllib2.HTTPError as httpError:
+      label = CRITICAL_HTTP_MESSAGE.format(httpError.code, query_url, traceback.format_exc())
+      return (RESULT_CODE_CRITICAL, [label])
+    except:
+      label = CRITICAL_CONNECTION_MESSAGE.format(query_url, traceback.format_exc())
+      return (RESULT_CODE_CRITICAL, [label])
+    finally:
+      if url_response is not None:
+        try:
+          url_response.close()
+        except:
+          pass
+
+
+  # if status is not in the response, we can't do any check; return CRIT
+  if 'status' not in json_response:
+    return (RESULT_CODE_CRITICAL, [CRITICAL_WEBHCAT_UNKNOWN_JSON_MESSAGE + str(json_response)])
+
+
+  # URL response received, parse it
+  try:
+    webhcat_status = json_response['status']
+  except:
+    return (RESULT_CODE_CRITICAL, [CRITICAL_WEBHCAT_UNKNOWN_JSON_MESSAGE + "\n" + traceback.format_exc()])
+
+
+  # proper JSON received, compare against known value
+  if webhcat_status.lower() == WEBHCAT_OK_RESPONSE:
+    result_code = RESULT_CODE_OK
+    label = OK_MESSAGE.format(total_time, query_url)
+  else:
+    result_code = RESULT_CODE_CRITICAL
+    label = CRITICAL_WEBHCAT_STATUS_MESSAGE.format(webhcat_status)
+
+  return (result_code, [label])


[32/50] [abbrv] bigtop git commit: removed commented code and comments

Posted by rv...@apache.org.
removed commented code and comments

removed commented code and comments


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

Branch: refs/heads/BIGTOP-2666
Commit: ad84696ea64867844c2cbd5f506f1b77cb865ba4
Parents: 80fe9cb
Author: roypradeep <ro...@us.ibm.com>
Authored: Tue Nov 1 14:50:12 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:03 2017 -0800

----------------------------------------------------------------------
 .../odpi/specs/runtime/hive/TestBeeline.java    | 466 ++++++++++---------
 1 file changed, 257 insertions(+), 209 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/ad84696e/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
index 6d34e47..37c71cf 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
@@ -34,224 +34,272 @@ public class TestBeeline {
 
 	public static final Log LOG = LogFactory.getLog(TestBeeline.class.getName());
 
-	  private static final String URL = "odpiHiveTestJdbcUrl";
-	  private static final String USER = "odpiHiveTestJdbcUser";
-	  private static final String PASSWD = "odpiHiveTestJdbcPassword";
-	  
-	  private static Map<String, String> results;
-	  
-	  private static String beelineUrl; 
-	  private static String beelineUser;
-	  private static String beelinePasswd;
-	  
-	  @BeforeClass
-	  public static void checkHiveHome(){
-		  results = HiveHelper.execCommand(new CommandLine("echo").addArgument("$HIVE_HOME"));
-		  Assert.assertEquals("HIVE_HOME is not in the current path.", "", Integer.parseInt(results.get("outputStream")));
-		  TestBeeline.beelineUrl = System.getProperty(URL);
-		  TestBeeline.beelineUser = System.getProperty(USER);
-		  TestBeeline.beelinePasswd = System.getProperty(PASSWD);
-		  
-		  // Create Url with username and/or passowrd to handle all ways to connect to beeline
-		  
-		  if (beelineUser != null && beelineUser != "") { beelineUrl = beelineUrl+" -n "+beelineUser; }
-		  else if (beelineUser != null && beelineUser != "" && beelinePasswd != null && beelinePasswd != "") { beelineUrl = beelineUrl+" -n "+beelineUser+" -p "+"beelinePasswd"; }
-		  
-	  }
-	  
-	  @Test
-	  public static void checkBeeline() {
-	    
-	    LOG.info("URL is " + beelineUrl); 
-	    LOG.info("User is " + beelineUser);
-	    LOG.info("Passwd is " + beelinePasswd); 
-	    LOG.info("Passwd is null " + (beelinePasswd == null));
-	    
-	    results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl));
-	    String consoleMsg = results.get("outputStream").toLowerCase();
-	    //System.out.println(consoleMsg);
-	    try {
-			Assert.assertEquals("beeline is using beelineUrl", true, consoleMsg.contains("connecting to "+beelineUrl) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-			LOG.info("Beeline -u PASSED.");
-		} catch (AssertionError e) {
-			// TODO Auto-generated catch block
-			LOG.error("Beeline -u FAILED.");
-			LOG.error(results.get("outputStream"));
+	private static final String URL = "odpiHiveTestJdbcUrl";
+	private static final String USER = "odpiHiveTestJdbcUser";
+	private static final String PASSWD = "odpiHiveTestJdbcPassword";
+
+	private static Map<String, String> results;
+
+	private static String beelineUrl; 
+	private static String beelineUser;
+	private static String beelinePasswd;
+	private static String testUrl;
+
+	//flags to check if username and password should be added as argument in some tests
+	private static boolean bothUserPass = false;
+	private static boolean onlyUser = false;
+
+	@BeforeClass
+	public static void initialSetup(){
+
+		TestBeeline.beelineUrl = System.getProperty(URL);
+		TestBeeline.beelineUser = System.getProperty(USER);
+		TestBeeline.beelinePasswd =System.getProperty(PASSWD);
+		TestBeeline.testUrl = System.getProperty(URL);
+
+		// Create Url with username and/or passowrd to handle all ways to connect to beeline
+		if (beelineUser != null && beelineUser != "" && beelinePasswd != null && beelinePasswd != "") 
+		{ 
+			testUrl = beelineUrl+" -n "+beelineUser+" -p "+beelinePasswd; 
+			bothUserPass=true;
 		}
-	    
- 	  }
-	  
-	  @Test
-	  public static void checkBeelineConnect(){
-		  try(PrintWriter out = new PrintWriter("connect.url")){ out.println("!connect " + beelineUrl+";"); out.println("!quit"); } 
-		  catch (FileNotFoundException e1) {
-			// TODO Auto-generated catch block
-			e1.printStackTrace();
+		else if (beelineUser != null && beelineUser != "") 
+		{ 
+			testUrl = beelineUrl+" -n "+beelineUser; 
+			onlyUser=true;
 		}
-		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -f connect.url",false));
-		  String consoleMsg = results.get("outputStream").toLowerCase();
-		   
-		    try {
-				Assert.assertEquals("beeline is able to connect to " +beelineUrl, true, consoleMsg.contains("connecting to "+beelineUrl) && !consoleMsg.contains("error") && !consoleMsg.contains("exception") );
-				LOG.info("Beeline !connect PASSED.");
-			} catch (AssertionError e) {
-				// TODO Auto-generated catch block
-				LOG.error("Beeline !connect FAILED.");
-				LOG.error(results.get("outputStream"));
-			}  
-	  }
-	  
-	  @Test
-	  public static void checkBeelineHelp(){
-		   results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("--help"));
-		  String consoleMsg = results.get("outputStream").toLowerCase();
-		    try {
-				Assert.assertEquals("beeline help works", true, consoleMsg.contains("usage: java org.apache.hive.cli.beeline.beeLine" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-				LOG.info("Beeline --help PASSED.");
-			} catch (AssertionError e) {
-				// TODO Auto-generated catch block
-				LOG.error("Beeline --help FAILED.");
-				LOG.error(results.get("outputStream"));
-			}  
-	  }
-
-	  @Test
-	  public static void checkBeelineQueryExecFromCmdLine(){
-		  results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("SHOW DATABASES"));
-		  
-		  if(!results.get("outputStream").contains("odpi_runtime_hive")){
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
-				
-			}else{
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+		System.out.println("Setting url"+testUrl); 
+
+		LOG.info("URL is " + beelineUrl); 
+		LOG.info("User is " + beelineUser);
+		LOG.info("Passwd is " + beelinePasswd); 
+		LOG.info("Passwd is null " + (beelinePasswd == null));
+	}
+
+	@Test
+	public void checkBeeline() {
+
+		System.out.println(beelineUrl);  
+
+		results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(testUrl));
+		String consoleMsg = results.get("outputStream").toLowerCase();
+		Assert.assertEquals("beeline -u FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("connecting to "+beelineUrl) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+
+
+	}
+
+	@Test
+	public void checkBeelineConnect(){
+		try(PrintWriter out = new PrintWriter("connect.url")){ out.println("!connect " + beelineUrl+" "+beelineUser+" "+beelinePasswd+";"); out.println("!quit;"); } 
+		catch (FileNotFoundException e1) {
 			
-			}
-		  String consoleMsg = results.get("outputStream").toLowerCase();
-		  try {
-				Assert.assertEquals("beeline execution works", true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-				LOG.info("Beeline -e PASSED.");
-			} catch (AssertionError e) {
-				// TODO Auto-generated catch block
-				LOG.error("Beeline -e FAILED.");
-				LOG.error(results.get("outputStream"));
-			}  
-		  	
-		  HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));		    
-	  }
-	  
-	  @Test
-	  public static void checkBeelineQueryExecFromFile() throws FileNotFoundException{
-		  
-			try(PrintWriter out = new PrintWriter("beeline-f1.sql")){ out.println("SHOW DATABASES;"); }
-			try(PrintWriter out = new PrintWriter("beeline-f2.sql")){ out.println("CREATE DATABASE odpi_runtime_hive;"); }
-			try(PrintWriter out = new PrintWriter("beeline-f3.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); out.println("CREATE DATABASE odpi_runtime_hive;"); }
-		 	try(PrintWriter out = new PrintWriter("beeline-f4.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); }
-		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -f beeline-f1.sql",false));
-
-		  if(!results.get("outputStream").contains("odpi_runtime_hive")){
-				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -f beeline-f2.sql",false));
-				
+			e1.printStackTrace();
+		}
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -f connect.url",false));
+		String consoleMsg = results.get("outputStream").toLowerCase();
+
+
+		Assert.assertEquals("beeline !connect FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("connecting to "+beelineUrl) && !consoleMsg.contains("error") && !consoleMsg.contains("exception") );  
+	}
+
+	@Test
+	public void checkBeelineHelp(){
+		results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("--help"));
+		String consoleMsg = results.get("outputStream").toLowerCase();
+		Assert.assertEquals("beeline --help FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("display this message" ) && consoleMsg.contains("usage: java org.apache.hive.cli.beeline.beeline") && !consoleMsg.contains("exception"));
+
+	}
+
+	@Test
+	public void checkBeelineQueryExecFromCmdLine(){
+
+		if (bothUserPass) 
+		{ 
+			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("SHOW DATABASES;"));
+
+			if(!results.get("outputStream").contains("odpi_runtime_hive")){
+
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("SHOW DATABASES;"));
 			}else{
-				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -f beeline-f3.sql",false));
+
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive;"));
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("SHOW DATABASES;"));
+
 			}
-		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -f beeline-f1.sql",false));
-		  
-		  String consoleMsg = results.get("outputStream").toLowerCase();
-		  try {
-				Assert.assertEquals("beeline execution with file works", true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-				LOG.info("Beeline -f PASSED.");
-			} catch (AssertionError e) {
-				// TODO Auto-generated catch block
-				LOG.error("Beeline -f FAILED.");
-				LOG.error(results.get("outputStream"));
-			}  
-		  
-		  HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -f beeline-f4.sql",false));		    
-	  }
-	  
-	  public static void checkBeelineInitFile() throws FileNotFoundException{
-		  
-			try(PrintWriter out = new PrintWriter("beeline-i1.sql")){ out.println("SHOW DATABASES;"); }
-			try(PrintWriter out = new PrintWriter("beeline-i2.sql")){ out.println("CREATE DATABASE odpi_runtime_beeline_init;"); }
-			try(PrintWriter out = new PrintWriter("beeline-i3.sql")){ out.println("DROP DATABASE odpi_runtime_beeline_init;"); out.println("CREATE DATABASE odpi_runtime_beeline_init;"); }
-		 	try(PrintWriter out = new PrintWriter("beeline-i4.sql")){ out.println("DROP DATABASE odpi_runtime_beeline_init;"); }
-		 	
-		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -i beeline-i1.sql",false));
-	  
-		  if(!results.get("outputStream").contains("odpi_runtime_beeline_init")){
-				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -i beeline-i2.sql",false));
-				
+			String consoleMsg = results.get("outputStream").toLowerCase();
+			Assert.assertEquals("beeline -e FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+
+			HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		}
+		else if (onlyUser) 
+		{ 
+			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("SHOW DATABASES;"));
+
+			if(!results.get("outputStream").contains("odpi_runtime_hive")){
+
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("SHOW DATABASES;"));
 			}else{
-				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -i beeline-i3.sql",false));
+
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive;"));
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("SHOW DATABASES;"));
+
 			}
-		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -i beeline-i1.sql",false));
-		  String consoleMsg = results.get("outputStream").toLowerCase();
-		  try {
-				Assert.assertEquals("beeline execution with init file works", true, consoleMsg.contains("odpi_runtime_beeline_init") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-				LOG.info("Beeline -i PASSED.");
-			} catch (AssertionError e) {
-				// TODO Auto-generated catch block
-				LOG.error("Beeline -i FAILED.");
-				LOG.error(results.get("outputStream"));
-			}  
-
-		  HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -i beeline-i4.sql",false));		    
-	  }
-	  
-	  public static void checkBeelineHiveVar() throws FileNotFoundException{
-		  
-			try(PrintWriter out = new PrintWriter("beeline-hv1.sql")){ out.println("SHOW DATABASES;"); }
-			try(PrintWriter out = new PrintWriter("beeline-hv2.sql")){ out.println("CREATE DATABASE ${db};"); }
-			try(PrintWriter out = new PrintWriter("beeline-hv3.sql")){ out.println("DROP DATABASE ${db};"); out.println("CREATE DATABASE ${db};"); }
-		 	try(PrintWriter out = new PrintWriter("beeline-hv4.sql")){ out.println("DROP DATABASE ${db};"); }
-		 	
-		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv1.sql",false));
-		  String consoleMsg = results.get("outputStream");
-		  
-		  if(!results.get("outputStream").contains("odpi_runtime_beeline_hivevar")){
-				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv2.sql",false));
-				
+			String consoleMsg = results.get("outputStream").toLowerCase();
+			Assert.assertEquals("beeline -e FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+
+			HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		}
+		else {
+			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("SHOW DATABASES;"));
+
+			if(!results.get("outputStream").contains("odpi_runtime_hive")){
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
+
 			}else{
-				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv3.sql",false));
-			}
-		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv1.sql",false));
-		  consoleMsg = results.get("outputStream").toLowerCase();
-
-		  try {
-				Assert.assertEquals("beeline execution with hivevar file works", true, consoleMsg.contains("odpi_runtime_beeline_hivevar") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-				LOG.info("Beeline --hivevar PASSED.");
-			} catch (AssertionError e) {
-				// TODO Auto-generated catch block
-				LOG.error("Beeline --hivevar FAILED.");
-				LOG.error(results.get("outputStream"));
-			}  
-		  	
-		  HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv4.sql",false));		    
-	  }
-	  
-	  @Test
-	  public static void CheckBeelineFastConnect(){
-		   results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("--fastConnect=false"));
-		  String consoleMsg = results.get("outputStream").toLowerCase();
-		    
-		    try {
-				Assert.assertEquals("beeline fastConnect works", true, consoleMsg.contains("set fastconnect to true to skip") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-				LOG.info("Beeline --fastConnect PASSED.");
-			} catch (AssertionError e) {
-				// TODO Auto-generated catch block
-				LOG.error("Beeline --fastConnect FAILED.");
-				LOG.error(results.get("outputStream"));
-			}  
-	  }
-	  
-	  @AfterClass
-	  public static void cleanup() throws FileNotFoundException {
-	    
-		  	results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf beeline*.sql", false));
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive;"));
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("SHOW DATABASES;"));
 			
-	  }
+
+			}
+			String consoleMsg = results.get("outputStream").toLowerCase();
+			Assert.assertEquals("beeline -e FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+
+			HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		}
+	}
+
+	@Test
+	public void checkBeelineQueryExecFromFile() throws FileNotFoundException{
+
+		try(PrintWriter out = new PrintWriter("beeline-f1.sql")){ out.println("SHOW DATABASES;"); }
+		try(PrintWriter out = new PrintWriter("beeline-f2.sql")){ out.println("CREATE DATABASE odpi_runtime_hive;"); }
+		try(PrintWriter out = new PrintWriter("beeline-f3.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); out.println("CREATE DATABASE odpi_runtime_hive;"); }
+		try(PrintWriter out = new PrintWriter("beeline-f4.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); }
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -f beeline-f1.sql",false));
+
+		if(!results.get("outputStream").contains("odpi_runtime_hive")){
+			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -f beeline-f2.sql",false));
+
+		}else{
+			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -f beeline-f3.sql",false));
+		}
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -f beeline-f1.sql",false));
+
+		String consoleMsg = results.get("outputStream").toLowerCase();
+		Assert.assertEquals("beeline -f FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+
+		HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -f beeline-f4.sql",false));		    
+	}
+
+	@Test
+	public void checkBeelineInitFile() throws FileNotFoundException{
+
+		try(PrintWriter out = new PrintWriter("beeline-i1.sql")){ out.println("SHOW DATABASES;"); }
+		try(PrintWriter out = new PrintWriter("beeline-i2.sql")){ out.println("CREATE DATABASE odpi_runtime_beeline_init;"); }
+		try(PrintWriter out = new PrintWriter("beeline-i3.sql")){ out.println("DROP DATABASE odpi_runtime_beeline_init;"); out.println("CREATE DATABASE odpi_runtime_beeline_init;"); }
+		try(PrintWriter out = new PrintWriter("beeline-i4.sql")){ out.println("DROP DATABASE odpi_runtime_beeline_init;"); }
+
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -i beeline-i1.sql",false));
+
+		if(!results.get("outputStream").contains("odpi_runtime_beeline_init")){
+			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -i beeline-i2.sql",false));
+
+		}else{
+			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -i beeline-i3.sql",false));
+		}
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -i beeline-i1.sql",false));
+		String consoleMsg = results.get("outputStream").toLowerCase();
+		Assert.assertEquals("beeline -i FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_beeline_init") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+
+		HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -i beeline-i4.sql",false));		    
+	}
+
+	@Test
+	public void checkBeelineHiveVar() throws FileNotFoundException{
+
+		try(PrintWriter out = new PrintWriter("beeline-hv1.sql")){ out.println("SHOW DATABASES;"); }
+		try(PrintWriter out = new PrintWriter("beeline-hv2.sql")){ out.println("CREATE DATABASE ${db};"); }
+		try(PrintWriter out = new PrintWriter("beeline-hv3.sql")){ out.println("DROP DATABASE ${db};"); out.println("CREATE DATABASE ${db};"); }
+		try(PrintWriter out = new PrintWriter("beeline-hv4.sql")){ out.println("DROP DATABASE ${db};"); }
+
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv1.sql",false));
+		String consoleMsg = results.get("outputStream");
+
+		if(!results.get("outputStream").contains("odpi_runtime_beeline_hivevar")){
+			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv2.sql",false));
+
+		}else{
+			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv3.sql",false));
+		}
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv1.sql",false));
+		consoleMsg = results.get("outputStream").toLowerCase();
+
+		Assert.assertEquals("beeline --hivevar FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_beeline_hivevar") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+
+		HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv4.sql",false));		    
+	}
+
+	@Test
+	public void checkBeelineFastConnect(){
+		results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(testUrl).addArgument("--fastConnect=false"));
+		String consoleMsg = results.get("outputStream").toLowerCase();
+		Assert.assertEquals("beeline --fastConnect FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("set fastconnect to true to skip")); 
+	}
+
+	@Test
+	public void checkBeelineVerbose(){
+
+		//explicit check for username password again as url containing -u -p is not working in single addArgument function with testUrl
+
+		if (bothUserPass) 
+		{ 
+			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("--verbose=true"));
+		}
+		else if (onlyUser) 
+		{ 
+			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("--verbose=true"));
+		}
+		else {
+			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("--verbose=true"));
+		}
+		String consoleMsg = results.get("outputStream").toLowerCase();
+		Assert.assertEquals("beeline --verbose FAILED using url "+testUrl+". \n" +results.get("outputStream"), true, consoleMsg.contains("issuing: !connect jdbc:hive2:") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+	}
+
+	@Test
+	public void checkBeelineShowHeader(){
+		
+		//explicit check for username password again as url containing -u -p is not working in single addArgument function with testUrl
+
+		if (bothUserPass) 
+		{ 
+			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("--showHeader=false").addArgument("-e").addArgument("SHOW DATABASES;"));
+		}
+		else if (onlyUser) 
+		{ 
+			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("--showHeader=false").addArgument("-e").addArgument("SHOW DATABASES;"));
+		}
+		else {
+			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("--showHeader=false").addArgument("-e").addArgument("SHOW DATABASES;"));
+		}
+		String consoleMsg = results.get("outputStream").toLowerCase();
+		Assert.assertEquals("beeline --showHeader FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("default")&&!consoleMsg.contains("database_name") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+
+	}
+
+	@AfterClass
+	public static void cleanup() throws FileNotFoundException {
+
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf beeline*.sql", false));
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf connect.url", false));
+
+	}
+
 
 
-	  
 }


[27/50] [abbrv] bigtop git commit: ODPI-197. Enable Kerberos in ODPi Ambari

Posted by rv...@apache.org.
ODPI-197. Enable Kerberos in ODPi Ambari


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

Branch: refs/heads/BIGTOP-2666
Commit: 166d2e01a35f8bd3f99fce86fc55c843ecdf3740
Parents: b1cc82c
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Fri Oct 28 10:55:30 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:01 2017 -0800

----------------------------------------------------------------------
 .../src/common/ambari/ODPi/1.0/kerberos.json    | 60 +++++++++++++
 .../ambari/ODPi/1.0/services/HIVE/kerberos.json | 17 ----
 .../ODPi/1.0/services/KERBEROS/metainfo.xml     | 26 ++++++
 .../src/common/ambari/ODPi/1.0/widgets.json     | 95 ++++++++++++++++++++
 4 files changed, 181 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/166d2e01/bigtop-packages/src/common/ambari/ODPi/1.0/kerberos.json
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/kerberos.json b/bigtop-packages/src/common/ambari/ODPi/1.0/kerberos.json
new file mode 100644
index 0000000..3aad080
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/kerberos.json
@@ -0,0 +1,60 @@
+{
+  "properties": {
+    "realm": "${kerberos-env/realm}",
+    "keytab_dir": "/etc/security/keytabs",
+    "additional_realms": ""
+  },
+  "identities": [
+    {
+      "name": "spnego",
+      "principal": {
+        "value": "HTTP/_HOST@${realm}",
+        "type" : "service"
+      },
+      "keytab": {
+        "file": "${keytab_dir}/spnego.service.keytab",
+        "owner": {
+          "name": "root",
+          "access": "r"
+        },
+        "group": {
+          "name": "${cluster-env/user_group}",
+          "access": "r"
+        }
+      }
+    },
+    {
+      "name": "smokeuser",
+      "principal": {
+        "value": "${cluster-env/smokeuser}-${cluster_name|toLower()}@${realm}",
+        "type" : "user",
+        "configuration": "cluster-env/smokeuser_principal_name",
+        "local_username" : "${cluster-env/smokeuser}"
+      },
+      "keytab": {
+        "file": "${keytab_dir}/smokeuser.headless.keytab",
+        "owner": {
+          "name": "${cluster-env/smokeuser}",
+          "access": "r"
+        },
+        "group": {
+          "name": "${cluster-env/user_group}",
+          "access": "r"
+        },
+        "configuration": "cluster-env/smokeuser_keytab"
+      }
+    },
+    {
+      "name": "ambari-server",
+      "principal": {
+        "value": "ambari-server-${cluster_name|toLower()}@${realm}",
+        "type" : "user",
+        "configuration": "cluster-env/ambari_principal_name"
+      },
+      "keytab": {
+        "file": "${keytab_dir}/ambari.server.keytab"
+      }
+    }
+  ]
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/166d2e01/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/kerberos.json
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/kerberos.json b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/kerberos.json
index 34bda73..4b31f7a 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/kerberos.json
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/kerberos.json
@@ -113,23 +113,6 @@
           ]
         },
         {
-          "name": "HIVE_SERVER_INTERACTIVE",
-          "identities": [
-            {
-              "name": "/HDFS/NAMENODE/hdfs"
-            },
-            {
-              "name": "/HIVE/HIVE_SERVER/hive_server_hive"
-            },
-            {
-              "name": "/HIVE/HIVE_SERVER/spnego"
-            },
-            {
-              "name": "/YARN/NODEMANAGER/llap_zk_hive"
-            }
-          ]
-        },
-        {
           "name": "WEBHCAT_SERVER",
           "identities": [
             {

http://git-wip-us.apache.org/repos/asf/bigtop/blob/166d2e01/bigtop-packages/src/common/ambari/ODPi/1.0/services/KERBEROS/metainfo.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/KERBEROS/metainfo.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/KERBEROS/metainfo.xml
new file mode 100755
index 0000000..25cfcc6
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/KERBEROS/metainfo.xml
@@ -0,0 +1,26 @@
+<?xml version="1.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.
+-->
+<metainfo>
+  <schemaVersion>2.0</schemaVersion>
+  <services>
+    <service>
+      <name>KERBEROS</name>
+      <extends>common-services/KERBEROS/1.10.3-10</extends>
+    </service>
+  </services>
+</metainfo>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/166d2e01/bigtop-packages/src/common/ambari/ODPi/1.0/widgets.json
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/widgets.json b/bigtop-packages/src/common/ambari/ODPi/1.0/widgets.json
new file mode 100644
index 0000000..3176354
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/widgets.json
@@ -0,0 +1,95 @@
+{
+  "layouts": [
+    {
+      "layout_name": "default_system_heatmap",
+      "display_name": "Heatmaps",
+      "section_name": "SYSTEM_HEATMAPS",
+      "widgetLayoutInfo": [
+        {
+          "widget_name": "Host Disk Space Used %",
+          "description": "",
+          "widget_type": "HEATMAP",
+          "is_visible": true,
+          "metrics": [
+            {
+              "name": "disk_free",
+              "metric_path": "metrics/disk/disk_free",
+              "service_name": "STACK"
+            },
+            {
+              "name": "disk_total",
+              "metric_path": "metrics/disk/disk_total",
+              "service_name": "STACK"
+            }
+          ],
+          "values": [
+            {
+              "name": "Host Disk Space Used %",
+              "value": "${((disk_total-disk_free)/disk_total)*100}"
+            }
+          ],
+          "properties": {
+            "display_unit": "%",
+            "max_limit": "100"
+          }
+        },
+        {
+          "widget_name": "Host Memory Used %",
+          "description": "",
+          "widget_type": "HEATMAP",
+          "is_visible": false,
+          "metrics": [
+            {
+              "name": "mem_total",
+              "metric_path": "metrics/memory/mem_total",
+              "service_name": "STACK"
+            },
+            {
+              "name": "mem_free",
+              "metric_path": "metrics/memory/mem_free",
+              "service_name": "STACK"
+            },
+            {
+              "name": "mem_cached",
+              "metric_path": "metrics/memory/mem_cached",
+              "service_name": "STACK"
+            }
+          ],
+          "values": [
+            {
+              "name": "Host Memory Used %",
+              "value": "${((mem_total-mem_free-mem_cached)/mem_total)*100}"
+            }
+          ],
+          "properties": {
+            "display_unit": "%",
+            "max_limit": "100"
+          }
+        },
+        {
+          "widget_name": "Host CPU Wait IO %",
+          "description": "",
+          "widget_type": "HEATMAP",
+          "is_visible": false,
+          "metrics": [
+            {
+              "name": "cpu_wio",
+              "metric_path": "metrics/cpu/cpu_wio",
+              "service_name": "STACK"
+            }
+          ],
+          "values": [
+            {
+              "name": "Host Memory Used %",
+              "value": "${cpu_wio*100}"
+            }
+          ],
+          "properties": {
+            "display_unit": "%",
+            "max_limit": "100"
+          }
+        }
+      ]
+    }
+  ]
+}


[04/50] [abbrv] bigtop git commit: Added SQL tests for basic SQL coverage

Posted by rv...@apache.org.
Added SQL tests for basic SQL coverage


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

Branch: refs/heads/BIGTOP-2666
Commit: ff03afa416a896f3947a0d0bccfce00ff0fb1582
Parents: 3de7271
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Oct 12 16:59:25 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:45:55 2017 -0800

----------------------------------------------------------------------
 .../org/odpi/specs/runtime/hive/TestSql.java    | 143 ++++++++++++++++++-
 1 file changed, 136 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/ff03afa4/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
index 3965f07..71ca12c 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
@@ -24,6 +24,8 @@ import org.junit.Test;
 import java.sql.SQLException;
 import java.sql.Statement;
 
+// This does not test every option that Hive supports, but does try to touch the major
+// options, especially anything unique to Hive.  See each test for areas tested and not tested.
 public class TestSql extends JdbcConnector {
   private static final Log LOG = LogFactory.getLog(TestSql.class.getName());
 
@@ -98,7 +100,7 @@ public class TestSql extends JdbcConnector {
           "stored as orc " +
           "tblproperties ('a' = 'b')");
 
-      // NOTES: Not testing SKEWED BY, ROW FORMAT, STORED BY (storage handler
+      // Not testing SKEWED BY, ROW FORMAT, STORED BY (storage handler
 
       stmt.execute("create temporary table " + table3 + " like " + table2);
 
@@ -111,8 +113,8 @@ public class TestSql extends JdbcConnector {
       stmt.execute("alter table " + table4 + " rename to " + table5);
       stmt.execute("alter table " + table2 + " set tblproperties ('c' = 'd')");
 
-      // NOTE: Not testing alter of clustered or sorted by, because that's suicidal
-      // NOTE: Not testing alter of skewed or serde properties since we didn't test it for create
+      // Not testing alter of clustered or sorted by, because that's suicidal
+      // Not testing alter of skewed or serde properties since we didn't test it for create
       // above.
 
       stmt.execute("drop table " + table1 + " purge");
@@ -144,7 +146,7 @@ public class TestSql extends JdbcConnector {
       stmt.execute("alter table " + table1 + " add columns (c3 float)");
       stmt.execute("alter table " + table1 + " drop partition (p1 = 'a')");
 
-      // NOTE: Not testing rename partition, exchange partition, msck repair, archive/unarchive,
+      // Not testing rename partition, exchange partition, msck repair, archive/unarchive,
       // set location, enable/disable no_drop/offline, compact (because not everyone may have
       // ACID on), change column
 
@@ -178,8 +180,8 @@ public class TestSql extends JdbcConnector {
     }
   }
 
-  // NOTE: Not testing indices because they are currently useless in Hive
-  // NOTE: Not testing macros because as far as I know no one uses them
+  // Not testing indices because they are currently useless in Hive
+  // Not testing macros because as far as I know no one uses them
 
   @Test
   public void function() throws SQLException {
@@ -198,9 +200,136 @@ public class TestSql extends JdbcConnector {
     }
   }
 
-  // NOTE: Not testing grant/revoke/roles as different vendors use different security solutions
+  // Not testing grant/revoke/roles as different vendors use different security solutions
   // and hence different things will work here.
 
+  // This covers insert (non-partitioned, partitioned, dynamic partitions, overwrite, with
+  // values and select), and multi-insert.  Load is not tested as there's no guarantee that the
+  // test machine has access to HDFS and thus the ability to upload a file.
+  @Test
+  public void insert() throws SQLException {
+    final String table1 = "odpi_insert_table1";
+    final String table2 = "odpi_insert_table2";
+    try (Statement stmt = conn.createStatement()) {
+      stmt.execute("drop table if exists " + table1);
+      stmt.execute("create table " + table1 +
+          "(c1 tinyint," +
+          " c2 smallint," +
+          " c3 int," +
+          " c4 bigint," +
+          " c5 float," +
+          " c6 double," +
+          " c7 decimal(8,2)," +
+          " c8 varchar(120)," +
+          " c9 char(10)," +
+          " c10 boolean)" +
+          " partitioned by (p1 string)");
+
+      // insert with partition
+      stmt.execute("insert into " + table1 + " partition (p1 = 'a') values " +
+          "(1, 2, 3, 4, 1.1, 2.2, 3.3, 'abcdef', 'ghi', true)," +
+          "(5, 6, 7, 8, 9.9, 8.8, 7.7, 'jklmno', 'pqr', true)");
+
+      stmt.execute("set hive.exec.dynamic.partition.mode=nonstrict");
+
+      // dynamic partition
+      stmt.execute("insert into " + table1 + " partition (p1) values " +
+          "(1, 2, 3, 4, 1.1, 2.2, 3.3, 'abcdef', 'ghi', true, 'b')," +
+          "(5, 6, 7, 8, 9.9, 8.8, 7.7, 'jklmno', 'pqr', true, 'b')");
+
+      stmt.execute("drop table if exists " + table2);
+
+      stmt.execute("create table " + table2 +
+          "(c1 tinyint," +
+          " c2 smallint," +
+          " c3 int," +
+          " c4 bigint," +
+          " c5 float," +
+          " c6 double," +
+          " c7 decimal(8,2)," +
+          " c8 varchar(120)," +
+          " c9 char(10)," +
+          " c10 boolean)");
+
+      stmt.execute("insert into " + table2 + " values " +
+          "(1, 2, 3, 4, 1.1, 2.2, 3.3, 'abcdef', 'ghi', true)," +
+          "(5, 6, 7, 8, 9.9, 8.8, 7.7, 'jklmno', 'pqr', true)");
+
+      stmt.execute("insert overwrite table " + table2 + " select c1, c2, c3, c4, c5, c6, c7, c8, " +
+          "c9, c10 from " + table1);
+
+      // multi-insert
+      stmt.execute("from " + table1 +
+          " insert into table " + table1 + " partition (p1 = 'c') " +
+          " select c1, c2, c3, c4, c5, c6, c7, c8, c9, c10" +
+          " insert into table " + table2 + " select c1, c2, c3, c4, c5, c6, c7, c8, c9, c10");
+    }
+  }
+
+  // This tests CTEs
+  @Test
+  public void cte() throws SQLException {
+    final String table1 = "odpi_cte_table1";
+    try (Statement stmt = conn.createStatement()) {
+      stmt.execute("drop table if exists " + table1);
+      stmt.execute("create table " + table1 + "(c1 int, c2 varchar(32))");
+      stmt.execute("insert into " + table1 + " values (1, 'abc'), (2, 'def')");
+      stmt.execute("with cte1 as (select c1 from " + table1 + " where c1 < 10) " +
+          " select c1 from cte1");
+    }
+  }
+
+  // This tests select, including CTEs, all/distinct, single tables, joins (inner & outer),
+  // group by (w/ and w/o having), order by, cluster by/distribute by/sort by, limit, union,
+  // subqueries, and over.
+
+  @Test
+  public void select() throws SQLException {
+    final String[] tables = {"odpi_select_table1", "odpi_select_table2"};
+    try (Statement stmt = conn.createStatement()) {
+      for (int i = 0; i < tables.length; i++) {
+        stmt.execute("drop table if exists " + tables[i]);
+        stmt.execute("create table " + tables[i] + "(c1 int, c2 varchar(32))");
+        stmt.execute("insert into " + tables[i] + " values (1, 'abc'), (2, 'def')");
+      }
+
+      // single table queries tested above in several places
+
+      stmt.execute("select all a.c2, SUM(a.c1), SUM(b.c1) " +
+          "from " + tables[0] + " a join " + tables[1] + " b on (a.c2 = b.c2) " +
+          "group by a.c2 " +
+          "order by a.c2 asc " +
+          "limit 10");
+
+      stmt.execute("select distinct a.c2 " +
+          "from " + tables[0] + " a left outer join " + tables[1] + " b on (a.c2 = b.c2) " +
+          "order by a.c2 desc ");
+
+      stmt.execute("select a.c2, SUM(a.c1) " +
+          "from " + tables[0] + " a right outer join " + tables[1] + " b on (a.c2 = b.c2) " +
+          "group by a.c2 " +
+          "having SUM(b.c1) > 0 " +
+          "order by a.c2 ");
+
+      stmt.execute("select a.c2, rank() over (partition by a.c1) " +
+          "from " + tables[0] + " a full outer join " + tables[1] + " b on (a.c2 = b.c2) ");
+
+      stmt.execute("select c2 from " + tables[0] + " union all select c2 from " + tables[1]);
+
+      stmt.execute("select * from " + tables[0] + " distribute by c1 sort by c2");
+      stmt.execute("select * from " + tables[0] + " cluster by c1");
+
+      stmt.execute("select * from (select c1 from " + tables[0] + ") t");
+      stmt.execute("select * from " + tables[0] + " where c1 in (select c1 from " + tables[1] +
+          ")");
+
+    }
+
+  }
+
+  // Update and delete are not tested because not everyone configures their system to run
+  // with ACID.
+
 
 }
 


[31/50] [abbrv] bigtop git commit: renamed URL USER and PASSWD

Posted by rv...@apache.org.
renamed URL USER and PASSWD


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

Branch: refs/heads/BIGTOP-2666
Commit: 3bc96f8c7be83ee898d36208c09c224082a0411a
Parents: ddeb29f
Author: roypradeep <ro...@us.ibm.com>
Authored: Mon Oct 31 15:14:44 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:02 2017 -0800

----------------------------------------------------------------------
 .../src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/3bc96f8c/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
index dcd105b..6d34e47 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
@@ -34,9 +34,9 @@ public class TestBeeline {
 
 	public static final Log LOG = LogFactory.getLog(TestBeeline.class.getName());
 
-	  private static final String URL = "odpiHiveTestBeelineUrl";
-	  private static final String USER = "odpiHiveTestBeelineUser";
-	  private static final String PASSWD = "odpiHiveTestBeelinePassword";
+	  private static final String URL = "odpiHiveTestJdbcUrl";
+	  private static final String USER = "odpiHiveTestJdbcUser";
+	  private static final String PASSWD = "odpiHiveTestJdbcPassword";
 	  
 	  private static Map<String, String> results;
 	  


[07/50] [abbrv] bigtop git commit: ODPI-184, tests for thrift.

Posted by rv...@apache.org.
ODPI-184, tests for thrift.


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

Branch: refs/heads/BIGTOP-2666
Commit: b1911695f2efeed36c3b4a27863217db8e0453b2
Parents: 56970d1
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Oct 19 16:30:10 2016 -0400
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:45:56 2017 -0800

----------------------------------------------------------------------
 bigtop-tests/spec-tests/runtime/build.gradle    |  12 +
 .../odpi/specs/runtime/hive/JdbcConnector.java  |   8 +
 .../org/odpi/specs/runtime/hive/TestThrift.java | 266 +++++++++++++++++++
 3 files changed, 286 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/b1911695/bigtop-tests/spec-tests/runtime/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/build.gradle b/bigtop-tests/spec-tests/runtime/build.gradle
index 1285a19..4df0dae 100644
--- a/bigtop-tests/spec-tests/runtime/build.gradle
+++ b/bigtop-tests/spec-tests/runtime/build.gradle
@@ -16,11 +16,23 @@
  * limitations under the License.
  */
 def junitVersion = '4.11'
+
+repositories {
+  maven {
+    url "http://conjars.org/repo/"
+  }
+}
 dependencies {
   compile group: 'junit', name: 'junit', version: junitVersion, transitive: 'true'
   compile group: 'commons-logging', name: 'commons-logging', version: '1.1.3'
   compile group: 'org.apache.hive', name: 'hive-jdbc', version: '1.2.1'
+  compile group: 'org.apache.hive', name: 'hive-metastore', version: '1.2.1'
+  compile group: 'org.apache.hive', name: 'hive-common', version: '1.2.1'
+  compile group: 'org.apache.thrift', name: 'libfb303', version: '0.9.3'
+  compile group: 'org.apache.thrift', name: 'libthrift', version: '0.9.3'
   testCompile group: 'org.apache.hadoop', name: 'hadoop-common', version: '2.7.2'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-mapreduce-client-core', version: '2.7.2'
+  testCompile group: 'org.apache.hive', name: 'hive-exec', version: '1.2.1'
   if (System.env.HADOOP_CONF_DIR) testRuntime files(System.env.HADOOP_CONF_DIR)
 }
 

http://git-wip-us.apache.org/repos/asf/bigtop/blob/b1911695/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
index 4f15ab4..f5cc379 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
@@ -34,6 +34,8 @@ public class JdbcConnector {
   protected static final String USER = "odpi.test.hive.jdbc.user";
   protected static final String PASSWD = "odpi.test.hive.jdbc.password";
   protected static final String LOCATION = "odpi.test.hive.location";
+  protected static final String METASTORE_URL = "odpi.test.hive.metastore.url";
+  protected static final String TEST_THRIFT = "odpi.test.hive.thrift.test";
 
   protected static Connection conn;
 
@@ -65,4 +67,10 @@ public class JdbcConnector {
     return val;
   }
 
+  protected static boolean testActive(String property, String description) {
+    String val = System.getProperty(property, "true");
+    LOG.debug(description + " is " + val);
+    return Boolean.valueOf(val);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/b1911695/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
new file mode 100644
index 0000000..1aede96
--- /dev/null
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
@@ -0,0 +1,266 @@
+/**
+ * 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.odpi.specs.runtime.hive;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DropPartitionsRequest;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.RequestPartsSpec;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
+import org.apache.thrift.TException;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Random;
+
+public class TestThrift {
+
+  private static ThriftHiveMetastore.Iface client = null;
+  private static HiveConf conf;
+
+  private Random rand;
+
+  @BeforeClass
+  public static void connect() throws MetaException {
+    if (JdbcConnector.testActive(JdbcConnector.TEST_THRIFT, "Test Thrift ")) {
+      String url = JdbcConnector.getProperty(JdbcConnector.METASTORE_URL, "Thrift metastore URL");
+      conf = new HiveConf();
+      conf.setVar(HiveConf.ConfVars.METASTOREURIS, url);
+      client = new HiveMetaStore.HMSHandler("ODPi test", conf, true);
+    }
+  }
+
+  @Before
+  public void checkIfActive() {
+    Assume.assumeTrue(JdbcConnector.testActive(JdbcConnector.TEST_THRIFT, "Test Thrift "));
+    rand = new Random();
+  }
+
+  @Test
+  public void db() throws TException {
+    final String dbName = "odpi_thrift_db_" + rand.nextInt(Integer.MAX_VALUE);
+
+    String location = JdbcConnector.getProperty(JdbcConnector.LOCATION, " HDFS location we can " +
+        "write to");
+    Database db = new Database(dbName, "a db", location, new HashMap<String, String>());
+    client.create_database(db);
+    db = client.get_database(dbName);
+    Assert.assertNotNull(db);
+    db = new Database(db);
+    db.getParameters().put("a", "b");
+    client.alter_database(dbName, db);
+    List<String> alldbs = client.get_databases("odpi_*");
+    Assert.assertNotNull(alldbs);
+    Assert.assertTrue(alldbs.size() > 0);
+    alldbs = client.get_all_databases();
+    Assert.assertNotNull(alldbs);
+    Assert.assertTrue(alldbs.size() > 0);
+    client.drop_database(dbName, true, true);
+  }
+
+  // Not testing types calls, as they aren't used AFAIK
+
+  @Test
+  public void nonPartitionedTable() throws TException {
+    final String tableName = "odpi_thrift_table_" + rand.nextInt(Integer.MAX_VALUE);
+    String location = JdbcConnector.getProperty(JdbcConnector.LOCATION, " HDFS location we can " +
+        "write to");
+
+    // I don't test every operation related to tables, but only those that are frequently used.
+    SerDeInfo serde = new SerDeInfo("default_serde",
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), new HashMap<String, String>());
+    FieldSchema fs = new FieldSchema("a", "int", "no comment");
+    StorageDescriptor sd = new StorageDescriptor(Collections.singletonList(fs), location,
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT),
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT), false, 0, serde, null, null,
+        new HashMap<String, String>());
+    Table table = new Table(tableName, "default", "me", 0, 0, 0, sd, null,
+        new HashMap<String, String>(), null, null, TableType.MANAGED_TABLE.toString());
+    EnvironmentContext envContext = new EnvironmentContext(new HashMap<String, String>());
+    client.create_table_with_environment_context(table, envContext);
+
+    table = client.get_table("default", tableName);
+    Assert.assertNotNull(table);
+
+    List<Table> tables =
+        client.get_table_objects_by_name("default", Collections.singletonList(tableName));
+    Assert.assertNotNull(tables);
+    Assert.assertEquals(1, tables.size());
+
+    List<String> tableNames = client.get_tables("default", "odpi_*");
+    Assert.assertNotNull(tableNames);
+    Assert.assertTrue(tableNames.size() >= 1);
+
+    tableNames = client.get_all_tables("default");
+    Assert.assertNotNull(tableNames);
+    Assert.assertTrue(tableNames.size() >= 1);
+
+    List<FieldSchema> cols = client.get_fields("default", tableName);
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(1, cols.size());
+
+    cols = client.get_schema_with_environment_context("default", tableName, envContext);
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(1, cols.size());
+
+    table = new Table(table);
+    table.getParameters().put("a", "b");
+    client.alter_table_with_cascade("default", tableName, table, false);
+
+    table.getParameters().put("c", "d");
+    client.alter_table_with_environment_context("default", tableName, table, envContext);
+
+    client.drop_table_with_environment_context("default", tableName, true, envContext);
+  }
+
+  @Test
+  public void partitionedTable() throws TException {
+    final String tableName = "odpi_thrift_partitioned_table_" + rand.nextInt(Integer.MAX_VALUE);
+    String location = JdbcConnector.getProperty(JdbcConnector.LOCATION, " HDFS location we can " +
+        "write to");
+
+    // I don't test every operation related to tables, but only those that are frequently used.
+    SerDeInfo serde = new SerDeInfo("default_serde",
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), new HashMap<String, String>());
+    FieldSchema fs = new FieldSchema("a", "int", "no comment");
+    StorageDescriptor sd = new StorageDescriptor(Collections.singletonList(fs), location,
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT),
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT), false, 0, serde, null, null,
+        new HashMap<String, String>());
+    FieldSchema pk = new FieldSchema("pk", "string", "");
+    Table table = new Table(tableName, "default", "me", 0, 0, 0, sd, Collections.singletonList(pk),
+        new HashMap<String, String>(), null, null, TableType.MANAGED_TABLE.toString());
+    EnvironmentContext envContext = new EnvironmentContext(new HashMap<String, String>());
+    client.create_table_with_environment_context(table, envContext);
+
+    sd = new StorageDescriptor(Collections.singletonList(fs), location + "/x",
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE),
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), false, 0, serde, null, null,
+        new HashMap<String, String>());
+    Partition partition = new Partition(Collections.singletonList("x"), "default", tableName, 0,
+        0, sd, new HashMap<String, String>());
+    client.add_partition_with_environment_context(partition, envContext);
+
+    sd = new StorageDescriptor(Collections.singletonList(fs), location + "/y",
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE),
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), false, 0, serde, null, null,
+        new HashMap<String, String>());
+    partition = new Partition(Collections.singletonList("y"), "default", tableName, 0,
+        0, sd, new HashMap<String, String>());
+    client.add_partitions(Collections.singletonList(partition));
+
+    sd = new StorageDescriptor(Collections.singletonList(fs), location + "/z",
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE),
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), false, 0, serde, null, null,
+        new HashMap<String, String>());
+    partition = new Partition(Collections.singletonList("z"), "default", tableName, 0,
+        0, sd, new HashMap<String, String>());
+    AddPartitionsRequest rqst = new AddPartitionsRequest("default", tableName,
+        Collections.singletonList(partition), true);
+    client.add_partitions_req(rqst);
+
+    List<Partition> parts = client.get_partitions("default", tableName, (short)-1);
+    Assert.assertNotNull(parts);
+    Assert.assertEquals(3, parts.size());
+
+    parts = client.get_partitions_with_auth("default", tableName, (short)-1, "me",
+        Collections.<String>emptyList());
+    Assert.assertNotNull(parts);
+    Assert.assertEquals(3, parts.size());
+
+    parts = client.get_partitions_ps("default", tableName, Collections.singletonList("x"),
+        (short)-1);
+    Assert.assertNotNull(parts);
+    Assert.assertEquals(1, parts.size());
+
+    parts = client.get_partitions_by_filter("default", tableName, "pk = \"x\"", (short)-1);
+    Assert.assertNotNull(parts);
+    Assert.assertEquals(1, parts.size());
+
+    parts = client.get_partitions_by_names("default", tableName, Collections.singletonList("pk=x"));
+    Assert.assertNotNull(parts);
+    Assert.assertEquals(1, parts.size());
+
+    partition = client.get_partition("default", tableName, Collections.singletonList("x"));
+    Assert.assertNotNull(partition);
+
+    partition = client.get_partition_by_name("default", tableName, "pk=x");
+    Assert.assertNotNull(partition);
+
+    partition = client.get_partition_with_auth("default", tableName, Collections.singletonList("x"),
+        "me", Collections.<String>emptyList());
+    Assert.assertNotNull(partition);
+
+    List<String> partitionNames = client.get_partition_names("default", tableName, (short)-1);
+    Assert.assertNotNull(partitionNames);
+    Assert.assertEquals(3, partitionNames.size());
+
+    partition = new Partition(partition);
+    partition.getParameters().put("a", "b");
+    client.alter_partition("default", tableName, partition);
+
+    for (Partition p : parts) p.getParameters().put("c", "d");
+    client.alter_partitions("default", tableName, parts);
+
+    // Not testing get_partitions_by_expr because I don't want to hard code some byte sequence
+    // from the parser.  The odds that anyone other than Hive parser would call this method seem
+    // low, since you'd have to exactly match the serliazation of the Hive parser.
+
+    // Not testing partition marking events, not used by anyone but Hive replication AFAIK
+
+    client.drop_partition_by_name_with_environment_context("default", tableName, "pk=x", true,
+        envContext);
+    client.drop_partition_with_environment_context("default", tableName,
+        Collections.singletonList("y"), true, envContext);
+    DropPartitionsRequest dropRequest = new DropPartitionsRequest("default", tableName,
+        RequestPartsSpec.names(Collections.singletonList("pk=z")));
+    client.drop_partitions_req(dropRequest);
+  }
+
+  // Not testing index calls, as no one uses indices
+
+
+  // Not sure if anyone uses stats calls or not.  Other query engines might.  Ignoring for now.
+
+  // Not sure if anyone else uses functions, though I'm guessing not as without Hive classes they
+  // won't be runable.
+
+  // Not testing authorization calls as AFAIK no one else uses Hive security
+
+  // Not testing transaction/locking calls, as those are used only by Hive.
+
+  // Not testing notification logging calls, as those are used only by Hive replication.
+
+}


[14/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.oracle.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.oracle.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.oracle.sql
new file mode 100755
index 0000000..6bd8df9
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.oracle.sql
@@ -0,0 +1,835 @@
+-- Table SEQUENCE_TABLE is an internal table required by DataNucleus.
+-- NOTE: Some versions of SchemaTool do not automatically generate this table.
+-- See http://www.datanucleus.org/servlet/jira/browse/NUCRDBMS-416
+CREATE TABLE SEQUENCE_TABLE
+(
+   SEQUENCE_NAME VARCHAR2(255) NOT NULL,
+   NEXT_VAL NUMBER NOT NULL
+);
+
+ALTER TABLE SEQUENCE_TABLE ADD CONSTRAINT PART_TABLE_PK PRIMARY KEY (SEQUENCE_NAME);
+
+-- Table NUCLEUS_TABLES is an internal table required by DataNucleus.
+-- This table is required if datanucleus.autoStartMechanism=SchemaTable
+-- NOTE: Some versions of SchemaTool do not automatically generate this table.
+-- See http://www.datanucleus.org/servlet/jira/browse/NUCRDBMS-416
+CREATE TABLE NUCLEUS_TABLES
+(
+   CLASS_NAME VARCHAR2(128) NOT NULL,
+   TABLE_NAME VARCHAR2(128) NOT NULL,
+   TYPE VARCHAR2(4) NOT NULL,
+   OWNER VARCHAR2(2) NOT NULL,
+   VERSION VARCHAR2(20) NOT NULL,
+   INTERFACE_NAME VARCHAR2(255) NULL
+);
+
+ALTER TABLE NUCLEUS_TABLES ADD CONSTRAINT NUCLEUS_TABLES_PK PRIMARY KEY (CLASS_NAME);
+
+-- Table PART_COL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege]
+CREATE TABLE PART_COL_PRIVS
+(
+    PART_COLUMN_GRANT_ID NUMBER NOT NULL,
+    "COLUMN_NAME" VARCHAR2(128) NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PART_ID NUMBER NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    PART_COL_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE PART_COL_PRIVS ADD CONSTRAINT PART_COL_PRIVS_PK PRIMARY KEY (PART_COLUMN_GRANT_ID);
+
+-- Table CDS.
+CREATE TABLE CDS
+(
+    CD_ID NUMBER NOT NULL
+);
+
+ALTER TABLE CDS ADD CONSTRAINT CDS_PK PRIMARY KEY (CD_ID);
+
+-- Table COLUMNS_V2 for join relationship
+CREATE TABLE COLUMNS_V2
+(
+    CD_ID NUMBER NOT NULL,
+    "COMMENT" VARCHAR2(256) NULL,
+    "COLUMN_NAME" VARCHAR2(128) NOT NULL,
+    TYPE_NAME VARCHAR2(4000) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE COLUMNS_V2 ADD CONSTRAINT COLUMNS_V2_PK PRIMARY KEY (CD_ID,"COLUMN_NAME");
+
+-- Table PARTITION_KEY_VALS for join relationship
+CREATE TABLE PARTITION_KEY_VALS
+(
+    PART_ID NUMBER NOT NULL,
+    PART_KEY_VAL VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE PARTITION_KEY_VALS ADD CONSTRAINT PARTITION_KEY_VALS_PK PRIMARY KEY (PART_ID,INTEGER_IDX);
+
+-- Table DBS for classes [org.apache.hadoop.hive.metastore.model.MDatabase]
+CREATE TABLE DBS
+(
+    DB_ID NUMBER NOT NULL,
+    "DESC" VARCHAR2(4000) NULL,
+    DB_LOCATION_URI VARCHAR2(4000) NOT NULL,
+    "NAME" VARCHAR2(128) NULL,
+    OWNER_NAME VARCHAR2(128) NULL,
+    OWNER_TYPE VARCHAR2(10) NULL
+);
+
+ALTER TABLE DBS ADD CONSTRAINT DBS_PK PRIMARY KEY (DB_ID);
+
+-- Table PARTITION_PARAMS for join relationship
+CREATE TABLE PARTITION_PARAMS
+(
+    PART_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE PARTITION_PARAMS ADD CONSTRAINT PARTITION_PARAMS_PK PRIMARY KEY (PART_ID,PARAM_KEY);
+
+-- Table SERDES for classes [org.apache.hadoop.hive.metastore.model.MSerDeInfo]
+CREATE TABLE SERDES
+(
+    SERDE_ID NUMBER NOT NULL,
+    "NAME" VARCHAR2(128) NULL,
+    SLIB VARCHAR2(4000) NULL
+);
+
+ALTER TABLE SERDES ADD CONSTRAINT SERDES_PK PRIMARY KEY (SERDE_ID);
+
+-- Table TYPES for classes [org.apache.hadoop.hive.metastore.model.MType]
+CREATE TABLE TYPES
+(
+    TYPES_ID NUMBER NOT NULL,
+    TYPE_NAME VARCHAR2(128) NULL,
+    TYPE1 VARCHAR2(767) NULL,
+    TYPE2 VARCHAR2(767) NULL
+);
+
+ALTER TABLE TYPES ADD CONSTRAINT TYPES_PK PRIMARY KEY (TYPES_ID);
+
+-- Table PARTITION_KEYS for join relationship
+CREATE TABLE PARTITION_KEYS
+(
+    TBL_ID NUMBER NOT NULL,
+    PKEY_COMMENT VARCHAR2(4000) NULL,
+    PKEY_NAME VARCHAR2(128) NOT NULL,
+    PKEY_TYPE VARCHAR2(767) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE PARTITION_KEYS ADD CONSTRAINT PARTITION_KEY_PK PRIMARY KEY (TBL_ID,PKEY_NAME);
+
+-- Table ROLES for classes [org.apache.hadoop.hive.metastore.model.MRole]
+CREATE TABLE ROLES
+(
+    ROLE_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    OWNER_NAME VARCHAR2(128) NULL,
+    ROLE_NAME VARCHAR2(128) NULL
+);
+
+ALTER TABLE ROLES ADD CONSTRAINT ROLES_PK PRIMARY KEY (ROLE_ID);
+
+-- Table PARTITIONS for classes [org.apache.hadoop.hive.metastore.model.MPartition]
+CREATE TABLE PARTITIONS
+(
+    PART_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    LAST_ACCESS_TIME NUMBER (10) NOT NULL,
+    PART_NAME VARCHAR2(767) NULL,
+    SD_ID NUMBER NULL,
+    TBL_ID NUMBER NULL
+);
+
+ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_PK PRIMARY KEY (PART_ID);
+
+-- Table INDEX_PARAMS for join relationship
+CREATE TABLE INDEX_PARAMS
+(
+    INDEX_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE INDEX_PARAMS ADD CONSTRAINT INDEX_PARAMS_PK PRIMARY KEY (INDEX_ID,PARAM_KEY);
+
+-- Table TBL_COL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege]
+CREATE TABLE TBL_COL_PRIVS
+(
+    TBL_COLUMN_GRANT_ID NUMBER NOT NULL,
+    "COLUMN_NAME" VARCHAR2(128) NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    TBL_COL_PRIV VARCHAR2(128) NULL,
+    TBL_ID NUMBER NULL
+);
+
+ALTER TABLE TBL_COL_PRIVS ADD CONSTRAINT TBL_COL_PRIVS_PK PRIMARY KEY (TBL_COLUMN_GRANT_ID);
+
+-- Table IDXS for classes [org.apache.hadoop.hive.metastore.model.MIndex]
+CREATE TABLE IDXS
+(
+    INDEX_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    DEFERRED_REBUILD NUMBER(1) NOT NULL CHECK (DEFERRED_REBUILD IN (1,0)),
+    INDEX_HANDLER_CLASS VARCHAR2(4000) NULL,
+    INDEX_NAME VARCHAR2(128) NULL,
+    INDEX_TBL_ID NUMBER NULL,
+    LAST_ACCESS_TIME NUMBER (10) NOT NULL,
+    ORIG_TBL_ID NUMBER NULL,
+    SD_ID NUMBER NULL
+);
+
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_PK PRIMARY KEY (INDEX_ID);
+
+-- Table BUCKETING_COLS for join relationship
+CREATE TABLE BUCKETING_COLS
+(
+    SD_ID NUMBER NOT NULL,
+    BUCKET_COL_NAME VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE BUCKETING_COLS ADD CONSTRAINT BUCKETING_COLS_PK PRIMARY KEY (SD_ID,INTEGER_IDX);
+
+-- Table TYPE_FIELDS for join relationship
+CREATE TABLE TYPE_FIELDS
+(
+    TYPE_NAME NUMBER NOT NULL,
+    "COMMENT" VARCHAR2(256) NULL,
+    FIELD_NAME VARCHAR2(128) NOT NULL,
+    FIELD_TYPE VARCHAR2(767) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE TYPE_FIELDS ADD CONSTRAINT TYPE_FIELDS_PK PRIMARY KEY (TYPE_NAME,FIELD_NAME);
+
+-- Table SD_PARAMS for join relationship
+CREATE TABLE SD_PARAMS
+(
+    SD_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE SD_PARAMS ADD CONSTRAINT SD_PARAMS_PK PRIMARY KEY (SD_ID,PARAM_KEY);
+
+-- Table GLOBAL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MGlobalPrivilege]
+CREATE TABLE GLOBAL_PRIVS
+(
+    USER_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    USER_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE GLOBAL_PRIVS ADD CONSTRAINT GLOBAL_PRIVS_PK PRIMARY KEY (USER_GRANT_ID);
+
+-- Table SDS for classes [org.apache.hadoop.hive.metastore.model.MStorageDescriptor]
+CREATE TABLE SDS
+(
+    SD_ID NUMBER NOT NULL,
+    CD_ID NUMBER NULL,
+    INPUT_FORMAT VARCHAR2(4000) NULL,
+    IS_COMPRESSED NUMBER(1) NOT NULL CHECK (IS_COMPRESSED IN (1,0)),
+    LOCATION VARCHAR2(4000) NULL,
+    NUM_BUCKETS NUMBER (10) NOT NULL,
+    OUTPUT_FORMAT VARCHAR2(4000) NULL,
+    SERDE_ID NUMBER NULL,
+    IS_STOREDASSUBDIRECTORIES NUMBER(1) NOT NULL CHECK (IS_STOREDASSUBDIRECTORIES IN (1,0))
+);
+
+ALTER TABLE SDS ADD CONSTRAINT SDS_PK PRIMARY KEY (SD_ID);
+
+-- Table TABLE_PARAMS for join relationship
+CREATE TABLE TABLE_PARAMS
+(
+    TBL_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE TABLE_PARAMS ADD CONSTRAINT TABLE_PARAMS_PK PRIMARY KEY (TBL_ID,PARAM_KEY);
+
+-- Table SORT_COLS for join relationship
+CREATE TABLE SORT_COLS
+(
+    SD_ID NUMBER NOT NULL,
+    "COLUMN_NAME" VARCHAR2(128) NULL,
+    "ORDER" NUMBER (10) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SORT_COLS ADD CONSTRAINT SORT_COLS_PK PRIMARY KEY (SD_ID,INTEGER_IDX);
+
+-- Table TBL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MTablePrivilege]
+CREATE TABLE TBL_PRIVS
+(
+    TBL_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    TBL_PRIV VARCHAR2(128) NULL,
+    TBL_ID NUMBER NULL
+);
+
+ALTER TABLE TBL_PRIVS ADD CONSTRAINT TBL_PRIVS_PK PRIMARY KEY (TBL_GRANT_ID);
+
+-- Table DATABASE_PARAMS for join relationship
+CREATE TABLE DATABASE_PARAMS
+(
+    DB_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(180) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE DATABASE_PARAMS ADD CONSTRAINT DATABASE_PARAMS_PK PRIMARY KEY (DB_ID,PARAM_KEY);
+
+-- Table ROLE_MAP for classes [org.apache.hadoop.hive.metastore.model.MRoleMap]
+CREATE TABLE ROLE_MAP
+(
+    ROLE_GRANT_ID NUMBER NOT NULL,
+    ADD_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    ROLE_ID NUMBER NULL
+);
+
+ALTER TABLE ROLE_MAP ADD CONSTRAINT ROLE_MAP_PK PRIMARY KEY (ROLE_GRANT_ID);
+
+-- Table SERDE_PARAMS for join relationship
+CREATE TABLE SERDE_PARAMS
+(
+    SERDE_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE SERDE_PARAMS ADD CONSTRAINT SERDE_PARAMS_PK PRIMARY KEY (SERDE_ID,PARAM_KEY);
+
+-- Table PART_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MPartitionPrivilege]
+CREATE TABLE PART_PRIVS
+(
+    PART_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PART_ID NUMBER NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    PART_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE PART_PRIVS ADD CONSTRAINT PART_PRIVS_PK PRIMARY KEY (PART_GRANT_ID);
+
+-- Table DB_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MDBPrivilege]
+CREATE TABLE DB_PRIVS
+(
+    DB_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    DB_ID NUMBER NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    DB_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE DB_PRIVS ADD CONSTRAINT DB_PRIVS_PK PRIMARY KEY (DB_GRANT_ID);
+
+-- Table TBLS for classes [org.apache.hadoop.hive.metastore.model.MTable]
+CREATE TABLE TBLS
+(
+    TBL_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    DB_ID NUMBER NULL,
+    LAST_ACCESS_TIME NUMBER (10) NOT NULL,
+    OWNER VARCHAR2(767) NULL,
+    RETENTION NUMBER (10) NOT NULL,
+    SD_ID NUMBER NULL,
+    TBL_NAME VARCHAR2(128) NULL,
+    TBL_TYPE VARCHAR2(128) NULL,
+    VIEW_EXPANDED_TEXT CLOB NULL,
+    VIEW_ORIGINAL_TEXT CLOB NULL
+);
+
+ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);
+
+-- Table PARTITION_EVENTS for classes [org.apache.hadoop.hive.metastore.model.MPartitionEvent]
+CREATE TABLE PARTITION_EVENTS
+(
+    PART_NAME_ID NUMBER NOT NULL,
+    DB_NAME VARCHAR2(128) NULL,
+    EVENT_TIME NUMBER NOT NULL,
+    EVENT_TYPE NUMBER (10) NOT NULL,
+    PARTITION_NAME VARCHAR2(767) NULL,
+    TBL_NAME VARCHAR2(128) NULL
+);
+
+ALTER TABLE PARTITION_EVENTS ADD CONSTRAINT PARTITION_EVENTS_PK PRIMARY KEY (PART_NAME_ID);
+
+-- Table SKEWED_STRING_LIST for classes [org.apache.hadoop.hive.metastore.model.MStringList]
+CREATE TABLE SKEWED_STRING_LIST
+(
+    STRING_LIST_ID NUMBER NOT NULL
+);
+
+ALTER TABLE SKEWED_STRING_LIST ADD CONSTRAINT SKEWED_STRING_LIST_PK PRIMARY KEY (STRING_LIST_ID);
+
+CREATE TABLE SKEWED_STRING_LIST_VALUES
+(
+    STRING_LIST_ID NUMBER NOT NULL,
+    "STRING_LIST_VALUE" VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SKEWED_STRING_LIST_VALUES ADD CONSTRAINT SKEWED_STRING_LIST_VALUES_PK PRIMARY KEY (STRING_LIST_ID,INTEGER_IDX);
+
+ALTER TABLE SKEWED_STRING_LIST_VALUES ADD CONSTRAINT SKEWED_STRING_LIST_VALUES_FK1 FOREIGN KEY (STRING_LIST_ID) REFERENCES SKEWED_STRING_LIST (STRING_LIST_ID) INITIALLY DEFERRED ;
+
+CREATE TABLE SKEWED_COL_NAMES
+(
+    SD_ID NUMBER NOT NULL,
+    "SKEWED_COL_NAME" VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SKEWED_COL_NAMES ADD CONSTRAINT SKEWED_COL_NAMES_PK PRIMARY KEY (SD_ID,INTEGER_IDX);
+
+ALTER TABLE SKEWED_COL_NAMES ADD CONSTRAINT SKEWED_COL_NAMES_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE TABLE SKEWED_COL_VALUE_LOC_MAP
+(
+    SD_ID NUMBER NOT NULL,
+    STRING_LIST_ID_KID NUMBER NOT NULL,
+    "LOCATION" VARCHAR2(4000) NULL
+);
+
+CREATE TABLE MASTER_KEYS
+(
+    KEY_ID NUMBER (10) NOT NULL,
+    MASTER_KEY VARCHAR2(767) NULL
+);
+
+CREATE TABLE DELEGATION_TOKENS
+(
+    TOKEN_IDENT VARCHAR2(767) NOT NULL,
+    TOKEN VARCHAR2(767) NULL
+);
+
+ALTER TABLE SKEWED_COL_VALUE_LOC_MAP ADD CONSTRAINT SKEWED_COL_VALUE_LOC_MAP_PK PRIMARY KEY (SD_ID,STRING_LIST_ID_KID);
+
+ALTER TABLE SKEWED_COL_VALUE_LOC_MAP ADD CONSTRAINT SKEWED_COL_VALUE_LOC_MAP_FK1 FOREIGN KEY (STRING_LIST_ID_KID) REFERENCES SKEWED_STRING_LIST (STRING_LIST_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE SKEWED_COL_VALUE_LOC_MAP ADD CONSTRAINT SKEWED_COL_VALUE_LOC_MAP_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE TABLE SKEWED_VALUES
+(
+    SD_ID_OID NUMBER NOT NULL,
+    STRING_LIST_ID_EID NUMBER NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_PK PRIMARY KEY (SD_ID_OID,INTEGER_IDX);
+
+ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_FK1 FOREIGN KEY (STRING_LIST_ID_EID) REFERENCES SKEWED_STRING_LIST (STRING_LIST_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_FK2 FOREIGN KEY (SD_ID_OID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+-- column statistics
+
+CREATE TABLE TAB_COL_STATS (
+ CS_ID NUMBER NOT NULL,
+ DB_NAME VARCHAR2(128) NOT NULL,
+ TABLE_NAME VARCHAR2(128) NOT NULL, 
+ COLUMN_NAME VARCHAR2(128) NOT NULL,
+ COLUMN_TYPE VARCHAR2(128) NOT NULL,
+ TBL_ID NUMBER NOT NULL,
+ LONG_LOW_VALUE NUMBER,
+ LONG_HIGH_VALUE NUMBER,
+ DOUBLE_LOW_VALUE NUMBER,
+ DOUBLE_HIGH_VALUE NUMBER,
+ BIG_DECIMAL_LOW_VALUE VARCHAR2(4000),
+ BIG_DECIMAL_HIGH_VALUE VARCHAR2(4000),
+ NUM_NULLS NUMBER NOT NULL,
+ NUM_DISTINCTS NUMBER,
+ AVG_COL_LEN NUMBER,
+ MAX_COL_LEN NUMBER,
+ NUM_TRUES NUMBER,
+ NUM_FALSES NUMBER,
+ LAST_ANALYZED NUMBER NOT NULL
+);
+
+CREATE TABLE VERSION (
+  VER_ID NUMBER NOT NULL,
+  SCHEMA_VERSION VARCHAR(127) NOT NULL,
+  VERSION_COMMENT VARCHAR(255)
+);
+ALTER TABLE VERSION ADD CONSTRAINT VERSION_PK PRIMARY KEY (VER_ID);
+
+ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_PKEY PRIMARY KEY (CS_ID);
+
+ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_FK FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TAB_COL_STATS_N49 ON TAB_COL_STATS(TBL_ID);
+
+CREATE TABLE PART_COL_STATS (
+ CS_ID NUMBER NOT NULL,
+ DB_NAME VARCHAR2(128) NOT NULL,
+ TABLE_NAME VARCHAR2(128) NOT NULL,
+ PARTITION_NAME VARCHAR2(767) NOT NULL,
+ COLUMN_NAME VARCHAR2(128) NOT NULL,
+ COLUMN_TYPE VARCHAR2(128) NOT NULL,
+ PART_ID NUMBER NOT NULL,
+ LONG_LOW_VALUE NUMBER,
+ LONG_HIGH_VALUE NUMBER,
+ DOUBLE_LOW_VALUE NUMBER,
+ DOUBLE_HIGH_VALUE NUMBER,
+ BIG_DECIMAL_LOW_VALUE VARCHAR2(4000),
+ BIG_DECIMAL_HIGH_VALUE VARCHAR2(4000),
+ NUM_NULLS NUMBER NOT NULL,
+ NUM_DISTINCTS NUMBER,
+ AVG_COL_LEN NUMBER,
+ MAX_COL_LEN NUMBER,
+ NUM_TRUES NUMBER,
+ NUM_FALSES NUMBER,
+ LAST_ANALYZED NUMBER NOT NULL
+);
+
+ALTER TABLE PART_COL_STATS ADD CONSTRAINT PART_COL_STATS_PKEY PRIMARY KEY (CS_ID);
+
+ALTER TABLE PART_COL_STATS ADD CONSTRAINT PART_COL_STATS_FK FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED;
+
+CREATE INDEX PART_COL_STATS_N49 ON PART_COL_STATS (PART_ID);
+
+CREATE TABLE FUNCS (
+  FUNC_ID NUMBER NOT NULL,
+  CLASS_NAME VARCHAR2(4000),
+  CREATE_TIME NUMBER(10) NOT NULL,
+  DB_ID NUMBER,
+  FUNC_NAME VARCHAR2(128),
+  FUNC_TYPE NUMBER(10) NOT NULL,
+  OWNER_NAME VARCHAR2(128),
+  OWNER_TYPE VARCHAR2(10)
+);
+
+ALTER TABLE FUNCS ADD CONSTRAINT FUNCS_PK PRIMARY KEY (FUNC_ID);
+
+CREATE TABLE FUNC_RU (
+  FUNC_ID NUMBER NOT NULL,
+  RESOURCE_TYPE NUMBER(10) NOT NULL,
+  RESOURCE_URI VARCHAR2(4000),
+  INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE FUNC_RU ADD CONSTRAINT FUNC_RU_PK PRIMARY KEY (FUNC_ID, INTEGER_IDX);
+
+
+-- Constraints for table PART_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege]
+ALTER TABLE PART_COL_PRIVS ADD CONSTRAINT PART_COL_PRIVS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PART_COL_PRIVS_N49 ON PART_COL_PRIVS (PART_ID);
+
+CREATE INDEX PARTITIONCOLUMNPRIVILEGEINDEX ON PART_COL_PRIVS (PART_ID,"COLUMN_NAME",PRINCIPAL_NAME,PRINCIPAL_TYPE,PART_COL_PRIV,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table COLUMNS_V2
+ALTER TABLE COLUMNS_V2 ADD CONSTRAINT COLUMNS_V2_FK1 FOREIGN KEY (CD_ID) REFERENCES CDS (CD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX COLUMNS_V2_N49 ON COLUMNS_V2 (CD_ID);
+
+
+-- Constraints for table PARTITION_KEY_VALS
+ALTER TABLE PARTITION_KEY_VALS ADD CONSTRAINT PARTITION_KEY_VALS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITION_KEY_VALS_N49 ON PARTITION_KEY_VALS (PART_ID);
+
+
+-- Constraints for table DBS for class(es) [org.apache.hadoop.hive.metastore.model.MDatabase]
+CREATE UNIQUE INDEX UNIQUE_DATABASE ON DBS ("NAME");
+
+
+-- Constraints for table PARTITION_PARAMS
+ALTER TABLE PARTITION_PARAMS ADD CONSTRAINT PARTITION_PARAMS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITION_PARAMS_N49 ON PARTITION_PARAMS (PART_ID);
+
+
+-- Constraints for table SERDES for class(es) [org.apache.hadoop.hive.metastore.model.MSerDeInfo]
+
+-- Constraints for table TYPES for class(es) [org.apache.hadoop.hive.metastore.model.MType]
+CREATE UNIQUE INDEX UNIQUE_TYPE ON TYPES (TYPE_NAME);
+
+
+-- Constraints for table PARTITION_KEYS
+ALTER TABLE PARTITION_KEYS ADD CONSTRAINT PARTITION_KEYS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITION_KEYS_N49 ON PARTITION_KEYS (TBL_ID);
+
+
+-- Constraints for table ROLES for class(es) [org.apache.hadoop.hive.metastore.model.MRole]
+CREATE UNIQUE INDEX ROLEENTITYINDEX ON ROLES (ROLE_NAME);
+
+
+-- Constraints for table PARTITIONS for class(es) [org.apache.hadoop.hive.metastore.model.MPartition]
+ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITIONS_N49 ON PARTITIONS (SD_ID);
+
+CREATE INDEX PARTITIONS_N50 ON PARTITIONS (TBL_ID);
+
+CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (PART_NAME,TBL_ID);
+
+
+-- Constraints for table INDEX_PARAMS
+ALTER TABLE INDEX_PARAMS ADD CONSTRAINT INDEX_PARAMS_FK1 FOREIGN KEY (INDEX_ID) REFERENCES IDXS (INDEX_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX INDEX_PARAMS_N49 ON INDEX_PARAMS (INDEX_ID);
+
+
+-- Constraints for table TBL_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege]
+ALTER TABLE TBL_COL_PRIVS ADD CONSTRAINT TBL_COL_PRIVS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TABLECOLUMNPRIVILEGEINDEX ON TBL_COL_PRIVS (TBL_ID,"COLUMN_NAME",PRINCIPAL_NAME,PRINCIPAL_TYPE,TBL_COL_PRIV,GRANTOR,GRANTOR_TYPE);
+
+CREATE INDEX TBL_COL_PRIVS_N49 ON TBL_COL_PRIVS (TBL_ID);
+
+
+-- Constraints for table IDXS for class(es) [org.apache.hadoop.hive.metastore.model.MIndex]
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK1 FOREIGN KEY (ORIG_TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK3 FOREIGN KEY (INDEX_TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE UNIQUE INDEX UNIQUEINDEX ON IDXS (INDEX_NAME,ORIG_TBL_ID);
+
+CREATE INDEX IDXS_N50 ON IDXS (INDEX_TBL_ID);
+
+CREATE INDEX IDXS_N51 ON IDXS (SD_ID);
+
+CREATE INDEX IDXS_N49 ON IDXS (ORIG_TBL_ID);
+
+
+-- Constraints for table BUCKETING_COLS
+ALTER TABLE BUCKETING_COLS ADD CONSTRAINT BUCKETING_COLS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX BUCKETING_COLS_N49 ON BUCKETING_COLS (SD_ID);
+
+
+-- Constraints for table TYPE_FIELDS
+ALTER TABLE TYPE_FIELDS ADD CONSTRAINT TYPE_FIELDS_FK1 FOREIGN KEY (TYPE_NAME) REFERENCES TYPES (TYPES_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TYPE_FIELDS_N49 ON TYPE_FIELDS (TYPE_NAME);
+
+
+-- Constraints for table SD_PARAMS
+ALTER TABLE SD_PARAMS ADD CONSTRAINT SD_PARAMS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SD_PARAMS_N49 ON SD_PARAMS (SD_ID);
+
+
+-- Constraints for table GLOBAL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MGlobalPrivilege]
+CREATE UNIQUE INDEX GLOBALPRIVILEGEINDEX ON GLOBAL_PRIVS (PRINCIPAL_NAME,PRINCIPAL_TYPE,USER_PRIV,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table SDS for class(es) [org.apache.hadoop.hive.metastore.model.MStorageDescriptor]
+ALTER TABLE SDS ADD CONSTRAINT SDS_FK1 FOREIGN KEY (SERDE_ID) REFERENCES SERDES (SERDE_ID) INITIALLY DEFERRED ;
+ALTER TABLE SDS ADD CONSTRAINT SDS_FK2 FOREIGN KEY (CD_ID) REFERENCES CDS (CD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SDS_N49 ON SDS (SERDE_ID);
+CREATE INDEX SDS_N50 ON SDS (CD_ID);
+
+
+-- Constraints for table TABLE_PARAMS
+ALTER TABLE TABLE_PARAMS ADD CONSTRAINT TABLE_PARAMS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TABLE_PARAMS_N49 ON TABLE_PARAMS (TBL_ID);
+
+
+-- Constraints for table SORT_COLS
+ALTER TABLE SORT_COLS ADD CONSTRAINT SORT_COLS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SORT_COLS_N49 ON SORT_COLS (SD_ID);
+
+
+-- Constraints for table TBL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MTablePrivilege]
+ALTER TABLE TBL_PRIVS ADD CONSTRAINT TBL_PRIVS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TBL_PRIVS_N49 ON TBL_PRIVS (TBL_ID);
+
+CREATE INDEX TABLEPRIVILEGEINDEX ON TBL_PRIVS (TBL_ID,PRINCIPAL_NAME,PRINCIPAL_TYPE,TBL_PRIV,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table DATABASE_PARAMS
+ALTER TABLE DATABASE_PARAMS ADD CONSTRAINT DATABASE_PARAMS_FK1 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX DATABASE_PARAMS_N49 ON DATABASE_PARAMS (DB_ID);
+
+
+-- Constraints for table ROLE_MAP for class(es) [org.apache.hadoop.hive.metastore.model.MRoleMap]
+ALTER TABLE ROLE_MAP ADD CONSTRAINT ROLE_MAP_FK1 FOREIGN KEY (ROLE_ID) REFERENCES ROLES (ROLE_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX ROLE_MAP_N49 ON ROLE_MAP (ROLE_ID);
+
+CREATE UNIQUE INDEX USERROLEMAPINDEX ON ROLE_MAP (PRINCIPAL_NAME,ROLE_ID,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table SERDE_PARAMS
+ALTER TABLE SERDE_PARAMS ADD CONSTRAINT SERDE_PARAMS_FK1 FOREIGN KEY (SERDE_ID) REFERENCES SERDES (SERDE_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SERDE_PARAMS_N49 ON SERDE_PARAMS (SERDE_ID);
+
+
+-- Constraints for table PART_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionPrivilege]
+ALTER TABLE PART_PRIVS ADD CONSTRAINT PART_PRIVS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTPRIVILEGEINDEX ON PART_PRIVS (PART_ID,PRINCIPAL_NAME,PRINCIPAL_TYPE,PART_PRIV,GRANTOR,GRANTOR_TYPE);
+
+CREATE INDEX PART_PRIVS_N49 ON PART_PRIVS (PART_ID);
+
+
+-- Constraints for table DB_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MDBPrivilege]
+ALTER TABLE DB_PRIVS ADD CONSTRAINT DB_PRIVS_FK1 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED ;
+
+CREATE UNIQUE INDEX DBPRIVILEGEINDEX ON DB_PRIVS (DB_ID,PRINCIPAL_NAME,PRINCIPAL_TYPE,DB_PRIV,GRANTOR,GRANTOR_TYPE);
+
+CREATE INDEX DB_PRIVS_N49 ON DB_PRIVS (DB_ID);
+
+
+-- Constraints for table TBLS for class(es) [org.apache.hadoop.hive.metastore.model.MTable]
+ALTER TABLE TBLS ADD CONSTRAINT TBLS_FK2 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE TBLS ADD CONSTRAINT TBLS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TBLS_N49 ON TBLS (DB_ID);
+
+CREATE UNIQUE INDEX UNIQUETABLE ON TBLS (TBL_NAME,DB_ID);
+
+CREATE INDEX TBLS_N50 ON TBLS (SD_ID);
+
+
+-- Constraints for table PARTITION_EVENTS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionEvent]
+CREATE INDEX PARTITIONEVENTINDEX ON PARTITION_EVENTS (PARTITION_NAME);
+
+
+-- Constraints for table FUNCS for class(es) [org.apache.hadoop.hive.metastore.model.MFunctions]
+ALTER TABLE FUNCS ADD CONSTRAINT FUNCS_FK1 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED;
+
+CREATE UNIQUE INDEX UNIQUEFUNCTION ON FUNCS (FUNC_NAME, DB_ID);
+
+CREATE INDEX FUNCS_N49 ON FUNCS (DB_ID);
+
+
+-- Constraints for table FUNC_RU for class(es) [org.apache.hadoop.hive.metastore.model.MFunctions]
+ALTER TABLE FUNC_RU ADD CONSTRAINT FUNC_RU_FK1 FOREIGN KEY (FUNC_ID) REFERENCES FUNCS (FUNC_ID) INITIALLY DEFERRED;
+
+CREATE INDEX FUNC_RU_N49 ON FUNC_RU (FUNC_ID);
+
+
+-- -----------------------------------------------------------------------------------------------------------------------------------------------
+-- Transaction and Lock Tables
+-- These are not part of package jdo, so if you are going to regenerate this file you need to manually add the following section back to the file.
+-- -----------------------------------------------------------------------------------------------------------------------------------------------
+
+CREATE TABLE TXNS (
+  TXN_ID NUMBER(19) PRIMARY KEY,
+  TXN_STATE char(1) NOT NULL,
+  TXN_STARTED NUMBER(19) NOT NULL,
+  TXN_LAST_HEARTBEAT NUMBER(19) NOT NULL,
+  TXN_USER varchar(128) NOT NULL,
+  TXN_HOST varchar(128) NOT NULL
+);
+
+CREATE TABLE TXN_COMPONENTS (
+  TC_TXNID NUMBER(19) REFERENCES TXNS (TXN_ID),
+  TC_DATABASE VARCHAR2(128) NOT NULL,
+  TC_TABLE VARCHAR2(128),
+  TC_PARTITION VARCHAR2(767) NULL
+);
+
+CREATE TABLE COMPLETED_TXN_COMPONENTS (
+  CTC_TXNID NUMBER(19),
+  CTC_DATABASE varchar(128) NOT NULL,
+  CTC_TABLE varchar(128),
+  CTC_PARTITION varchar(767)
+);
+
+CREATE TABLE NEXT_TXN_ID (
+  NTXN_NEXT NUMBER(19) NOT NULL
+);
+INSERT INTO NEXT_TXN_ID VALUES(1);
+
+CREATE TABLE HIVE_LOCKS (
+  HL_LOCK_EXT_ID NUMBER(19) NOT NULL,
+  HL_LOCK_INT_ID NUMBER(19) NOT NULL,
+  HL_TXNID NUMBER(19),
+  HL_DB VARCHAR2(128) NOT NULL,
+  HL_TABLE VARCHAR2(128),
+  HL_PARTITION VARCHAR2(767),
+  HL_LOCK_STATE CHAR(1) NOT NULL,
+  HL_LOCK_TYPE CHAR(1) NOT NULL,
+  HL_LAST_HEARTBEAT NUMBER(19) NOT NULL,
+  HL_ACQUIRED_AT NUMBER(19),
+  HL_USER varchar(128) NOT NULL,
+  HL_HOST varchar(128) NOT NULL,
+  PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID)
+); 
+
+CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID);
+
+CREATE TABLE NEXT_LOCK_ID (
+  NL_NEXT NUMBER(19) NOT NULL
+);
+INSERT INTO NEXT_LOCK_ID VALUES(1);
+
+CREATE TABLE COMPACTION_QUEUE (
+  CQ_ID NUMBER(19) PRIMARY KEY,
+  CQ_DATABASE varchar(128) NOT NULL,
+  CQ_TABLE varchar(128) NOT NULL,
+  CQ_PARTITION varchar(767),
+  CQ_STATE char(1) NOT NULL,
+  CQ_TYPE char(1) NOT NULL,
+  CQ_WORKER_ID varchar(128),
+  CQ_START NUMBER(19),
+  CQ_RUN_AS varchar(128)
+);
+
+CREATE TABLE NEXT_COMPACTION_QUEUE_ID (
+  NCQ_NEXT NUMBER(19) NOT NULL
+);
+INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1);
+
+
+-- -----------------------------------------------------------------
+-- Record schema version. Should be the last step in the init script
+-- -----------------------------------------------------------------
+INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.13.0', 'Hive release version 0.13.0');
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.postgres.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.postgres.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.postgres.sql
new file mode 100755
index 0000000..7b886e1
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.postgres.sql
@@ -0,0 +1,1538 @@
+--
+-- PostgreSQL database dump
+--
+
+SET statement_timeout = 0;
+SET client_encoding = 'UTF8';
+SET standard_conforming_strings = off;
+SET check_function_bodies = false;
+SET client_min_messages = warning;
+SET escape_string_warning = off;
+
+SET search_path = public, pg_catalog;
+
+SET default_tablespace = '';
+
+SET default_with_oids = false;
+
+--
+-- Name: BUCKETING_COLS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "BUCKETING_COLS" (
+    "SD_ID" bigint NOT NULL,
+    "BUCKET_COL_NAME" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: CDS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "CDS" (
+    "CD_ID" bigint NOT NULL
+);
+
+
+--
+-- Name: COLUMNS_OLD; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "COLUMNS_OLD" (
+    "SD_ID" bigint NOT NULL,
+    "COMMENT" character varying(256) DEFAULT NULL::character varying,
+    "COLUMN_NAME" character varying(128) NOT NULL,
+    "TYPE_NAME" character varying(4000) NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: COLUMNS_V2; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "COLUMNS_V2" (
+    "CD_ID" bigint NOT NULL,
+    "COMMENT" character varying(4000),
+    "COLUMN_NAME" character varying(128) NOT NULL,
+    "TYPE_NAME" character varying(4000),
+    "INTEGER_IDX" integer NOT NULL
+);
+
+
+--
+-- Name: DATABASE_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "DATABASE_PARAMS" (
+    "DB_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(180) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: DBS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "DBS" (
+    "DB_ID" bigint NOT NULL,
+    "DESC" character varying(4000) DEFAULT NULL::character varying,
+    "DB_LOCATION_URI" character varying(4000) NOT NULL,
+    "NAME" character varying(128) DEFAULT NULL::character varying,
+    "OWNER_NAME" character varying(128) DEFAULT NULL::character varying,
+    "OWNER_TYPE" character varying(10) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: DB_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "DB_PRIVS" (
+    "DB_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "DB_ID" bigint,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "DB_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: GLOBAL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "GLOBAL_PRIVS" (
+    "USER_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "USER_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: IDXS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "IDXS" (
+    "INDEX_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "DEFERRED_REBUILD" boolean NOT NULL,
+    "INDEX_HANDLER_CLASS" character varying(4000) DEFAULT NULL::character varying,
+    "INDEX_NAME" character varying(128) DEFAULT NULL::character varying,
+    "INDEX_TBL_ID" bigint,
+    "LAST_ACCESS_TIME" bigint NOT NULL,
+    "ORIG_TBL_ID" bigint,
+    "SD_ID" bigint
+);
+
+
+--
+-- Name: INDEX_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "INDEX_PARAMS" (
+    "INDEX_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: NUCLEUS_TABLES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "NUCLEUS_TABLES" (
+    "CLASS_NAME" character varying(128) NOT NULL,
+    "TABLE_NAME" character varying(128) NOT NULL,
+    "TYPE" character varying(4) NOT NULL,
+    "OWNER" character varying(2) NOT NULL,
+    "VERSION" character varying(20) NOT NULL,
+    "INTERFACE_NAME" character varying(255) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: PARTITIONS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITIONS" (
+    "PART_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "LAST_ACCESS_TIME" bigint NOT NULL,
+    "PART_NAME" character varying(767) DEFAULT NULL::character varying,
+    "SD_ID" bigint,
+    "TBL_ID" bigint
+);
+
+
+--
+-- Name: PARTITION_EVENTS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_EVENTS" (
+    "PART_NAME_ID" bigint NOT NULL,
+    "DB_NAME" character varying(128),
+    "EVENT_TIME" bigint NOT NULL,
+    "EVENT_TYPE" integer NOT NULL,
+    "PARTITION_NAME" character varying(767),
+    "TBL_NAME" character varying(128)
+);
+
+
+--
+-- Name: PARTITION_KEYS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_KEYS" (
+    "TBL_ID" bigint NOT NULL,
+    "PKEY_COMMENT" character varying(4000) DEFAULT NULL::character varying,
+    "PKEY_NAME" character varying(128) NOT NULL,
+    "PKEY_TYPE" character varying(767) NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: PARTITION_KEY_VALS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_KEY_VALS" (
+    "PART_ID" bigint NOT NULL,
+    "PART_KEY_VAL" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: PARTITION_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_PARAMS" (
+    "PART_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: PART_COL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PART_COL_PRIVS" (
+    "PART_COLUMN_GRANT_ID" bigint NOT NULL,
+    "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_ID" bigint,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_COL_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: PART_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PART_PRIVS" (
+    "PART_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_ID" bigint,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: ROLES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "ROLES" (
+    "ROLE_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "OWNER_NAME" character varying(128) DEFAULT NULL::character varying,
+    "ROLE_NAME" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: ROLE_MAP; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "ROLE_MAP" (
+    "ROLE_GRANT_ID" bigint NOT NULL,
+    "ADD_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "ROLE_ID" bigint
+);
+
+
+--
+-- Name: SDS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SDS" (
+    "SD_ID" bigint NOT NULL,
+    "INPUT_FORMAT" character varying(4000) DEFAULT NULL::character varying,
+    "IS_COMPRESSED" boolean NOT NULL,
+    "LOCATION" character varying(4000) DEFAULT NULL::character varying,
+    "NUM_BUCKETS" bigint NOT NULL,
+    "OUTPUT_FORMAT" character varying(4000) DEFAULT NULL::character varying,
+    "SERDE_ID" bigint,
+    "CD_ID" bigint,
+    "IS_STOREDASSUBDIRECTORIES" boolean NOT NULL
+);
+
+
+--
+-- Name: SD_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SD_PARAMS" (
+    "SD_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: SEQUENCE_TABLE; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SEQUENCE_TABLE" (
+    "SEQUENCE_NAME" character varying(255) NOT NULL,
+    "NEXT_VAL" bigint NOT NULL
+);
+
+
+--
+-- Name: SERDES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SERDES" (
+    "SERDE_ID" bigint NOT NULL,
+    "NAME" character varying(128) DEFAULT NULL::character varying,
+    "SLIB" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: SERDE_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SERDE_PARAMS" (
+    "SERDE_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: SORT_COLS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SORT_COLS" (
+    "SD_ID" bigint NOT NULL,
+    "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+    "ORDER" bigint NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: TABLE_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TABLE_PARAMS" (
+    "TBL_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: TBLS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TBLS" (
+    "TBL_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "DB_ID" bigint,
+    "LAST_ACCESS_TIME" bigint NOT NULL,
+    "OWNER" character varying(767) DEFAULT NULL::character varying,
+    "RETENTION" bigint NOT NULL,
+    "SD_ID" bigint,
+    "TBL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "TBL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "VIEW_EXPANDED_TEXT" text,
+    "VIEW_ORIGINAL_TEXT" text
+);
+
+
+--
+-- Name: TBL_COL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TBL_COL_PRIVS" (
+    "TBL_COLUMN_GRANT_ID" bigint NOT NULL,
+    "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "TBL_COL_PRIV" character varying(128) DEFAULT NULL::character varying,
+    "TBL_ID" bigint
+);
+
+
+--
+-- Name: TBL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TBL_PRIVS" (
+    "TBL_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "TBL_PRIV" character varying(128) DEFAULT NULL::character varying,
+    "TBL_ID" bigint
+);
+
+
+--
+-- Name: TYPES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TYPES" (
+    "TYPES_ID" bigint NOT NULL,
+    "TYPE_NAME" character varying(128) DEFAULT NULL::character varying,
+    "TYPE1" character varying(767) DEFAULT NULL::character varying,
+    "TYPE2" character varying(767) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: TYPE_FIELDS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TYPE_FIELDS" (
+    "TYPE_NAME" bigint NOT NULL,
+    "COMMENT" character varying(256) DEFAULT NULL::character varying,
+    "FIELD_NAME" character varying(128) NOT NULL,
+    "FIELD_TYPE" character varying(767) NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_STRING_LIST" (
+    "STRING_LIST_ID" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_STRING_LIST_VALUES" (
+    "STRING_LIST_ID" bigint NOT NULL,
+    "STRING_LIST_VALUE" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_COL_NAMES" (
+    "SD_ID" bigint NOT NULL,
+    "SKEWED_COL_NAME" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_COL_VALUE_LOC_MAP" (
+    "SD_ID" bigint NOT NULL,
+    "STRING_LIST_ID_KID" bigint NOT NULL,
+    "LOCATION" character varying(4000) DEFAULT NULL::character varying
+);
+
+CREATE TABLE "SKEWED_VALUES" (
+    "SD_ID_OID" bigint NOT NULL,
+    "STRING_LIST_ID_EID" bigint NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: TAB_COL_STATS Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE  "MASTER_KEYS"
+(
+    "KEY_ID" SERIAL,
+    "MASTER_KEY" varchar(767) NULL,
+    PRIMARY KEY ("KEY_ID")
+);
+
+CREATE TABLE  "DELEGATION_TOKENS"
+(
+    "TOKEN_IDENT" varchar(767) NOT NULL,
+    "TOKEN" varchar(767) NULL,
+    PRIMARY KEY ("TOKEN_IDENT")
+);
+
+CREATE TABLE "TAB_COL_STATS" (
+ "CS_ID" bigint NOT NULL,
+ "DB_NAME" character varying(128) DEFAULT NULL::character varying,
+ "TABLE_NAME" character varying(128) DEFAULT NULL::character varying,
+ "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+ "COLUMN_TYPE" character varying(128) DEFAULT NULL::character varying,
+ "TBL_ID" bigint NOT NULL,
+ "LONG_LOW_VALUE" bigint,
+ "LONG_HIGH_VALUE" bigint,
+ "DOUBLE_LOW_VALUE" double precision,
+ "DOUBLE_HIGH_VALUE" double precision,
+ "BIG_DECIMAL_LOW_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "BIG_DECIMAL_HIGH_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "NUM_NULLS" bigint NOT NULL,
+ "NUM_DISTINCTS" bigint,
+ "AVG_COL_LEN" double precision,
+ "MAX_COL_LEN" bigint,
+ "NUM_TRUES" bigint,
+ "NUM_FALSES" bigint,
+ "LAST_ANALYZED" bigint NOT NULL
+);
+
+--
+-- Table structure for VERSION
+--
+CREATE TABLE "VERSION" (
+  "VER_ID" bigint,
+  "SCHEMA_VERSION" character varying(127) NOT NULL,
+  "VERSION_COMMENT" character varying(255) NOT NULL
+);
+
+--
+-- Name: PART_COL_STATS Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PART_COL_STATS" (
+ "CS_ID" bigint NOT NULL,
+ "DB_NAME" character varying(128) DEFAULT NULL::character varying,
+ "TABLE_NAME" character varying(128) DEFAULT NULL::character varying,
+ "PARTITION_NAME" character varying(767) DEFAULT NULL::character varying,
+ "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+ "COLUMN_TYPE" character varying(128) DEFAULT NULL::character varying,
+ "PART_ID" bigint NOT NULL,
+ "LONG_LOW_VALUE" bigint,
+ "LONG_HIGH_VALUE" bigint,
+ "DOUBLE_LOW_VALUE" double precision,
+ "DOUBLE_HIGH_VALUE" double precision,
+ "BIG_DECIMAL_LOW_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "BIG_DECIMAL_HIGH_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "NUM_NULLS" bigint NOT NULL,
+ "NUM_DISTINCTS" bigint,
+ "AVG_COL_LEN" double precision,
+ "MAX_COL_LEN" bigint,
+ "NUM_TRUES" bigint,
+ "NUM_FALSES" bigint,
+ "LAST_ANALYZED" bigint NOT NULL
+);
+
+--
+-- Table structure for FUNCS
+--
+CREATE TABLE "FUNCS" (
+  "FUNC_ID" BIGINT NOT NULL,
+  "CLASS_NAME" VARCHAR(4000),
+  "CREATE_TIME" INTEGER NOT NULL,
+  "DB_ID" BIGINT,
+  "FUNC_NAME" VARCHAR(128),
+  "FUNC_TYPE" INTEGER NOT NULL,
+  "OWNER_NAME" VARCHAR(128),
+  "OWNER_TYPE" VARCHAR(10),
+  PRIMARY KEY ("FUNC_ID")
+);
+
+--
+-- Table structure for FUNC_RU
+--
+CREATE TABLE "FUNC_RU" (
+  "FUNC_ID" BIGINT NOT NULL,
+  "RESOURCE_TYPE" INTEGER NOT NULL,
+  "RESOURCE_URI" VARCHAR(4000),
+  "INTEGER_IDX" INTEGER NOT NULL,
+  PRIMARY KEY ("FUNC_ID", "INTEGER_IDX")
+);
+
+--
+-- Name: BUCKETING_COLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "BUCKETING_COLS"
+    ADD CONSTRAINT "BUCKETING_COLS_pkey" PRIMARY KEY ("SD_ID", "INTEGER_IDX");
+
+
+--
+-- Name: CDS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "CDS"
+    ADD CONSTRAINT "CDS_pkey" PRIMARY KEY ("CD_ID");
+
+
+--
+-- Name: COLUMNS_V2_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "COLUMNS_V2"
+    ADD CONSTRAINT "COLUMNS_V2_pkey" PRIMARY KEY ("CD_ID", "COLUMN_NAME");
+
+
+--
+-- Name: COLUMNS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "COLUMNS_OLD"
+    ADD CONSTRAINT "COLUMNS_pkey" PRIMARY KEY ("SD_ID", "COLUMN_NAME");
+
+
+--
+-- Name: DATABASE_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DATABASE_PARAMS"
+    ADD CONSTRAINT "DATABASE_PARAMS_pkey" PRIMARY KEY ("DB_ID", "PARAM_KEY");
+
+
+--
+-- Name: DBPRIVILEGEINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DB_PRIVS"
+    ADD CONSTRAINT "DBPRIVILEGEINDEX" UNIQUE ("DB_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DB_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: DBS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DBS"
+    ADD CONSTRAINT "DBS_pkey" PRIMARY KEY ("DB_ID");
+
+
+--
+-- Name: DB_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DB_PRIVS"
+    ADD CONSTRAINT "DB_PRIVS_pkey" PRIMARY KEY ("DB_GRANT_ID");
+
+
+--
+-- Name: GLOBALPRIVILEGEINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "GLOBAL_PRIVS"
+    ADD CONSTRAINT "GLOBALPRIVILEGEINDEX" UNIQUE ("PRINCIPAL_NAME", "PRINCIPAL_TYPE", "USER_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: GLOBAL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "GLOBAL_PRIVS"
+    ADD CONSTRAINT "GLOBAL_PRIVS_pkey" PRIMARY KEY ("USER_GRANT_ID");
+
+
+--
+-- Name: IDXS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_pkey" PRIMARY KEY ("INDEX_ID");
+
+
+--
+-- Name: INDEX_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "INDEX_PARAMS"
+    ADD CONSTRAINT "INDEX_PARAMS_pkey" PRIMARY KEY ("INDEX_ID", "PARAM_KEY");
+
+
+--
+-- Name: NUCLEUS_TABLES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "NUCLEUS_TABLES"
+    ADD CONSTRAINT "NUCLEUS_TABLES_pkey" PRIMARY KEY ("CLASS_NAME");
+
+
+--
+-- Name: PARTITIONS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "PARTITIONS_pkey" PRIMARY KEY ("PART_ID");
+
+
+--
+-- Name: PARTITION_EVENTS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_EVENTS"
+    ADD CONSTRAINT "PARTITION_EVENTS_pkey" PRIMARY KEY ("PART_NAME_ID");
+
+
+--
+-- Name: PARTITION_KEYS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_KEYS"
+    ADD CONSTRAINT "PARTITION_KEYS_pkey" PRIMARY KEY ("TBL_ID", "PKEY_NAME");
+
+
+--
+-- Name: PARTITION_KEY_VALS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_KEY_VALS"
+    ADD CONSTRAINT "PARTITION_KEY_VALS_pkey" PRIMARY KEY ("PART_ID", "INTEGER_IDX");
+
+
+--
+-- Name: PARTITION_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_PARAMS"
+    ADD CONSTRAINT "PARTITION_PARAMS_pkey" PRIMARY KEY ("PART_ID", "PARAM_KEY");
+
+
+--
+-- Name: PART_COL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PART_COL_PRIVS"
+    ADD CONSTRAINT "PART_COL_PRIVS_pkey" PRIMARY KEY ("PART_COLUMN_GRANT_ID");
+
+
+--
+-- Name: PART_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PART_PRIVS"
+    ADD CONSTRAINT "PART_PRIVS_pkey" PRIMARY KEY ("PART_GRANT_ID");
+
+
+--
+-- Name: ROLEENTITYINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLES"
+    ADD CONSTRAINT "ROLEENTITYINDEX" UNIQUE ("ROLE_NAME");
+
+
+--
+-- Name: ROLES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLES"
+    ADD CONSTRAINT "ROLES_pkey" PRIMARY KEY ("ROLE_ID");
+
+
+--
+-- Name: ROLE_MAP_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLE_MAP"
+    ADD CONSTRAINT "ROLE_MAP_pkey" PRIMARY KEY ("ROLE_GRANT_ID");
+
+
+--
+-- Name: SDS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SDS"
+    ADD CONSTRAINT "SDS_pkey" PRIMARY KEY ("SD_ID");
+
+
+--
+-- Name: SD_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SD_PARAMS"
+    ADD CONSTRAINT "SD_PARAMS_pkey" PRIMARY KEY ("SD_ID", "PARAM_KEY");
+
+
+--
+-- Name: SEQUENCE_TABLE_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SEQUENCE_TABLE"
+    ADD CONSTRAINT "SEQUENCE_TABLE_pkey" PRIMARY KEY ("SEQUENCE_NAME");
+
+
+--
+-- Name: SERDES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SERDES"
+    ADD CONSTRAINT "SERDES_pkey" PRIMARY KEY ("SERDE_ID");
+
+
+--
+-- Name: SERDE_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SERDE_PARAMS"
+    ADD CONSTRAINT "SERDE_PARAMS_pkey" PRIMARY KEY ("SERDE_ID", "PARAM_KEY");
+
+
+--
+-- Name: SORT_COLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SORT_COLS"
+    ADD CONSTRAINT "SORT_COLS_pkey" PRIMARY KEY ("SD_ID", "INTEGER_IDX");
+
+
+--
+-- Name: TABLE_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TABLE_PARAMS"
+    ADD CONSTRAINT "TABLE_PARAMS_pkey" PRIMARY KEY ("TBL_ID", "PARAM_KEY");
+
+
+--
+-- Name: TBLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "TBLS_pkey" PRIMARY KEY ("TBL_ID");
+
+
+--
+-- Name: TBL_COL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBL_COL_PRIVS"
+    ADD CONSTRAINT "TBL_COL_PRIVS_pkey" PRIMARY KEY ("TBL_COLUMN_GRANT_ID");
+
+
+--
+-- Name: TBL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBL_PRIVS"
+    ADD CONSTRAINT "TBL_PRIVS_pkey" PRIMARY KEY ("TBL_GRANT_ID");
+
+
+--
+-- Name: TYPES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TYPES"
+    ADD CONSTRAINT "TYPES_pkey" PRIMARY KEY ("TYPES_ID");
+
+
+--
+-- Name: TYPE_FIELDS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TYPE_FIELDS"
+    ADD CONSTRAINT "TYPE_FIELDS_pkey" PRIMARY KEY ("TYPE_NAME", "FIELD_NAME");
+
+ALTER TABLE ONLY "SKEWED_STRING_LIST"
+    ADD CONSTRAINT "SKEWED_STRING_LIST_pkey" PRIMARY KEY ("STRING_LIST_ID");
+
+ALTER TABLE ONLY "SKEWED_STRING_LIST_VALUES"
+    ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_pkey" PRIMARY KEY ("STRING_LIST_ID", "INTEGER_IDX");
+
+
+ALTER TABLE ONLY "SKEWED_COL_NAMES"
+    ADD CONSTRAINT "SKEWED_COL_NAMES_pkey" PRIMARY KEY ("SD_ID", "INTEGER_IDX");
+
+ALTER TABLE ONLY "SKEWED_COL_VALUE_LOC_MAP"
+    ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_pkey" PRIMARY KEY ("SD_ID", "STRING_LIST_ID_KID");
+
+ALTER TABLE ONLY "SKEWED_VALUES"
+    ADD CONSTRAINT "SKEWED_VALUES_pkey" PRIMARY KEY ("SD_ID_OID", "INTEGER_IDX");
+
+--
+-- Name: TAB_COL_STATS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+ALTER TABLE ONLY "TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_pkey" PRIMARY KEY("CS_ID");
+
+--
+-- Name: PART_COL_STATS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+ALTER TABLE ONLY "PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_pkey" PRIMARY KEY("CS_ID");
+
+--
+-- Name: UNIQUEINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "UNIQUEINDEX" UNIQUE ("INDEX_NAME", "ORIG_TBL_ID");
+
+
+--
+-- Name: UNIQUEPARTITION; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "UNIQUEPARTITION" UNIQUE ("PART_NAME", "TBL_ID");
+
+
+--
+-- Name: UNIQUETABLE; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "UNIQUETABLE" UNIQUE ("TBL_NAME", "DB_ID");
+
+
+--
+-- Name: UNIQUE_DATABASE; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DBS"
+    ADD CONSTRAINT "UNIQUE_DATABASE" UNIQUE ("NAME");
+
+
+--
+-- Name: UNIQUE_TYPE; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TYPES"
+    ADD CONSTRAINT "UNIQUE_TYPE" UNIQUE ("TYPE_NAME");
+
+
+--
+-- Name: USERROLEMAPINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLE_MAP"
+    ADD CONSTRAINT "USERROLEMAPINDEX" UNIQUE ("PRINCIPAL_NAME", "ROLE_ID", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: BUCKETING_COLS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "BUCKETING_COLS_N49" ON "BUCKETING_COLS" USING btree ("SD_ID");
+
+
+--
+-- Name: COLUMNS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "COLUMNS_N49" ON "COLUMNS_OLD" USING btree ("SD_ID");
+
+
+--
+-- Name: DATABASE_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "DATABASE_PARAMS_N49" ON "DATABASE_PARAMS" USING btree ("DB_ID");
+
+
+--
+-- Name: DB_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "DB_PRIVS_N49" ON "DB_PRIVS" USING btree ("DB_ID");
+
+
+--
+-- Name: IDXS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "IDXS_N49" ON "IDXS" USING btree ("ORIG_TBL_ID");
+
+
+--
+-- Name: IDXS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "IDXS_N50" ON "IDXS" USING btree ("INDEX_TBL_ID");
+
+
+--
+-- Name: IDXS_N51; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "IDXS_N51" ON "IDXS" USING btree ("SD_ID");
+
+
+--
+-- Name: INDEX_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "INDEX_PARAMS_N49" ON "INDEX_PARAMS" USING btree ("INDEX_ID");
+
+
+--
+-- Name: PARTITIONCOLUMNPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONCOLUMNPRIVILEGEINDEX" ON "PART_COL_PRIVS" USING btree ("PART_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_COL_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: PARTITIONEVENTINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONEVENTINDEX" ON "PARTITION_EVENTS" USING btree ("PARTITION_NAME");
+
+
+--
+-- Name: PARTITIONS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONS_N49" ON "PARTITIONS" USING btree ("TBL_ID");
+
+
+--
+-- Name: PARTITIONS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONS_N50" ON "PARTITIONS" USING btree ("SD_ID");
+
+
+--
+-- Name: PARTITION_KEYS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITION_KEYS_N49" ON "PARTITION_KEYS" USING btree ("TBL_ID");
+
+
+--
+-- Name: PARTITION_KEY_VALS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITION_KEY_VALS_N49" ON "PARTITION_KEY_VALS" USING btree ("PART_ID");
+
+
+--
+-- Name: PARTITION_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITION_PARAMS_N49" ON "PARTITION_PARAMS" USING btree ("PART_ID");
+
+
+--
+-- Name: PARTPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTPRIVILEGEINDEX" ON "PART_PRIVS" USING btree ("PART_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: PART_COL_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PART_COL_PRIVS_N49" ON "PART_COL_PRIVS" USING btree ("PART_ID");
+
+
+--
+-- Name: PART_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PART_PRIVS_N49" ON "PART_PRIVS" USING btree ("PART_ID");
+
+
+--
+-- Name: ROLE_MAP_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "ROLE_MAP_N49" ON "ROLE_MAP" USING btree ("ROLE_ID");
+
+
+--
+-- Name: SDS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SDS_N49" ON "SDS" USING btree ("SERDE_ID");
+
+
+--
+-- Name: SD_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SD_PARAMS_N49" ON "SD_PARAMS" USING btree ("SD_ID");
+
+
+--
+-- Name: SERDE_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SERDE_PARAMS_N49" ON "SERDE_PARAMS" USING btree ("SERDE_ID");
+
+
+--
+-- Name: SORT_COLS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SORT_COLS_N49" ON "SORT_COLS" USING btree ("SD_ID");
+
+
+--
+-- Name: TABLECOLUMNPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TABLECOLUMNPRIVILEGEINDEX" ON "TBL_COL_PRIVS" USING btree ("TBL_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_COL_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: TABLEPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TABLEPRIVILEGEINDEX" ON "TBL_PRIVS" USING btree ("TBL_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: TABLE_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TABLE_PARAMS_N49" ON "TABLE_PARAMS" USING btree ("TBL_ID");
+
+
+--
+-- Name: TBLS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBLS_N49" ON "TBLS" USING btree ("DB_ID");
+
+
+--
+-- Name: TBLS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBLS_N50" ON "TBLS" USING btree ("SD_ID");
+
+
+--
+-- Name: TBL_COL_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBL_COL_PRIVS_N49" ON "TBL_COL_PRIVS" USING btree ("TBL_ID");
+
+
+--
+-- Name: TBL_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBL_PRIVS_N49" ON "TBL_PRIVS" USING btree ("TBL_ID");
+
+
+--
+-- Name: TYPE_FIELDS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TYPE_FIELDS_N49" ON "TYPE_FIELDS" USING btree ("TYPE_NAME");
+
+--
+-- Name: TAB_COL_STATS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TAB_COL_STATS_N49" ON "TAB_COL_STATS" USING btree ("TBL_ID");
+
+--
+-- Name: PART_COL_STATS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PART_COL_STATS_N49" ON "PART_COL_STATS" USING btree ("PART_ID");
+
+--
+-- Name: UNIQUEFUNCTION; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE UNIQUE INDEX "UNIQUEFUNCTION" ON "FUNCS" ("FUNC_NAME", "DB_ID");
+
+--
+-- Name: FUNCS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "FUNCS_N49" ON "FUNCS" ("DB_ID");
+
+--
+-- Name: FUNC_RU_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "FUNC_RU_N49" ON "FUNC_RU" ("FUNC_ID");
+
+
+ALTER TABLE ONLY "SKEWED_STRING_LIST_VALUES"
+    ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_fkey" FOREIGN KEY ("STRING_LIST_ID") REFERENCES "SKEWED_STRING_LIST"("STRING_LIST_ID") DEFERRABLE;
+
+
+ALTER TABLE ONLY "SKEWED_COL_NAMES"
+    ADD CONSTRAINT "SKEWED_COL_NAMES_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+ALTER TABLE ONLY "SKEWED_COL_VALUE_LOC_MAP"
+    ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_fkey1" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+ALTER TABLE ONLY "SKEWED_COL_VALUE_LOC_MAP"
+    ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_fkey2" FOREIGN KEY ("STRING_LIST_ID_KID") REFERENCES "SKEWED_STRING_LIST"("STRING_LIST_ID") DEFERRABLE;
+
+ALTER TABLE ONLY "SKEWED_VALUES"
+    ADD CONSTRAINT "SKEWED_VALUES_fkey1" FOREIGN KEY ("STRING_LIST_ID_EID") REFERENCES "SKEWED_STRING_LIST"("STRING_LIST_ID") DEFERRABLE;
+
+ALTER TABLE ONLY "SKEWED_VALUES"
+    ADD CONSTRAINT "SKEWED_VALUES_fkey2" FOREIGN KEY ("SD_ID_OID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: BUCKETING_COLS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "BUCKETING_COLS"
+    ADD CONSTRAINT "BUCKETING_COLS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: COLUMNS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "COLUMNS_OLD"
+    ADD CONSTRAINT "COLUMNS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: COLUMNS_V2_CD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "COLUMNS_V2"
+    ADD CONSTRAINT "COLUMNS_V2_CD_ID_fkey" FOREIGN KEY ("CD_ID") REFERENCES "CDS"("CD_ID") DEFERRABLE;
+
+
+--
+-- Name: DATABASE_PARAMS_DB_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "DATABASE_PARAMS"
+    ADD CONSTRAINT "DATABASE_PARAMS_DB_ID_fkey" FOREIGN KEY ("DB_ID") REFERENCES "DBS"("DB_ID") DEFERRABLE;
+
+
+--
+-- Name: DB_PRIVS_DB_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "DB_PRIVS"
+    ADD CONSTRAINT "DB_PRIVS_DB_ID_fkey" FOREIGN KEY ("DB_ID") REFERENCES "DBS"("DB_ID") DEFERRABLE;
+
+
+--
+-- Name: IDXS_INDEX_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_INDEX_TBL_ID_fkey" FOREIGN KEY ("INDEX_TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: IDXS_ORIG_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_ORIG_TBL_ID_fkey" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: IDXS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: INDEX_PARAMS_INDEX_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "INDEX_PARAMS"
+    ADD CONSTRAINT "INDEX_PARAMS_INDEX_ID_fkey" FOREIGN KEY ("INDEX_ID") REFERENCES "IDXS"("INDEX_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITIONS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "PARTITIONS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITIONS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "PARTITIONS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITION_KEYS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITION_KEYS"
+    ADD CONSTRAINT "PARTITION_KEYS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITION_KEY_VALS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITION_KEY_VALS"
+    ADD CONSTRAINT "PARTITION_KEY_VALS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITION_PARAMS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITION_PARAMS"
+    ADD CONSTRAINT "PARTITION_PARAMS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: PART_COL_PRIVS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PART_COL_PRIVS"
+    ADD CONSTRAINT "PART_COL_PRIVS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: PART_PRIVS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PART_PRIVS"
+    ADD CONSTRAINT "PART_PRIVS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: ROLE_MAP_ROLE_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "ROLE_MAP"
+    ADD CONSTRAINT "ROLE_MAP_ROLE_ID_fkey" FOREIGN KEY ("ROLE_ID") REFERENCES "ROLES"("ROLE_ID") DEFERRABLE;
+
+
+--
+-- Name: SDS_CD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SDS"
+    ADD CONSTRAINT "SDS_CD_ID_fkey" FOREIGN KEY ("CD_ID") REFERENCES "CDS"("CD_ID") DEFERRABLE;
+
+
+--
+-- Name: SDS_SERDE_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SDS"
+    ADD CONSTRAINT "SDS_SERDE_ID_fkey" FOREIGN KEY ("SERDE_ID") REFERENCES "SERDES"("SERDE_ID") DEFERRABLE;
+
+
+--
+-- Name: SD_PARAMS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SD_PARAMS"
+    ADD CONSTRAINT "SD_PARAMS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: SERDE_PARAMS_SERDE_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SERDE_PARAMS"
+    ADD CONSTRAINT "SERDE_PARAMS_SERDE_ID_fkey" FOREIGN KEY ("SERDE_ID") REFERENCES "SERDES"("SERDE_ID") DEFERRABLE;
+
+
+--
+-- Name: SORT_COLS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SORT_COLS"
+    ADD CONSTRAINT "SORT_COLS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: TABLE_PARAMS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TABLE_PARAMS"
+    ADD CONSTRAINT "TABLE_PARAMS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: TBLS_DB_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "TBLS_DB_ID_fkey" FOREIGN KEY ("DB_ID") REFERENCES "DBS"("DB_ID") DEFERRABLE;
+
+
+--
+-- Name: TBLS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "TBLS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: TBL_COL_PRIVS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBL_COL_PRIVS"
+    ADD CONSTRAINT "TBL_COL_PRIVS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: TBL_PRIVS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBL_PRIVS"
+    ADD CONSTRAINT "TBL_PRIVS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: TYPE_FIELDS_TYPE_NAME_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TYPE_FIELDS"
+    ADD CONSTRAINT "TYPE_FIELDS_TYPE_NAME_fkey" FOREIGN KEY ("TYPE_NAME") REFERENCES "TYPES"("TYPES_ID") DEFERRABLE;
+
+--
+-- Name: TAB_COL_STATS_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+ALTER TABLE ONLY "TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_fkey" FOREIGN KEY("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: PART_COL_STATS_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+ALTER TABLE ONLY "PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_fkey" FOREIGN KEY("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+ALTER TABLE ONLY "VERSION" ADD CONSTRAINT "VERSION_pkey" PRIMARY KEY ("VER_ID");
+
+-- Name: FUNCS_FK1; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+ALTER TABLE ONLY "FUNCS"
+    ADD CONSTRAINT "FUNCS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "DBS" ("DB_ID") DEFERRABLE;
+
+-- Name: FUNC_RU_FK1; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+ALTER TABLE ONLY "FUNC_RU"
+    ADD CONSTRAINT "FUNC_RU_FK1" FOREIGN KEY ("FUNC_ID") REFERENCES "FUNCS" ("FUNC_ID") DEFERRABLE;
+
+--
+-- Name: public; Type: ACL; Schema: -; Owner: hiveuser
+--
+
+REVOKE ALL ON SCHEMA public FROM PUBLIC;
+GRANT ALL ON SCHEMA public TO PUBLIC;
+
+
+--
+-- PostgreSQL database dump complete
+--
+
+-- -----------------------------------------------------------------------------------------------------------------------------------------------
+-- Transaction and lock tables
+-- These are not part of package jdo, so if you are going to regenerate this file you need to manually add the following section back to the file.
+-- -----------------------------------------------------------------------------------------------------------------------------------------------
+
+CREATE TABLE TXNS (
+  TXN_ID bigint PRIMARY KEY,
+  TXN_STATE char(1) NOT NULL,
+  TXN_STARTED bigint NOT NULL,
+  TXN_LAST_HEARTBEAT bigint NOT NULL,
+  TXN_USER varchar(128) NOT NULL,
+  TXN_HOST varchar(128) NOT NULL
+);
+
+CREATE TABLE TXN_COMPONENTS (
+  TC_TXNID bigint REFERENCES TXNS (TXN_ID),
+  TC_DATABASE varchar(128) NOT NULL,
+  TC_TABLE varchar(128),
+  TC_PARTITION varchar(767) DEFAULT NULL
+);
+
+CREATE TABLE COMPLETED_TXN_COMPONENTS (
+  CTC_TXNID bigint,
+  CTC_DATABASE varchar(128) NOT NULL,
+  CTC_TABLE varchar(128),
+  CTC_PARTITION varchar(767)
+);
+
+CREATE TABLE NEXT_TXN_ID (
+  NTXN_NEXT bigint NOT NULL
+);
+INSERT INTO NEXT_TXN_ID VALUES(1);
+
+CREATE TABLE HIVE_LOCKS (
+  HL_LOCK_EXT_ID bigint NOT NULL,
+  HL_LOCK_INT_ID bigint NOT NULL,
+  HL_TXNID bigint,
+  HL_DB varchar(128) NOT NULL,
+  HL_TABLE varchar(128),
+  HL_PARTITION varchar(767) DEFAULT NULL,
+  HL_LOCK_STATE char(1) NOT NULL,
+  HL_LOCK_TYPE char(1) NOT NULL,
+  HL_LAST_HEARTBEAT bigint NOT NULL,
+  HL_ACQUIRED_AT bigint,
+  HL_USER varchar(128) NOT NULL,
+  HL_HOST varchar(128) NOT NULL,
+  PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID)
+); 
+
+CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS USING hash (HL_TXNID);
+
+CREATE TABLE NEXT_LOCK_ID (
+  NL_NEXT bigint NOT NULL
+);
+INSERT INTO NEXT_LOCK_ID VALUES(1);
+
+CREATE TABLE COMPACTION_QUEUE (
+  CQ_ID bigint PRIMARY KEY,
+  CQ_DATABASE varchar(128) NOT NULL,
+  CQ_TABLE varchar(128) NOT NULL,
+  CQ_PARTITION varchar(767),
+  CQ_STATE char(1) NOT NULL,
+  CQ_TYPE char(1) NOT NULL,
+  CQ_WORKER_ID varchar(128),
+  CQ_START bigint,
+  CQ_RUN_AS varchar(128)
+);
+
+CREATE TABLE NEXT_COMPACTION_QUEUE_ID (
+  NCQ_NEXT bigint NOT NULL
+);
+INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1);
+
+
+-- -----------------------------------------------------------------
+-- Record schema version. Should be the last step in the init script
+-- -----------------------------------------------------------------
+INSERT INTO "VERSION" ("VER_ID", "SCHEMA_VERSION", "VERSION_COMMENT") VALUES (1, '0.13.0', 'Hive release version 0.13.0');
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/upgrade-0.12.0-to-0.13.0.oracle.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/upgrade-0.12.0-to-0.13.0.oracle.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/upgrade-0.12.0-to-0.13.0.oracle.sql
new file mode 100755
index 0000000..d08b985
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/upgrade-0.12.0-to-0.13.0.oracle.sql
@@ -0,0 +1,165 @@
+SELECT 'Upgrading MetaStore schema from 0.12.0 to 0.13.0' AS Status from dual;
+
+-- 15-HIVE-5700.oracle.sql
+-- Normalize the date partition column values as best we can. No schema changes.
+
+CREATE FUNCTION hive13_to_date(date_str IN VARCHAR2) RETURN DATE IS dt DATE; BEGIN dt := TO_DATE(date_str, 'YYYY-MM-DD'); RETURN dt; EXCEPTION WHEN others THEN RETURN null; END;/
+
+MERGE INTO PARTITION_KEY_VALS
+USING (
+  SELECT SRC.PART_ID as IPART_ID, SRC.INTEGER_IDX as IINTEGER_IDX, 
+     NVL(TO_CHAR(hive13_to_date(PART_KEY_VAL),'YYYY-MM-DD'), PART_KEY_VAL) as NORM
+  FROM PARTITION_KEY_VALS SRC
+    INNER JOIN PARTITIONS ON SRC.PART_ID = PARTITIONS.PART_ID
+    INNER JOIN PARTITION_KEYS ON PARTITION_KEYS.TBL_ID = PARTITIONS.TBL_ID
+      AND PARTITION_KEYS.INTEGER_IDX = SRC.INTEGER_IDX AND PARTITION_KEYS.PKEY_TYPE = 'date'
+) ON (IPART_ID = PARTITION_KEY_VALS.PART_ID AND IINTEGER_IDX = PARTITION_KEY_VALS.INTEGER_IDX)
+WHEN MATCHED THEN UPDATE SET PART_KEY_VAL = NORM;
+
+DROP FUNCTION hive13_to_date;
+
+-- 16-HIVE-6386.oracle.sql
+ALTER TABLE DBS ADD OWNER_NAME VARCHAR2(128);
+ALTER TABLE DBS ADD OWNER_TYPE VARCHAR2(10);
+
+-- 17-HIVE-6458.oracle.sql
+CREATE TABLE FUNCS (
+  FUNC_ID NUMBER NOT NULL,
+  CLASS_NAME VARCHAR2(4000),
+  CREATE_TIME NUMBER(10) NOT NULL,
+  DB_ID NUMBER,
+  FUNC_NAME VARCHAR2(128),
+  FUNC_TYPE NUMBER(10) NOT NULL,
+  OWNER_NAME VARCHAR2(128),
+  OWNER_TYPE VARCHAR2(10)
+);
+
+ALTER TABLE FUNCS ADD CONSTRAINT FUNCS_PK PRIMARY KEY (FUNC_ID);
+ALTER TABLE FUNCS ADD CONSTRAINT FUNCS_FK1 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED;
+CREATE UNIQUE INDEX UNIQUEFUNCTION ON FUNCS (FUNC_NAME, DB_ID);
+CREATE INDEX FUNCS_N49 ON FUNCS (DB_ID);
+
+CREATE TABLE FUNC_RU (
+  FUNC_ID NUMBER NOT NULL,
+  RESOURCE_TYPE NUMBER(10) NOT NULL,
+  RESOURCE_URI VARCHAR2(4000),
+  INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE FUNC_RU ADD CONSTRAINT FUNC_RU_PK PRIMARY KEY (FUNC_ID, INTEGER_IDX);
+ALTER TABLE FUNC_RU ADD CONSTRAINT FUNC_RU_FK1 FOREIGN KEY (FUNC_ID) REFERENCES FUNCS (FUNC_ID) INITIALLY DEFERRED;
+CREATE INDEX FUNC_RU_N49 ON FUNC_RU (FUNC_ID);
+
+-- 18-HIVE-6757.oracle.sql
+UPDATE SDS
+  SET INPUT_FORMAT = 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+WHERE
+  INPUT_FORMAT= 'parquet.hive.DeprecatedParquetInputFormat' or
+  INPUT_FORMAT = 'parquet.hive.MapredParquetInputFormat'
+;
+
+UPDATE SDS
+  SET OUTPUT_FORMAT = 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+WHERE
+  OUTPUT_FORMAT = 'parquet.hive.DeprecatedParquetOutputFormat'  or
+  OUTPUT_FORMAT = 'parquet.hive.MapredParquetOutputFormat'
+;
+
+UPDATE SERDES
+  SET SLIB='org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+WHERE
+  SLIB = 'parquet.hive.serde.ParquetHiveSerDe'
+;
+
+-- hive-txn-schema-0.13.0.oracle.sql
+
+-- 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.
+
+--
+-- Tables for transaction management
+-- 
+
+CREATE TABLE TXNS (
+  TXN_ID NUMBER(19) PRIMARY KEY,
+  TXN_STATE char(1) NOT NULL,
+  TXN_STARTED NUMBER(19) NOT NULL,
+  TXN_LAST_HEARTBEAT NUMBER(19) NOT NULL,
+  TXN_USER varchar(128) NOT NULL,
+  TXN_HOST varchar(128) NOT NULL
+);
+
+CREATE TABLE TXN_COMPONENTS (
+  TC_TXNID NUMBER(19) REFERENCES TXNS (TXN_ID),
+  TC_DATABASE VARCHAR2(128) NOT NULL,
+  TC_TABLE VARCHAR2(128),
+  TC_PARTITION VARCHAR2(767) NULL
+);
+
+CREATE TABLE COMPLETED_TXN_COMPONENTS (
+  CTC_TXNID NUMBER(19),
+  CTC_DATABASE varchar(128) NOT NULL,
+  CTC_TABLE varchar(128),
+  CTC_PARTITION varchar(767)
+);
+
+CREATE TABLE NEXT_TXN_ID (
+  NTXN_NEXT NUMBER(19) NOT NULL
+);
+INSERT INTO NEXT_TXN_ID VALUES(1);
+
+CREATE TABLE HIVE_LOCKS (
+  HL_LOCK_EXT_ID NUMBER(19) NOT NULL,
+  HL_LOCK_INT_ID NUMBER(19) NOT NULL,
+  HL_TXNID NUMBER(19),
+  HL_DB VARCHAR2(128) NOT NULL,
+  HL_TABLE VARCHAR2(128),
+  HL_PARTITION VARCHAR2(767),
+  HL_LOCK_STATE CHAR(1) NOT NULL,
+  HL_LOCK_TYPE CHAR(1) NOT NULL,
+  HL_LAST_HEARTBEAT NUMBER(19) NOT NULL,
+  HL_ACQUIRED_AT NUMBER(19),
+  HL_USER varchar(128) NOT NULL,
+  HL_HOST varchar(128) NOT NULL,
+  PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID)
+); 
+
+CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID);
+
+CREATE TABLE NEXT_LOCK_ID (
+  NL_NEXT NUMBER(19) NOT NULL
+);
+INSERT INTO NEXT_LOCK_ID VALUES(1);
+
+CREATE TABLE COMPACTION_QUEUE (
+  CQ_ID NUMBER(19) PRIMARY KEY,
+  CQ_DATABASE varchar(128) NOT NULL,
+  CQ_TABLE varchar(128) NOT NULL,
+  CQ_PARTITION varchar(767),
+  CQ_STATE char(1) NOT NULL,
+  CQ_TYPE char(1) NOT NULL,
+  CQ_WORKER_ID varchar(128),
+  CQ_START NUMBER(19),
+  CQ_RUN_AS varchar(128)
+);
+
+CREATE TABLE NEXT_COMPACTION_QUEUE_ID (
+  NCQ_NEXT NUMBER(19) NOT NULL
+);
+INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1);
+
+
+UPDATE VERSION SET SCHEMA_VERSION='0.13.0', VERSION_COMMENT='Hive release version 0.13.0' where VER_ID=1;
+SELECT 'Finished upgrading MetaStore schema from 0.12.0 to 0.13.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/upgrade-0.13.0.oracle.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/upgrade-0.13.0.oracle.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/upgrade-0.13.0.oracle.sql
new file mode 100755
index 0000000..b34f406
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/upgrade-0.13.0.oracle.sql
@@ -0,0 +1,38 @@
+ALTER TABLE TXNS MODIFY (
+  TXN_ID NUMBER(19),
+  TXN_STARTED NUMBER(19),
+  TXN_LAST_HEARTBEAT NUMBER(19)
+);
+
+ALTER TABLE TXN_COMPONENTS MODIFY (
+  TC_TXNID NUMBER(19)
+);
+
+ALTER TABLE COMPLETED_TXN_COMPONENTS MODIFY (
+  CTC_TXNID NUMBER(19)
+);
+
+ALTER TABLE NEXT_TXN_ID MODIFY (
+  NTXN_NEXT NUMBER(19)
+);
+
+ALTER TABLE HIVE_LOCKS MODIFY (
+  HL_LOCK_EXT_ID NUMBER(19),
+  HL_LOCK_INT_ID NUMBER(19),
+  HL_TXNID NUMBER(19),
+  HL_LAST_HEARTBEAT NUMBER(19),
+  HL_ACQUIRED_AT NUMBER(19)
+);
+
+ALTER TABLE NEXT_LOCK_ID MODIFY (
+  NL_NEXT NUMBER(19)
+);
+
+ALTER TABLE COMPACTION_QUEUE MODIFY (
+  CQ_ID NUMBER(19),
+  CQ_START NUMBER(19)
+);
+
+ALTER TABLE NEXT_COMPACTION_QUEUE_ID MODIFY (
+  NCQ_NEXT NUMBER(19)
+);

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/kerberos.json
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/kerberos.json b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/kerberos.json
new file mode 100755
index 0000000..34bda73
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/kerberos.json
@@ -0,0 +1,149 @@
+{
+  "services": [
+    {
+      "name": "HIVE",
+      "identities": [
+        {
+          "name": "/spnego"
+        },
+        {
+          "name": "/smokeuser"
+        }
+      ],
+      "configurations": [
+        {
+          "hive-site": {
+            "hive.metastore.sasl.enabled": "true",
+            "hive.server2.authentication": "KERBEROS"
+          }
+        },
+        {
+          "webhcat-site": {
+            "templeton.kerberos.secret": "secret",
+            "templeton.hive.properties": "hive.metastore.local=false,hive.metastore.uris=${clusterHostInfo/hive_metastore_host|each(thrift://%s:9083, \\\\,, \\s*\\,\\s*)},hive.metastore.sasl.enabled=true,hive.metastore.execute.setugi=true,hive.metastore.warehouse.dir=/apps/hive/warehouse,hive.exec.mode.local.auto=false,hive.metastore.kerberos.principal=hive/_HOST@${realm}"
+          }
+        },
+        {
+          "core-site": {
+            "hadoop.proxyuser.HTTP.hosts": "${clusterHostInfo/webhcat_server_host}"
+          }
+        },
+        {
+          "ranger-hive-audit": {
+            "xasecure.audit.jaas.Client.loginModuleName": "com.sun.security.auth.module.Krb5LoginModule",
+            "xasecure.audit.jaas.Client.loginModuleControlFlag": "required",
+            "xasecure.audit.jaas.Client.option.useKeyTab": "true",
+            "xasecure.audit.jaas.Client.option.storeKey": "false",
+            "xasecure.audit.jaas.Client.option.serviceName": "solr",
+            "xasecure.audit.destination.solr.force.use.inmemory.jaas.config": "true"
+          }
+        }
+      ],
+      "components": [
+        {
+          "name": "HIVE_METASTORE",
+          "identities": [
+            {
+              "name": "/HIVE/HIVE_SERVER/hive_server_hive",
+              "principal": {
+                "configuration": "hive-site/hive.metastore.kerberos.principal"
+              },
+              "keytab": {
+                "configuration": "hive-site/hive.metastore.kerberos.keytab.file"
+              }
+            }
+          ]
+        },
+        {
+          "name": "HIVE_SERVER",
+          "identities": [
+            {
+              "name": "/HDFS/NAMENODE/hdfs"
+            },
+            {
+              "name": "hive_server_hive",
+              "principal": {
+                "value": "hive/_HOST@${realm}",
+                "type" : "service",
+                "configuration": "hive-site/hive.server2.authentication.kerberos.principal",
+                "local_username": "${hive-env/hive_user}"
+              },
+              "keytab": {
+                "file": "${keytab_dir}/hive.service.keytab",
+                "owner": {
+                  "name": "${hive-env/hive_user}",
+                  "access": "r"
+                },
+                "group": {
+                  "name": "${cluster-env/user_group}",
+                  "access": "r"
+                },
+                "configuration": "hive-site/hive.server2.authentication.kerberos.keytab"
+              }
+            },
+            {
+              "name": "atlas_kafka",
+              "reference": "/HIVE/HIVE_SERVER/hive_server_hive",
+              "principal": {
+                "configuration": "hive-atlas-application.properties/atlas.jaas.KafkaClient.option.principal"
+              },
+              "keytab": {
+                "configuration": "hive-atlas-application.properties/atlas.jaas.KafkaClient.option.keyTab"
+              }
+            },
+            {
+              "name": "/spnego",
+              "principal": {
+                "configuration": "hive-site/hive.server2.authentication.spnego.principal"
+              },
+              "keytab": {
+                "configuration": "hive-site/hive.server2.authentication.spnego.keytab"
+              }
+            },
+            {
+              "name": "ranger_audit",
+              "reference": "/HIVE/HIVE_SERVER/hive_server_hive",
+              "principal": {
+                "configuration": "ranger-hive-audit/xasecure.audit.jaas.Client.option.principal"
+              },
+              "keytab": {
+                "configuration": "ranger-hive-audit/xasecure.audit.jaas.Client.option.keyTab"
+              }
+            }
+          ]
+        },
+        {
+          "name": "HIVE_SERVER_INTERACTIVE",
+          "identities": [
+            {
+              "name": "/HDFS/NAMENODE/hdfs"
+            },
+            {
+              "name": "/HIVE/HIVE_SERVER/hive_server_hive"
+            },
+            {
+              "name": "/HIVE/HIVE_SERVER/spnego"
+            },
+            {
+              "name": "/YARN/NODEMANAGER/llap_zk_hive"
+            }
+          ]
+        },
+        {
+          "name": "WEBHCAT_SERVER",
+          "identities": [
+            {
+              "name": "/spnego",
+              "principal": {
+                "configuration": "webhcat-site/templeton.kerberos.principal"
+              },
+              "keytab": {
+                "configuration": "webhcat-site/templeton.kerberos.keytab"
+              }
+            }
+          ]
+        }
+      ]
+    }
+  ]
+}


[20/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation


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

Branch: refs/heads/BIGTOP-2666
Commit: 4522c95962009c8eb3d9d98bd533510a73ed4a9f
Parents: c6da69b
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Sat Oct 22 20:33:46 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:45:58 2017 -0800

----------------------------------------------------------------------
 .../ambari/ODPi/1.0/services/HIVE/alerts.json   |  232 ++
 .../HIVE/configuration/beeline-log4j2.xml       |   62 +
 .../services/HIVE/configuration/hcat-env.xml    |   41 +
 .../hive-atlas-application.properties.xml       |   67 +
 .../services/HIVE/configuration/hive-env.xml    |  540 ++++
 .../HIVE/configuration/hive-exec-log4j.xml      |   96 +
 .../HIVE/configuration/hive-exec-log4j2.xml     |   83 +
 .../HIVE/configuration/hive-interactive-env.xml |  373 +++
 .../configuration/hive-interactive-site.xml     |  909 ++++++
 .../services/HIVE/configuration/hive-log4j.xml  |  106 +
 .../services/HIVE/configuration/hive-log4j2.xml |   90 +
 .../services/HIVE/configuration/hive-site.xml   | 2796 ++++++++++++++++++
 .../HIVE/configuration/hivemetastore-site.xml   |   43 +
 .../hiveserver2-interactive-site.xml            |   56 +
 .../HIVE/configuration/hiveserver2-site.xml     |  122 +
 .../HIVE/configuration/llap-cli-log4j2.xml      |   91 +
 .../HIVE/configuration/llap-daemon-log4j.xml    |  158 +
 .../HIVE/configuration/ranger-hive-audit.xml    |  136 +
 .../ranger-hive-plugin-properties.xml           |   63 +
 .../configuration/ranger-hive-policymgr-ssl.xml |   71 +
 .../HIVE/configuration/ranger-hive-security.xml |   81 +
 .../HIVE/configuration/tez-interactive-site.xml |  144 +
 .../services/HIVE/configuration/webhcat-env.xml |   38 +
 .../HIVE/configuration/webhcat-log4j.xml        |   63 +
 .../HIVE/configuration/webhcat-site.xml         |  287 ++
 .../HIVE/etc/hive-schema-0.12.0.mysql.sql       |  777 +++++
 .../HIVE/etc/hive-schema-0.12.0.oracle.sql      |  718 +++++
 .../HIVE/etc/hive-schema-0.12.0.postgres.sql    | 1406 +++++++++
 .../HIVE/etc/hive-schema-0.13.0.mysql.sql       |  889 ++++++
 .../HIVE/etc/hive-schema-0.13.0.oracle.sql      |  835 ++++++
 .../HIVE/etc/hive-schema-0.13.0.postgres.sql    | 1538 ++++++++++
 .../etc/upgrade-0.12.0-to-0.13.0.oracle.sql     |  165 ++
 .../services/HIVE/etc/upgrade-0.13.0.oracle.sql |   38 +
 .../ambari/ODPi/1.0/services/HIVE/kerberos.json |  149 +
 .../ambari/ODPi/1.0/services/HIVE/metainfo.xml  |  518 ++++
 .../alert_hive_interactive_thrift_port.py       |  216 ++
 .../HIVE/package/alerts/alert_hive_metastore.py |  270 ++
 .../package/alerts/alert_hive_thrift_port.py    |  274 ++
 .../package/alerts/alert_llap_app_status.py     |  299 ++
 .../HIVE/package/alerts/alert_webhcat_server.py |  228 ++
 .../package/etc/hive-schema-0.12.0.mysql.sql    |  777 +++++
 .../package/etc/hive-schema-0.12.0.oracle.sql   |  718 +++++
 .../package/etc/hive-schema-0.12.0.postgres.sql | 1406 +++++++++
 .../services/HIVE/package/files/addMysqlUser.sh |   39 +
 .../services/HIVE/package/files/hcatSmoke.sh    |   41 +
 .../services/HIVE/package/files/hiveSmoke.sh    |   24 +
 .../HIVE/package/files/hiveTezSetup.cmd         |   58 +
 .../services/HIVE/package/files/hiveserver2.sql |   23 +
 .../HIVE/package/files/hiveserver2Smoke.sh      |   32 +
 .../1.0/services/HIVE/package/files/pigSmoke.sh |   18 +
 .../HIVE/package/files/removeMysqlUser.sh       |   33 +
 .../HIVE/package/files/startMetastore.sh        |   25 +
 .../HIVE/package/files/templetonSmoke.sh        |   95 +
 .../services/HIVE/package/scripts/__init__.py   |   19 +
 .../1.0/services/HIVE/package/scripts/hcat.py   |   81 +
 .../HIVE/package/scripts/hcat_client.py         |   85 +
 .../HIVE/package/scripts/hcat_service_check.py  |   86 +
 .../1.0/services/HIVE/package/scripts/hive.py   |  481 +++
 .../HIVE/package/scripts/hive_client.py         |   68 +
 .../HIVE/package/scripts/hive_interactive.py    |  302 ++
 .../HIVE/package/scripts/hive_metastore.py      |  259 ++
 .../HIVE/package/scripts/hive_server.py         |  211 ++
 .../package/scripts/hive_server_interactive.py  |  535 ++++
 .../HIVE/package/scripts/hive_server_upgrade.py |  141 +
 .../HIVE/package/scripts/hive_service.py        |  187 ++
 .../package/scripts/hive_service_interactive.py |  109 +
 .../HIVE/package/scripts/mysql_server.py        |   64 +
 .../HIVE/package/scripts/mysql_service.py       |   49 +
 .../HIVE/package/scripts/mysql_users.py         |   70 +
 .../HIVE/package/scripts/mysql_utils.py         |   35 +
 .../1.0/services/HIVE/package/scripts/params.py |   29 +
 .../HIVE/package/scripts/params_linux.py        |  736 +++++
 .../HIVE/package/scripts/params_windows.py      |   74 +
 .../HIVE/package/scripts/service_check.py       |  190 ++
 .../HIVE/package/scripts/setup_ranger_hive.py   |   98 +
 .../scripts/setup_ranger_hive_interactive.py    |   78 +
 .../HIVE/package/scripts/status_params.py       |  118 +
 .../services/HIVE/package/scripts/webhcat.py    |  145 +
 .../HIVE/package/scripts/webhcat_server.py      |  164 +
 .../HIVE/package/scripts/webhcat_service.py     |   96 +
 .../package/scripts/webhcat_service_check.py    |  128 +
 .../hadoop-metrics2-hivemetastore.properties.j2 |   54 +
 .../hadoop-metrics2-hiveserver2.properties.j2   |   54 +
 .../templates/hadoop-metrics2-llapdaemon.j2     |   52 +
 .../hadoop-metrics2-llaptaskscheduler.j2        |   52 +
 .../HIVE/package/templates/hive.conf.j2         |   35 +
 .../package/templates/startHiveserver2.sh.j2    |   24 +
 .../templates/startHiveserver2Interactive.sh.j2 |   24 +
 .../package/templates/templeton_smoke.pig.j2    |   24 +
 89 files changed, 23314 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/alerts.json
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/alerts.json b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/alerts.json
new file mode 100755
index 0000000..e2431c3
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/alerts.json
@@ -0,0 +1,232 @@
+{
+  "HIVE": {
+    "service": [],
+    "HIVE_METASTORE": [
+      {
+        "name": "hive_metastore_process",
+        "label": "Hive Metastore Process",
+        "description": "This host-level alert is triggered if the Hive Metastore process cannot be determined to be up and listening on the network.",
+        "interval": 3,
+        "scope": "ANY",
+        "enabled": true,
+        "source": {
+          "type": "SCRIPT",
+          "path": "HIVE/0.12.0.2.0/package/alerts/alert_hive_metastore.py",
+          "parameters": [
+            {
+              "name": "check.command.timeout",
+              "display_name": "Command Timeout",
+              "value": 60.0,
+              "type": "NUMERIC",
+              "description": "The maximum time before check command will be killed by timeout",
+              "units": "seconds",
+              "threshold": "CRITICAL"
+            },
+            {
+              "name": "default.smoke.user",
+              "display_name": "Default Smoke User",
+              "value": "ambari-qa",
+              "type": "STRING",
+              "description": "The user that will run the Hive commands if not specified in cluster-env/smokeuser",
+              "visibility": "HIDDEN"
+            },
+            {
+              "name": "default.smoke.principal",
+              "display_name": "Default Smoke Principal",
+              "value": "ambari-qa@EXAMPLE.COM",
+              "type": "STRING",
+              "description": "The principal to use when retrieving the kerberos ticket if not specified in cluster-env/smokeuser_principal_name",
+              "visibility": "HIDDEN"
+            },
+            {
+              "name": "default.smoke.keytab",
+              "display_name": "Default Smoke Keytab",
+              "value": "/etc/security/keytabs/smokeuser.headless.keytab",
+              "type": "STRING",
+              "description": "The keytab to use when retrieving the kerberos ticket if not specified in cluster-env/smokeuser_keytab",
+              "visibility": "HIDDEN"
+            }
+          ]
+        }
+      }
+    ],
+    "HIVE_SERVER": [
+      {
+        "name": "hive_server_process",
+        "label": "HiveServer2 Process",
+        "description": "This host-level alert is triggered if the HiveServer cannot be determined to be up and responding to client requests.",
+        "interval": 3,
+        "scope": "ANY",
+        "enabled": true,
+        "source": {
+          "type": "SCRIPT",
+          "path": "HIVE/0.12.0.2.0/package/alerts/alert_hive_thrift_port.py",
+          "parameters": [
+            {
+              "name": "check.command.timeout",
+              "display_name": "Command Timeout",
+              "value": 60.0,
+              "type": "NUMERIC",
+              "description": "The maximum time before check command will be killed by timeout",
+              "units": "seconds",
+              "threshold": "CRITICAL"
+            },
+            {
+              "name": "default.smoke.user",
+              "display_name": "Default Smoke User",
+              "value": "ambari-qa",
+              "type": "STRING",
+              "description": "The user that will run the Hive commands if not specified in cluster-env/smokeuser",
+              "visibility": "HIDDEN"
+            },
+            {
+              "name": "default.smoke.principal",
+              "display_name": "Default Smoke Principal",
+              "value": "ambari-qa@EXAMPLE.COM",
+              "type": "STRING",
+              "description": "The principal to use when retrieving the kerberos ticket if not specified in cluster-env/smokeuser_principal_name",
+              "visibility": "HIDDEN"
+            },
+            {
+              "name": "default.smoke.keytab",
+              "display_name": "Default Smoke Keytab",
+              "value": "/etc/security/keytabs/smokeuser.headless.keytab",
+              "type": "STRING",
+              "description": "The keytab to use when retrieving the kerberos ticket if not specified in cluster-env/smokeuser_keytab",
+              "visibility": "HIDDEN"
+            }
+          ]
+        }
+      }
+    ],
+    "HIVE_SERVER_INTERACTIVE": [
+      {
+        "name": "hive_server_interactive_process",
+        "label": "HiveServer2 Interactive Process",
+        "description": "This host-level alert is triggered if the HiveServerInteractive cannot be determined to be up and responding to client requests.",
+        "interval": 3,
+        "scope": "ANY",
+        "enabled": true,
+        "source": {
+          "type": "SCRIPT",
+          "path": "HIVE/0.12.0.2.0/package/alerts/alert_hive_interactive_thrift_port.py",
+          "parameters": [
+            {
+              "name": "check.command.timeout",
+              "display_name": "Command Timeout",
+              "value": 60.0,
+              "type": "NUMERIC",
+              "description": "The maximum time before check command will be killed by timeout",
+              "units": "seconds",
+              "threshold": "CRITICAL"
+            },
+            {
+              "name": "default.smoke.user",
+              "display_name": "Default Smoke User",
+              "value": "ambari-qa",
+              "type": "STRING",
+              "description": "The user that will run the Hive commands if not specified in cluster-env/smokeuser",
+              "visibility": "HIDDEN"
+            },
+            {
+              "name": "default.smoke.principal",
+              "display_name": "Default Smoke Principal",
+              "value": "ambari-qa@EXAMPLE.COM",
+              "type": "STRING",
+              "description": "The principal to use when retrieving the kerberos ticket if not specified in cluster-env/smokeuser_principal_name",
+              "visibility": "HIDDEN"
+            },
+            {
+              "name": "default.smoke.keytab",
+              "display_name": "Default Smoke Keytab",
+              "value": "/etc/security/keytabs/smokeuser.headless.keytab",
+              "type": "STRING",
+              "description": "The keytab to use when retrieving the kerberos ticket if not specified in cluster-env/smokeuser_keytab",
+              "visibility": "HIDDEN"
+            }
+          ]
+        }
+      },
+      {
+        "name": "llap_application",
+        "label": "LLAP Application",
+        "description": "This alert is triggered if the LLAP Application cannot be determined to be up and responding to requests.",
+        "interval": 3,
+        "scope": "ANY",
+        "enabled": true,
+        "source": {
+          "type": "SCRIPT",
+          "path": "HIVE/0.12.0.2.0/package/alerts/alert_llap_app_status.py",
+          "parameters": [
+            {
+              "name": "check.command.timeout",
+              "display_name": "Command Timeout",
+              "value": 120.0,
+              "type": "NUMERIC",
+              "description": "The maximum time before check command will be killed by timeout",
+              "units": "seconds",
+              "threshold": "CRITICAL"
+            },
+            {
+              "name": "default.hive.user",
+              "display_name": "Default HIVE User",
+              "value": "hive",
+              "type": "STRING",
+              "description": "The user that will run the Hive commands if not specified in cluster-env",
+              "visibility": "HIDDEN"
+            },
+            {
+              "name": "default.hive.principal",
+              "display_name": "Default HIVE Principal",
+              "value": "hive/_HOST@EXAMPLE.COM",
+              "type": "STRING",
+              "description": "The principal to use when retrieving the kerberos ticket if not specified in cluster-env",
+              "visibility": "HIDDEN"
+            },
+            {
+              "name": "default.hive.keytab",
+              "display_name": "Default HIVE Keytab",
+              "value": "/etc/security/keytabs/hive.llap.zk.sm.keytab",
+              "type": "STRING",
+              "description": "The keytab to use when retrieving the kerberos ticket if not specified in cluster-env.",
+              "visibility": "HIDDEN"
+            }
+          ]
+        }
+      }
+    ],
+    "WEBHCAT_SERVER": [
+      {
+        "name": "hive_webhcat_server_status",
+        "label": "WebHCat Server Status",
+        "description": "This host-level alert is triggered if the templeton server status is not healthy.",
+        "interval": 1,
+        "scope": "ANY",
+        "enabled": true,
+        "source": {
+          "type": "SCRIPT",
+          "path": "HIVE/0.12.0.2.0/package/alerts/alert_webhcat_server.py",
+          "parameters": [
+            {
+              "name": "default.smoke.user",
+              "display_name": "Default Smoke User",
+              "value": "ambari-qa",
+              "type": "STRING",
+              "description": "The user that will run the Hive commands if not specified in cluster-env/smokeuser",
+              "visibility": "HIDDEN"
+            },
+            {
+              "name": "connection.timeout",
+              "display_name": "Connection Timeout",
+              "value": 5.0,
+              "type": "NUMERIC",
+              "description": "The maximum time before this alert is considered to be CRITICAL",
+              "units": "seconds",
+              "threshold": "CRITICAL"
+            }
+          ]
+        }
+      }    
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/beeline-log4j2.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/beeline-log4j2.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/beeline-log4j2.xml
new file mode 100755
index 0000000..03de64e
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/beeline-log4j2.xml
@@ -0,0 +1,62 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+# 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.
+
+status = INFO
+name = BeelineLog4j2
+packages = org.apache.hadoop.hive.ql.log
+
+# list of properties
+property.hive.log.level = WARN
+property.hive.root.logger = console
+
+# list of all appenders
+appenders = console
+
+# console appender
+appender.console.type = Console
+appender.console.name = console
+appender.console.target = SYSTEM_ERR
+appender.console.layout.type = PatternLayout
+appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n
+
+# list of all loggers
+loggers = HiveConnection
+
+# HiveConnection logs useful info for dynamic service discovery
+logger.HiveConnection.name = org.apache.hive.jdbc.HiveConnection
+logger.HiveConnection.level = INFO
+
+# root logger
+rootLogger.level = ${sys:hive.log.level}
+rootLogger.appenderRefs = root
+rootLogger.appenderRef.root.ref = ${sys:hive.root.logger}
+  </value>
+    <description>Custom beeline-log4j2.properties</description>
+    <display-name>beeline-log4j template</display-name>
+    <filename>beeline-log4j2.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+        <show-property-name>false</show-property-name>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hcat-env.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hcat-env.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hcat-env.xml
new file mode 100755
index 0000000..3908d61
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hcat-env.xml
@@ -0,0 +1,41 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+      # 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.
+
+      JAVA_HOME={{java64_home}}
+      HCAT_PID_DIR={{hcat_pid_dir}}/
+      HCAT_LOG_DIR={{hcat_log_dir}}/
+      HCAT_CONF_DIR={{hcat_conf_dir}}
+      HADOOP_HOME=${HADOOP_HOME:-{{hadoop_home}}}
+      #DBROOT is the path where the connector jars are downloaded
+      DBROOT={{hcat_dbroot}}
+      USER={{hcat_user}}
+      METASTORE_PORT={{hive_metastore_port}}
+    </value>
+    <description>This is the jinja template for hcat-env.sh file</description>
+    <display-name>hcat-env template</display-name>
+    <filename>hcat-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-atlas-application.properties.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-atlas-application.properties.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-atlas-application.properties.xml
new file mode 100755
index 0000000..7eb72ef
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-atlas-application.properties.xml
@@ -0,0 +1,67 @@
+<configuration><property require-input="false">
+    <name>atlas.hook.hive.synchronous</name>
+    <value>false</value>
+    <description></description>
+    <filename>hive-atlas-application.properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>atlas.hook.hive.numRetries</name>
+    <value>3</value>
+    <description></description>
+    <filename>hive-atlas-application.properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>atlas.hook.hive.minThreads</name>
+    <value>5</value>
+    <description></description>
+    <filename>hive-atlas-application.properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>atlas.hook.hive.maxThreads</name>
+    <value>5</value>
+    <description></description>
+    <filename>hive-atlas-application.properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>atlas.hook.hive.keepAliveTime</name>
+    <value>10</value>
+    <description></description>
+    <filename>hive-atlas-application.properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>atlas.hook.hive.queueSize</name>
+    <value>1000</value>
+    <description></description>
+    <filename>hive-atlas-application.properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-env.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-env.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-env.xml
new file mode 100755
index 0000000..e5ed319
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-env.xml
@@ -0,0 +1,540 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+      export HADOOP_USER_CLASSPATH_FIRST=true  #this prevents old metrics libs from mapreduce lib from bringing in old jar deps overriding HIVE_LIB
+      if [ "$SERVICE" = "cli" ]; then
+      if [ -z "$DEBUG" ]; then
+      export HADOOP_OPTS="$HADOOP_OPTS -XX:NewRatio=12 -XX:MaxHeapFreeRatio=40 -XX:MinHeapFreeRatio=15 -XX:+UseNUMA -XX:+UseParallelGC -XX:-UseGCOverheadLimit"
+      else
+      export HADOOP_OPTS="$HADOOP_OPTS -XX:NewRatio=12 -XX:MaxHeapFreeRatio=40 -XX:MinHeapFreeRatio=15 -XX:-UseGCOverheadLimit"
+      fi
+      fi
+
+      # The heap size of the jvm stared by hive shell script can be controlled via:
+
+      if [ "$SERVICE" = "metastore" ]; then
+      export HADOOP_HEAPSIZE={{hive_metastore_heapsize}} # Setting for HiveMetastore
+      else
+      export HADOOP_HEAPSIZE={{hive_heapsize}} # Setting for HiveServer2 and Client
+      fi
+
+      export HADOOP_CLIENT_OPTS="$HADOOP_CLIENT_OPTS  -Xmx${HADOOP_HEAPSIZE}m"
+
+      # Larger heap size may be required when running queries over large number of files or partitions.
+      # By default hive shell scripts use a heap size of 256 (MB).  Larger heap size would also be
+      # appropriate for hive server (hwi etc).
+
+
+      # Set HADOOP_HOME to point to a specific hadoop install directory
+      HADOOP_HOME=${HADOOP_HOME:-{{hadoop_home}}}
+
+      export HIVE_HOME=${HIVE_HOME:-{{hive_home_dir}}}
+
+      # Hive Configuration Directory can be controlled by:
+      export HIVE_CONF_DIR=${HIVE_CONF_DIR:-{{hive_config_dir}}}
+
+      # Folder containing extra libraries required for hive compilation/execution can be controlled by:
+      if [ "${HIVE_AUX_JARS_PATH}" != "" ]; then
+      if [ -f "${HIVE_AUX_JARS_PATH}" ]; then
+      export HIVE_AUX_JARS_PATH=${HIVE_AUX_JARS_PATH}
+      elif [ -d "/usr/hdp/current/hive-webhcat/share/hcatalog" ]; then
+      export HIVE_AUX_JARS_PATH=/usr/hdp/current/hive-webhcat/share/hcatalog/hive-hcatalog-core.jar
+      fi
+      elif [ -d "/usr/hdp/current/hive-webhcat/share/hcatalog" ]; then
+      export HIVE_AUX_JARS_PATH=/usr/hdp/current/hive-webhcat/share/hcatalog/hive-hcatalog-core.jar
+      fi
+
+      export METASTORE_PORT={{hive_metastore_port}}
+
+      {% if sqla_db_used or lib_dir_available %}
+      export LD_LIBRARY_PATH="$LD_LIBRARY_PATH:{{jdbc_libs_dir}}"
+      export JAVA_LIBRARY_PATH="$JAVA_LIBRARY_PATH:{{jdbc_libs_dir}}"
+      {% endif %}
+        </value>
+    <description>This is the jinja template for hive-env.sh file</description>
+    <display-name>hive-env template</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.heapsize</name>
+    <value>512</value>
+    <description>Hive Java heap size</description>
+    <display-name>HiveServer2 Heap Size</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <maximum>2048</maximum>
+        <minimum>512</minimum>
+        <unit>MB</unit>
+        <overridable>false</overridable>
+        <increment-step>512</increment-step>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive_exec_orc_storage_strategy</name>
+    <value>SPEED</value>
+    <description>
+      Define the encoding strategy to use while writing data. Changing this will only affect the light weight encoding for integers.
+      This flag will not change the compression level of higher level compression codec (like ZLIB). Possible options are SPEED and COMPRESSION.
+    </description>
+    <display-name>ORC Storage Strategy</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>SPEED</value>
+                <label>Speed</label>
+            </entry>
+            <entry>
+                <value>COMPRESSION</value>
+                <label>Compression</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.exec.orc.encoding.strategy</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.exec.orc.compression.strategy</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive_txn_acid</name>
+    <value>off</value>
+    <display-name>ACID Transactions</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>on</value>
+                <label>On</label>
+            </entry>
+            <entry>
+                <value>off</value>
+                <label>Off</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.enforce.bucketing</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.exec.dynamic.partition.mode</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.support.concurrency</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.txn.manager</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.compactor.initiator.on</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.compactor.worker.threads</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive_security_authorization</name>
+    <value>None</value>
+    <description>
+      Authorization mode, default NONE. Options are NONE, Ranger, SQLStdAuth.
+      SQL standard authorization provides grant/revoke functionality at database, table level. 
+      Ranger provides a centralized authorization interface for Hive and provides more granular
+      access control at column level through the Hive plugin.
+    </description>
+    <display-name>Choose Authorization</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>None</value>
+                <label>None</label>
+            </entry>
+            <entry>
+                <value>SQLStdAuth</value>
+                <label>SQLStdAuth</label>
+            </entry>
+            <entry>
+                <value>Ranger</value>
+                <label>Ranger</label>
+            </entry>
+        </entries>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>ranger-hive-plugin-enabled</name>
+            <type>ranger-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.server2.enable.doAs</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.security.authenticator.manager</name>
+            <type>hiveserver2-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.server2.enable.doAs</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.security.authorization.enabled</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.conf.restricted.list</name>
+            <type>hiveserver2-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.security.authenticator.manager</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.security.authorization.manager</name>
+            <type>hiveserver2-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.security.authorization.manager</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.security.authorization.enabled</name>
+            <type>hiveserver2-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.security.metastore.authorization.manager</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive_timeline_logging_enabled</name>
+    <value>true</value>
+    <display-name>Use ATS Logging</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.exec.pre.hooks</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.exec.post.hooks</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.exec.failure.hooks</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive.client.heapsize</name>
+    <value>512</value>
+    <description>Hive Client Java heap size</description>
+    <display-name>Client Heap Size</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <maximum>2048</maximum>
+        <minimum>512</minimum>
+        <unit>MB</unit>
+        <overridable>false</overridable>
+        <increment-step>512</increment-step>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.heapsize</name>
+    <value>1024</value>
+    <description>Hive Metastore Java heap size</description>
+    <display-name>Metastore Heap Size</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <maximum>2048</maximum>
+        <minimum>512</minimum>
+        <unit>MB</unit>
+        <increment-step>512</increment-step>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive_database_type</name>
+    <value>mysql</value>
+    <description>Default HIVE DB type.</description>
+    <display-name>Hive Database Type</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_database</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive_database</name>
+    <value>New MySQL Database</value>
+    <description>
+      Property that determines whether the HIVE DB is managed by Ambari.
+    </description>
+    <display-name>Hive Database</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive_database_type</name>
+            <type>hive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>javax.jdo.option.ConnectionURL</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>javax.jdo.option.ConnectionDriverName</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>datanucleus.rdbms.datastoreAdapterClassName</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive_ambari_database</name>
+    <value>MySQL</value>
+    <description>Database type.</description>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive_database_name</name>
+    <value>hive</value>
+    <description>Database name.</description>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>database</type>
+        <visible>false</visible>
+        <overridable>false</overridable>
+        <editable-only-at-install>true</editable-only-at-install>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive_log_dir</name>
+    <value>/var/log/hive</value>
+    <description>Directory for Hive Log files.</description>
+    <display-name>Hive Log Dir</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>directory</type>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive_pid_dir</name>
+    <value>/var/run/hive</value>
+    <description>Hive PID Dir.</description>
+    <display-name>Hive PID Dir</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>directory</type>
+        <overridable>false</overridable>
+        <editable-only-at-install>true</editable-only-at-install>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive_user</name>
+    <value>hive</value>
+    <description>Hive User.</description>
+    <display-name>Hive User</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type>USER</property-type>
+    <value-attributes>
+        <type>user</type>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>ranger.plugins.hive.serviceuser</name>
+            <type>ranger-admin-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>ranger.kms.service.user.hive</name>
+            <type>ranger-admin-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hcat_log_dir</name>
+    <value>/var/log/webhcat</value>
+    <description>WebHCat Log Dir.</description>
+    <display-name>WebHCat Log Dir</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>directory</type>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hcat_pid_dir</name>
+    <value>/var/run/webhcat</value>
+    <description>WebHCat Pid Dir.</description>
+    <display-name>WebHCat Pid Dir</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>directory</type>
+        <overridable>false</overridable>
+        <editable-only-at-install>true</editable-only-at-install>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hcat_user</name>
+    <value>hcat</value>
+    <description>HCat User.</description>
+    <display-name>HCat User</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type>USER</property-type>
+    <value-attributes>
+        <type>user</type>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>webhcat_user</name>
+    <value>hcat</value>
+    <description>WebHCat User.</description>
+    <display-name>WebHCat User</display-name>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type>USER</property-type>
+    <value-attributes>
+        <type>user</type>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive_user_nofile_limit</name>
+    <value>32000</value>
+    <description>Max open files limit setting for HIVE user.</description>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive_user_nproc_limit</name>
+    <value>16000</value>
+    <description>Max number of processes limit setting for HIVE user.</description>
+    <filename>hive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j.xml
new file mode 100755
index 0000000..538334c
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j.xml
@@ -0,0 +1,96 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+# 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.
+
+# Define some default values that can be overridden by system properties
+
+hive.log.threshold=ALL
+hive.root.logger=INFO,FA
+hive.log.dir=${java.io.tmpdir}/${user.name}
+hive.query.id=hadoop
+hive.log.file=${hive.query.id}.log
+
+# Define the root logger to the system property "hadoop.root.logger".
+log4j.rootLogger=${hive.root.logger}, EventCounter
+
+# Logging Threshold
+log4j.threshhold=${hive.log.threshold}
+
+#
+# File Appender
+#
+
+log4j.appender.FA=org.apache.log4j.FileAppender
+log4j.appender.FA.File=${hive.log.dir}/${hive.log.file}
+log4j.appender.FA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+log4j.appender.FA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this
+#
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
+
+#custom logging levels
+#log4j.logger.xxx=DEBUG
+
+#
+# Event Counter Appender
+# Sends counts of logging messages at different severity levels to Hadoop Metrics.
+#
+log4j.appender.EventCounter=org.apache.hadoop.hive.shims.HiveEventCounter
+
+
+log4j.category.DataNucleus=ERROR,FA
+log4j.category.Datastore=ERROR,FA
+log4j.category.Datastore.Schema=ERROR,FA
+log4j.category.JPOX.Datastore=ERROR,FA
+log4j.category.JPOX.Plugin=ERROR,FA
+log4j.category.JPOX.MetaData=ERROR,FA
+log4j.category.JPOX.Query=ERROR,FA
+log4j.category.JPOX.General=ERROR,FA
+log4j.category.JPOX.Enhancer=ERROR,FA
+
+
+# Silence useless ZK logs
+log4j.logger.org.apache.zookeeper.server.NIOServerCnxn=WARN,FA
+log4j.logger.org.apache.zookeeper.ClientCnxnSocketNIO=WARN,FA
+
+    </value>
+    <description>Custom hive-exec-log4j</description>
+    <display-name>hive-exec-log4j template</display-name>
+    <filename>hive-exec-log4j.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+        <show-property-name>false</show-property-name>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j2.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j2.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j2.xml
new file mode 100755
index 0000000..c818d43
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j2.xml
@@ -0,0 +1,83 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+# 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.
+
+status = INFO
+name = HiveExecLog4j2
+packages = org.apache.hadoop.hive.ql.log
+
+# list of properties
+property.hive.log.level = INFO
+property.hive.root.logger = FA
+property.hive.query.id = hadoop
+property.hive.log.dir = ${sys:java.io.tmpdir}/${sys:user.name}
+property.hive.log.file = ${sys:hive.query.id}.log
+
+# list of all appenders
+appenders = console, FA
+
+# console appender
+appender.console.type = Console
+appender.console.name = console
+appender.console.target = SYSTEM_ERR
+appender.console.layout.type = PatternLayout
+appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n
+
+# simple file appender
+appender.FA.type = File
+appender.FA.name = FA
+appender.FA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
+appender.FA.layout.type = PatternLayout
+appender.FA.layout.pattern = %d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n
+
+# list of all loggers
+loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX
+
+logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn
+logger.NIOServerCnxn.level = WARN
+
+logger.ClientCnxnSocketNIO.name = org.apache.zookeeper.ClientCnxnSocketNIO
+logger.ClientCnxnSocketNIO.level = WARN
+
+logger.DataNucleus.name = DataNucleus
+logger.DataNucleus.level = ERROR
+
+logger.Datastore.name = Datastore
+logger.Datastore.level = ERROR
+
+logger.JPOX.name = JPOX
+logger.JPOX.level = ERROR
+
+# root logger
+rootLogger.level = ${sys:hive.log.level}
+rootLogger.appenderRefs = root
+rootLogger.appenderRef.root.ref = ${sys:hive.root.logger}
+  </value>
+    <description>Custom hive-exec-log4j2.properties</description>
+    <display-name>hive-exec-log4j2 template</display-name>
+    <filename>hive-exec-log4j2.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+        <show-property-name>false</show-property-name>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-env.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-env.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-env.xml
new file mode 100755
index 0000000..7035283
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-env.xml
@@ -0,0 +1,373 @@
+<configuration><property require-input="false">
+    <name>enable_hive_interactive</name>
+    <value>false</value>
+    <description>Enable or disable interactive query in this cluster.</description>
+    <display-name>Enable Interactive Query (Tech Preview)</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <overridable>false</overridable>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>Yes</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>No</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.llap.daemon.num.executors</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>llap_queue_capacity</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>llap_heap_size</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>tez.am.resource.memory.mb</name>
+            <type>tez-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.llap.daemon.yarn.container.mb</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>capacity-scheduler</name>
+            <type>capacity-scheduler</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.server2.tez.sessions.per.default.queue</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.llap.io.memory.size</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>num_llap_nodes</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>slider_am_container_mb</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive_server_interactive_host</name>
+    <value>localhost</value>
+    <description>HiveServer2 Interactive Host</description>
+    <display-name>HiveServer2 Interactive Host</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>llap_queue_capacity</name>
+    <value>0</value>
+    <description>Percentage of the cluster dedicated to interactive query.</description>
+    <display-name>% of Cluster Capacity</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <maximum>100</maximum>
+        <minimum>20</minimum>
+        <increment-step>1</increment-step>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </property>
+        <property>
+            <name>enable_hive_interactive</name>
+            <type>hive-interactive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.llap.daemon.num.executors</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>llap_heap_size</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>tez.am.resource.memory.mb</name>
+            <type>tez-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.llap.daemon.yarn.container.mb</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>capacity-scheduler</name>
+            <type>capacity-scheduler</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.server2.tez.sessions.per.default.queue</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.llap.io.memory.size</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>num_llap_nodes</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>slider_am_container_mb</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>num_llap_nodes</name>
+    <value>1</value>
+    <description>The number of Hive LLAP daemons to run.</description>
+    <display-name>Number of LLAP Daemons</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <minimum>1</minimum>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </property>
+        <property>
+            <name>llap_queue_capacity</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>enable_hive_interactive</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>hive.server2.tez.sessions.per.default.queue</name>
+            <type>hive-interactive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>num_retries_for_checking_llap_status</name>
+    <value>10</value>
+    <description>After starting LLAP app, retry count to check LLAP status before starting HiveServer2.</description>
+    <display-name>Number of retries while checking LLAP app status</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <maximum>20</maximum>
+        <minimum>0</minimum>
+        <increment-step>1</increment-step>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>llap_heap_size</name>
+    <value>0</value>
+    <description>Heap Size used by LLAP app.</description>
+    <display-name>LLAP heap size</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>tez.am.resource.memory.mb</name>
+            <type>tez-site</type>
+        </property>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </property>
+        <property>
+            <name>llap_queue_capacity</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>enable_hive_interactive</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>hive.server2.tez.sessions.per.default.queue</name>
+            <type>hive-interactive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>slider_am_container_mb</name>
+    <value>341</value>
+    <description>Slider's app master container size in MB.</description>
+    <display-name>Slider AM container size</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <unit>MB</unit>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </property>
+        <property>
+            <name>llap_queue_capacity</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>enable_hive_interactive</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>hive.server2.tez.sessions.per.default.queue</name>
+            <type>hive-interactive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>llap_log_level</name>
+    <value>INFO</value>
+    <description>LLAP app logging level (WARN/INFO/DEBUG/TRACE)</description>
+    <display-name>LLAP app logging level (WARN/INFO/DEBUG/TRACE)</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>llap_app_name</name>
+    <value>llap0</value>
+    <description>LLAP app name</description>
+    <display-name>LLAP app name</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>llap_java_opts</name>
+    <value>-XX:+AlwaysPreTouch {% if java_version &gt; 7 %}-XX:+UseG1GC -XX:TLABSize=8m -XX:+ResizeTLAB -XX:+UseNUMA -XX:+AggressiveOpts -XX:MetaspaceSize=1024m -XX:InitiatingHeapOccupancyPercent=80 -XX:MaxGCPauseMillis=200{% else %}-XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC{% endif %}</value>
+    <description>Java opts for llap application</description>
+    <display-name>LLAP app java opts</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>content</name>
+    <value>
+      if [ "$SERVICE" = "cli" ]; then
+      if [ -z "$DEBUG" ]; then
+      export HADOOP_OPTS="$HADOOP_OPTS -XX:NewRatio=12 -XX:MaxHeapFreeRatio=40 -XX:MinHeapFreeRatio=15 -XX:+UseParNewGC -XX:-UseGCOverheadLimit"
+      else
+      export HADOOP_OPTS="$HADOOP_OPTS -XX:NewRatio=12 -XX:MaxHeapFreeRatio=40 -XX:MinHeapFreeRatio=15 -XX:-UseGCOverheadLimit"
+      fi
+      fi
+
+      # The heap size of the jvm stared by hive shell script can be controlled via:
+
+      if [ "$SERVICE" = "metastore" ]; then
+      export HADOOP_HEAPSIZE={{hive_metastore_heapsize}} # Setting for HiveMetastore
+      else
+      export HADOOP_HEAPSIZE={{hive_heapsize}} # Setting for HiveServer2 and Client
+      fi
+
+      export HADOOP_CLIENT_OPTS="$HADOOP_CLIENT_OPTS  -Xmx${HADOOP_HEAPSIZE}m"
+
+      # Larger heap size may be required when running queries over large number of files or partitions.
+      # By default hive shell scripts use a heap size of 256 (MB).  Larger heap size would also be
+      # appropriate for hive server (hwi etc).
+
+
+      # Set HADOOP_HOME to point to a specific hadoop install directory
+      HADOOP_HOME=${HADOOP_HOME:-{{hadoop_home}}}
+
+      # Hive Configuration Directory can be controlled by:
+      export HIVE_CONF_DIR={{hive_server_interactive_conf_dir}}
+
+      # Add additional hcatalog jars
+      if [ "${HIVE_AUX_JARS_PATH}" != "" ]; then
+      export HIVE_AUX_JARS_PATH=${HIVE_AUX_JARS_PATH}
+      else
+      export HIVE_AUX_JARS_PATH=/usr/hdp/current/hive-server2-hive2/lib/hive-hcatalog-core.jar
+      fi
+
+      export METASTORE_PORT={{hive_metastore_port}}
+
+      # Spark assembly contains a conflicting copy of HiveConf from hive-1.2
+      export HIVE_SKIP_SPARK_ASSEMBLY=true
+
+    </value>
+    <description>This is the jinja template for hive-env.sh file</description>
+    <display-name>hive-interactive-env template</display-name>
+    <filename>hive-interactive-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file


[18/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-site.xml
new file mode 100755
index 0000000..cddb624
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-site.xml
@@ -0,0 +1,2796 @@
+<configuration><property require-input="false">
+    <name>hive.default.fileformat.managed</name>
+    <value>TextFile</value>
+    <description>
+      Default file format for CREATE TABLE statement applied to managed tables only. 
+      External tables will be created with default file format. Leaving this null 
+      will result in using the default file format for all tables.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>datanucleus.rdbms.datastoreAdapterClassName</name>
+    <description>Datanucleus Class, This property used only when hive db is SQL Anywhere</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive_database</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>atlas.hook.hive.minThreads</name>
+    <value>1</value>
+    <description>
+      Minimum number of threads maintained by Atlas hook.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>atlas.hook.hive.maxThreads</name>
+    <value>1</value>
+    <description>
+      Maximum number of threads used by Atlas hook.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.cbo.enable</name>
+    <value>true</value>
+    <description>Flag to control enabling Cost Based Optimizations using Calcite framework.</description>
+    <display-name>Enable Cost Based Optimizer</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>On</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>Off</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.compute.query.using.stats</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.stats.fetch.partition.stats</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.stats.fetch.column.stats</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive.zookeeper.quorum</name>
+    <value>localhost:2181</value>
+    <description>List of ZooKeeper servers to talk to. This is needed for: 1.
+      Read/write locks - when hive.lock.manager is set to
+      org.apache.hadoop.hive.ql.lockmgr.zookeeper.ZooKeeperHiveLockManager,
+      2. When HiveServer2 supports service discovery via Zookeeper.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>multiLine</type>
+        <empty-value-valid>true</empty-value-valid>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.connect.retries</name>
+    <value>24</value>
+    <description>Number of retries while opening a connection to metastore</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.failure.retries</name>
+    <value>24</value>
+    <description>Number of retries upon failure of Thrift metastore calls</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.client.connect.retry.delay</name>
+    <value>5s</value>
+    <description>
+      Expects a time value with unit (d/day, h/hour, m/min, s/sec, ms/msec, us/usec, ns/nsec), which is sec if not specified.
+      Number of seconds for the client to wait between consecutive connection attempts
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.client.socket.timeout</name>
+    <value>1800s</value>
+    <description>
+      Expects a time value with unit (d/day, h/hour, m/min, s/sec, ms/msec, us/usec, ns/nsec), which is sec if not specified.
+      MetaStore Client socket timeout in seconds
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.mapjoin.bucket.cache.size</name>
+    <value>10000</value>
+    <description></description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.security.authorization.manager</name>
+    <value>org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdConfOnlyAuthorizerFactory</value>
+    <description>
+      The Hive client authorization manager class name. The user defined authorization class should implement
+      interface org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive_security_authorization</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.cluster.delegation.token.store.class</name>
+    <value>org.apache.hadoop.hive.thrift.ZooKeeperTokenStore</value>
+    <description>The delegation token store implementation.
+      Set to org.apache.hadoop.hive.thrift.ZooKeeperTokenStore for load-balanced cluster.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.cluster.delegation.token.store.zookeeper.connectString</name>
+    <value>localhost:2181</value>
+    <description>The ZooKeeper token store connect string.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.support.dynamic.service.discovery</name>
+    <value>true</value>
+    <description>Whether HiveServer2 supports dynamic service discovery for its clients.
+      To support this, each instance of HiveServer2 currently uses ZooKeeper to register itself,
+      when it is brought up. JDBC/ODBC clients should use the ZooKeeper ensemble: hive.zookeeper.quorum
+      in their connection string.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>boolean</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.scratchdir</name>
+    <value>/tmp/hive</value>
+    <description>HDFS root scratch dir for Hive jobs which gets created with write all (733) permission. For each connecting user, an HDFS scratch dir: ${hive.exec.scratchdir}/&lt;username&gt; is created, with ${hive.scratch.dir.permission}.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.submitviachild</name>
+    <value>false</value>
+    <description></description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.submit.local.task.via.child</name>
+    <value>true</value>
+    <description>
+      Determines whether local tasks (typically mapjoin hashtable generation phase) runs in
+      separate JVM (true recommended) or not.
+      Avoids the overhead of spawning new JVM, but can lead to out-of-memory issues.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.compress.output</name>
+    <value>false</value>
+    <description>
+      This controls whether the final outputs of a query (to a local/HDFS file or a Hive table) is compressed.
+      The compression codec and other options are determined from Hadoop config variables mapred.output.compress*
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.compress.intermediate</name>
+    <value>false</value>
+    <description>
+      This controls whether intermediate files produced by Hive between multiple map-reduce jobs are compressed.
+      The compression codec and other options are determined from Hadoop config variables mapred.output.compress*
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.reducers.bytes.per.reducer</name>
+    <value>67108864</value>
+    <description>Defines the size per reducer. For example, if it is set to 64M, given 256M input size, 4 reducers will be used.</description>
+    <display-name>Data per Reducer</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <maximum>4294967296</maximum>
+        <minimum>64</minimum>
+        <unit>B</unit>
+        <increment-step></increment-step>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.reducers.max</name>
+    <value>1009</value>
+    <description>
+      max number of reducers will be used. If the one specified in the configuration parameter mapred.reduce.tasks is
+      negative, Hive will use this one as the max number of reducers when automatically determine number of reducers.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.pre.hooks</name>
+    <value>org.apache.hadoop.hive.ql.hooks.ATSHook</value>
+    <description>
+      Comma-separated list of pre-execution hooks to be invoked for each statement.
+      A pre-execution hook is specified as the name of a Java class which implements the
+      org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive_timeline_logging_enabled</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.post.hooks</name>
+    <value>org.apache.hadoop.hive.ql.hooks.ATSHook</value>
+    <description>
+      Comma-separated list of post-execution hooks to be invoked for each statement.
+      A post-execution hook is specified as the name of a Java class which implements the
+      org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>atlas.server.https.port</name>
+            <type>application-properties</type>
+        </property>
+        <property>
+            <name>atlas.server.http.port</name>
+            <type>application-properties</type>
+        </property>
+        <property>
+            <name>hive_timeline_logging_enabled</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.failure.hooks</name>
+    <value>org.apache.hadoop.hive.ql.hooks.ATSHook</value>
+    <description>
+      Comma-separated list of on-failure hooks to be invoked for each statement.
+      An on-failure hook is specified as the name of Java class which implements the
+      org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive_timeline_logging_enabled</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.parallel</name>
+    <value>false</value>
+    <description>Whether to execute jobs in parallel</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.parallel.thread.number</name>
+    <value>8</value>
+    <description>How many jobs at most can be executed in parallel</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.mapred.reduce.tasks.speculative.execution</name>
+    <value>false</value>
+    <description>Whether speculative execution for reducers should be turned on. </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.dynamic.partition</name>
+    <value>true</value>
+    <description>Whether or not to allow dynamic partitions in DML/DDL.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.dynamic.partition.mode</name>
+    <value>nonstrict</value>
+    <description>
+      In strict mode, the user must specify at least one static partition
+      in case the user accidentally overwrites all partitions.
+      NonStrict allows all partitions of a table to be dynamic.
+    </description>
+    <display-name>Allow all partitions to be Dynamic</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>nonstrict</value>
+                <label>On</label>
+            </entry>
+            <entry>
+                <value>strict</value>
+                <label>Off</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_txn_acid</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.max.dynamic.partitions</name>
+    <value>5000</value>
+    <description>Maximum number of dynamic partitions allowed to be created in total.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.max.dynamic.partitions.pernode</name>
+    <value>2000</value>
+    <description>Maximum number of dynamic partitions allowed to be created in each mapper/reducer node.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.max.created.files</name>
+    <value>100000</value>
+    <description>Maximum number of HDFS files created by all mappers/reducers in a MapReduce job.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.warehouse.dir</name>
+    <value>/apps/hive/warehouse</value>
+    <description>location of default database for the warehouse</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type>NOT_MANAGED_HDFS_PATH</property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.uris</name>
+    <value>thrift://localhost:9083</value>
+    <description>Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="true">
+    <name>javax.jdo.option.ConnectionPassword</name>
+    <value></value>
+    <description>password to use against metastore database</description>
+    <display-name>Database Password</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type>PASSWORD</property-type>
+    <value-attributes>
+        <type>password</type>
+        <overridable>false</overridable>
+        <hidden>HIVE_CLIENT,WEBHCAT_SERVER,HCAT,CONFIG_DOWNLOAD</hidden>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>javax.jdo.option.ConnectionURL</name>
+    <value>jdbc:mysql://localhost/hive?createDatabaseIfNotExist=true</value>
+    <description>JDBC connect string for a JDBC metastore</description>
+    <display-name>Database URL</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_database</name>
+            <type>hive-env</type>
+        </property>
+        <property>
+            <name>ambari.hive.db.schema.name</name>
+            <type>hive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.server.max.threads</name>
+    <value>100000</value>
+    <description>Maximum number of worker threads in the Thrift server's pool.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.kerberos.keytab.file</name>
+    <value>/etc/security/keytabs/hive.service.keytab</value>
+    <description>The path to the Kerberos Keytab file containing the metastore Thrift server's service principal.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.kerberos.principal</name>
+    <value>hive/_HOST@EXAMPLE.COM</value>
+    <description>
+      The service principal for the metastore Thrift server.
+      The special string _HOST will be replaced automatically with the correct host name.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.cluster.delegation.token.store.zookeeper.znode</name>
+    <value>/hive/cluster/delegation</value>
+    <description>The root path for token store data.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.cache.pinobjtypes</name>
+    <value>Table,Database,Type,FieldSchema,Order</value>
+    <description>List of comma separated metastore object types that should be pinned in the cache</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.pre.event.listeners</name>
+    <value>org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener</value>
+    <description>List of comma separated listeners for metastore events.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.authorization.storage.checks</name>
+    <value>false</value>
+    <description>
+      Should the metastore do authorization checks against the underlying storage (usually hdfs)
+      for operations like drop-partition (disallow the drop-partition if the user in
+      question doesn't have permissions to delete the corresponding directory
+      on the storage).
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>javax.jdo.option.ConnectionDriverName</name>
+    <value>com.mysql.jdbc.Driver</value>
+    <description>Driver class name for a JDBC metastore</description>
+    <display-name>JDBC Driver Class</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_database</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>javax.jdo.option.ConnectionUserName</name>
+    <value>hive</value>
+    <description>username to use against metastore database</description>
+    <display-name>Database Username</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>db_user</type>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.map.aggr</name>
+    <value>true</value>
+    <description>Whether to use map-side aggregation in Hive Group By queries</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.cbo.enable</name>
+    <value>true</value>
+    <description>Flag to control enabling Cost Based Optimizations using Calcite framework.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.mapjoin.optimized.hashtable</name>
+    <value>true</value>
+    <description>
+      Whether Hive should use memory-optimized hash table for MapJoin. Only works on Tez,
+      because memory-optimized hashtable cannot be serialized.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.smbjoin.cache.rows</name>
+    <value>10000</value>
+    <description>How many rows with the same key value should be cached in memory per smb joined table.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.map.aggr.hash.percentmemory</name>
+    <value>0.5</value>
+    <description>Portion of total memory to be used by map-side group aggregation hash table</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.map.aggr.hash.force.flush.memory.threshold</name>
+    <value>0.9</value>
+    <description>
+      The max memory to be used by map-side group aggregation hash table.
+      If the memory usage is higher than this number, force to flush data
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.map.aggr.hash.min.reduction</name>
+    <value>0.5</value>
+    <description>
+      Hash aggregation will be turned off if the ratio between hash  table size and input rows is bigger than this number.
+      Set to 1 to make sure hash aggregation is never turned off.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.merge.mapfiles</name>
+    <value>true</value>
+    <description>Merge small files at the end of a map-only job</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.merge.mapredfiles</name>
+    <value>false</value>
+    <description>Merge small files at the end of a map-reduce job</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.merge.tezfiles</name>
+    <value>false</value>
+    <description>Merge small files at the end of a Tez DAG</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.merge.size.per.task</name>
+    <value>256000000</value>
+    <description>Size of merged files at the end of the job</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.merge.smallfiles.avgsize</name>
+    <value>16000000</value>
+    <description>
+      When the average output file size of a job is less than this number, Hive will start an additional
+      map-reduce job to merge the output files into bigger files. This is only done for map-only jobs
+      if hive.merge.mapfiles is true, and for map-reduce jobs if hive.merge.mapredfiles is true.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.merge.rcfile.block.level</name>
+    <value>true</value>
+    <description></description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.merge.orcfile.stripe.level</name>
+    <value>true</value>
+    <description>
+      When hive.merge.mapfiles or hive.merge.mapredfiles is enabled while writing a
+      table with ORC file format, enabling this config will do stripe level fast merge
+      for small ORC files. Note that enabling this config will not honor padding tolerance
+      config (hive.exec.orc.block.padding.tolerance).
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.orc.default.stripe.size</name>
+    <value>67108864</value>
+    <description>Define the default ORC stripe size</description>
+    <display-name>Default ORC Stripe Size</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <maximum>268435456</maximum>
+        <minimum>8388608</minimum>
+        <unit>B</unit>
+        <increment-step>8388608</increment-step>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.orc.default.compress</name>
+    <value>ZLIB</value>
+    <description>Define the default compression codec for ORC file</description>
+    <display-name>ORC Compression Algorithm</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>ZLIB</value>
+                <label>zlib Compression Library</label>
+            </entry>
+            <entry>
+                <value>SNAPPY</value>
+                <label>Snappy Compression Library</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.orc.splits.include.file.footer</name>
+    <value>false</value>
+    <description>
+      If turned on splits generated by orc will include metadata about the stripes in the file. This
+      data is read remotely (from the client or HS2 machine) and sent to all the tasks.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.orc.compute.splits.num.threads</name>
+    <value>10</value>
+    <description>How many threads orc should use to create splits in parallel.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.auto.convert.join</name>
+    <value>true</value>
+    <description>Whether Hive enables the optimization about converting common join into mapjoin based on the input file size</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.auto.convert.join.noconditionaltask</name>
+    <value>true</value>
+    <description>
+      Whether Hive enables the optimization about converting common join into mapjoin based on the input file size.
+      If this parameter is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than the
+      specified size, the join is directly converted to a mapjoin (there is no conditional task).
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.limit.optimize.enable</name>
+    <value>true</value>
+    <description>Whether to enable to optimization to trying a smaller subset of data for simple LIMIT first.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.cpu.vcores</name>
+    <value>-1</value>
+    <description>By default Tez will ask for however many cpus map-reduce is configured to use per container. This can be used to overwrite.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.log.level</name>
+    <value>INFO</value>
+    <description>
+      The log level to use for tasks executing as part of the DAG.
+      Used only if hive.tez.java.opts is used to configure Java options.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.enforce.bucketing</name>
+    <value>true</value>
+    <description>Whether bucketing is enforced. If true, while inserting into the table, bucketing is enforced.</description>
+    <display-name>Enforce bucketing</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_txn_acid</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.enforce.sorting</name>
+    <value>true</value>
+    <description>Whether sorting is enforced. If true, while inserting into the table, sorting is enforced.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.enforce.sortmergebucketmapjoin</name>
+    <value>true</value>
+    <description>If the user asked for sort-merge bucketed map-side join, and it cannot be performed, should the query fail or not ?</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.auto.convert.sortmerge.join</name>
+    <value>true</value>
+    <description>Will the join be automatically converted to a sort-merge join, if the joined tables pass the criteria for sort-merge join.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.auto.convert.sortmerge.join.to.mapjoin</name>
+    <value>false</value>
+    <description>
+      If hive.auto.convert.sortmerge.join is set to true, and a join was converted to a sort-merge join,
+      this parameter decides whether each table should be tried as a big table, and effectively a map-join should be
+      tried. That would create a conditional task with n+1 children for a n-way join (1 child for each table as the
+      big table), and the backup task will be the sort-merge join. In some cases, a map-join would be faster than a
+      sort-merge join, if there is no advantage of having the output bucketed and sorted. For example, if a very big sorted
+      and bucketed table with few files (say 10 files) are being joined with a very small sorter and bucketed table
+      with few files (10 files), the sort-merge join will only use 10 mappers, and a simple map-only join might be faster
+      if the complete small table can fit in memory, and a map-join can be performed.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.optimize.constant.propagation</name>
+    <value>true</value>
+    <description>Whether to enable constant propagation optimizer</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.optimize.metadataonly</name>
+    <value>true</value>
+    <description></description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.optimize.null.scan</name>
+    <value>true</value>
+    <description>Dont scan relations which are guaranteed to not generate any rows</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.optimize.bucketmapjoin</name>
+    <value>true</value>
+    <description>Whether to try bucket mapjoin</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.optimize.reducededuplication</name>
+    <value>true</value>
+    <description>
+      Remove extra map-reduce jobs if the data is already clustered by the same key which needs to be used again.
+      This should always be set to true. Since it is a new feature, it has been made configurable.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.optimize.reducededuplication.min.reducer</name>
+    <value>4</value>
+    <description>
+      Reduce deduplication merges two RSs by moving key/parts/reducer-num of the child RS to parent RS.
+      That means if reducer-num of the child RS is fixed (order by or forced bucketing) and small, it can make very slow, single MR.
+      The optimization will be automatically disabled if number of reducers would be less than specified value.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.optimize.sort.dynamic.partition</name>
+    <value>false</value>
+    <description>
+      When enabled dynamic partitioning column will be globally sorted.
+      This way we can keep only one record writer open for each partition value
+      in the reducer thereby reducing the memory pressure on reducers.
+    </description>
+    <display-name>Sort Partitions Dynamically</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.stats.autogather</name>
+    <value>true</value>
+    <description>A flag to gather statistics automatically during the INSERT OVERWRITE command.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.stats.dbclass</name>
+    <value>fs</value>
+    <description>
+      Expects one of the pattern in [jdbc(:.*), hbase, counter, custom, fs].
+      The storage that stores temporary Hive statistics. Currently, jdbc, hbase, counter and custom type are supported.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.stats.fetch.partition.stats</name>
+    <value>true</value>
+    <description>
+      Annotation of operator tree with statistics information requires partition level basic
+      statistics like number of rows, data size and file size. Partition statistics are fetched from
+      metastore. Fetching partition statistics for each needed partition can be expensive when the
+      number of partitions is high. This flag can be used to disable fetching of partition statistics
+      from metastore. When this flag is disabled, Hive will make calls to filesystem to get file sizes
+      and will estimate the number of rows from row schema.
+    </description>
+    <display-name>Fetch partition stats at compiler</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>On</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>Off</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive.cbo.enable</name>
+            <type>hive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.stats.fetch.column.stats</name>
+    <value>false</value>
+    <description>
+      Annotation of operator tree with statistics information requires column statistics.
+      Column statistics are fetched from metastore. Fetching column statistics for each needed column
+      can be expensive when the number of columns is high. This flag can be used to disable fetching
+      of column statistics from metastore.
+    </description>
+    <display-name>Fetch column stats at compiler</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>On</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>Off</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive.cbo.enable</name>
+            <type>hive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.zookeeper.client.port</name>
+    <value>2181</value>
+    <description>The port of ZooKeeper servers to talk to. If the list of Zookeeper servers specified in hive.zookeeper.quorum,does not contain port numbers, this value is used.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.zookeeper.namespace</name>
+    <value>hive_zookeeper_namespace</value>
+    <description>The parent node under which all ZooKeeper nodes are created.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.txn.manager</name>
+    <value>org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager</value>
+    <description></description>
+    <display-name>Transaction Manager</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager</value>
+                <label>org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager (off)</label>
+            </entry>
+            <entry>
+                <value>org.apache.hadoop.hive.ql.lockmgr.DbTxnManager</value>
+                <label>org.apache.hadoop.hive.ql.lockmgr.DbTxnManager (on)</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_txn_acid</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.txn.max.open.batch</name>
+    <value>1000</value>
+    <description>
+      Maximum number of transactions that can be fetched in one call to open_txns().
+      Increasing this will decrease the number of delta files created when
+      streaming data into Hive.  But it will also increase the number of
+      open transactions at any given time, possibly impacting read performance.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.support.concurrency</name>
+    <value>false</value>
+    <description>
+      Support concurrency and use locks, needed for Transactions. Requires Zookeeper.
+    </description>
+    <display-name>Use Locking</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_txn_acid</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.cli.print.header</name>
+    <value>false</value>
+    <description>
+      Whether to print the names of the columns in query output.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.compactor.worker.timeout</name>
+    <value>86400L</value>
+    <description>
+      Expects a time value with unit (d/day, h/hour, m/min, s/sec, ms/msec, us/usec, ns/nsec), which is sec if not specified.
+      Time before a given compaction in working state is declared a failure
+      and returned to the initiated state.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.compactor.check.interval</name>
+    <value>300L</value>
+    <description>
+      Expects a time value with unit (d/day, h/hour, m/min, s/sec, ms/msec, us/usec, ns/nsec), which is sec if not specified.
+      Time between checks to see if any partitions need compacted.
+      This should be kept high because each check for compaction requires many calls against the NameNode.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.compactor.delta.pct.threshold</name>
+    <value>0.1f</value>
+    <description>Percentage (by size) of base that deltas can be before major compaction is initiated.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.fetch.task.conversion</name>
+    <value>more</value>
+    <description>
+      Expects one of [none, minimal, more].
+      Some select queries can be converted to single FETCH task minimizing latency.
+      Currently the query should be single sourced not having any subquery and should not have
+      any aggregations or distincts (which incurs RS), lateral views and joins.
+      0. none : disable hive.fetch.task.conversion
+      1. minimal : SELECT STAR, FILTER on partition columns, LIMIT only
+      2. more    : SELECT, FILTER, LIMIT only (support TABLESAMPLE and virtual columns)
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.fetch.task.conversion.threshold</name>
+    <value>1073741824</value>
+    <description>
+      Input threshold for applying hive.fetch.task.conversion. If target table is native, input length
+      is calculated by summation of file lengths. If it's not native, storage handler for the table
+      can optionally implement org.apache.hadoop.hive.ql.metadata.InputEstimator interface.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.fetch.task.aggr</name>
+    <value>false</value>
+    <description>
+      Aggregation queries with no group-by clause (for example, select count(*) from src) execute
+      final aggregations in single reduce task. If this is set true, Hive delegates final aggregation
+      stage to fetch task, possibly decreasing the query time.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.security.authorization.enabled</name>
+    <value>false</value>
+    <description>enable or disable the Hive client authorization</description>
+    <display-name>Enable Authorization</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_security_authorization</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.security.authenticator.manager</name>
+    <value>org.apache.hadoop.hive.ql.security.ProxyUserAuthenticator</value>
+    <description>
+      hive client authenticator manager class name. The user defined authenticator should implement
+      interface org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive_security_authorization</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.security.metastore.authorization.manager</name>
+    <value>org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider</value>
+    <description>
+      authorization manager class name to be used in the metastore for authorization.
+      The user defined authorization class should implement interface
+      org.apache.hadoop.hive.ql.security.authorization.HiveMetastoreAuthorizationProvider.
+    </description>
+    <display-name>Hive Authorization Manager</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive_security_authorization</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.security.metastore.authorization.auth.reads</name>
+    <value>true</value>
+    <description>If this is true, metastore authorizer authorizes read actions on database, table</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.security.metastore.authenticator.manager</name>
+    <value>org.apache.hadoop.hive.ql.security.HadoopDefaultMetastoreAuthenticator</value>
+    <description>
+      authenticator manager class name to be used in the metastore for authentication.
+      The user defined authenticator should implement interface org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.logging.operation.enabled</name>
+    <value>true</value>
+    <description>When true, HS2 will save operation logs</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.logging.operation.log.location</name>
+    <value>/tmp/hive/operation_logs</value>
+    <description>Top level directory where operation logs are stored if logging functionality is enabled</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.zookeeper.namespace</name>
+    <value>hiveserver2</value>
+    <description>The parent node in ZooKeeper used by HiveServer2 when supporting dynamic service discovery.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.thrift.http.port</name>
+    <value>10001</value>
+    <description>Port number of HiveServer2 Thrift interface when hive.server2.transport.mode is 'http'.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.thrift.http.path</name>
+    <value>cliservice</value>
+    <description>Path component of URL endpoint when in HTTP mode.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.thrift.port</name>
+    <value>10000</value>
+    <description>
+      TCP port number to listen on, default 10000.
+    </description>
+    <display-name>HiveServer2 Port</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.thrift.sasl.qop</name>
+    <value>auth</value>
+    <description>
+      Expects one of [auth, auth-int, auth-conf].
+      Sasl QOP value; Set it to one of following values to enable higher levels of
+      protection for HiveServer2 communication with clients.
+      "auth" - authentication only (default)
+      "auth-int" - authentication plus integrity protection
+      "auth-conf" - authentication plus integrity and confidentiality protection
+      This is applicable only if HiveServer2 is configured to use Kerberos authentication.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.thrift.max.worker.threads</name>
+    <value>500</value>
+    <description>Maximum number of Thrift worker threads</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.allow.user.substitution</name>
+    <value>true</value>
+    <description>Allow alternate user to be specified as part of HiveServer2 open connection request.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.authentication.spnego.keytab</name>
+    <value>HTTP/_HOST@EXAMPLE.COM</value>
+    <description>
+      keytab file for SPNego principal, optional,
+      typical value would look like /etc/security/keytabs/spnego.service.keytab,
+      This keytab would be used by HiveServer2 when Kerberos security is enabled and
+      HTTP transport mode is used.
+      This needs to be set only if SPNEGO is to be used in authentication.
+      SPNego authentication would be honored only if valid
+      hive.server2.authentication.spnego.principal
+      and
+      hive.server2.authentication.spnego.keytab
+      are specified.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.authentication</name>
+    <value>NONE</value>
+    <description>Authentication mode, default NONE. Options are NONE, NOSASL, KERBEROS, LDAP, PAM and CUSTOM</description>
+    <display-name>HiveServer2 Authentication</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>NONE</value>
+                <label>None</label>
+            </entry>
+            <entry>
+                <value>LDAP</value>
+                <label>LDAP</label>
+            </entry>
+            <entry>
+                <value>KERBEROS</value>
+                <label>Kerberos</label>
+            </entry>
+            <entry>
+                <value>PAM</value>
+                <label>PAM</label>
+            </entry>
+            <entry>
+                <value>CUSTOM</value>
+                <label>Custom</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.server2.custom.authentication.class</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.server2.authentication.kerberos.principal</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.server2.authentication.kerberos.keytab</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.server2.authentication.ldap.url</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.server2.authentication.ldap.baseDN</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.server2.authentication.pam.services</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive.server2.authentication.spnego.principal</name>
+    <value>/etc/security/keytabs/spnego.service.keytab</value>
+    <description>
+      SPNego service principal, optional,
+      typical value would look like HTTP/_HOST@EXAMPLE.COM
+      SPNego service principal would be used by HiveServer2 when Kerberos security is enabled
+      and HTTP transport mode is used.
+      This needs to be set only if SPNEGO is to be used in authentication.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.enable.doAs</name>
+    <value>true</value>
+    <description>
+      Setting this property to true will have HiveServer2 execute
+      Hive operations as the user making the calls to it.
+    </description>
+    <display-name>Run as end user instead of Hive user</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_security_authorization</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.table.type.mapping</name>
+    <value>CLASSIC</value>
+    <description>
+      Expects one of [classic, hive].
+      This setting reflects how HiveServer2 will report the table types for JDBC and other
+      client implementations that retrieve the available tables and supported table types
+      HIVE : Exposes Hive's native table types like MANAGED_TABLE, EXTERNAL_TABLE, VIRTUAL_VIEW
+      CLASSIC : More generic types like TABLE and VIEW
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.use.SSL</name>
+    <value>false</value>
+    <description>
+      Set this to true for using SSL encryption in HiveServer2.
+    </description>
+    <display-name>Use SSL</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.conf.restricted.list</name>
+    <value>hive.security.authenticator.manager,hive.security.authorization.manager,hive.users.in.admin.role</value>
+    <description>Comma separated list of configuration options which are immutable at runtime</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.user.install.directory</name>
+    <value>/user/</value>
+    <description>
+      If hive (in tez mode only) cannot find a usable hive jar in "hive.jar.directory",
+      it will upload the hive jar to "hive.user.install.directory/user.name"
+      and use it to run queries.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.vectorized.groupby.maxentries</name>
+    <value>100000</value>
+    <description>
+      Max number of entries in the vector group by aggregation hashtables.
+      Exceeding this will trigger a flush irrelevant of memory pressure condition.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.prewarm.enabled</name>
+    <value>false</value>
+    <description>Enables container prewarm for Tez (Hadoop 2 only)</description>
+    <display-name>Hold Containers to Reduce Latency</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.prewarm.numcontainers</name>
+    <value>3</value>
+    <description>Controls the number of containers to prewarm for Tez (Hadoop 2 only)</description>
+    <display-name>Number of Containers Held</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <maximum>20</maximum>
+        <minimum>1</minimum>
+        <increment-step>1</increment-step>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.convert.join.bucket.mapjoin.tez</name>
+    <value>false</value>
+    <description>
+      Whether joins can be automatically converted to bucket map joins in hive
+      when tez is used as the execution engine.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.auto.reducer.parallelism</name>
+    <value>false</value>
+    <description>
+      Turn on Tez' auto reducer parallelism feature. When enabled, Hive will still estimate data sizes
+      and set parallelism estimates. Tez will sample source vertices' output sizes and adjust the estimates at runtime as
+      necessary.
+    </description>
+    <display-name>Allow dynamic numbers of reducers</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.max.partition.factor</name>
+    <value>2.0</value>
+    <description>When auto reducer parallelism is enabled this factor will be used to over-partition data in shuffle edges.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.min.partition.factor</name>
+    <value>0.25</value>
+    <description>
+      When auto reducer parallelism is enabled this factor will be used to put a lower limit to the number
+      of reducers that tez specifies.
+    </description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.dynamic.partition.pruning</name>
+    <value>true</value>
+    <description>When dynamic pruning is enabled, joins on partition keys will be processed by sending events from the processing vertices to the tez application master. These events will be used to prune unnecessary partitions.</description>
+    <display-name>Allow dynamic partition pruning</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.dynamic.partition.pruning.max.event.size</name>
+    <value>1048576</value>
+    <description>Maximum size of events sent by processors in dynamic pruning. If this size is crossed no pruning will take place.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.dynamic.partition.pruning.max.data.size</name>
+    <value>104857600</value>
+    <description>Maximum total data size of events in dynamic pruning.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.smb.number.waves</name>
+    <value>0.5</value>
+    <description>The number of waves in which to run the SMB join. Account for cluster being occupied. Ideally should be 1 wave.</description>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>ambari.hive.db.schema.name</name>
+    <value>hive</value>
+    <description>Database name used as the Hive Metastore</description>
+    <display-name>Database Name</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>database</type>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>javax.jdo.option.ConnectionURL</name>
+            <type>hive-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive.vectorized.execution.enabled</name>
+    <value>true</value>
+    <description>
+      This flag should be set to true to enable vectorized mode of query execution.
+      The default value is false.
+    </description>
+    <display-name>Enable Vectorization and Map Vectorization</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.auto.convert.join.noconditionaltask.size</name>
+    <value>52428800</value>
+    <description>If hive.auto.convert.join.noconditionaltask is off, this parameter does not take affect. However, if it
+      is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than this size, the join is directly
+      converted to a mapjoin(there is no conditional task).
+    </description>
+    <display-name>For Map Join, per Map memory threshold</display-name>
+    <filename>hive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <maximum>17179869184</maximum>
+       

<TRUNCATED>

[30/50] [abbrv] bigtop git commit: initial commit for odpi beeline tests

Posted by rv...@apache.org.
initial commit for odpi beeline tests


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

Branch: refs/heads/BIGTOP-2666
Commit: ddeb29f06b83c12d701ba5ccd8dffdadf18f4f00
Parents: 06adbb5
Author: roypr <ro...@us.ibm.com>
Authored: Mon Oct 31 14:33:48 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:02 2017 -0800

----------------------------------------------------------------------
 .../odpi/specs/runtime/hive/TestBeeline.java    | 257 +++++++++++++++++++
 1 file changed, 257 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/ddeb29f0/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
new file mode 100644
index 0000000..dcd105b
--- /dev/null
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
@@ -0,0 +1,257 @@
+package org.odpi.specs.runtime.hive;
+
+import org.apache.commons.exec.CommandLine;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import java.io.FileNotFoundException;
+import java.io.PrintWriter;
+import java.util.Map;
+
+
+/**
+ * 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.
+ */
+
+public class TestBeeline {
+
+	public static final Log LOG = LogFactory.getLog(TestBeeline.class.getName());
+
+	  private static final String URL = "odpiHiveTestBeelineUrl";
+	  private static final String USER = "odpiHiveTestBeelineUser";
+	  private static final String PASSWD = "odpiHiveTestBeelinePassword";
+	  
+	  private static Map<String, String> results;
+	  
+	  private static String beelineUrl; 
+	  private static String beelineUser;
+	  private static String beelinePasswd;
+	  
+	  @BeforeClass
+	  public static void checkHiveHome(){
+		  results = HiveHelper.execCommand(new CommandLine("echo").addArgument("$HIVE_HOME"));
+		  Assert.assertEquals("HIVE_HOME is not in the current path.", "", Integer.parseInt(results.get("outputStream")));
+		  TestBeeline.beelineUrl = System.getProperty(URL);
+		  TestBeeline.beelineUser = System.getProperty(USER);
+		  TestBeeline.beelinePasswd = System.getProperty(PASSWD);
+		  
+		  // Create Url with username and/or passowrd to handle all ways to connect to beeline
+		  
+		  if (beelineUser != null && beelineUser != "") { beelineUrl = beelineUrl+" -n "+beelineUser; }
+		  else if (beelineUser != null && beelineUser != "" && beelinePasswd != null && beelinePasswd != "") { beelineUrl = beelineUrl+" -n "+beelineUser+" -p "+"beelinePasswd"; }
+		  
+	  }
+	  
+	  @Test
+	  public static void checkBeeline() {
+	    
+	    LOG.info("URL is " + beelineUrl); 
+	    LOG.info("User is " + beelineUser);
+	    LOG.info("Passwd is " + beelinePasswd); 
+	    LOG.info("Passwd is null " + (beelinePasswd == null));
+	    
+	    results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl));
+	    String consoleMsg = results.get("outputStream").toLowerCase();
+	    //System.out.println(consoleMsg);
+	    try {
+			Assert.assertEquals("beeline is using beelineUrl", true, consoleMsg.contains("connecting to "+beelineUrl) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+			LOG.info("Beeline -u PASSED.");
+		} catch (AssertionError e) {
+			// TODO Auto-generated catch block
+			LOG.error("Beeline -u FAILED.");
+			LOG.error(results.get("outputStream"));
+		}
+	    
+ 	  }
+	  
+	  @Test
+	  public static void checkBeelineConnect(){
+		  try(PrintWriter out = new PrintWriter("connect.url")){ out.println("!connect " + beelineUrl+";"); out.println("!quit"); } 
+		  catch (FileNotFoundException e1) {
+			// TODO Auto-generated catch block
+			e1.printStackTrace();
+		}
+		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -f connect.url",false));
+		  String consoleMsg = results.get("outputStream").toLowerCase();
+		   
+		    try {
+				Assert.assertEquals("beeline is able to connect to " +beelineUrl, true, consoleMsg.contains("connecting to "+beelineUrl) && !consoleMsg.contains("error") && !consoleMsg.contains("exception") );
+				LOG.info("Beeline !connect PASSED.");
+			} catch (AssertionError e) {
+				// TODO Auto-generated catch block
+				LOG.error("Beeline !connect FAILED.");
+				LOG.error(results.get("outputStream"));
+			}  
+	  }
+	  
+	  @Test
+	  public static void checkBeelineHelp(){
+		   results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("--help"));
+		  String consoleMsg = results.get("outputStream").toLowerCase();
+		    try {
+				Assert.assertEquals("beeline help works", true, consoleMsg.contains("usage: java org.apache.hive.cli.beeline.beeLine" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+				LOG.info("Beeline --help PASSED.");
+			} catch (AssertionError e) {
+				// TODO Auto-generated catch block
+				LOG.error("Beeline --help FAILED.");
+				LOG.error(results.get("outputStream"));
+			}  
+	  }
+
+	  @Test
+	  public static void checkBeelineQueryExecFromCmdLine(){
+		  results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("SHOW DATABASES"));
+		  
+		  if(!results.get("outputStream").contains("odpi_runtime_hive")){
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+				
+			}else{
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			
+			}
+		  String consoleMsg = results.get("outputStream").toLowerCase();
+		  try {
+				Assert.assertEquals("beeline execution works", true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+				LOG.info("Beeline -e PASSED.");
+			} catch (AssertionError e) {
+				// TODO Auto-generated catch block
+				LOG.error("Beeline -e FAILED.");
+				LOG.error(results.get("outputStream"));
+			}  
+		  	
+		  HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));		    
+	  }
+	  
+	  @Test
+	  public static void checkBeelineQueryExecFromFile() throws FileNotFoundException{
+		  
+			try(PrintWriter out = new PrintWriter("beeline-f1.sql")){ out.println("SHOW DATABASES;"); }
+			try(PrintWriter out = new PrintWriter("beeline-f2.sql")){ out.println("CREATE DATABASE odpi_runtime_hive;"); }
+			try(PrintWriter out = new PrintWriter("beeline-f3.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); out.println("CREATE DATABASE odpi_runtime_hive;"); }
+		 	try(PrintWriter out = new PrintWriter("beeline-f4.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); }
+		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -f beeline-f1.sql",false));
+
+		  if(!results.get("outputStream").contains("odpi_runtime_hive")){
+				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -f beeline-f2.sql",false));
+				
+			}else{
+				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -f beeline-f3.sql",false));
+			}
+		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -f beeline-f1.sql",false));
+		  
+		  String consoleMsg = results.get("outputStream").toLowerCase();
+		  try {
+				Assert.assertEquals("beeline execution with file works", true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+				LOG.info("Beeline -f PASSED.");
+			} catch (AssertionError e) {
+				// TODO Auto-generated catch block
+				LOG.error("Beeline -f FAILED.");
+				LOG.error(results.get("outputStream"));
+			}  
+		  
+		  HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -f beeline-f4.sql",false));		    
+	  }
+	  
+	  public static void checkBeelineInitFile() throws FileNotFoundException{
+		  
+			try(PrintWriter out = new PrintWriter("beeline-i1.sql")){ out.println("SHOW DATABASES;"); }
+			try(PrintWriter out = new PrintWriter("beeline-i2.sql")){ out.println("CREATE DATABASE odpi_runtime_beeline_init;"); }
+			try(PrintWriter out = new PrintWriter("beeline-i3.sql")){ out.println("DROP DATABASE odpi_runtime_beeline_init;"); out.println("CREATE DATABASE odpi_runtime_beeline_init;"); }
+		 	try(PrintWriter out = new PrintWriter("beeline-i4.sql")){ out.println("DROP DATABASE odpi_runtime_beeline_init;"); }
+		 	
+		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -i beeline-i1.sql",false));
+	  
+		  if(!results.get("outputStream").contains("odpi_runtime_beeline_init")){
+				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -i beeline-i2.sql",false));
+				
+			}else{
+				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -i beeline-i3.sql",false));
+			}
+		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -i beeline-i1.sql",false));
+		  String consoleMsg = results.get("outputStream").toLowerCase();
+		  try {
+				Assert.assertEquals("beeline execution with init file works", true, consoleMsg.contains("odpi_runtime_beeline_init") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+				LOG.info("Beeline -i PASSED.");
+			} catch (AssertionError e) {
+				// TODO Auto-generated catch block
+				LOG.error("Beeline -i FAILED.");
+				LOG.error(results.get("outputStream"));
+			}  
+
+		  HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" -i beeline-i4.sql",false));		    
+	  }
+	  
+	  public static void checkBeelineHiveVar() throws FileNotFoundException{
+		  
+			try(PrintWriter out = new PrintWriter("beeline-hv1.sql")){ out.println("SHOW DATABASES;"); }
+			try(PrintWriter out = new PrintWriter("beeline-hv2.sql")){ out.println("CREATE DATABASE ${db};"); }
+			try(PrintWriter out = new PrintWriter("beeline-hv3.sql")){ out.println("DROP DATABASE ${db};"); out.println("CREATE DATABASE ${db};"); }
+		 	try(PrintWriter out = new PrintWriter("beeline-hv4.sql")){ out.println("DROP DATABASE ${db};"); }
+		 	
+		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv1.sql",false));
+		  String consoleMsg = results.get("outputStream");
+		  
+		  if(!results.get("outputStream").contains("odpi_runtime_beeline_hivevar")){
+				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv2.sql",false));
+				
+			}else{
+				results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv3.sql",false));
+			}
+		  results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv1.sql",false));
+		  consoleMsg = results.get("outputStream").toLowerCase();
+
+		  try {
+				Assert.assertEquals("beeline execution with hivevar file works", true, consoleMsg.contains("odpi_runtime_beeline_hivevar") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+				LOG.info("Beeline --hivevar PASSED.");
+			} catch (AssertionError e) {
+				// TODO Auto-generated catch block
+				LOG.error("Beeline --hivevar FAILED.");
+				LOG.error(results.get("outputStream"));
+			}  
+		  	
+		  HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+beelineUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv4.sql",false));		    
+	  }
+	  
+	  @Test
+	  public static void CheckBeelineFastConnect(){
+		   results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("--fastConnect=false"));
+		  String consoleMsg = results.get("outputStream").toLowerCase();
+		    
+		    try {
+				Assert.assertEquals("beeline fastConnect works", true, consoleMsg.contains("set fastconnect to true to skip") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+				LOG.info("Beeline --fastConnect PASSED.");
+			} catch (AssertionError e) {
+				// TODO Auto-generated catch block
+				LOG.error("Beeline --fastConnect FAILED.");
+				LOG.error(results.get("outputStream"));
+			}  
+	  }
+	  
+	  @AfterClass
+	  public static void cleanup() throws FileNotFoundException {
+	    
+		  	results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf beeline*.sql", false));
+			
+	  }
+
+
+	  
+}


[39/50] [abbrv] bigtop git commit: ODPI-143. java.io.IOException: Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being

Posted by rv...@apache.org.
ODPI-143. java.io.IOException: Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being


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

Branch: refs/heads/BIGTOP-2666
Commit: 19654438387dfd13166a66baa2fd5c082c101101
Parents: bebdd09
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Mon Nov 7 23:39:12 2016 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:47:33 2017 -0800

----------------------------------------------------------------------
 bigtop-deploy/puppet/modules/hadoop/templates/hdfs-site.xml | 7 -------
 docker/sandbox/site.yaml.template.centos-6_hadoop           | 9 ++++++---
 2 files changed, 6 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/19654438/bigtop-deploy/puppet/modules/hadoop/templates/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/hadoop/templates/hdfs-site.xml b/bigtop-deploy/puppet/modules/hadoop/templates/hdfs-site.xml
index 5a36484..e95150a 100644
--- a/bigtop-deploy/puppet/modules/hadoop/templates/hdfs-site.xml
+++ b/bigtop-deploy/puppet/modules/hadoop/templates/hdfs-site.xml
@@ -242,13 +242,6 @@
     <value><%= @hdfs_data_dirs.map { |dir| "file://#{dir}" }.join(",") %></value>
   </property>
   
-  <!-- BEGIN: Specific change for ODPi reference build ONLY -->
-  <property>
-    <name>dfs.client.block.write.replace-datanode-on-failure.enable</name>
-    <value>false</value>
-  </property>
-  <!-- END: Specific change for ODPi reference build ONLY -->
-
 <% if @hdfs_replace_datanode_on_failure %>
   <property>
     <name>dfs.client.block.write.replace-datanode-on-failure.enable</name>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/19654438/docker/sandbox/site.yaml.template.centos-6_hadoop
----------------------------------------------------------------------
diff --git a/docker/sandbox/site.yaml.template.centos-6_hadoop b/docker/sandbox/site.yaml.template.centos-6_hadoop
index 71a7c33..63d0325 100644
--- a/docker/sandbox/site.yaml.template.centos-6_hadoop
+++ b/docker/sandbox/site.yaml.template.centos-6_hadoop
@@ -15,6 +15,9 @@
 
 bigtop::hadoop_head_node: "head.node.fqdn"
 hadoop::hadoop_storage_dirs: [/data/1, /data/2]
-bigtop::bigtop_repo_uri: http://bigtop-repos.s3.amazonaws.com/releases/1.1.0/centos/6/x86_64
-hadoop_cluster_node::cluster_components: [hadoop, yarn]
-bigtop::jdk_package_name: java-1.7.0-openjdk-devel.x86_64
+bigtop::bigtop_repo_uri: $2
+hadoop_cluster_node::cluster_components: $3
+bigtop::jdk_package_name: $4
+hadoop::common_hdfs::testonly_hdfs_sshkeys: "yes"
+hadoop::common_hdfs::hdfs_replace_datanode_on_failure: "false"
+EOF


[23/50] [abbrv] bigtop git commit: Added logging and return in catch blocks from Alan's comments.

Posted by rv...@apache.org.
Added logging and return in catch blocks from Alan's comments.


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

Branch: refs/heads/BIGTOP-2666
Commit: 307c4d05ca4518ee172ff39b95e86c8b1ed5e2f7
Parents: 066e1cf
Author: Raj Desai <rd...@us.ibm.com>
Authored: Thu Oct 27 10:31:24 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:00 2017 -0800

----------------------------------------------------------------------
 .../java/org/odpi/specs/runtime/hive/HiveHelper.java  | 14 ++++++++++++--
 1 file changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/307c4d05/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
index 2ac9cc8..a4477ff 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
@@ -36,8 +36,7 @@ import org.apache.commons.logging.LogFactory;
 public class HiveHelper {
 	
 	private static final Log LOG = LogFactory.getLog(HiveHelper.class.getName());
-	
-	
+		
 	public static Map<String, String> execCommand(CommandLine commandline) {
 		
 		System.out.println("Executing command:");
@@ -48,6 +47,7 @@ public class HiveHelper {
 			env = EnvironmentUtils.getProcEnvironment();
 		} catch (IOException e1) {
 			// TODO Auto-generated catch block
+			LOG.debug("Failed to get process environment: "+ e1.getMessage());
 			e1.printStackTrace();
 		}
 
@@ -63,10 +63,20 @@ public class HiveHelper {
 			executor.execute(commandline, env, resultHandler);
 		} catch (ExecuteException e) {
 			// TODO Auto-generated catch block
+			LOG.debug("Failed to execute command with exit value: "+ String.valueOf(resultHandler.getExitValue()));
+			LOG.debug("outputStream: "+ outputStream.toString());
+			entry.put("exitValue", String.valueOf(resultHandler.getExitValue()));
+			entry.put("outputStream", outputStream.toString() + e.getMessage());
 			e.printStackTrace();
+			return entry;
 		} catch (IOException e) {
 			// TODO Auto-generated catch block
+			LOG.debug("Failed to execute command with exit value: "+ String.valueOf(resultHandler.getExitValue()));
+			LOG.debug("outputStream: "+ outputStream.toString());
+			entry.put("exitValue", String.valueOf(resultHandler.getExitValue()));
+			entry.put("outputStream", outputStream.toString() + e.getMessage());
 			e.printStackTrace();
+			return entry;
 		}
 		
 		try {


[15/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.postgres.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.postgres.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.postgres.sql
new file mode 100755
index 0000000..bc6486b
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.postgres.sql
@@ -0,0 +1,1406 @@
+--
+-- PostgreSQL database dump
+--
+
+SET statement_timeout = 0;
+SET client_encoding = 'UTF8';
+SET standard_conforming_strings = off;
+SET check_function_bodies = false;
+SET client_min_messages = warning;
+SET escape_string_warning = off;
+
+SET search_path = public, pg_catalog;
+
+SET default_tablespace = '';
+
+SET default_with_oids = false;
+
+--
+-- Name: BUCKETING_COLS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "BUCKETING_COLS" (
+    "SD_ID" bigint NOT NULL,
+    "BUCKET_COL_NAME" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: CDS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "CDS" (
+    "CD_ID" bigint NOT NULL
+);
+
+
+--
+-- Name: COLUMNS_OLD; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "COLUMNS_OLD" (
+    "SD_ID" bigint NOT NULL,
+    "COMMENT" character varying(256) DEFAULT NULL::character varying,
+    "COLUMN_NAME" character varying(128) NOT NULL,
+    "TYPE_NAME" character varying(4000) NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: COLUMNS_V2; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "COLUMNS_V2" (
+    "CD_ID" bigint NOT NULL,
+    "COMMENT" character varying(4000),
+    "COLUMN_NAME" character varying(128) NOT NULL,
+    "TYPE_NAME" character varying(4000),
+    "INTEGER_IDX" integer NOT NULL
+);
+
+
+--
+-- Name: DATABASE_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "DATABASE_PARAMS" (
+    "DB_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(180) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: DBS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "DBS" (
+    "DB_ID" bigint NOT NULL,
+    "DESC" character varying(4000) DEFAULT NULL::character varying,
+    "DB_LOCATION_URI" character varying(4000) NOT NULL,
+    "NAME" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: DB_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "DB_PRIVS" (
+    "DB_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "DB_ID" bigint,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "DB_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: GLOBAL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "GLOBAL_PRIVS" (
+    "USER_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "USER_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: IDXS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "IDXS" (
+    "INDEX_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "DEFERRED_REBUILD" boolean NOT NULL,
+    "INDEX_HANDLER_CLASS" character varying(4000) DEFAULT NULL::character varying,
+    "INDEX_NAME" character varying(128) DEFAULT NULL::character varying,
+    "INDEX_TBL_ID" bigint,
+    "LAST_ACCESS_TIME" bigint NOT NULL,
+    "ORIG_TBL_ID" bigint,
+    "SD_ID" bigint
+);
+
+
+--
+-- Name: INDEX_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "INDEX_PARAMS" (
+    "INDEX_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: NUCLEUS_TABLES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "NUCLEUS_TABLES" (
+    "CLASS_NAME" character varying(128) NOT NULL,
+    "TABLE_NAME" character varying(128) NOT NULL,
+    "TYPE" character varying(4) NOT NULL,
+    "OWNER" character varying(2) NOT NULL,
+    "VERSION" character varying(20) NOT NULL,
+    "INTERFACE_NAME" character varying(255) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: PARTITIONS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITIONS" (
+    "PART_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "LAST_ACCESS_TIME" bigint NOT NULL,
+    "PART_NAME" character varying(767) DEFAULT NULL::character varying,
+    "SD_ID" bigint,
+    "TBL_ID" bigint
+);
+
+
+--
+-- Name: PARTITION_EVENTS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_EVENTS" (
+    "PART_NAME_ID" bigint NOT NULL,
+    "DB_NAME" character varying(128),
+    "EVENT_TIME" bigint NOT NULL,
+    "EVENT_TYPE" integer NOT NULL,
+    "PARTITION_NAME" character varying(767),
+    "TBL_NAME" character varying(128)
+);
+
+
+--
+-- Name: PARTITION_KEYS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_KEYS" (
+    "TBL_ID" bigint NOT NULL,
+    "PKEY_COMMENT" character varying(4000) DEFAULT NULL::character varying,
+    "PKEY_NAME" character varying(128) NOT NULL,
+    "PKEY_TYPE" character varying(767) NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: PARTITION_KEY_VALS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_KEY_VALS" (
+    "PART_ID" bigint NOT NULL,
+    "PART_KEY_VAL" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: PARTITION_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_PARAMS" (
+    "PART_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: PART_COL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PART_COL_PRIVS" (
+    "PART_COLUMN_GRANT_ID" bigint NOT NULL,
+    "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_ID" bigint,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_COL_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: PART_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PART_PRIVS" (
+    "PART_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_ID" bigint,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: ROLES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "ROLES" (
+    "ROLE_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "OWNER_NAME" character varying(128) DEFAULT NULL::character varying,
+    "ROLE_NAME" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: ROLE_MAP; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "ROLE_MAP" (
+    "ROLE_GRANT_ID" bigint NOT NULL,
+    "ADD_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "ROLE_ID" bigint
+);
+
+
+--
+-- Name: SDS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SDS" (
+    "SD_ID" bigint NOT NULL,
+    "INPUT_FORMAT" character varying(4000) DEFAULT NULL::character varying,
+    "IS_COMPRESSED" boolean NOT NULL,
+    "LOCATION" character varying(4000) DEFAULT NULL::character varying,
+    "NUM_BUCKETS" bigint NOT NULL,
+    "OUTPUT_FORMAT" character varying(4000) DEFAULT NULL::character varying,
+    "SERDE_ID" bigint,
+    "CD_ID" bigint,
+    "IS_STOREDASSUBDIRECTORIES" boolean NOT NULL
+);
+
+
+--
+-- Name: SD_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SD_PARAMS" (
+    "SD_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: SEQUENCE_TABLE; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SEQUENCE_TABLE" (
+    "SEQUENCE_NAME" character varying(255) NOT NULL,
+    "NEXT_VAL" bigint NOT NULL
+);
+
+
+--
+-- Name: SERDES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SERDES" (
+    "SERDE_ID" bigint NOT NULL,
+    "NAME" character varying(128) DEFAULT NULL::character varying,
+    "SLIB" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: SERDE_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SERDE_PARAMS" (
+    "SERDE_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: SORT_COLS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SORT_COLS" (
+    "SD_ID" bigint NOT NULL,
+    "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+    "ORDER" bigint NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: TABLE_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TABLE_PARAMS" (
+    "TBL_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: TBLS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TBLS" (
+    "TBL_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "DB_ID" bigint,
+    "LAST_ACCESS_TIME" bigint NOT NULL,
+    "OWNER" character varying(767) DEFAULT NULL::character varying,
+    "RETENTION" bigint NOT NULL,
+    "SD_ID" bigint,
+    "TBL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "TBL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "VIEW_EXPANDED_TEXT" text,
+    "VIEW_ORIGINAL_TEXT" text
+);
+
+
+--
+-- Name: TBL_COL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TBL_COL_PRIVS" (
+    "TBL_COLUMN_GRANT_ID" bigint NOT NULL,
+    "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "TBL_COL_PRIV" character varying(128) DEFAULT NULL::character varying,
+    "TBL_ID" bigint
+);
+
+
+--
+-- Name: TBL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TBL_PRIVS" (
+    "TBL_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "TBL_PRIV" character varying(128) DEFAULT NULL::character varying,
+    "TBL_ID" bigint
+);
+
+
+--
+-- Name: TYPES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TYPES" (
+    "TYPES_ID" bigint NOT NULL,
+    "TYPE_NAME" character varying(128) DEFAULT NULL::character varying,
+    "TYPE1" character varying(767) DEFAULT NULL::character varying,
+    "TYPE2" character varying(767) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: TYPE_FIELDS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TYPE_FIELDS" (
+    "TYPE_NAME" bigint NOT NULL,
+    "COMMENT" character varying(256) DEFAULT NULL::character varying,
+    "FIELD_NAME" character varying(128) NOT NULL,
+    "FIELD_TYPE" character varying(767) NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_STRING_LIST" (
+    "STRING_LIST_ID" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_STRING_LIST_VALUES" (
+    "STRING_LIST_ID" bigint NOT NULL,
+    "STRING_LIST_VALUE" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_COL_NAMES" (
+    "SD_ID" bigint NOT NULL,
+    "SKEWED_COL_NAME" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_COL_VALUE_LOC_MAP" (
+    "SD_ID" bigint NOT NULL,
+    "STRING_LIST_ID_KID" bigint NOT NULL,
+    "LOCATION" character varying(4000) DEFAULT NULL::character varying
+);
+
+CREATE TABLE "SKEWED_VALUES" (
+    "SD_ID_OID" bigint NOT NULL,
+    "STRING_LIST_ID_EID" bigint NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: TAB_COL_STATS Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE  "MASTER_KEYS"
+(
+    "KEY_ID" SERIAL,
+    "MASTER_KEY" varchar(767) NULL,
+    PRIMARY KEY ("KEY_ID")
+);
+
+CREATE TABLE  "DELEGATION_TOKENS"
+(
+    "TOKEN_IDENT" varchar(767) NOT NULL,
+    "TOKEN" varchar(767) NULL,
+    PRIMARY KEY ("TOKEN_IDENT")
+);
+
+CREATE TABLE "TAB_COL_STATS" (
+ "CS_ID" bigint NOT NULL,
+ "DB_NAME" character varying(128) DEFAULT NULL::character varying,
+ "TABLE_NAME" character varying(128) DEFAULT NULL::character varying,
+ "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+ "COLUMN_TYPE" character varying(128) DEFAULT NULL::character varying,
+ "TBL_ID" bigint NOT NULL,
+ "LONG_LOW_VALUE" bigint,
+ "LONG_HIGH_VALUE" bigint,
+ "DOUBLE_LOW_VALUE" double precision,
+ "DOUBLE_HIGH_VALUE" double precision,
+ "BIG_DECIMAL_LOW_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "BIG_DECIMAL_HIGH_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "NUM_NULLS" bigint NOT NULL,
+ "NUM_DISTINCTS" bigint,
+ "AVG_COL_LEN" double precision,
+ "MAX_COL_LEN" bigint,
+ "NUM_TRUES" bigint,
+ "NUM_FALSES" bigint,
+ "LAST_ANALYZED" bigint NOT NULL
+);
+
+--
+-- Table structure for VERSION
+--
+CREATE TABLE "VERSION" (
+  "VER_ID" bigint,
+  "SCHEMA_VERSION" character varying(127) NOT NULL,
+  "VERSION_COMMENT" character varying(255) NOT NULL
+);
+
+--
+-- Name: PART_COL_STATS Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PART_COL_STATS" (
+ "CS_ID" bigint NOT NULL,
+ "DB_NAME" character varying(128) DEFAULT NULL::character varying,
+ "TABLE_NAME" character varying(128) DEFAULT NULL::character varying,
+ "PARTITION_NAME" character varying(767) DEFAULT NULL::character varying,
+ "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+ "COLUMN_TYPE" character varying(128) DEFAULT NULL::character varying,
+ "PART_ID" bigint NOT NULL,
+ "LONG_LOW_VALUE" bigint,
+ "LONG_HIGH_VALUE" bigint,
+ "DOUBLE_LOW_VALUE" double precision,
+ "DOUBLE_HIGH_VALUE" double precision,
+ "BIG_DECIMAL_LOW_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "BIG_DECIMAL_HIGH_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "NUM_NULLS" bigint NOT NULL,
+ "NUM_DISTINCTS" bigint,
+ "AVG_COL_LEN" double precision,
+ "MAX_COL_LEN" bigint,
+ "NUM_TRUES" bigint,
+ "NUM_FALSES" bigint,
+ "LAST_ANALYZED" bigint NOT NULL
+);
+
+--
+-- Name: BUCKETING_COLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "BUCKETING_COLS"
+    ADD CONSTRAINT "BUCKETING_COLS_pkey" PRIMARY KEY ("SD_ID", "INTEGER_IDX");
+
+
+--
+-- Name: CDS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "CDS"
+    ADD CONSTRAINT "CDS_pkey" PRIMARY KEY ("CD_ID");
+
+
+--
+-- Name: COLUMNS_V2_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "COLUMNS_V2"
+    ADD CONSTRAINT "COLUMNS_V2_pkey" PRIMARY KEY ("CD_ID", "COLUMN_NAME");
+
+
+--
+-- Name: COLUMNS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "COLUMNS_OLD"
+    ADD CONSTRAINT "COLUMNS_pkey" PRIMARY KEY ("SD_ID", "COLUMN_NAME");
+
+
+--
+-- Name: DATABASE_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DATABASE_PARAMS"
+    ADD CONSTRAINT "DATABASE_PARAMS_pkey" PRIMARY KEY ("DB_ID", "PARAM_KEY");
+
+
+--
+-- Name: DBPRIVILEGEINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DB_PRIVS"
+    ADD CONSTRAINT "DBPRIVILEGEINDEX" UNIQUE ("DB_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DB_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: DBS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DBS"
+    ADD CONSTRAINT "DBS_pkey" PRIMARY KEY ("DB_ID");
+
+
+--
+-- Name: DB_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DB_PRIVS"
+    ADD CONSTRAINT "DB_PRIVS_pkey" PRIMARY KEY ("DB_GRANT_ID");
+
+
+--
+-- Name: GLOBALPRIVILEGEINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "GLOBAL_PRIVS"
+    ADD CONSTRAINT "GLOBALPRIVILEGEINDEX" UNIQUE ("PRINCIPAL_NAME", "PRINCIPAL_TYPE", "USER_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: GLOBAL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "GLOBAL_PRIVS"
+    ADD CONSTRAINT "GLOBAL_PRIVS_pkey" PRIMARY KEY ("USER_GRANT_ID");
+
+
+--
+-- Name: IDXS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_pkey" PRIMARY KEY ("INDEX_ID");
+
+
+--
+-- Name: INDEX_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "INDEX_PARAMS"
+    ADD CONSTRAINT "INDEX_PARAMS_pkey" PRIMARY KEY ("INDEX_ID", "PARAM_KEY");
+
+
+--
+-- Name: NUCLEUS_TABLES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "NUCLEUS_TABLES"
+    ADD CONSTRAINT "NUCLEUS_TABLES_pkey" PRIMARY KEY ("CLASS_NAME");
+
+
+--
+-- Name: PARTITIONS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "PARTITIONS_pkey" PRIMARY KEY ("PART_ID");
+
+
+--
+-- Name: PARTITION_EVENTS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_EVENTS"
+    ADD CONSTRAINT "PARTITION_EVENTS_pkey" PRIMARY KEY ("PART_NAME_ID");
+
+
+--
+-- Name: PARTITION_KEYS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_KEYS"
+    ADD CONSTRAINT "PARTITION_KEYS_pkey" PRIMARY KEY ("TBL_ID", "PKEY_NAME");
+
+
+--
+-- Name: PARTITION_KEY_VALS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_KEY_VALS"
+    ADD CONSTRAINT "PARTITION_KEY_VALS_pkey" PRIMARY KEY ("PART_ID", "INTEGER_IDX");
+
+
+--
+-- Name: PARTITION_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_PARAMS"
+    ADD CONSTRAINT "PARTITION_PARAMS_pkey" PRIMARY KEY ("PART_ID", "PARAM_KEY");
+
+
+--
+-- Name: PART_COL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PART_COL_PRIVS"
+    ADD CONSTRAINT "PART_COL_PRIVS_pkey" PRIMARY KEY ("PART_COLUMN_GRANT_ID");
+
+
+--
+-- Name: PART_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PART_PRIVS"
+    ADD CONSTRAINT "PART_PRIVS_pkey" PRIMARY KEY ("PART_GRANT_ID");
+
+
+--
+-- Name: ROLEENTITYINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLES"
+    ADD CONSTRAINT "ROLEENTITYINDEX" UNIQUE ("ROLE_NAME");
+
+
+--
+-- Name: ROLES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLES"
+    ADD CONSTRAINT "ROLES_pkey" PRIMARY KEY ("ROLE_ID");
+
+
+--
+-- Name: ROLE_MAP_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLE_MAP"
+    ADD CONSTRAINT "ROLE_MAP_pkey" PRIMARY KEY ("ROLE_GRANT_ID");
+
+
+--
+-- Name: SDS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SDS"
+    ADD CONSTRAINT "SDS_pkey" PRIMARY KEY ("SD_ID");
+
+
+--
+-- Name: SD_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SD_PARAMS"
+    ADD CONSTRAINT "SD_PARAMS_pkey" PRIMARY KEY ("SD_ID", "PARAM_KEY");
+
+
+--
+-- Name: SEQUENCE_TABLE_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SEQUENCE_TABLE"
+    ADD CONSTRAINT "SEQUENCE_TABLE_pkey" PRIMARY KEY ("SEQUENCE_NAME");
+
+
+--
+-- Name: SERDES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SERDES"
+    ADD CONSTRAINT "SERDES_pkey" PRIMARY KEY ("SERDE_ID");
+
+
+--
+-- Name: SERDE_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SERDE_PARAMS"
+    ADD CONSTRAINT "SERDE_PARAMS_pkey" PRIMARY KEY ("SERDE_ID", "PARAM_KEY");
+
+
+--
+-- Name: SORT_COLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SORT_COLS"
+    ADD CONSTRAINT "SORT_COLS_pkey" PRIMARY KEY ("SD_ID", "INTEGER_IDX");
+
+
+--
+-- Name: TABLE_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TABLE_PARAMS"
+    ADD CONSTRAINT "TABLE_PARAMS_pkey" PRIMARY KEY ("TBL_ID", "PARAM_KEY");
+
+
+--
+-- Name: TBLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "TBLS_pkey" PRIMARY KEY ("TBL_ID");
+
+
+--
+-- Name: TBL_COL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBL_COL_PRIVS"
+    ADD CONSTRAINT "TBL_COL_PRIVS_pkey" PRIMARY KEY ("TBL_COLUMN_GRANT_ID");
+
+
+--
+-- Name: TBL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBL_PRIVS"
+    ADD CONSTRAINT "TBL_PRIVS_pkey" PRIMARY KEY ("TBL_GRANT_ID");
+
+
+--
+-- Name: TYPES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TYPES"
+    ADD CONSTRAINT "TYPES_pkey" PRIMARY KEY ("TYPES_ID");
+
+
+--
+-- Name: TYPE_FIELDS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TYPE_FIELDS"
+    ADD CONSTRAINT "TYPE_FIELDS_pkey" PRIMARY KEY ("TYPE_NAME", "FIELD_NAME");
+
+ALTER TABLE ONLY "SKEWED_STRING_LIST"
+    ADD CONSTRAINT "SKEWED_STRING_LIST_pkey" PRIMARY KEY ("STRING_LIST_ID");
+
+ALTER TABLE ONLY "SKEWED_STRING_LIST_VALUES"
+    ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_pkey" PRIMARY KEY ("STRING_LIST_ID", "INTEGER_IDX");
+
+
+ALTER TABLE ONLY "SKEWED_COL_NAMES"
+    ADD CONSTRAINT "SKEWED_COL_NAMES_pkey" PRIMARY KEY ("SD_ID", "INTEGER_IDX");
+
+ALTER TABLE ONLY "SKEWED_COL_VALUE_LOC_MAP"
+    ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_pkey" PRIMARY KEY ("SD_ID", "STRING_LIST_ID_KID");
+
+ALTER TABLE ONLY "SKEWED_VALUES"
+    ADD CONSTRAINT "SKEWED_VALUES_pkey" PRIMARY KEY ("SD_ID_OID", "INTEGER_IDX");
+
+--
+-- Name: TAB_COL_STATS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+ALTER TABLE ONLY "TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_pkey" PRIMARY KEY("CS_ID");
+
+--
+-- Name: PART_COL_STATS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+ALTER TABLE ONLY "PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_pkey" PRIMARY KEY("CS_ID");
+
+--
+-- Name: UNIQUEINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "UNIQUEINDEX" UNIQUE ("INDEX_NAME", "ORIG_TBL_ID");
+
+
+--
+-- Name: UNIQUEPARTITION; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "UNIQUEPARTITION" UNIQUE ("PART_NAME", "TBL_ID");
+
+
+--
+-- Name: UNIQUETABLE; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "UNIQUETABLE" UNIQUE ("TBL_NAME", "DB_ID");
+
+
+--
+-- Name: UNIQUE_DATABASE; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DBS"
+    ADD CONSTRAINT "UNIQUE_DATABASE" UNIQUE ("NAME");
+
+
+--
+-- Name: UNIQUE_TYPE; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TYPES"
+    ADD CONSTRAINT "UNIQUE_TYPE" UNIQUE ("TYPE_NAME");
+
+
+--
+-- Name: USERROLEMAPINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLE_MAP"
+    ADD CONSTRAINT "USERROLEMAPINDEX" UNIQUE ("PRINCIPAL_NAME", "ROLE_ID", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: BUCKETING_COLS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "BUCKETING_COLS_N49" ON "BUCKETING_COLS" USING btree ("SD_ID");
+
+
+--
+-- Name: COLUMNS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "COLUMNS_N49" ON "COLUMNS_OLD" USING btree ("SD_ID");
+
+
+--
+-- Name: DATABASE_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "DATABASE_PARAMS_N49" ON "DATABASE_PARAMS" USING btree ("DB_ID");
+
+
+--
+-- Name: DB_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "DB_PRIVS_N49" ON "DB_PRIVS" USING btree ("DB_ID");
+
+
+--
+-- Name: IDXS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "IDXS_N49" ON "IDXS" USING btree ("ORIG_TBL_ID");
+
+
+--
+-- Name: IDXS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "IDXS_N50" ON "IDXS" USING btree ("INDEX_TBL_ID");
+
+
+--
+-- Name: IDXS_N51; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "IDXS_N51" ON "IDXS" USING btree ("SD_ID");
+
+
+--
+-- Name: INDEX_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "INDEX_PARAMS_N49" ON "INDEX_PARAMS" USING btree ("INDEX_ID");
+
+
+--
+-- Name: PARTITIONCOLUMNPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONCOLUMNPRIVILEGEINDEX" ON "PART_COL_PRIVS" USING btree ("PART_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_COL_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: PARTITIONEVENTINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONEVENTINDEX" ON "PARTITION_EVENTS" USING btree ("PARTITION_NAME");
+
+
+--
+-- Name: PARTITIONS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONS_N49" ON "PARTITIONS" USING btree ("TBL_ID");
+
+
+--
+-- Name: PARTITIONS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONS_N50" ON "PARTITIONS" USING btree ("SD_ID");
+
+
+--
+-- Name: PARTITION_KEYS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITION_KEYS_N49" ON "PARTITION_KEYS" USING btree ("TBL_ID");
+
+
+--
+-- Name: PARTITION_KEY_VALS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITION_KEY_VALS_N49" ON "PARTITION_KEY_VALS" USING btree ("PART_ID");
+
+
+--
+-- Name: PARTITION_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITION_PARAMS_N49" ON "PARTITION_PARAMS" USING btree ("PART_ID");
+
+
+--
+-- Name: PARTPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTPRIVILEGEINDEX" ON "PART_PRIVS" USING btree ("PART_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: PART_COL_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PART_COL_PRIVS_N49" ON "PART_COL_PRIVS" USING btree ("PART_ID");
+
+
+--
+-- Name: PART_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PART_PRIVS_N49" ON "PART_PRIVS" USING btree ("PART_ID");
+
+
+--
+-- Name: ROLE_MAP_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "ROLE_MAP_N49" ON "ROLE_MAP" USING btree ("ROLE_ID");
+
+
+--
+-- Name: SDS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SDS_N49" ON "SDS" USING btree ("SERDE_ID");
+
+
+--
+-- Name: SD_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SD_PARAMS_N49" ON "SD_PARAMS" USING btree ("SD_ID");
+
+
+--
+-- Name: SERDE_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SERDE_PARAMS_N49" ON "SERDE_PARAMS" USING btree ("SERDE_ID");
+
+
+--
+-- Name: SORT_COLS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SORT_COLS_N49" ON "SORT_COLS" USING btree ("SD_ID");
+
+
+--
+-- Name: TABLECOLUMNPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TABLECOLUMNPRIVILEGEINDEX" ON "TBL_COL_PRIVS" USING btree ("TBL_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_COL_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: TABLEPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TABLEPRIVILEGEINDEX" ON "TBL_PRIVS" USING btree ("TBL_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: TABLE_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TABLE_PARAMS_N49" ON "TABLE_PARAMS" USING btree ("TBL_ID");
+
+
+--
+-- Name: TBLS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBLS_N49" ON "TBLS" USING btree ("DB_ID");
+
+
+--
+-- Name: TBLS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBLS_N50" ON "TBLS" USING btree ("SD_ID");
+
+
+--
+-- Name: TBL_COL_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBL_COL_PRIVS_N49" ON "TBL_COL_PRIVS" USING btree ("TBL_ID");
+
+
+--
+-- Name: TBL_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBL_PRIVS_N49" ON "TBL_PRIVS" USING btree ("TBL_ID");
+
+
+--
+-- Name: TYPE_FIELDS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TYPE_FIELDS_N49" ON "TYPE_FIELDS" USING btree ("TYPE_NAME");
+
+--
+-- Name: TAB_COL_STATS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TAB_COL_STATS_N49" ON "TAB_COL_STATS" USING btree ("TBL_ID");
+
+--
+-- Name: PART_COL_STATS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PART_COL_STATS_N49" ON "PART_COL_STATS" USING btree ("PART_ID");
+
+
+ALTER TABLE ONLY "SKEWED_STRING_LIST_VALUES"
+    ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_fkey" FOREIGN KEY ("STRING_LIST_ID") REFERENCES "SKEWED_STRING_LIST"("STRING_LIST_ID") DEFERRABLE;
+
+
+ALTER TABLE ONLY "SKEWED_COL_NAMES"
+    ADD CONSTRAINT "SKEWED_COL_NAMES_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+ALTER TABLE ONLY "SKEWED_COL_VALUE_LOC_MAP"
+    ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_fkey1" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+ALTER TABLE ONLY "SKEWED_COL_VALUE_LOC_MAP"
+    ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_fkey2" FOREIGN KEY ("STRING_LIST_ID_KID") REFERENCES "SKEWED_STRING_LIST"("STRING_LIST_ID") DEFERRABLE;
+
+ALTER TABLE ONLY "SKEWED_VALUES"
+    ADD CONSTRAINT "SKEWED_VALUES_fkey1" FOREIGN KEY ("STRING_LIST_ID_EID") REFERENCES "SKEWED_STRING_LIST"("STRING_LIST_ID") DEFERRABLE;
+
+ALTER TABLE ONLY "SKEWED_VALUES"
+    ADD CONSTRAINT "SKEWED_VALUES_fkey2" FOREIGN KEY ("SD_ID_OID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: BUCKETING_COLS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "BUCKETING_COLS"
+    ADD CONSTRAINT "BUCKETING_COLS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: COLUMNS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "COLUMNS_OLD"
+    ADD CONSTRAINT "COLUMNS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: COLUMNS_V2_CD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "COLUMNS_V2"
+    ADD CONSTRAINT "COLUMNS_V2_CD_ID_fkey" FOREIGN KEY ("CD_ID") REFERENCES "CDS"("CD_ID") DEFERRABLE;
+
+
+--
+-- Name: DATABASE_PARAMS_DB_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "DATABASE_PARAMS"
+    ADD CONSTRAINT "DATABASE_PARAMS_DB_ID_fkey" FOREIGN KEY ("DB_ID") REFERENCES "DBS"("DB_ID") DEFERRABLE;
+
+
+--
+-- Name: DB_PRIVS_DB_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "DB_PRIVS"
+    ADD CONSTRAINT "DB_PRIVS_DB_ID_fkey" FOREIGN KEY ("DB_ID") REFERENCES "DBS"("DB_ID") DEFERRABLE;
+
+
+--
+-- Name: IDXS_INDEX_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_INDEX_TBL_ID_fkey" FOREIGN KEY ("INDEX_TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: IDXS_ORIG_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_ORIG_TBL_ID_fkey" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: IDXS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: INDEX_PARAMS_INDEX_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "INDEX_PARAMS"
+    ADD CONSTRAINT "INDEX_PARAMS_INDEX_ID_fkey" FOREIGN KEY ("INDEX_ID") REFERENCES "IDXS"("INDEX_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITIONS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "PARTITIONS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITIONS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "PARTITIONS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITION_KEYS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITION_KEYS"
+    ADD CONSTRAINT "PARTITION_KEYS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITION_KEY_VALS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITION_KEY_VALS"
+    ADD CONSTRAINT "PARTITION_KEY_VALS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITION_PARAMS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITION_PARAMS"
+    ADD CONSTRAINT "PARTITION_PARAMS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: PART_COL_PRIVS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PART_COL_PRIVS"
+    ADD CONSTRAINT "PART_COL_PRIVS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: PART_PRIVS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PART_PRIVS"
+    ADD CONSTRAINT "PART_PRIVS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: ROLE_MAP_ROLE_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "ROLE_MAP"
+    ADD CONSTRAINT "ROLE_MAP_ROLE_ID_fkey" FOREIGN KEY ("ROLE_ID") REFERENCES "ROLES"("ROLE_ID") DEFERRABLE;
+
+
+--
+-- Name: SDS_CD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SDS"
+    ADD CONSTRAINT "SDS_CD_ID_fkey" FOREIGN KEY ("CD_ID") REFERENCES "CDS"("CD_ID") DEFERRABLE;
+
+
+--
+-- Name: SDS_SERDE_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SDS"
+    ADD CONSTRAINT "SDS_SERDE_ID_fkey" FOREIGN KEY ("SERDE_ID") REFERENCES "SERDES"("SERDE_ID") DEFERRABLE;
+
+
+--
+-- Name: SD_PARAMS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SD_PARAMS"
+    ADD CONSTRAINT "SD_PARAMS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: SERDE_PARAMS_SERDE_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SERDE_PARAMS"
+    ADD CONSTRAINT "SERDE_PARAMS_SERDE_ID_fkey" FOREIGN KEY ("SERDE_ID") REFERENCES "SERDES"("SERDE_ID") DEFERRABLE;
+
+
+--
+-- Name: SORT_COLS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SORT_COLS"
+    ADD CONSTRAINT "SORT_COLS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: TABLE_PARAMS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TABLE_PARAMS"
+    ADD CONSTRAINT "TABLE_PARAMS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: TBLS_DB_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "TBLS_DB_ID_fkey" FOREIGN KEY ("DB_ID") REFERENCES "DBS"("DB_ID") DEFERRABLE;
+
+
+--
+-- Name: TBLS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "TBLS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: TBL_COL_PRIVS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBL_COL_PRIVS"
+    ADD CONSTRAINT "TBL_COL_PRIVS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: TBL_PRIVS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBL_PRIVS"
+    ADD CONSTRAINT "TBL_PRIVS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: TYPE_FIELDS_TYPE_NAME_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TYPE_FIELDS"
+    ADD CONSTRAINT "TYPE_FIELDS_TYPE_NAME_fkey" FOREIGN KEY ("TYPE_NAME") REFERENCES "TYPES"("TYPES_ID") DEFERRABLE;
+
+--
+-- Name: TAB_COL_STATS_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+ALTER TABLE ONLY "TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_fkey" FOREIGN KEY("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: PART_COL_STATS_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+ALTER TABLE ONLY "PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_fkey" FOREIGN KEY("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+ALTER TABLE ONLY "VERSION" ADD CONSTRAINT "VERSION_pkey" PRIMARY KEY ("VER_ID");
+
+--
+-- Name: public; Type: ACL; Schema: -; Owner: hiveuser
+--
+
+REVOKE ALL ON SCHEMA public FROM PUBLIC;
+GRANT ALL ON SCHEMA public TO PUBLIC;
+
+
+INSERT INTO "VERSION" ("VER_ID", "SCHEMA_VERSION", "VERSION_COMMENT") VALUES (1, '0.12.0', 'Hive release version 0.12.0');
+--
+-- PostgreSQL database dump complete
+--
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.mysql.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.mysql.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.mysql.sql
new file mode 100755
index 0000000..89ce15d
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.13.0.mysql.sql
@@ -0,0 +1,889 @@
+-- MySQL dump 10.13  Distrib 5.5.25, for osx10.6 (i386)
+--
+-- Host: localhost    Database: test
+-- ------------------------------------------------------
+-- Server version	5.5.25
+
+/*!40101 SET @OLD_CHARACTER_SET_CLIENT=@@CHARACTER_SET_CLIENT */;
+/*!40101 SET @OLD_CHARACTER_SET_RESULTS=@@CHARACTER_SET_RESULTS */;
+/*!40101 SET @OLD_COLLATION_CONNECTION=@@COLLATION_CONNECTION */;
+/*!40101 SET NAMES utf8 */;
+/*!40103 SET @OLD_TIME_ZONE=@@TIME_ZONE */;
+/*!40103 SET TIME_ZONE='+00:00' */;
+/*!40014 SET @OLD_UNIQUE_CHECKS=@@UNIQUE_CHECKS, UNIQUE_CHECKS=0 */;
+/*!40014 SET @OLD_FOREIGN_KEY_CHECKS=@@FOREIGN_KEY_CHECKS, FOREIGN_KEY_CHECKS=0 */;
+/*!40101 SET @OLD_SQL_MODE=@@SQL_MODE, SQL_MODE='NO_AUTO_VALUE_ON_ZERO' */;
+/*!40111 SET @OLD_SQL_NOTES=@@SQL_NOTES, SQL_NOTES=0 */;
+
+--
+-- Table structure for table `BUCKETING_COLS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `BUCKETING_COLS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `BUCKET_COL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
+  KEY `BUCKETING_COLS_N49` (`SD_ID`),
+  CONSTRAINT `BUCKETING_COLS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `CDS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `CDS` (
+  `CD_ID` bigint(20) NOT NULL,
+  PRIMARY KEY (`CD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `COLUMNS_V2`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `COLUMNS_V2` (
+  `CD_ID` bigint(20) NOT NULL,
+  `COMMENT` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TYPE_NAME` varchar(4000) DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`CD_ID`,`COLUMN_NAME`),
+  KEY `COLUMNS_V2_N49` (`CD_ID`),
+  CONSTRAINT `COLUMNS_V2_FK1` FOREIGN KEY (`CD_ID`) REFERENCES `CDS` (`CD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `DATABASE_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` (
+  `DB_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(180) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`DB_ID`,`PARAM_KEY`),
+  KEY `DATABASE_PARAMS_N49` (`DB_ID`),
+  CONSTRAINT `DATABASE_PARAMS_FK1` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `DBS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `DBS` (
+  `DB_ID` bigint(20) NOT NULL,
+  `DESC` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `DB_LOCATION_URI` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `OWNER_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `OWNER_TYPE` varchar(10) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`DB_ID`),
+  UNIQUE KEY `UNIQUE_DATABASE` (`NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `DB_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `DB_PRIVS` (
+  `DB_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `DB_ID` bigint(20) DEFAULT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `DB_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`DB_GRANT_ID`),
+  UNIQUE KEY `DBPRIVILEGEINDEX` (`DB_ID`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`DB_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `DB_PRIVS_N49` (`DB_ID`),
+  CONSTRAINT `DB_PRIVS_FK1` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `GLOBAL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `GLOBAL_PRIVS` (
+  `USER_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `USER_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`USER_GRANT_ID`),
+  UNIQUE KEY `GLOBALPRIVILEGEINDEX` (`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`USER_PRIV`,`GRANTOR`,`GRANTOR_TYPE`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `IDXS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `IDXS` (
+  `INDEX_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `DEFERRED_REBUILD` bit(1) NOT NULL,
+  `INDEX_HANDLER_CLASS` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INDEX_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INDEX_TBL_ID` bigint(20) DEFAULT NULL,
+  `LAST_ACCESS_TIME` int(11) NOT NULL,
+  `ORIG_TBL_ID` bigint(20) DEFAULT NULL,
+  `SD_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`INDEX_ID`),
+  UNIQUE KEY `UNIQUEINDEX` (`INDEX_NAME`,`ORIG_TBL_ID`),
+  KEY `IDXS_N51` (`SD_ID`),
+  KEY `IDXS_N50` (`INDEX_TBL_ID`),
+  KEY `IDXS_N49` (`ORIG_TBL_ID`),
+  CONSTRAINT `IDXS_FK1` FOREIGN KEY (`ORIG_TBL_ID`) REFERENCES `TBLS` (`TBL_ID`),
+  CONSTRAINT `IDXS_FK2` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`),
+  CONSTRAINT `IDXS_FK3` FOREIGN KEY (`INDEX_TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `INDEX_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `INDEX_PARAMS` (
+  `INDEX_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`),
+  KEY `INDEX_PARAMS_N49` (`INDEX_ID`),
+  CONSTRAINT `INDEX_PARAMS_FK1` FOREIGN KEY (`INDEX_ID`) REFERENCES `IDXS` (`INDEX_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `NUCLEUS_TABLES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `NUCLEUS_TABLES` (
+  `CLASS_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TABLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TYPE` varchar(4) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `OWNER` varchar(2) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `VERSION` varchar(20) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `INTERFACE_NAME` varchar(255) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`CLASS_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITIONS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITIONS` (
+  `PART_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `LAST_ACCESS_TIME` int(11) NOT NULL,
+  `PART_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SD_ID` bigint(20) DEFAULT NULL,
+  `TBL_ID` bigint(20) DEFAULT NULL,
+  `LINK_TARGET_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`PART_ID`),
+  UNIQUE KEY `UNIQUEPARTITION` (`PART_NAME`,`TBL_ID`),
+  KEY `PARTITIONS_N49` (`TBL_ID`),
+  KEY `PARTITIONS_N50` (`SD_ID`),
+  KEY `PARTITIONS_N51` (`LINK_TARGET_ID`),
+  CONSTRAINT `PARTITIONS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`),
+  CONSTRAINT `PARTITIONS_FK2` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`),
+  CONSTRAINT `PARTITIONS_FK3` FOREIGN KEY (`LINK_TARGET_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_EVENTS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_EVENTS` (
+  `PART_NAME_ID` bigint(20) NOT NULL,
+  `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `EVENT_TIME` bigint(20) NOT NULL,
+  `EVENT_TYPE` int(11) NOT NULL,
+  `PARTITION_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_NAME_ID`),
+  KEY `PARTITIONEVENTINDEX` (`PARTITION_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_KEYS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_KEYS` (
+  `TBL_ID` bigint(20) NOT NULL,
+  `PKEY_COMMENT` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PKEY_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PKEY_TYPE` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`TBL_ID`,`PKEY_NAME`),
+  KEY `PARTITION_KEYS_N49` (`TBL_ID`),
+  CONSTRAINT `PARTITION_KEYS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_KEY_VALS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_KEY_VALS` (
+  `PART_ID` bigint(20) NOT NULL,
+  `PART_KEY_VAL` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`PART_ID`,`INTEGER_IDX`),
+  KEY `PARTITION_KEY_VALS_N49` (`PART_ID`),
+  CONSTRAINT `PARTITION_KEY_VALS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_PARAMS` (
+  `PART_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_ID`,`PARAM_KEY`),
+  KEY `PARTITION_PARAMS_N49` (`PART_ID`),
+  CONSTRAINT `PARTITION_PARAMS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PART_COL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PART_COL_PRIVS` (
+  `PART_COLUMN_GRANT_ID` bigint(20) NOT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_ID` bigint(20) DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_COL_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_COLUMN_GRANT_ID`),
+  KEY `PART_COL_PRIVS_N49` (`PART_ID`),
+  KEY `PARTITIONCOLUMNPRIVILEGEINDEX` (`PART_ID`,`COLUMN_NAME`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`PART_COL_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  CONSTRAINT `PART_COL_PRIVS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PART_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PART_PRIVS` (
+  `PART_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_ID` bigint(20) DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_GRANT_ID`),
+  KEY `PARTPRIVILEGEINDEX` (`PART_ID`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`PART_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `PART_PRIVS_N49` (`PART_ID`),
+  CONSTRAINT `PART_PRIVS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `ROLES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `ROLES` (
+  `ROLE_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `OWNER_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `ROLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`ROLE_ID`),
+  UNIQUE KEY `ROLEENTITYINDEX` (`ROLE_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `ROLE_MAP`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `ROLE_MAP` (
+  `ROLE_GRANT_ID` bigint(20) NOT NULL,
+  `ADD_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `ROLE_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`ROLE_GRANT_ID`),
+  UNIQUE KEY `USERROLEMAPINDEX` (`PRINCIPAL_NAME`,`ROLE_ID`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `ROLE_MAP_N49` (`ROLE_ID`),
+  CONSTRAINT `ROLE_MAP_FK1` FOREIGN KEY (`ROLE_ID`) REFERENCES `ROLES` (`ROLE_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SDS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SDS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `CD_ID` bigint(20) DEFAULT NULL,
+  `INPUT_FORMAT` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `IS_COMPRESSED` bit(1) NOT NULL,
+  `IS_STOREDASSUBDIRECTORIES` bit(1) NOT NULL,
+  `LOCATION` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `NUM_BUCKETS` int(11) NOT NULL,
+  `OUTPUT_FORMAT` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SERDE_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`SD_ID`),
+  KEY `SDS_N49` (`SERDE_ID`),
+  KEY `SDS_N50` (`CD_ID`),
+  CONSTRAINT `SDS_FK1` FOREIGN KEY (`SERDE_ID`) REFERENCES `SERDES` (`SERDE_ID`),
+  CONSTRAINT `SDS_FK2` FOREIGN KEY (`CD_ID`) REFERENCES `CDS` (`CD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SD_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SD_PARAMS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SD_ID`,`PARAM_KEY`),
+  KEY `SD_PARAMS_N49` (`SD_ID`),
+  CONSTRAINT `SD_PARAMS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SEQUENCE_TABLE`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SEQUENCE_TABLE` (
+  `SEQUENCE_NAME` varchar(255) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `NEXT_VAL` bigint(20) NOT NULL,
+  PRIMARY KEY (`SEQUENCE_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SERDES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SERDES` (
+  `SERDE_ID` bigint(20) NOT NULL,
+  `NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SLIB` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SERDE_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SERDE_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SERDE_PARAMS` (
+  `SERDE_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`),
+  KEY `SERDE_PARAMS_N49` (`SERDE_ID`),
+  CONSTRAINT `SERDE_PARAMS_FK1` FOREIGN KEY (`SERDE_ID`) REFERENCES `SERDES` (`SERDE_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_COL_NAMES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_COL_NAMES` (
+  `SD_ID` bigint(20) NOT NULL,
+  `SKEWED_COL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
+  KEY `SKEWED_COL_NAMES_N49` (`SD_ID`),
+  CONSTRAINT `SKEWED_COL_NAMES_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_COL_VALUE_LOC_MAP`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_COL_VALUE_LOC_MAP` (
+  `SD_ID` bigint(20) NOT NULL,
+  `STRING_LIST_ID_KID` bigint(20) NOT NULL,
+  `LOCATION` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`),
+  KEY `SKEWED_COL_VALUE_LOC_MAP_N49` (`STRING_LIST_ID_KID`),
+  KEY `SKEWED_COL_VALUE_LOC_MAP_N50` (`SD_ID`),
+  CONSTRAINT `SKEWED_COL_VALUE_LOC_MAP_FK2` FOREIGN KEY (`STRING_LIST_ID_KID`) REFERENCES `SKEWED_STRING_LIST` (`STRING_LIST_ID`),
+  CONSTRAINT `SKEWED_COL_VALUE_LOC_MAP_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_STRING_LIST`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST` (
+  `STRING_LIST_ID` bigint(20) NOT NULL,
+  PRIMARY KEY (`STRING_LIST_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_STRING_LIST_VALUES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST_VALUES` (
+  `STRING_LIST_ID` bigint(20) NOT NULL,
+  `STRING_LIST_VALUE` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`),
+  KEY `SKEWED_STRING_LIST_VALUES_N49` (`STRING_LIST_ID`),
+  CONSTRAINT `SKEWED_STRING_LIST_VALUES_FK1` FOREIGN KEY (`STRING_LIST_ID`) REFERENCES `SKEWED_STRING_LIST` (`STRING_LIST_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_VALUES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_VALUES` (
+  `SD_ID_OID` bigint(20) NOT NULL,
+  `STRING_LIST_ID_EID` bigint(20) NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`),
+  KEY `SKEWED_VALUES_N50` (`SD_ID_OID`),
+  KEY `SKEWED_VALUES_N49` (`STRING_LIST_ID_EID`),
+  CONSTRAINT `SKEWED_VALUES_FK2` FOREIGN KEY (`STRING_LIST_ID_EID`) REFERENCES `SKEWED_STRING_LIST` (`STRING_LIST_ID`),
+  CONSTRAINT `SKEWED_VALUES_FK1` FOREIGN KEY (`SD_ID_OID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SORT_COLS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SORT_COLS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `ORDER` int(11) NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
+  KEY `SORT_COLS_N49` (`SD_ID`),
+  CONSTRAINT `SORT_COLS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TABLE_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` (
+  `TBL_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`TBL_ID`,`PARAM_KEY`),
+  KEY `TABLE_PARAMS_N49` (`TBL_ID`),
+  CONSTRAINT `TABLE_PARAMS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TBLS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TBLS` (
+  `TBL_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `DB_ID` bigint(20) DEFAULT NULL,
+  `LAST_ACCESS_TIME` int(11) NOT NULL,
+  `OWNER` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `RETENTION` int(11) NOT NULL,
+  `SD_ID` bigint(20) DEFAULT NULL,
+  `TBL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `VIEW_EXPANDED_TEXT` mediumtext,
+  `VIEW_ORIGINAL_TEXT` mediumtext,
+  `LINK_TARGET_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`TBL_ID`),
+  UNIQUE KEY `UNIQUETABLE` (`TBL_NAME`,`DB_ID`),
+  KEY `TBLS_N50` (`SD_ID`),
+  KEY `TBLS_N49` (`DB_ID`),
+  KEY `TBLS_N51` (`LINK_TARGET_ID`),
+  CONSTRAINT `TBLS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`),
+  CONSTRAINT `TBLS_FK2` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`),
+  CONSTRAINT `TBLS_FK3` FOREIGN KEY (`LINK_TARGET_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TBL_COL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` (
+  `TBL_COLUMN_GRANT_ID` bigint(20) NOT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_COL_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`TBL_COLUMN_GRANT_ID`),
+  KEY `TABLECOLUMNPRIVILEGEINDEX` (`TBL_ID`,`COLUMN_NAME`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`TBL_COL_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `TBL_COL_PRIVS_N49` (`TBL_ID`),
+  CONSTRAINT `TBL_COL_PRIVS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TBL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TBL_PRIVS` (
+  `TBL_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`TBL_GRANT_ID`),
+  KEY `TBL_PRIVS_N49` (`TBL_ID`),
+  KEY `TABLEPRIVILEGEINDEX` (`TBL_ID`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`TBL_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  CONSTRAINT `TBL_PRIVS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TAB_COL_STATS`
+--
+CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` (
+ `CS_ID` bigint(20) NOT NULL,
+ `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `TABLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `TBL_ID` bigint(20) NOT NULL,
+ `LONG_LOW_VALUE` bigint(20),
+ `LONG_HIGH_VALUE` bigint(20),
+ `DOUBLE_HIGH_VALUE` double(53,4),
+ `DOUBLE_LOW_VALUE` double(53,4),
+ `BIG_DECIMAL_LOW_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `BIG_DECIMAL_HIGH_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `NUM_NULLS` bigint(20) NOT NULL,
+ `NUM_DISTINCTS` bigint(20),
+ `AVG_COL_LEN` double(53,4),
+ `MAX_COL_LEN` bigint(20),
+ `NUM_TRUES` bigint(20),
+ `NUM_FALSES` bigint(20),
+ `LAST_ANALYZED` bigint(20) NOT NULL,
+  PRIMARY KEY (`CS_ID`),
+  CONSTRAINT `TAB_COL_STATS_FK` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for table `PART_COL_STATS`
+--
+CREATE TABLE IF NOT EXISTS `PART_COL_STATS` (
+ `CS_ID` bigint(20) NOT NULL,
+ `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `TABLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `PARTITION_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `PART_ID` bigint(20) NOT NULL,
+ `LONG_LOW_VALUE` bigint(20),
+ `LONG_HIGH_VALUE` bigint(20),
+ `DOUBLE_HIGH_VALUE` double(53,4),
+ `DOUBLE_LOW_VALUE` double(53,4),
+ `BIG_DECIMAL_LOW_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `BIG_DECIMAL_HIGH_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `NUM_NULLS` bigint(20) NOT NULL,
+ `NUM_DISTINCTS` bigint(20),
+ `AVG_COL_LEN` double(53,4),
+ `MAX_COL_LEN` bigint(20),
+ `NUM_TRUES` bigint(20),
+ `NUM_FALSES` bigint(20),
+ `LAST_ANALYZED` bigint(20) NOT NULL,
+  PRIMARY KEY (`CS_ID`),
+  CONSTRAINT `PART_COL_STATS_FK` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for table `TYPES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TYPES` (
+  `TYPES_ID` bigint(20) NOT NULL,
+  `TYPE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TYPE1` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TYPE2` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`TYPES_ID`),
+  UNIQUE KEY `UNIQUE_TYPE` (`TYPE_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TYPE_FIELDS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TYPE_FIELDS` (
+  `TYPE_NAME` bigint(20) NOT NULL,
+  `COMMENT` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `FIELD_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `FIELD_TYPE` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`TYPE_NAME`,`FIELD_NAME`),
+  KEY `TYPE_FIELDS_N49` (`TYPE_NAME`),
+  CONSTRAINT `TYPE_FIELDS_FK1` FOREIGN KEY (`TYPE_NAME`) REFERENCES `TYPES` (`TYPES_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+-- Table `MASTER_KEYS` for classes [org.apache.hadoop.hive.metastore.model.MMasterKey]
+CREATE TABLE IF NOT EXISTS `MASTER_KEYS` 
+(
+    `KEY_ID` INTEGER NOT NULL AUTO_INCREMENT,
+    `MASTER_KEY` VARCHAR(767) BINARY NULL,
+    PRIMARY KEY (`KEY_ID`)
+) ENGINE=INNODB DEFAULT CHARSET=latin1;
+
+-- Table `DELEGATION_TOKENS` for classes [org.apache.hadoop.hive.metastore.model.MDelegationToken]
+CREATE TABLE IF NOT EXISTS `DELEGATION_TOKENS`
+(
+    `TOKEN_IDENT` VARCHAR(767) BINARY NOT NULL,
+    `TOKEN` VARCHAR(767) BINARY NULL,
+    PRIMARY KEY (`TOKEN_IDENT`)
+) ENGINE=INNODB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for VERSION
+--
+CREATE TABLE IF NOT EXISTS `VERSION` (
+  `VER_ID` BIGINT NOT NULL,
+  `SCHEMA_VERSION` VARCHAR(127) NOT NULL,
+  `VERSION_COMMENT` VARCHAR(255),
+  PRIMARY KEY (`VER_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for table FUNCS
+--
+CREATE TABLE IF NOT EXISTS `FUNCS` (
+  `FUNC_ID` BIGINT(20) NOT NULL,
+  `CLASS_NAME` VARCHAR(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+  `CREATE_TIME` INT(11) NOT NULL,
+  `DB_ID` BIGINT(20),
+  `FUNC_NAME` VARCHAR(128) CHARACTER SET latin1 COLLATE latin1_bin,
+  `FUNC_TYPE` INT(11) NOT NULL,
+  `OWNER_NAME` VARCHAR(128) CHARACTER SET latin1 COLLATE latin1_bin,
+  `OWNER_TYPE` VARCHAR(10) CHARACTER SET latin1 COLLATE latin1_bin,
+  PRIMARY KEY (`FUNC_ID`),
+  UNIQUE KEY `UNIQUEFUNCTION` (`FUNC_NAME`, `DB_ID`),
+  KEY `FUNCS_N49` (`DB_ID`),
+  CONSTRAINT `FUNCS_FK1` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for table FUNC_RU
+--
+CREATE TABLE IF NOT EXISTS `FUNC_RU` (
+  `FUNC_ID` BIGINT(20) NOT NULL,
+  `RESOURCE_TYPE` INT(11) NOT NULL,
+  `RESOURCE_URI` VARCHAR(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+  `INTEGER_IDX` INT(11) NOT NULL,
+  PRIMARY KEY (`FUNC_ID`, `INTEGER_IDX`),
+  CONSTRAINT `FUNC_RU_FK1` FOREIGN KEY (`FUNC_ID`) REFERENCES `FUNCS` (`FUNC_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+
+-- -----------------------------------------------------------------------------------------------------------------------------------------------
+-- Transaction and Lock Tables
+-- These are not part of package jdo, so if you are going to regenerate this file you need to manually add the following section back to the file.
+-- -----------------------------------------------------------------------------------------------------------------------------------------------
+
+CREATE TABLE TXNS (
+  TXN_ID bigint PRIMARY KEY,
+  TXN_STATE char(1) NOT NULL,
+  TXN_STARTED bigint NOT NULL,
+  TXN_LAST_HEARTBEAT bigint NOT NULL,
+  TXN_USER varchar(128) NOT NULL,
+  TXN_HOST varchar(128) NOT NULL
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE TABLE TXN_COMPONENTS (
+  TC_TXNID bigint,
+  TC_DATABASE varchar(128) NOT NULL,
+  TC_TABLE varchar(128),
+  TC_PARTITION varchar(767),
+  FOREIGN KEY (TC_TXNID) REFERENCES TXNS (TXN_ID)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE TABLE COMPLETED_TXN_COMPONENTS (
+  CTC_TXNID bigint,
+  CTC_DATABASE varchar(128) NOT NULL,
+  CTC_TABLE varchar(128),
+  CTC_PARTITION varchar(767)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE TABLE NEXT_TXN_ID (
+  NTXN_NEXT bigint NOT NULL
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+INSERT INTO NEXT_TXN_ID VALUES(1);
+
+CREATE TABLE HIVE_LOCKS (
+  HL_LOCK_EXT_ID bigint NOT NULL,
+  HL_LOCK_INT_ID bigint NOT NULL,
+  HL_TXNID bigint,
+  HL_DB varchar(128) NOT NULL,
+  HL_TABLE varchar(128),
+  HL_PARTITION varchar(767),
+  HL_LOCK_STATE char(1) not null,
+  HL_LOCK_TYPE char(1) not null,
+  HL_LAST_HEARTBEAT bigint NOT NULL,
+  HL_ACQUIRED_AT bigint,
+  HL_USER varchar(128) NOT NULL,
+  HL_HOST varchar(128) NOT NULL,
+  PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID),
+  KEY HIVE_LOCK_TXNID_INDEX (HL_TXNID)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE INDEX HL_TXNID_IDX ON HIVE_LOCKS (HL_TXNID);
+
+CREATE TABLE NEXT_LOCK_ID (
+  NL_NEXT bigint NOT NULL
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+INSERT INTO NEXT_LOCK_ID VALUES(1);
+
+CREATE TABLE COMPACTION_QUEUE (
+  CQ_ID bigint PRIMARY KEY,
+  CQ_DATABASE varchar(128) NOT NULL,
+  CQ_TABLE varchar(128) NOT NULL,
+  CQ_PARTITION varchar(767),
+  CQ_STATE char(1) NOT NULL,
+  CQ_TYPE char(1) NOT NULL,
+  CQ_WORKER_ID varchar(128),
+  CQ_START bigint,
+  CQ_RUN_AS varchar(128)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE TABLE NEXT_COMPACTION_QUEUE_ID (
+  NCQ_NEXT bigint NOT NULL
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1);
+
+
+-- -----------------------------------------------------------------
+-- Record schema version. Should be the last step in the init script
+-- -----------------------------------------------------------------
+INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.13.0', 'Hive release version 0.13.0');
+
+/*!40101 SET character_set_client = @saved_cs_client */;
+/*!40103 SET TIME_ZONE=@OLD_TIME_ZONE */;
+
+/*!40101 SET SQL_MODE=@OLD_SQL_MODE */;
+/*!40014 SET FOREIGN_KEY_CHECKS=@OLD_FOREIGN_KEY_CHECKS */;
+/*!40014 SET UNIQUE_CHECKS=@OLD_UNIQUE_CHECKS */;
+/*!40101 SET CHARACTER_SET_CLIENT=@OLD_CHARACTER_SET_CLIENT */;
+/*!40101 SET CHARACTER_SET_RESULTS=@OLD_CHARACTER_SET_RESULTS */;
+/*!40101 SET COLLATION_CONNECTION=@OLD_COLLATION_CONNECTION */;
+/*!40111 SET SQL_NOTES=@OLD_SQL_NOTES */;
+
+-- Dump completed on 2012-08-23  0:56:31


[29/50] [abbrv] bigtop git commit: Making ODPi Ambari stack compatible with ODPi packaging (getting rid of /usr/hdp)

Posted by rv...@apache.org.
Making ODPi Ambari stack compatible with ODPi packaging (getting rid of /usr/hdp)


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

Branch: refs/heads/BIGTOP-2666
Commit: 06adbb5214639b7e813ac777d0d11553d911603d
Parents: 166d2e0
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Fri Oct 28 10:56:11 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:02 2017 -0800

----------------------------------------------------------------------
 .../ODPi/1.0/services/HIVE/package/scripts/params_linux.py    | 7 +++++--
 .../ODPi/1.0/services/HIVE/package/scripts/status_params.py   | 3 ++-
 2 files changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/06adbb52/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
index b33d715..9d79e12 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
@@ -195,8 +195,11 @@ purge_tables = 'true'
 
 # this is NOT a typo.  Configs for hcatalog/webhcat point to a
 # specific directory which is NOT called 'conf'
-hcat_conf_dir = format('{stack_root}/current/hive-webhcat/etc/hcatalog')
-config_dir = format('{stack_root}/current/hive-webhcat/etc/webhcat')
+# FIXME: ODPi
+# hcat_conf_dir = format('{stack_root}/current/hive-webhcat/etc/hcatalog')
+# config_dir = format('{stack_root}/current/hive-webhcat/etc/webhcat')
+hcat_conf_dir = format('/etc/hive-hcatalog/conf')
+config_dir = format('/etc/hive-webhcat/conf')
 
 hive_metastore_site_supported = True
 

http://git-wip-us.apache.org/repos/asf/bigtop/blob/06adbb52/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
index 024f3df..a7b2e3f 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
@@ -108,7 +108,8 @@ else:
   if check_stack_feature(StackFeature.HIVE_WEBHCAT_SPECIFIC_CONFIGS, stack_version_formatted_major):
     # this is NOT a typo. Configs for hcatalog/webhcat point to a
     # specific directory which is NOT called 'conf'
-    webhcat_conf_dir = format("{stack_root}/current/hive-webhcat/etc/webhcat")
+    #  FIXME ODPi: webhcat_conf_dir = format("{stack_root}/current/hive-webhcat/etc/webhcat")
+    webhcat_conf_dir = format("/etc/hive-webhcat/conf")
 
   # if stack version supports hive serve interactive
   if check_stack_feature(StackFeature.HIVE_SERVER_INTERACTIVE, stack_version_formatted_major):


[06/50] [abbrv] bigtop git commit: ODPI-195 The asssertValueExists method is making a call to String::startsWith using an empty string which results in a passing test when it should not be passing.

Posted by rv...@apache.org.
ODPI-195 The asssertValueExists method is making a call to
String::startsWith using an empty string which results in a passing test
when it should not be passing.


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

Branch: refs/heads/BIGTOP-2666
Commit: c6da69b8b8e6258bf1860033bbdbac9a2d559550
Parents: b191169
Author: Clint Edwards <cl...@sas.com>
Authored: Thu Oct 20 14:36:28 2016 -0400
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:45:56 2017 -0800

----------------------------------------------------------------------
 .../bigtop/itest/httpfs/TestHttpFs.groovy       | 30 +++++++++++++++-----
 1 file changed, 23 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/c6da69b8/bigtop-tests/test-artifacts/httpfs/src/main/groovy/org/apache/bigtop/itest/httpfs/TestHttpFs.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/httpfs/src/main/groovy/org/apache/bigtop/itest/httpfs/TestHttpFs.groovy b/bigtop-tests/test-artifacts/httpfs/src/main/groovy/org/apache/bigtop/itest/httpfs/TestHttpFs.groovy
index 78d7a7a..bcec657 100644
--- a/bigtop-tests/test-artifacts/httpfs/src/main/groovy/org/apache/bigtop/itest/httpfs/TestHttpFs.groovy
+++ b/bigtop-tests/test-artifacts/httpfs/src/main/groovy/org/apache/bigtop/itest/httpfs/TestHttpFs.groovy
@@ -40,8 +40,13 @@ public class TestHttpFs {
 
   private static final String HTTPFS_PREFIX = "http://$HTTPFS_PROXY/webhdfs/v1";
   private static final String HTTPFS_SUCCESS = "{\"boolean\":true}";
+  private static final String HTTP_OK = "HTTP/1.1 200 OK";
+  private static final String HTTP_CREATE = "HTTP/1.1 201 Created";
+  private static final String HTTP_TMP_REDIR = "HTTP/1.1 307 TEMPORARY_REDIRECT";
 
-  private static final String DATA_DIR = System.getProperty("data.dir", "text-files");
+  public static final String HTTPFS_SOURCE = "bigtop-tests/test-artifacts/httpfs/src/main/resources/"
+  def httpfs_source = System.getenv("BIGTOP_HOME") + "/" + HTTPFS_SOURCE;
+  def DATA_DIR = httpfs_source + "/" + "text-files";
 
   private static String testHttpFsFolder = "/tmp/httpfssmoke-" + (new Date().getTime());
   private static String testHttpFsFolderRenamed = "$testHttpFsFolder-renamed";
@@ -74,7 +79,17 @@ public class TestHttpFs {
   public void assertValueExists(List<String> values, String expected) {
     boolean exists = false;
     for (String value : values) {
-      if (expected.startsWith(value)) {
+      if (value.length() && expected.startsWith(value)) {
+        exists = true;
+      }
+    }
+    assertTrue(expected + " NOT found!", exists == true);
+  }
+
+  public void assertValueContains(List<String> values, String expected) {
+    boolean exists = false;
+    for (String value : values) {
+      if (value.length() && value.contains(expected)) {
         exists = true;
       }
     }
@@ -119,8 +134,8 @@ public class TestHttpFs {
     assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
     sh.exec("curl -i '$HTTPFS_PREFIX$testHttpFsFolder?user.name=$USERNAME&op=GETFILESTATUS'");
     assertTrue("curl command to create a dir failed", sh.getRet() == 0);
-    assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
-    assertValueExists(sh.getOut(), "DIRECTORY");
+    assertValueContains(sh.getOut(), "DIRECTORY");
+    assertValueExists(sh.getOut(), HTTP_OK);
   }
 
   @Test
@@ -140,13 +155,14 @@ public class TestHttpFs {
       }
     }
     LOG.debug("Datanode location: $datanodeLocation");
-    assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
+    assertValueExists(sh.getOut(), HTTP_TMP_REDIR);
+    assertNotNull("Datanode location not in response", datanodeLocation);
     sh.exec("curl -i -T $DATA_DIR/$filename '$datanodeLocation' --header 'Content-Type:application/octet-stream'");
     assertTrue("curl command to create a file failed", sh.getRet() == 0);
-    assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
+    assertValueExists(sh.getOut(), HTTP_CREATE);
     sh.exec("curl -i -L '$HTTPFS_PREFIX$testHttpFsFolder/$filename?user.name=$USERNAME&op=OPEN'");
     assertTrue("curl command to create a file failed", sh.getRet() == 0);
-    assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
+    assertValueExists(sh.getOut(), HTTP_OK);
     assertValueExists(sh.getOut(), filenameContent);
   }
 }


[45/50] [abbrv] bigtop git commit: ODPI-200. Added hive support for run_itest.sh and made some enhancements.

Posted by rv...@apache.org.
ODPI-200. Added hive support for run_itest.sh and made some enhancements.


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

Branch: refs/heads/BIGTOP-2666
Commit: bbc4b47bd84897937a8c17903177507e920cd494
Parents: f827574
Author: Raj Desai <rd...@us.ibm.com>
Authored: Thu Nov 10 15:09:33 2016 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:25 2017 -0800

----------------------------------------------------------------------
 bigtop-tests/build.gradle             |   8 +-
 bigtop-tests/smoke-tests/run_itest.sh | 182 ------------------
 run_itest.sh                          | 293 +++++++++++++++++++++++++++++
 3 files changed, 296 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/bbc4b47b/bigtop-tests/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/build.gradle b/bigtop-tests/build.gradle
index f49546e..1940998 100644
--- a/bigtop-tests/build.gradle
+++ b/bigtop-tests/build.gradle
@@ -32,12 +32,13 @@ distributions {
         include 'gradlew'
         include '*.gradle'
         include 'gradle/**'
+        include 'run_itest.sh'
       }
       from ("$rootDir/buildSrc") {
         into 'buildSrc'
       }
       from ("$rootDir/bigtop-packages/src/common/bigtop-utils") {
-        into 'bin'
+        into 'bigtop-packages/src/common/bigtop-utils'
         include '*'
       }
       from ('smoke-tests') {
@@ -47,11 +48,9 @@ distributions {
         include 'hdfs/**/*'
         include 'mapreduce/**/*'
         include 'yarn/**/*'
+        include 'hive/**/*'
         include 'logger-test-config/**'
       }
-      from ('smoke-tests') { // Put the driver script to the top-level
-        include 'run_itest.sh'
-      }
       from ('spec-tests') {
         into 'bigtop-tests/spec-tests'
         include '**/*'
@@ -67,4 +66,3 @@ distributions {
 
 distZip.dependsOn clean
 distTar.dependsOn clean
-

http://git-wip-us.apache.org/repos/asf/bigtop/blob/bbc4b47b/bigtop-tests/smoke-tests/run_itest.sh
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/run_itest.sh b/bigtop-tests/smoke-tests/run_itest.sh
deleted file mode 100755
index d93bd5a..0000000
--- a/bigtop-tests/smoke-tests/run_itest.sh
+++ /dev/null
@@ -1,182 +0,0 @@
-#!/bin/bash
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-## Borrowed heavily from
-##  https://github.com/jctanner/odp-scripts/blob/master/run_itest.sh
-## Kudos to https://github.com/jctanner
-
-# https://github.com/apache/bigtop/tree/master/bigtop-test-framework
-
-# "ITEST" is an integration testing framework written for and by the
-# apache bigtop project. It consists of typical java tools/libraries
-# such as junit, gradle and maven.
-
-# This script is a helper to run itest on any hadoop system without
-# requiring intimate knowledge of bigtop. If running for the first
-# time, simply execute ./run_itest.sh without any arguments. If you
-# want more information, use these additional parameters:
-#
-#   --info          - turns on the log4j output
-#   --debug         - turns up the log4j output to maximum
-#   --traceback     - shows tracebacks from tests
-
-set_java_home() {
-
-    #####################################################################
-    # Use bigtop's bigtop-detect-javahome if JAVA_HOME is not already set
-    #####################################################################
-
-    if [ -z "$JAVA_HOME" ]; then
-        source bin/bigtop-detect-javahome
-    fi    
-}
-
-set_hadoop_vars() {
-
-    #####################################################################
-    # Set the HADOOP_MAPRED_HOME and HADOOP_CONF vars
-    #####################################################################
-
-    # ITEST wants the MR dir with the examples jar ...
-    # java.lang.AssertionError: Can't find hadoop-examples.jar file
-
-    if ( [ -z "$HADOOP_HOME" ] && [ -d /usr/lib/hadoop ] ); then
-      export HADOOP_HOME=/usr/lib/hadoop
-    fi
-    if ( [ -z "$HADOOP_CONF_DIR" ] && [ -d /etc/hadoop/conf ] ); then
-      export HADOOP_CONF_DIR=/etc/hadoop/conf
-    fi
-    if ( [ -z "$HADOOP_MAPRED_HOME" ] && [ -d /usr/lib/hadoop-mapreduce-client ] ); then
-      export HADOOP_MAPRED_HOME=/usr/lib/hadoop-mapreduce-client
-    elif ( [ -z "$HADOOP_MAPRED_HOME" ] && [ -d /usr/lib/hadoop-mapreduce ] ); then
-      export HADOOP_MAPRED_HOME=/usr/lib/hadoop-mapreduce
-    fi
-}
-
-
-print_cluster_info() {
-
-  # ODPI-87
-
-  echo "######################################################"
-  echo "#               CLUSTER INFORMATION                  #"
-  echo "######################################################"
-
-  which facter >/dev/null 2>&1
-  RC=$?
-  if [[ $RC == 0 ]]; then
-    echo "# OS: $(facter lsbdistdescription)"
-    echo "# ARCH: $(facter architecture)"
-    echo "# KERNEL: $(facter kernelrelease)"
-    echo "# MEMORY: $(facter memorysize)"
-  else
-    echo "# OS: $(cat /etc/issue | tr '\n' ' ')"
-    echo "# ARCH: $(uname -i)"
-    echo "# KERNEL: $(uname -a)"
-    echo "# MEMORY: $(head -n1 /proc/meminfo)"
-  fi
-
-  YARN_NODES=$(yarn node -list 2>/dev/null | egrep ^Total | sed 's/Total Nodes:/TOTAL YARN NODES: /g')
-  echo "# $YARN_NODES"
-
-  HADOOP_VERSION=$(hadoop version 2>/dev/null | head -n1)
-  echo "# HADOOP_VERSION: $HADOOP_VERSION"
-  echo "# HADOOP_CONF_DIR: $HADOOP_CONF_DIR"
-  echo "# HADOOP_MAPRED_HOME: $HADOOP_MAPRED_HOME"
-  
-  echo "# BASH_VERSION: $BASH_VERSION"
-  echo "# SH_VERSION: $(/bin/sh -c 'echo $BASH_VERSION')"
-  
-  echo "# JAVA_HOME: $JAVA_HOME"
-  JAVA_VERSION=$(java -version 2>&1 | head -n 1 | awk -F '"' '{print $2}')
-  echo "# JAVA_VERSION: $JAVA_VERSION"
-}
-
-print_tests() {
-  echo "######################################################"
-  echo "#                     RESULTS                        #"
-  echo "######################################################"
-
-  pushd `pwd`
-  for TEST in $(echo $ITESTS | tr ',' '\n'); do
-    TESTDIR=bigtop-tests/smoke-tests/$TEST/build
-
-    if [ -d $TESTDIR ]; then
-      cd $TESTDIR
-
-      for FILE in $(find -L reports/tests/classes -type f -name "*.html"); do
-        echo "## $TESTDIR/$FILE"
-        if [ $(which links) ]; then
-            links $FILE -dump
-        else
-            echo "PLEASE INSTALL LINKS: sudo yum -y install links"
-        fi
-        echo ""
-      done
-    fi
-  done
-
-  popd
-  for TEST in $SPEC_TESTS; do
-    TESTDIR=bigtop-tests/spec-tests/$TEST/build
-
-    if [ -d $TESTDIR ]; then
-      cd $TESTDIR
-
-      for FILE in $(find -L reports/tests/classes -type f -name "*.html"); do
-        echo "## $TESTDIR/$FILE"
-        if [ $(which links) ]; then
-            links $FILE -dump
-        else
-            echo "PLEASE INSTALL LINKS: sudo yum -y install links"
-        fi
-        echo ""
-      done
-    fi
-  done
-}
-
-# SET JAVA_HOME
-set_java_home
-
-# SET HADOOP SERVICE HOMES
-set_hadoop_vars
-
-# ODPI-87
-print_cluster_info
-
-echo "######################################################"
-echo "#                 STARTING ITEST                     #"
-echo "######################################################"
-echo "# Use --debug/--info/--stacktrace for more details"
-
-# SET THE DEFAULT TESTS
-if [ -z "$ITESTS" ]; then
-  export ITESTS="hcfs,hdfs,yarn,mapreduce"
-fi
-SPEC_TESTS="runtime"
-for s in `echo $ITESTS | sed -e 's#,# #g'`; do
-  ALL_SMOKE_TASKS="$ALL_SMOKE_TASKS bigtop-tests:smoke-tests:$s:test"
-done
-for s in $SPEC_TESTS; do
-  ALL_SPEC_TASKS="$ALL_SPEC_TASKS bigtop-tests:spec-tests:$s:test"
-done
-
-# CALL THE GRADLE WRAPPER TO RUN THE FRAMEWORK
-./gradlew -q --continue clean -Psmoke.tests $ALL_SMOKE_TASKS -Pspec.tests $ALL_SPEC_TASKS $@
-
-# SHOW RESULTS (HTML)
-print_tests

http://git-wip-us.apache.org/repos/asf/bigtop/blob/bbc4b47b/run_itest.sh
----------------------------------------------------------------------
diff --git a/run_itest.sh b/run_itest.sh
new file mode 100644
index 0000000..e6184e2
--- /dev/null
+++ b/run_itest.sh
@@ -0,0 +1,293 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+## Borrowed heavily from
+##  https://github.com/jctanner/odp-scripts/blob/master/run_itest.sh
+## Kudos to https://github.com/jctanner
+
+# https://github.com/apache/bigtop/tree/master/bigtop-test-framework
+
+# "ITEST" is an integration testing framework written for and by the
+# apache bigtop project. It consists of typical java tools/libraries
+# such as junit, gradle and maven.
+
+# This script is a helper to run itest on any hadoop system without
+# requiring intimate knowledge of bigtop. If running for the first
+# time, simply execute ./run_itest.sh without any arguments. If you
+# want more information, use these additional parameters:
+#
+#   --info          - turns on the log4j output
+#   --debug         - turns up the log4j output to maximum
+#   --traceback     - shows tracebacks from tests
+
+usage ()
+{
+  echo "Usage : $0 [-options]"
+  echo
+  echo "   -j, --hivejdbc             hive jdbc url - default: e.g. jdbc:hive2://localhost:10000"
+  echo "   -m, --hivemeta             hive metastore url - default: thrift://localhost:9083"
+  echo "   -l, --hivelocation         location of hdfs for hive to write to - default: /user/<current user>"
+  echo "   -u, --hiveuser             hive user - default: current user"
+  echo "   -p, --hivepassword         hive user password - default: current user"
+  echo "   -t, --hivethrift           optional: true/false to test thrift, defaults to true"
+  echo "   -c, --hivecatalog          optional: true/false to test HCatalog, default to true"
+  echo "   -C, --hiveconf             hive conf dir - default: /etc/hive/conf"
+  echo "   -F, --hadoopconf           hadoop user - default: /etc/hadoop/conf"
+  echo "   -i, --info                 optional: info/debug"
+  echo "   -h, --help                 display this help and exit"
+  echo
+  exit
+}
+
+while [ "$1" != "" ]; do
+case $1 in
+        -j | --hivejdbc )       shift
+                                HIVE_JDBC_URL=$1
+                                ;;
+        -m | --hivemeta )       shift
+                                HIVE_METASTORE_URL=$1
+                                ;;
+        -l | --hivelocation )   shift
+                                HIVE_HDFS_LOCATION=$1
+                                ;;
+        -u | --hiveuser )       shift
+                                HIVE_USER=$1
+                                ;;
+        -p | --hivepassword )   shift
+                                HIVE_PASSWORD=$1
+                                ;;
+        -t | --hivethrift )     shift
+                                TEST_THRIFT=$1
+                                ;;
+        -c | --hivecatalog )     shift
+                                TEST_HCATALOG=$1
+                                ;;
+        -C | --hiveconf )       shift
+                                HIVE_CONF_DIR=$1
+                                ;;
+        -F | --hadoopconf )     shift
+                                HADOOP_CONF_DIR=$1
+                                ;;
+        -i | --info )           shift
+                                LOGGING=$1
+                                ;;
+        -h | --help )
+                                usage  # Call your function
+                                exit 0
+                                ;;
+    esac
+    shift
+done
+
+if [ ! -f /etc/hive/conf/hive-site.xml ]; then
+    echo "Could not find hive site configuration file, please specify hive params!"
+    usage
+fi
+
+if [ -z "$HIVE_JDBC_URL" ]; then
+    HIVE_PORT=`sed -n '/hive.server2.thrift.port/{n;p}' /etc/hive/conf/hive-site.xml | sed -n 's:.*<value>\(.*\)</value>.*:\1:p'`
+    netstat -nltp | grep $HIVE_PORT > /dev/null 2>&1
+    if [ $? -eq 0 ]; then
+        HIVE_JDBC_URL=jdbc:hive2://localhost:$HIVE_PORT
+    else
+        echo "Could not find hive server 2 service, please specify --hivejdbc argument."
+        usage
+    fi
+fi
+if [ -z "$HIVE_METASTORE_URL" ]; then
+    HIVE_METASTORE_URL=`sed -n '/hive.metastore.uris/{n;p}' /etc/hive/conf/hive-site.xml | sed -n 's:.*<value>\(.*\)</value>.*:\1:p'`
+fi
+if [ -z "$HIVE_HDFS_LOCATION" ]; then
+    HIVE_HDFS_LOCATION=/tmp/`id -u -n`
+fi
+if [ -z "$HIVE_USER" ]; then
+    HIVE_USER=`id -u -n`
+fi
+if [ -z "$HIVE_PASSWORD" ]; then
+    HIVE_PASSWORD=`id -u -n`
+fi
+if [ -z "$HIVE_CONF_DIR" ]; then
+    export HIVE_CONF_DIR=/etc/hive/conf
+fi
+if [ -z "$HADOOP_CONF_DIR" ]; then
+    export HADOOP_CONF_DIR=/etc/hadoop/conf
+fi
+if [ -z "$TEST_THRIFT" ]; then
+    TEST_THRIFT=true
+fi
+if [ -z "$TEST_HCATALOG" ]; then
+    TEST_HCATALOG=true
+fi
+if [ "$LOGGING" == "info" ]; then
+    LOGGING="--info --stacktrace"
+elif [ "$LOGGING" == "debug" ]; then
+    LOGGING="--debug --stacktrace"
+else
+    LOGGING=""
+fi
+
+export DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+
+set_java_home() {
+
+    #####################################################################
+    # Use bigtop's bigtop-detect-javahome if JAVA_HOME is not already set
+    #####################################################################
+
+    if [ -z "$JAVA_HOME" ]; then
+        source $DIR/bigtop-packages/src/common/bigtop-utils/bigtop-detect-javahome
+    fi
+}
+
+set_hadoop_vars() {
+
+    #####################################################################
+    # Set the HADOOP_MAPRED_HOME and HADOOP_CONF vars
+    #####################################################################
+
+    # ITEST wants the MR dir with the examples jar ...
+    # java.lang.AssertionError: Can't find hadoop-examples.jar file
+
+    if ( [ -z "$HADOOP_HOME" ] && [ -d /usr/lib/hadoop ] ); then
+      export HADOOP_HOME=/usr/lib/hadoop
+    fi
+    if ( [ -z "$HADOOP_CONF_DIR" ] && [ -d /etc/hadoop/conf ] ); then
+      export HADOOP_CONF_DIR=/etc/hadoop/conf
+    fi
+    if ( [ -z "$HADOOP_MAPRED_HOME" ] && [ -d /usr/lib/hadoop-mapreduce-client ] ); then
+      export HADOOP_MAPRED_HOME=/usr/lib/hadoop-mapreduce-client
+    elif ( [ -z "$HADOOP_MAPRED_HOME" ] && [ -d /usr/lib/hadoop-mapreduce ] ); then
+      export HADOOP_MAPRED_HOME=/usr/lib/hadoop-mapreduce
+    fi
+    if ( [ -z "$HIVE_HOME" ] && [ -d /usr/lib/hive ] ); then
+      export HIVE_HOME=/usr/lib/hive
+    fi
+}
+
+
+print_cluster_info() {
+
+  # ODPI-87
+
+  echo "######################################################"
+  echo "#               CLUSTER INFORMATION                  #"
+  echo "######################################################"
+
+  which facter >/dev/null 2>&1
+  RC=$?
+  if [[ $RC == 0 ]]; then
+    echo "# OS: $(facter lsbdistdescription)"
+    echo "# ARCH: $(facter architecture)"
+    echo "# KERNEL: $(facter kernelrelease)"
+    echo "# MEMORY: $(facter memorysize)"
+  else
+    echo "# OS: $(cat /etc/issue | tr '\n' ' ')"
+    echo "# ARCH: $(uname -i)"
+    echo "# KERNEL: $(uname -a)"
+    echo "# MEMORY: $(head -n1 /proc/meminfo)"
+  fi
+
+  YARN_NODES=$(yarn node -list 2>/dev/null | egrep ^Total | sed 's/Total Nodes:/TOTAL YARN NODES: /g')
+  echo "# $YARN_NODES"
+
+  HADOOP_VERSION=$(hadoop version 2>/dev/null | head -n1)
+  echo "# HADOOP_VERSION: $HADOOP_VERSION"
+  echo "# HADOOP_CONF_DIR: $HADOOP_CONF_DIR"
+  echo "# HADOOP_MAPRED_HOME: $HADOOP_MAPRED_HOME"
+
+  echo "# BASH_VERSION: $BASH_VERSION"
+  echo "# SH_VERSION: $(/bin/sh -c 'echo $BASH_VERSION')"
+
+  echo "# JAVA_HOME: $JAVA_HOME"
+  JAVA_VERSION=$(java -version 2>&1 | head -n 1 | awk -F '"' '{print $2}')
+  echo "# JAVA_VERSION: $JAVA_VERSION"
+}
+
+print_tests() {
+  echo "######################################################"
+  echo "#                     RESULTS                        #"
+  echo "######################################################"
+
+  pushd `pwd`
+  for TEST in $(echo $ITESTS | tr ',' '\n'); do
+    TESTDIR=$DIR/bigtop-tests/smoke-tests/$TEST/build
+
+    if [ -d $TESTDIR ]; then
+      cd $TESTDIR
+
+      for FILE in $(find -L reports/tests/classes -type f -name "*.html"); do
+        echo "## $TESTDIR/$FILE"
+        if [ $(which links) ]; then
+            links $FILE -dump
+        else
+            echo "PLEASE INSTALL LINKS: sudo yum -y install links"
+        fi
+        echo ""
+      done
+    fi
+  done
+
+  popd
+  for TEST in $SPEC_TESTS; do
+    TESTDIR=$DIR/bigtop-tests/spec-tests/$TEST/build
+
+    if [ -d $TESTDIR ]; then
+      cd $TESTDIR
+
+      for FILE in $(find -L reports/tests/classes -type f -name "*.html"); do
+        echo "## $TESTDIR/$FILE"
+        if [ $(which links) ]; then
+            links $FILE -dump
+        else
+            echo "PLEASE INSTALL LINKS: sudo yum -y install links"
+        fi
+        echo ""
+      done
+    fi
+  done
+}
+
+# SET JAVA_HOME
+set_java_home
+
+# SET HADOOP SERVICE HOMES
+set_hadoop_vars
+
+# ODPI-87
+print_cluster_info
+
+echo "######################################################"
+echo "#                 STARTING ITEST                     #"
+echo "######################################################"
+echo "# Use --debug/--info for more details"
+
+# SET THE DEFAULT TESTS
+if [ -z "$ITESTS" ]; then
+  export ITESTS="hcfs,hdfs,yarn,mapreduce,hive"
+fi
+SPEC_TESTS="runtime"
+for s in `echo $ITESTS | sed -e 's#,# #g'`; do
+  ALL_SMOKE_TASKS="$ALL_SMOKE_TASKS bigtop-tests:smoke-tests:$s:test"
+done
+for s in $SPEC_TESTS; do
+  ALL_SPEC_TASKS="$ALL_SPEC_TASKS bigtop-tests:spec-tests:$s:test"
+done
+
+# CALL THE GRADLE WRAPPER TO RUN THE FRAMEWORK
+$DIR/gradlew -q --continue clean -Psmoke.tests $ALL_SMOKE_TASKS -Pspec.tests $ALL_SPEC_TASKS -Dodpi.test.hive.jdbc.url=$HIVE_JDBC_URL -Dodpi.test.hive.metastore.url=$HIVE_METASTORE_URL -Dodpi.test.hive.location=$HIVE_HDFS_LOCATION -Dodpi.test.hive.jdbc.user=$HIVE_USER -Dodpi.test.hive.jdbc.password=$HIVE_PASSWORD -Dodpi.test.hive.conf.dir=$HIVE_CONF_DIR -Dodpi.test.hadoop.conf.dir=$HADOOP_CONF_DIR -Dodpi.test.hive.thrift.test=$TEST_THRIFT -Dodpi.test.hive.hcatalog.test=$TEST_HCATALOG $LOGGING
+
+# SHOW RESULTS (HTML)
+print_tests


[40/50] [abbrv] bigtop git commit: Progress so far. Doesn't work yet, but committing to avoid another data loss.

Posted by rv...@apache.org.
Progress so far.  Doesn't work yet, but committing to avoid another data loss.


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

Branch: refs/heads/BIGTOP-2666
Commit: 27ba88b8238a5a482fb77e7d09d7902ea07314e6
Parents: 1965443
Author: Alan Gates <ga...@hortonworks.com>
Authored: Mon Nov 7 15:39:54 2016 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:23 2017 -0800

----------------------------------------------------------------------
 bigtop-tests/spec-tests/runtime/build.gradle    |   8 +-
 .../org/odpi/specs/runtime/hive/HCatalogMR.java | 124 ++++++++++
 .../odpi/specs/runtime/hive/JdbcConnector.java  |   3 +
 .../odpi/specs/runtime/hive/TestHCatalog.java   | 224 +++++++++++++++++++
 .../org/odpi/specs/runtime/hive/TestThrift.java |   2 +-
 5 files changed, 359 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/27ba88b8/bigtop-tests/spec-tests/runtime/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/build.gradle b/bigtop-tests/spec-tests/runtime/build.gradle
index 5505550..f0166c9 100644
--- a/bigtop-tests/spec-tests/runtime/build.gradle
+++ b/bigtop-tests/spec-tests/runtime/build.gradle
@@ -17,6 +17,8 @@
  */
 def junitVersion = '4.11'
 
+apply plugin: 'java'
+
 repositories {
   maven {
     url "http://conjars.org/repo/"
@@ -31,8 +33,12 @@ dependencies {
   compile group: 'org.apache.hive', name: 'hive-common', version: '1.2.1'
   compile group: 'org.apache.thrift', name: 'libfb303', version: '0.9.3'
   compile group: 'org.apache.thrift', name: 'libthrift', version: '0.9.3'
-  testCompile group: 'org.apache.hadoop', name: 'hadoop-common', version: '2.7.2'
+  compile group: 'org.apache.hadoop', name: 'hadoop-common', version: '2.7.2'
+  compile group: 'org.apache.hive.hcatalog', name: 'hive-hcatalog-core', version: '1.2.1'
   testCompile group: 'org.apache.hadoop', name: 'hadoop-mapreduce-client-core', version: '2.7.2'
+  compile group: 'org.apache.hadoop', name: 'hadoop-mapreduce-client-jobclient', version: '2.7.2'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-mapreduce-client-common', version: '2.7.2'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-hdfs', version: '2.7.2'
   testCompile group: 'org.apache.hive', name: 'hive-exec', version: '1.2.1'
   if (System.env.HADOOP_CONF_DIR) testRuntime files(System.env.HADOOP_CONF_DIR)
 }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/27ba88b8/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java b/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
new file mode 100644
index 0000000..4a733d6
--- /dev/null
+++ b/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
@@ -0,0 +1,124 @@
+/**
+ * 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.odpi.specs.runtime.hive;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.Tool;
+import org.apache.hive.hcatalog.data.DefaultHCatRecord;
+import org.apache.hive.hcatalog.data.HCatRecord;
+import org.apache.hive.hcatalog.data.schema.HCatSchema;
+import org.apache.hive.hcatalog.data.schema.HCatSchemaUtils;
+import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
+import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.StringTokenizer;
+
+public class HCatalogMR extends Configured implements Tool {
+  private final static String INPUT_SCHEMA = "odpi.test.hcat.schema.input";
+  private final static String OUTPUT_SCHEMA = "odpi.test.hcat.schema.output";
+
+  @Override
+  public int run(String[] args) throws Exception {
+    Configuration conf = getConf();
+    args = new GenericOptionsParser(conf, args).getRemainingArgs();
+
+    String inputTable = args[0];
+    String outputTable = args[1];
+    String inputSchemaStr = args[2];
+    String outputSchemaStr = args[3];
+
+    conf.set(INPUT_SCHEMA, inputSchemaStr);
+    conf.set(OUTPUT_SCHEMA, outputSchemaStr);
+
+    Job job = new Job(conf, "odpi_hcat_test");
+    HCatInputFormat.setInput(job, "default", inputTable);
+
+    job.setInputFormatClass(HCatInputFormat.class);
+    job.setJarByClass(HCatalogMR.class);
+    job.setMapperClass(Map.class);
+    job.setReducerClass(Reduce.class);
+    job.setMapOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(IntWritable.class);
+    job.setOutputKeyClass(WritableComparable.class);
+    job.setOutputValueClass(HCatRecord.class);
+    HCatOutputFormat.setOutput(job, OutputJobInfo.create("default", outputTable, null));
+    HCatOutputFormat.setSchema(job, HCatSchemaUtils.getHCatSchema(outputSchemaStr));
+    job.setOutputFormatClass(HCatOutputFormat.class);
+
+    job.addCacheArchive(new URI("hdfs:/user/gates/hive-hcatalog-core-1.2.1.jar"));
+    job.addCacheArchive(new URI("hdfs:/user/gates/hive-metastore-1.2.1.jar"));
+    job.addCacheArchive(new URI("hdfs:/user/gates/hive-exec-1.2.1.jar"));
+
+    return job.waitForCompletion(true) ? 0 : 1;
+
+
+  }
+  public static class Map extends Mapper<WritableComparable,
+          HCatRecord, Text, IntWritable> {
+    private final static IntWritable one = new IntWritable(1);
+    private Text word = new Text();
+    private HCatSchema inputSchema = null;
+
+    @Override
+    protected void map(WritableComparable key, HCatRecord value, Context context)
+        throws IOException, InterruptedException {
+      if (inputSchema == null) {
+        inputSchema =
+            HCatSchemaUtils.getHCatSchema(context.getConfiguration().get(INPUT_SCHEMA));
+      }
+      String line = value.getString("line", inputSchema);
+      StringTokenizer tokenizer = new StringTokenizer(line);
+      while (tokenizer.hasMoreTokens()) {
+        word.set(tokenizer.nextToken());
+        context.write(word, one);
+      }
+    }
+  }
+
+  public static class Reduce extends Reducer<Text, IntWritable, WritableComparable, HCatRecord> {
+    private HCatSchema outputSchema = null;
+
+    @Override
+    protected void reduce(Text key, Iterable<IntWritable> values, Context context) throws
+        IOException, InterruptedException {
+      if (outputSchema == null) {
+        outputSchema =
+            HCatSchemaUtils.getHCatSchema(context.getConfiguration().get(OUTPUT_SCHEMA));
+      }
+      int sum = 0;
+      for (IntWritable i : values) {
+        sum += i.get();
+      }
+      HCatRecord output = new DefaultHCatRecord(2);
+      output.set("word", outputSchema, key);
+      output.set("count", outputSchema, sum);
+      context.write(null, output);
+    }
+  }
+ }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/27ba88b8/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
index f5cc379..7512dab 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
@@ -36,6 +36,9 @@ public class JdbcConnector {
   protected static final String LOCATION = "odpi.test.hive.location";
   protected static final String METASTORE_URL = "odpi.test.hive.metastore.url";
   protected static final String TEST_THRIFT = "odpi.test.hive.thrift.test";
+  protected static final String TEST_HCATALOG = "odpi.test.hive.hcatalog.test";
+  protected static final String HIVE_CONF_DIR = "odpi.test.hive.conf.dir";
+  protected static final String HADOOP_CONF_DIR = "odpi.test.hadoop.conf.dir";
 
   protected static Connection conn;
 

http://git-wip-us.apache.org/repos/asf/bigtop/blob/27ba88b8/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
new file mode 100644
index 0000000..4b61131
--- /dev/null
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
@@ -0,0 +1,224 @@
+/**
+ * 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.odpi.specs.runtime.hive;
+
+import org.apache.commons.exec.CommandLine;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hive.hcatalog.data.DefaultHCatRecord;
+import org.apache.hive.hcatalog.data.HCatRecord;
+import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
+import org.apache.hive.hcatalog.data.schema.HCatSchema;
+import org.apache.hive.hcatalog.data.schema.HCatSchemaUtils;
+import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
+import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
+import org.apache.thrift.TException;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.StringTokenizer;
+
+
+public class TestHCatalog {
+
+  private static final Log LOG = LogFactory.getLog(TestHCatalog.class.getName());
+
+  private static IMetaStoreClient client = null;
+  private static HiveConf conf;
+  private static HCatSchema inputSchema;
+  private static HCatSchema outputSchema;
+
+  private Random rand;
+
+  @BeforeClass
+  public static void connect() throws MetaException {
+    if (JdbcConnector.testActive(JdbcConnector.TEST_HCATALOG, "Test HCatalog ")) {
+      String hiveConfDir = JdbcConnector.getProperty(JdbcConnector.HIVE_CONF_DIR,
+          "Hive conf directory ");
+      String hadoopConfDir = JdbcConnector.getProperty(JdbcConnector.HADOOP_CONF_DIR,
+          "Hadoop conf directory ");
+      conf = new HiveConf();
+      String fileSep = System.getProperty("file.separator");
+      conf.addResource(new Path(hadoopConfDir + fileSep + "core-site.xml"));
+      conf.addResource(new Path(hadoopConfDir + fileSep + "hdfs-site.xml"));
+      conf.addResource(new Path(hadoopConfDir + fileSep + "yarn-site.xml"));
+      conf.addResource(new Path(hadoopConfDir + fileSep + "mapred-site.xml"));
+      conf.addResource(new Path(hiveConfDir + fileSep + "hive-site.xml"));
+      client = new HiveMetaStoreClient(conf);
+
+    }
+  }
+
+  @Before
+  public void checkIfActive() {
+    Assume.assumeTrue(JdbcConnector.testActive(JdbcConnector.TEST_HCATALOG, "Test HCatalog "));
+    rand = new Random();
+  }
+
+  @Test
+  public void hcatInputFormatOutputFormat() throws TException, IOException, ClassNotFoundException,
+      InterruptedException, URISyntaxException {
+    // Create a table to write to
+    final String inputTable = "odpi_hcat_input_table_" + rand.nextInt(Integer.MAX_VALUE);
+    SerDeInfo serde = new SerDeInfo("default_serde",
+        conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), new HashMap<String, String>());
+    FieldSchema schema = new FieldSchema("line", "string", "");
+    inputSchema = new HCatSchema(Collections.singletonList(new HCatFieldSchema(schema.getName(),
+        HCatFieldSchema.Type.STRING, schema.getComment())));
+    StorageDescriptor sd = new StorageDescriptor(Collections.singletonList(schema), null,
+        "org.apache.hadoop.mapred.TextInputFormat",
+        "org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat", false, 0, serde, null, null,
+        new HashMap<String, String>());
+    Table table = new Table(inputTable, "default", "me", 0, 0, 0, sd, null,
+        new HashMap<String, String>(), null, null, TableType.MANAGED_TABLE.toString());
+    client.createTable(table);
+
+    final String outputTable = "odpi_hcat_output_table_" + rand.nextInt(Integer.MAX_VALUE);
+    sd = new StorageDescriptor(Arrays.asList(
+          new FieldSchema("word", "string", ""),
+          new FieldSchema("count", "int", "")),
+        null, "org.apache.hadoop.mapred.TextInputFormat",
+        "org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat", false, 0, serde, null, null,
+        new HashMap<String, String>());
+    table = new Table(outputTable, "default", "me", 0, 0, 0, sd, null,
+        new HashMap<String, String>(), null, null, TableType.MANAGED_TABLE.toString());
+    client.createTable(table);
+    outputSchema = new HCatSchema(Arrays.asList(
+        new HCatFieldSchema("word", HCatFieldSchema.Type.STRING, ""),
+        new HCatFieldSchema("count", HCatFieldSchema.Type.INT, "")));
+
+    // TODO Could I use HCatWriter hear and the reader to read it?
+    // Write some stuff into a file in the location of the table
+    table = client.getTable("default", inputTable);
+    String inputFile = table.getSd().getLocation() + "/input";
+    /*
+    String inputFile = JdbcConnector.getProperty(JdbcConnector.LOCATION,
+        "Directory to write a file in ") + "/odpi_hcat_input_" + rand.nextInt(Integer.MAX_VALUE);
+        */
+    Path inputPath = new Path(inputFile);
+    FileSystem fs = FileSystem.get(conf);
+    FSDataOutputStream out = fs.create(inputPath);
+    out.writeChars("Mary had a little lamb\n");
+    out.writeChars("its fleece was white as snow\n");
+    out.writeChars("and everywhere that Mary went\n");
+    out.writeChars("the lamb was sure to go\n");
+    out.close();
+
+    Map<String, String> results = HiveHelper.execCommand(new CommandLine("hadoop")
+        .addArgument("jar")
+        .addArgument("/Users/gates/git/bigtop/runtime-1.2.0-SNAPSHOT.jar")
+        .addArgument(HCatalogMR.class.getName())
+        .addArgument(inputTable)
+        .addArgument(outputTable)
+        .addArgument(inputSchema.getSchemaAsTypeString())
+        .addArgument(outputSchema.getSchemaAsTypeString()));
+    Assert.assertEquals("HCat job failed", 0, Integer.parseInt(results.get("exitValue")));
+
+
+
+    /*
+    Job job = new Job(conf, "odpi_hcat_test");
+    HCatInputFormat.setInput(job, "default", inputTable);
+
+    job.setInputFormatClass(HCatInputFormat.class);
+    job.setJarByClass(TestHCatalog.class);
+    job.setMapperClass(Map.class);
+    job.setReducerClass(Reduce.class);
+    job.setMapOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(IntWritable.class);
+    job.setOutputKeyClass(WritableComparable.class);
+    job.setOutputValueClass(HCatRecord.class);
+    HCatOutputFormat.setOutput(job, OutputJobInfo.create("default", outputTable, null));
+    HCatOutputFormat.setSchema(job, outputSchema);
+    job.setOutputFormatClass(HCatOutputFormat.class);
+
+    job.addCacheArchive(new URI("hdfs:/user/gates/hive-hcatalog-core-1.2.1.jar"));
+    job.addCacheArchive(new URI("hdfs:/user/gates/hive-metastore-1.2.1.jar"));
+    job.addCacheArchive(new URI("hdfs:/user/gates/hive-exec-1.2.1.jar"));
+
+    Assert.assertTrue(job.waitForCompletion(true));
+    */
+
+    client.dropTable("default", inputTable);
+    client.dropTable("default", outputTable);
+  }
+
+  /*
+  public static class Map extends Mapper<WritableComparable,
+        HCatRecord, Text, IntWritable> {
+    private final static IntWritable one = new IntWritable(1);
+    private Text word = new Text();
+
+    @Override
+    protected void map(WritableComparable key, HCatRecord value, Context context)
+        throws IOException, InterruptedException {
+      String line = value.getString("line", inputSchema);
+      StringTokenizer tokenizer = new StringTokenizer(line);
+      while (tokenizer.hasMoreTokens()) {
+        word.set(tokenizer.nextToken());
+        context.write(word, one);
+      }
+    }
+  }
+
+  public static class Reduce extends Reducer<Text, IntWritable, WritableComparable, HCatRecord> {
+    @Override
+    protected void reduce(Text key, Iterable<IntWritable> values, Context context) throws
+        IOException, InterruptedException {
+      int sum = 0;
+      for (IntWritable i : values) {
+        sum += i.get();
+      }
+      HCatRecord output = new DefaultHCatRecord(2);
+      output.set("word", outputSchema, key);
+      output.set("count", outputSchema, sum);
+      context.write(null, output);
+    }
+  }
+  */
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/27ba88b8/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
index 5eaab95..8e0abda 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
@@ -45,7 +45,7 @@ import java.util.Random;
 
 public class TestThrift {
 
-  private static final Log LOG = LogFactory.getLog(JdbcConnector.class.getName());
+  private static final Log LOG = LogFactory.getLog(TestThrift.class.getName());
 
   private static IMetaStoreClient client = null;
   private static HiveConf conf;


[19/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-site.xml
new file mode 100755
index 0000000..27d3541
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-site.xml
@@ -0,0 +1,909 @@
+<configuration><property require-input="false">
+    <name>hive.server2.thrift.port</name>
+    <value>10500</value>
+    <description>
+      TCP port number to listen on, default 10500.
+    </description>
+    <display-name>HiveServer2 Port</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.thrift.http.port</name>
+    <value>10501</value>
+    <description>Port number of HiveServer2 Thrift interface when hive.server2.transport.mode is 'http'.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.tez.sessions.per.default.queue</name>
+    <value>1</value>
+    <description>
+      The maximum number of queries the Hive Interactive cluster will be able to handle concurrently.
+    </description>
+    <display-name>Maximum Total Concurrent Queries</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <maximum>10</maximum>
+        <minimum>1</minimum>
+        <increment-step>1</increment-step>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </property>
+        <property>
+            <name>llap_queue_capacity</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>enable_hive_interactive</name>
+            <type>hive-interactive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.llap.daemon.num.executors</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>llap_heap_size</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>tez.am.resource.memory.mb</name>
+            <type>tez-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.llap.daemon.yarn.container.mb</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.llap.io.memory.size</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>num_llap_nodes</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>slider_am_container_mb</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive.metastore.uris</name>
+    <value></value>
+    <description>Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <empty-value-valid>true</empty-value-valid>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.enable.doAs</name>
+    <value>false</value>
+    <description>
+      Setting this property to true will have HiveServer2 execute
+      Hive operations as the user making the calls to it.
+    </description>
+    <display-name>Run as end user instead of Hive user</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_security_authorization</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.prewarm.enabled</name>
+    <value>false</value>
+    <description>Enables container prewarm for Tez (Hadoop 2 only)</description>
+    <display-name>Hold Containers to Reduce Latency</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.vectorized.execution.reduce.enabled</name>
+    <value>true</value>
+    <description>
+      This flag should be set to true to enable vectorized mode of the reduce-side of
+      query execution.
+    </description>
+    <display-name>Enable Reduce Vectorization</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.tez.default.queues</name>
+    <value>default</value>
+    <description>
+      A list of comma separated values corresponding to YARN queues of the same name.
+      When HiveServer2 is launched in Tez mode, this configuration needs to be set
+      for multiple Tez sessions to run in parallel on the cluster.
+    </description>
+    <display-name>Default query queues</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>combo</type>
+        <entries>
+            <entry>
+                <value>default</value>
+                <label>Default</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1+</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.tez.initialize.default.sessions</name>
+    <value>true</value>
+    <description>
+      This flag is used in HiveServer2 to enable a user to use HiveServer2 without
+      turning on Tez for HiveServer2. The user could potentially want to run queries
+      over Tez without the pool of sessions.
+    </description>
+    <display-name>Start Tez session at Initialization</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.driver.parallel.compilation</name>
+    <value>true</value>
+    <description>
+      This flag allows HiveServer2 to compile queries in parallel.
+    </description>
+    <display-name>Compile queries in parallel</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.webui.port</name>
+    <value>10502</value>
+    <description>Web UI port address</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.webui.use.ssl</name>
+    <value>false</value>
+    <description>Enable SSL for HiveServer2 Interactive</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.server2.zookeeper.namespace</name>
+    <value>hiveserver2-hive2</value>
+    <description>The parent node in ZooKeeper used by HiveServer2 when supporting dynamic service discovery.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.daemon.queue.name</name>
+    <value>default</value>
+    <description>Choose the YARN queue in this cluster that is dedicated to interactive query.</description>
+    <display-name>Interactive Query Queue</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>combo</type>
+        <entries>
+            <entry>
+                <value>default</value>
+                <label>Default</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>enable_hive_interactive</name>
+            <type>hive-interactive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.server2.tez.default.queues</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.llap.daemon.num.executors</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>llap_queue_capacity</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>llap_heap_size</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>tez.am.resource.memory.mb</name>
+            <type>tez-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.llap.daemon.yarn.container.mb</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.server2.tez.sessions.per.default.queue</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>hive.llap.io.memory.size</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>num_llap_nodes</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+        <dependedByProperties>
+            <name>slider_am_container_mb</name>
+            <type>hive-interactive-env</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive.llap.daemon.yarn.shuffle.port</name>
+    <value>15551</value>
+    <description>YARN shuffle port for LLAP-daemon-hosted shuffle.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.execution.engine</name>
+    <value>tez</value>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.execution.mode</name>
+    <value>llap</value>
+    <description>Chooses whether query fragments will run in container or in llap</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.execution.mode</name>
+    <value>all</value>
+    <description>Chooses which fragments of a query will run in llap</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.io.enabled</name>
+    <value>true</value>
+    <description>Whether the LLAP IO layer is enabled.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive.llap.io.memory.size</name>
+            <type>hive-interactive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.io.use.lrfu</name>
+    <value>true</value>
+    <description>Whether ORC low-level cache should use LRFU cache policy instead of default (FIFO).</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.auto.allow.uber</name>
+    <value>false</value>
+    <description>Whether or not to allow the planner to run vertices in the AM.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.object.cache.enabled</name>
+    <value>true</value>
+    <description>Cache objects (plans, hashtables, etc) in llap</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.input.generate.consistent.splits</name>
+    <value>true</value>
+    <description>Whether to generate consistent split locations when generating splits in the AM</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.client.consistent.splits</name>
+    <value>true</value>
+    <description>
+      Whether to setup split locations to match nodes on which llap daemons are running,
+      instead of using the locations provided by the split itself.
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.task.scheduler.locality.delay</name>
+    <value>-1</value>
+    <description>
+      Amount of time to wait before allocating a request which contains location information,
+      to a location other than the ones requested. Set to -1 for an infinite delay, 0
+      for no delay.
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.exec.orc.split.strategy</name>
+    <value>HYBRID</value>
+    <description>
+      This is not a user level config. BI strategy is used when the requirement is to spend less time in split generation
+      as opposed to query execution (split generation does not read or cache file footers).
+      ETL strategy is used when spending little more time in split generation is acceptable
+      (split generation reads and caches file footers). HYBRID chooses between the above strategies
+      based on heuristics.
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.daemon.service.hosts</name>
+    <value>@llap0</value>
+    <description>
+      Explicitly specified hosts to use for LLAP scheduling. Useful for testing. By default,
+      YARN registry is used.
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.daemon.allow.permanent.fns</name>
+    <value>false</value>
+    <description>Whether LLAP daemon should localize the resources for permanent UDFs.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.io.memory.size</name>
+    <value>0</value>
+    <description>The amount of memory reserved for Hive's optimized in-memory cache.</description>
+    <display-name>In-Memory Cache per Daemon</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <unit>MB</unit>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>tez.am.resource.memory.mb</name>
+            <type>tez-site</type>
+        </property>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </property>
+        <property>
+            <name>llap_queue_capacity</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>enable_hive_interactive</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>hive.server2.tez.sessions.per.default.queue</name>
+            <type>hive-interactive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.llap.io.enabled</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive.llap.daemon.num.executors</name>
+    <value>1</value>
+    <description>The maximum number of CPUs a single LLAP daemon will use. Usually this should be equal to the number of available CPUs.</description>
+    <display-name>Maximum CPUs per Daemon</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>tez.am.resource.memory.mb</name>
+            <type>tez-site</type>
+        </property>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </property>
+        <property>
+            <name>llap_queue_capacity</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>enable_hive_interactive</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>hive.server2.tez.sessions.per.default.queue</name>
+            <type>hive-interactive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by>
+        <dependedByProperties>
+            <name>hive.llap.io.threadpool.size</name>
+            <type>hive-interactive-site</type>
+        </dependedByProperties>
+    </property_depended_by>
+</property><property require-input="false">
+    <name>hive.llap.daemon.vcpus.per.instance</name>
+    <value>${hive.llap.daemon.num.executors}</value>
+    <description>The total number of vcpus to use for the executors inside LLAP.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.daemon.yarn.container.mb</name>
+    <value>341</value>
+    <description>Total memory used by individual LLAP daemons. This includes memory for the cache as well as for the query execution.</description>
+    <display-name>Memory per daemon</display-name>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+        <unit>MB</unit>
+        <overridable>false</overridable>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </property>
+        <property>
+            <name>llap_queue_capacity</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>enable_hive_interactive</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>hive.server2.tez.sessions.per.default.queue</name>
+            <type>hive-interactive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>llap.shuffle.connection-keep-alive.enable</name>
+    <value>true</value>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>llap.shuffle.connection-keep-alive.timeout</name>
+    <value>60</value>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.io.threadpool.size</name>
+    <value>2</value>
+    <description>Specify the number of threads to use for low-level IO thread pool.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive.llap.daemon.num.executors</name>
+            <type>hive-interactive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.daemon.rpc.port</name>
+    <value>15001</value>
+    <description>The LLAP daemon RPC port.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.management.rpc.port</name>
+    <value>15004</value>
+    <description>RPC port for LLAP daemon management service.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.daemon.task.scheduler.enable.preemption</name>
+    <value>true</value>
+    <description>hive.llap.daemon.task.scheduler.enable.preemption</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.exec.print.summary</name>
+    <value>true</value>
+    <description>Display breakdown of execution steps, for every query executed by the shell.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.vectorized.execution.mapjoin.native.enabled</name>
+    <value>true</value>
+    <description>
+      This flag should be set to true to enable native (i.e. non-pass through) vectorization
+      of queries using MapJoin.
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.vectorized.execution.mapjoin.minmax.enabled</name>
+    <value>true</value>
+    <description>
+      This flag should be set to true to enable vector map join hash tables to
+      use max / max filtering for integer join queries using MapJoin.
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.vectorized.execution.mapjoin.native.fast.hashtable.enabled</name>
+    <value>true</value>
+    <description>
+      This flag should be set to true to enable use of native fast vector map join hash tables in
+      queries using MapJoin.
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.mapjoin.hybridgrace.hashtable</name>
+    <value>false</value>
+    <description>Whether to use hybrid grace hash join as the join method for mapjoin. Tez only.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.tez.bucket.pruning</name>
+    <value>true</value>
+    <description>
+      When pruning is enabled, filters on bucket columns will be processed by
+      filtering the splits against a bitset of included buckets. This needs predicates
+      produced by hive.optimize.ppd and hive.optimize.index.filters.
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.optimize.dynamic.partition.hashjoin</name>
+    <value>true</value>
+    <description>
+      Whether to enable dynamically partitioned hash join optimization.
+      This setting is also dependent on enabling hive.auto.convert.join
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.zk.sm.connectionString</name>
+    <value>localhost:2181</value>
+    <description>ZooKeeper connection string for ZooKeeper SecretManager.</description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>clientPort</name>
+            <type>zoo.cfg</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.llap.io.memory.mode</name>
+    <value></value>
+    <description>
+      LLAP IO memory usage; 'cache' (the default) uses data and metadata cache with a
+      custom off-heap allocator, 'allocator' uses the custom allocator without the caches,
+      'none' doesn't use either (this mode may result in significant performance degradation)
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <empty-value-valid>true</empty-value-valid>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.metastore.event.listeners</name>
+    <value></value>
+    <description>
+      Listeners for metastore events
+    </description>
+    <filename>hive-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <empty-value-valid>true</empty-value-valid>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j.xml
new file mode 100755
index 0000000..3ecb24a
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j.xml
@@ -0,0 +1,106 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+# 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.
+
+# Define some default values that can be overridden by system properties
+hive.log.threshold=ALL
+hive.root.logger=INFO,DRFA
+hive.log.dir=${java.io.tmpdir}/${user.name}
+hive.log.file=hive.log
+
+# Define the root logger to the system property "hadoop.root.logger".
+log4j.rootLogger=${hive.root.logger}, EventCounter
+
+# Logging Threshold
+log4j.threshold=${hive.log.threshold}
+
+#
+# Daily Rolling File Appender
+#
+# Use the PidDailyerRollingFileAppend class instead if you want to use separate log files
+# for different CLI session.
+#
+# log4j.appender.DRFA=org.apache.hadoop.hive.ql.log.PidDailyRollingFileAppender
+
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+
+log4j.appender.DRFA.File=${hive.log.dir}/${hive.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this
+#
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n
+log4j.appender.console.encoding=UTF-8
+
+#custom logging levels
+#log4j.logger.xxx=DEBUG
+
+#
+# Event Counter Appender
+# Sends counts of logging messages at different severity levels to Hadoop Metrics.
+#
+log4j.appender.EventCounter=org.apache.hadoop.hive.shims.HiveEventCounter
+
+
+log4j.category.DataNucleus=ERROR,DRFA
+log4j.category.Datastore=ERROR,DRFA
+log4j.category.Datastore.Schema=ERROR,DRFA
+log4j.category.JPOX.Datastore=ERROR,DRFA
+log4j.category.JPOX.Plugin=ERROR,DRFA
+log4j.category.JPOX.MetaData=ERROR,DRFA
+log4j.category.JPOX.Query=ERROR,DRFA
+log4j.category.JPOX.General=ERROR,DRFA
+log4j.category.JPOX.Enhancer=ERROR,DRFA
+
+
+# Silence useless ZK logs
+log4j.logger.org.apache.zookeeper.server.NIOServerCnxn=WARN,DRFA
+log4j.logger.org.apache.zookeeper.ClientCnxnSocketNIO=WARN,DRFA
+
+    </value>
+    <description>Custom log4j.properties</description>
+    <display-name>hive-log4j template</display-name>
+    <filename>hive-log4j.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+        <show-property-name>false</show-property-name>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j2.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j2.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j2.xml
new file mode 100755
index 0000000..798063b
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j2.xml
@@ -0,0 +1,90 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+# 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.
+
+status = INFO
+name = HiveLog4j2
+packages = org.apache.hadoop.hive.ql.log
+
+# list of properties
+property.hive.log.level = INFO
+property.hive.root.logger = DRFA
+property.hive.log.dir = ${sys:java.io.tmpdir}/${sys:user.name}
+property.hive.log.file = hive.log
+
+# list of all appenders
+appenders = console, DRFA
+
+# console appender
+appender.console.type = Console
+appender.console.name = console
+appender.console.target = SYSTEM_ERR
+appender.console.layout.type = PatternLayout
+appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n
+
+# daily rolling file appender
+appender.DRFA.type = RollingFile
+appender.DRFA.name = DRFA
+appender.DRFA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
+# Use %pid in the filePattern to append process-id@host-name to the filename if you want separate log files for different CLI session
+appender.DRFA.filePattern = ${sys:hive.log.dir}/${sys:hive.log.file}.%d{yyyy-MM-dd}.gz
+appender.DRFA.layout.type = PatternLayout
+appender.DRFA.layout.pattern = %d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n
+appender.DRFA.policies.type = Policies
+appender.DRFA.policies.time.type = TimeBasedTriggeringPolicy
+appender.DRFA.policies.time.interval = 1
+appender.DRFA.policies.time.modulate = true
+appender.DRFA.strategy.type = DefaultRolloverStrategy
+appender.DRFA.strategy.max = 30
+
+# list of all loggers
+loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX
+
+logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn
+logger.NIOServerCnxn.level = WARN
+
+logger.ClientCnxnSocketNIO.name = org.apache.zookeeper.ClientCnxnSocketNIO
+logger.ClientCnxnSocketNIO.level = WARN
+
+logger.DataNucleus.name = DataNucleus
+logger.DataNucleus.level = ERROR
+
+logger.Datastore.name = Datastore
+logger.Datastore.level = ERROR
+
+logger.JPOX.name = JPOX
+logger.JPOX.level = ERROR
+
+# root logger
+rootLogger.level = ${sys:hive.log.level}
+rootLogger.appenderRefs = root
+rootLogger.appenderRef.root.ref = ${sys:hive.root.logger}
+  </value>
+    <description>Custom hive-log4j2.properties</description>
+    <display-name>hive-log4j2 template</display-name>
+    <filename>hive-log4j2.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+        <show-property-name>false</show-property-name>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file


[48/50] [abbrv] bigtop git commit: Include keys for arguments passed to HCatalogMR

Posted by rv...@apache.org.
Include keys for arguments passed to HCatalogMR


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

Branch: refs/heads/BIGTOP-2666
Commit: 1df0c5d55f9e5c2c618a9261707e831e30687f8e
Parents: ebe22bc
Author: Raj Desai <rd...@us.ibm.com>
Authored: Thu Jan 19 15:12:12 2017 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:26 2017 -0800

----------------------------------------------------------------------
 .../src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java  | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/1df0c5d5/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
index b51db02..0ea49ce 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
@@ -140,9 +140,13 @@ public class TestHCatalog {
         .addArgument("jar")
         .addArgument(System.getProperty(JOBJAR))
         .addArgument(HCatalogMR.class.getName())
+        .addArgument("-it")
         .addArgument(inputTable)
+        .addArgument("-ot")
         .addArgument(outputTable)
+        .addArgument("-is")
         .addArgument(inputSchema.getSchemaAsTypeString())
+        .addArgument("-os")
         .addArgument(outputSchema.getSchemaAsTypeString()), env);
     LOG.info(results.toString());
     Assert.assertEquals("HCat job failed", 0, Integer.parseInt(results.get("exitValue")));


[10/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive.py
new file mode 100755
index 0000000..4f53ea9
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive.py
@@ -0,0 +1,481 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import os
+import glob
+from urlparse import urlparse
+
+from resource_management.libraries.script.script import Script
+from resource_management.libraries.resources.hdfs_resource import HdfsResource
+from resource_management.libraries.functions.copy_tarball import copy_to_hdfs
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.core.resources.service import ServiceConfig
+from resource_management.core.resources.system import File, Execute, Directory
+from resource_management.core.source import StaticFile, Template, DownloadSource, InlineTemplate
+from resource_management.core.shell import as_user
+from resource_management.libraries.functions.is_empty import is_empty
+from resource_management.libraries.resources.xml_config import XmlConfig
+from resource_management.libraries.functions.format import format
+from resource_management.core.exceptions import Fail
+from resource_management.core.shell import as_sudo
+from resource_management.core.shell import quote_bash_args
+from resource_management.core.logger import Logger
+from resource_management.core import utils
+from resource_management.libraries.functions.setup_atlas_hook import has_atlas_in_cluster, setup_atlas_hook
+from ambari_commons.constants import SERVICE
+
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def hive(name=None):
+  import params
+
+  XmlConfig("hive-site.xml",
+            conf_dir = params.hive_conf_dir,
+            configurations = params.config['configurations']['hive-site'],
+            owner=params.hive_user,
+            configuration_attributes=params.config['configuration_attributes']['hive-site']
+  )
+
+  if name in ["hiveserver2","metastore"]:
+    # Manually overriding service logon user & password set by the installation package
+    service_name = params.service_map[name]
+    ServiceConfig(service_name,
+                  action="change_user",
+                  username = params.hive_user,
+                  password = Script.get_password(params.hive_user))
+    Execute(format("cmd /c hadoop fs -mkdir -p {hive_warehouse_dir}"), logoutput=True, user=params.hadoop_user)
+
+  if name == 'metastore':
+    if params.init_metastore_schema:
+      check_schema_created_cmd = format('cmd /c "{hive_bin}\\hive.cmd --service schematool -info '
+                                        '-dbType {hive_metastore_db_type} '
+                                        '-userName {hive_metastore_user_name} '
+                                        '-passWord {hive_metastore_user_passwd!p}'
+                                        '&set EXITCODE=%ERRORLEVEL%&exit /B %EXITCODE%"', #cmd "feature", propagate the process exit code manually
+                                        hive_bin=params.hive_bin,
+                                        hive_metastore_db_type=params.hive_metastore_db_type,
+                                        hive_metastore_user_name=params.hive_metastore_user_name,
+                                        hive_metastore_user_passwd=params.hive_metastore_user_passwd)
+      try:
+        Execute(check_schema_created_cmd)
+      except Fail:
+        create_schema_cmd = format('cmd /c {hive_bin}\\hive.cmd --service schematool -initSchema '
+                                   '-dbType {hive_metastore_db_type} '
+                                   '-userName {hive_metastore_user_name} '
+                                   '-passWord {hive_metastore_user_passwd!p}',
+                                   hive_bin=params.hive_bin,
+                                   hive_metastore_db_type=params.hive_metastore_db_type,
+                                   hive_metastore_user_name=params.hive_metastore_user_name,
+                                   hive_metastore_user_passwd=params.hive_metastore_user_passwd)
+        Execute(create_schema_cmd,
+                user = params.hive_user,
+                logoutput=True
+        )
+
+  if name == "hiveserver2":
+    if params.hive_execution_engine == "tez":
+      # Init the tez app dir in hadoop
+      script_file = __file__.replace('/', os.sep)
+      cmd_file = os.path.normpath(os.path.join(os.path.dirname(script_file), "..", "files", "hiveTezSetup.cmd"))
+
+      Execute("cmd /c " + cmd_file, logoutput=True, user=params.hadoop_user)
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def hive(name=None):
+  import params
+
+  if name == 'hiveserver2':
+    # copy tarball to HDFS feature not supported
+    if not (params.stack_version_formatted_major and check_stack_feature(StackFeature.COPY_TARBALL_TO_HDFS, params.stack_version_formatted_major)):  
+      params.HdfsResource(params.webhcat_apps_dir,
+                            type="directory",
+                            action="create_on_execute",
+                            owner=params.webhcat_user,
+                            mode=0755
+                          )
+    
+    # Create webhcat dirs.
+    if params.hcat_hdfs_user_dir != params.webhcat_hdfs_user_dir:
+      params.HdfsResource(params.hcat_hdfs_user_dir,
+                           type="directory",
+                           action="create_on_execute",
+                           owner=params.hcat_user,
+                           mode=params.hcat_hdfs_user_mode
+      )
+
+    params.HdfsResource(params.webhcat_hdfs_user_dir,
+                         type="directory",
+                         action="create_on_execute",
+                         owner=params.webhcat_user,
+                         mode=params.webhcat_hdfs_user_mode
+    )
+
+    # ****** Begin Copy Tarballs ******
+    # *********************************
+    #  if copy tarball to HDFS feature  supported copy mapreduce.tar.gz and tez.tar.gz to HDFS
+    if params.stack_version_formatted_major and check_stack_feature(StackFeature.COPY_TARBALL_TO_HDFS, params.stack_version_formatted_major):
+      copy_to_hdfs("mapreduce", params.user_group, params.hdfs_user, host_sys_prepped=params.host_sys_prepped)
+      copy_to_hdfs("tez", params.user_group, params.hdfs_user, host_sys_prepped=params.host_sys_prepped)
+
+    # Always copy pig.tar.gz and hive.tar.gz using the appropriate mode.
+    # This can use a different source and dest location to account
+    copy_to_hdfs("pig",
+                 params.user_group,
+                 params.hdfs_user,
+                 file_mode=params.tarballs_mode,
+                 custom_source_file=params.pig_tar_source,
+                 custom_dest_file=params.pig_tar_dest_file,
+                 host_sys_prepped=params.host_sys_prepped)
+    copy_to_hdfs("hive",
+                 params.user_group,
+                 params.hdfs_user,
+                 file_mode=params.tarballs_mode,
+                 custom_source_file=params.hive_tar_source,
+                 custom_dest_file=params.hive_tar_dest_file,
+                 host_sys_prepped=params.host_sys_prepped)
+
+    wildcard_tarballs = ["sqoop", "hadoop_streaming"]
+    for tarball_name in wildcard_tarballs:
+      source_file_pattern = eval("params." + tarball_name + "_tar_source")
+      dest_dir = eval("params." + tarball_name + "_tar_dest_dir")
+
+      if source_file_pattern is None or dest_dir is None:
+        continue
+
+      source_files = glob.glob(source_file_pattern) if "*" in source_file_pattern else [source_file_pattern]
+      for source_file in source_files:
+        src_filename = os.path.basename(source_file)
+        dest_file = os.path.join(dest_dir, src_filename)
+
+        copy_to_hdfs(tarball_name,
+                     params.user_group,
+                     params.hdfs_user,
+                     file_mode=params.tarballs_mode,
+                     custom_source_file=source_file,
+                     custom_dest_file=dest_file,
+                     host_sys_prepped=params.host_sys_prepped)
+    # ******* End Copy Tarballs *******
+    # *********************************
+    
+    # if warehouse directory is in DFS
+    if not params.whs_dir_protocol or params.whs_dir_protocol == urlparse(params.default_fs).scheme:
+      # Create Hive Metastore Warehouse Dir
+      params.HdfsResource(params.hive_apps_whs_dir,
+                           type="directory",
+                            action="create_on_execute",
+                            owner=params.hive_user,
+                            mode=0777
+      )
+    else:
+      Logger.info(format("Not creating warehouse directory '{hive_apps_whs_dir}', as the location is not in DFS."))
+
+    # Create Hive User Dir
+    params.HdfsResource(params.hive_hdfs_user_dir,
+                         type="directory",
+                          action="create_on_execute",
+                          owner=params.hive_user,
+                          mode=params.hive_hdfs_user_mode
+    )
+    
+    if not is_empty(params.hive_exec_scratchdir) and not urlparse(params.hive_exec_scratchdir).path.startswith("/tmp"):
+      params.HdfsResource(params.hive_exec_scratchdir,
+                           type="directory",
+                           action="create_on_execute",
+                           owner=params.hive_user,
+                           group=params.hdfs_user,
+                           mode=0777) # Hive expects this dir to be writeable by everyone as it is used as a temp dir
+      
+    params.HdfsResource(None, action="execute")
+
+  Directory(params.hive_etc_dir_prefix,
+            mode=0755
+  )
+
+  # We should change configurations for client as well as for server.
+  # The reason is that stale-configs are service-level, not component.
+  Logger.info("Directories to fill with configs: %s" % str(params.hive_conf_dirs_list))
+  for conf_dir in params.hive_conf_dirs_list:
+    fill_conf_dir(conf_dir)
+
+  XmlConfig("hive-site.xml",
+            conf_dir=params.hive_config_dir,
+            configurations=params.hive_site_config,
+            configuration_attributes=params.config['configuration_attributes']['hive-site'],
+            owner=params.hive_user,
+            group=params.user_group,
+            mode=0644)
+
+  # Generate atlas-application.properties.xml file
+  if has_atlas_in_cluster():
+    atlas_hook_filepath = os.path.join(params.hive_config_dir, params.atlas_hook_filename)
+    setup_atlas_hook(SERVICE.HIVE, params.hive_atlas_application_properties, atlas_hook_filepath, params.hive_user, params.user_group)
+  
+  if name == 'hiveserver2':
+    XmlConfig("hiveserver2-site.xml",
+              conf_dir=params.hive_server_conf_dir,
+              configurations=params.config['configurations']['hiveserver2-site'],
+              configuration_attributes=params.config['configuration_attributes']['hiveserver2-site'],
+              owner=params.hive_user,
+              group=params.user_group,
+              mode=0644)
+
+  if params.hive_metastore_site_supported and name == 'metastore':
+    XmlConfig("hivemetastore-site.xml",
+              conf_dir=params.hive_server_conf_dir,
+              configurations=params.config['configurations']['hivemetastore-site'],
+              configuration_attributes=params.config['configuration_attributes']['hivemetastore-site'],
+              owner=params.hive_user,
+              group=params.user_group,
+              mode=0644)
+  
+  File(format("{hive_config_dir}/hive-env.sh"),
+       owner=params.hive_user,
+       group=params.user_group,
+       content=InlineTemplate(params.hive_env_sh_template)
+  )
+
+  # On some OS this folder could be not exists, so we will create it before pushing there files
+  Directory(params.limits_conf_dir,
+            create_parents = True,
+            owner='root',
+            group='root'
+            )
+
+  File(os.path.join(params.limits_conf_dir, 'hive.conf'),
+       owner='root',
+       group='root',
+       mode=0644,
+       content=Template("hive.conf.j2")
+       )
+
+  if name == 'metastore' or name == 'hiveserver2':
+    if params.hive_jdbc_target is not None and not os.path.exists(params.hive_jdbc_target):
+      jdbc_connector(params.hive_jdbc_target, params.hive_previous_jdbc_jar)
+    if params.hive2_jdbc_target is not None and not os.path.exists(params.hive2_jdbc_target):
+      jdbc_connector(params.hive2_jdbc_target, params.hive2_previous_jdbc_jar)
+
+  File(format("/usr/lib/ambari-agent/{check_db_connection_jar_name}"),
+       content = DownloadSource(format("{jdk_location}{check_db_connection_jar_name}")),
+       mode = 0644,
+  )
+
+  if name == 'metastore':
+    File(os.path.join(params.hive_server_conf_dir, "hadoop-metrics2-hivemetastore.properties"),
+         owner=params.hive_user,
+         group=params.user_group,
+         content=Template("hadoop-metrics2-hivemetastore.properties.j2")
+    )
+
+    File(params.start_metastore_path,
+         mode=0755,
+         content=StaticFile('startMetastore.sh')
+    )
+    if params.init_metastore_schema:
+      create_schema_cmd = format("export HIVE_CONF_DIR={hive_server_conf_dir} ; "
+                                 "{hive_schematool_bin}/schematool -initSchema "
+                                 "-dbType {hive_metastore_db_type} "
+                                 "-userName {hive_metastore_user_name} "
+                                 "-passWord {hive_metastore_user_passwd!p} -verbose")
+
+      check_schema_created_cmd = as_user(format("export HIVE_CONF_DIR={hive_server_conf_dir} ; "
+                                        "{hive_schematool_bin}/schematool -info "
+                                        "-dbType {hive_metastore_db_type} "
+                                        "-userName {hive_metastore_user_name} "
+                                        "-passWord {hive_metastore_user_passwd!p} -verbose"), params.hive_user)
+
+      # HACK: in cases with quoted passwords and as_user (which does the quoting as well) !p won't work for hiding passwords.
+      # Fixing it with the hack below:
+      quoted_hive_metastore_user_passwd = quote_bash_args(quote_bash_args(params.hive_metastore_user_passwd))
+      if quoted_hive_metastore_user_passwd[0] == "'" and quoted_hive_metastore_user_passwd[-1] == "'" \
+          or quoted_hive_metastore_user_passwd[0] == '"' and quoted_hive_metastore_user_passwd[-1] == '"':
+        quoted_hive_metastore_user_passwd = quoted_hive_metastore_user_passwd[1:-1]
+      Logger.sensitive_strings[repr(check_schema_created_cmd)] = repr(check_schema_created_cmd.replace(
+          format("-passWord {quoted_hive_metastore_user_passwd}"), "-passWord " + utils.PASSWORDS_HIDE_STRING))
+
+      Execute(create_schema_cmd,
+              not_if = check_schema_created_cmd,
+              user = params.hive_user
+      )
+  elif name == 'hiveserver2':
+    File(params.start_hiveserver2_path,
+         mode=0755,
+         content=Template(format('{start_hiveserver2_script}'))
+    )
+
+    File(os.path.join(params.hive_server_conf_dir, "hadoop-metrics2-hiveserver2.properties"),
+         owner=params.hive_user,
+         group=params.user_group,
+         content=Template("hadoop-metrics2-hiveserver2.properties.j2")
+    )
+
+  if name != "client":
+    Directory(params.hive_pid_dir,
+              create_parents = True,
+              cd_access='a',
+              owner=params.hive_user,
+              group=params.user_group,
+              mode=0755)
+    Directory(params.hive_log_dir,
+              create_parents = True,
+              cd_access='a',
+              owner=params.hive_user,
+              group=params.user_group,
+              mode=0755)
+    Directory(params.hive_var_lib,
+              create_parents = True,
+              cd_access='a',
+              owner=params.hive_user,
+              group=params.user_group,
+              mode=0755)
+
+"""
+Writes configuration files required by Hive.
+"""
+def fill_conf_dir(component_conf_dir):
+  import params
+
+  Directory(component_conf_dir,
+            owner=params.hive_user,
+            group=params.user_group,
+            create_parents = True
+  )
+
+  XmlConfig("mapred-site.xml",
+            conf_dir=component_conf_dir,
+            configurations=params.config['configurations']['mapred-site'],
+            configuration_attributes=params.config['configuration_attributes']['mapred-site'],
+            owner=params.hive_user,
+            group=params.user_group,
+            mode=0644)
+
+
+  File(format("{component_conf_dir}/hive-default.xml.template"),
+       owner=params.hive_user,
+       group=params.user_group
+  )
+
+  File(format("{component_conf_dir}/hive-env.sh.template"),
+       owner=params.hive_user,
+       group=params.user_group
+  )
+
+  # Create hive-log4j.properties and hive-exec-log4j.properties
+  # in /etc/hive/conf and not in /etc/hive2/conf
+  if params.log4j_version == '1':
+    log4j_exec_filename = 'hive-exec-log4j.properties'
+    if (params.log4j_exec_props != None):
+      File(format("{component_conf_dir}/{log4j_exec_filename}"),
+           mode=0644,
+           group=params.user_group,
+           owner=params.hive_user,
+           content=params.log4j_exec_props
+      )
+    elif (os.path.exists("{component_conf_dir}/{log4j_exec_filename}.template")):
+      File(format("{component_conf_dir}/{log4j_exec_filename}"),
+           mode=0644,
+           group=params.user_group,
+           owner=params.hive_user,
+           content=StaticFile(format("{component_conf_dir}/{log4j_exec_filename}.template"))
+      )
+
+    log4j_filename = 'hive-log4j.properties'
+    if (params.log4j_props != None):
+      File(format("{component_conf_dir}/{log4j_filename}"),
+           mode=0644,
+           group=params.user_group,
+           owner=params.hive_user,
+           content=params.log4j_props
+      )
+    elif (os.path.exists("{component_conf_dir}/{log4j_filename}.template")):
+      File(format("{component_conf_dir}/{log4j_filename}"),
+           mode=0644,
+           group=params.user_group,
+           owner=params.hive_user,
+           content=StaticFile(format("{component_conf_dir}/{log4j_filename}.template"))
+      )
+    pass # if params.log4j_version == '1'
+
+
+def jdbc_connector(target, hive_previous_jdbc_jar):
+  """
+  Shared by Hive Batch, Hive Metastore, and Hive Interactive
+  :param target: Target of jdbc jar name, which could be for any of the components above.
+  """
+  import params
+
+  if not params.jdbc_jar_name:
+    return
+
+  if params.hive_jdbc_driver in params.hive_jdbc_drivers_list and params.hive_use_existing_db:
+    environment = {
+      "no_proxy": format("{ambari_server_hostname}")
+    }
+
+    if hive_previous_jdbc_jar and os.path.isfile(hive_previous_jdbc_jar):
+      File(hive_previous_jdbc_jar, action='delete')
+
+    # TODO: should be removed after ranger_hive_plugin will not provide jdbc
+    if params.prepackaged_jdbc_name != params.jdbc_jar_name:
+      Execute(('rm', '-f', params.prepackaged_ojdbc_symlink),
+              path=["/bin", "/usr/bin/"],
+              sudo = True)
+    
+    File(params.downloaded_custom_connector,
+         content = DownloadSource(params.driver_curl_source))
+
+    # maybe it will be more correcvly to use db type
+    if params.sqla_db_used:
+      untar_sqla_type2_driver = ('tar', '-xvf', params.downloaded_custom_connector, '-C', params.tmp_dir)
+
+      Execute(untar_sqla_type2_driver, sudo = True)
+
+      Execute(format("yes | {sudo} cp {jars_path_in_archive} {hive_lib}"))
+
+      Directory(params.jdbc_libs_dir,
+                create_parents = True)
+
+      Execute(format("yes | {sudo} cp {libs_path_in_archive} {jdbc_libs_dir}"))
+
+      Execute(format("{sudo} chown -R {hive_user}:{user_group} {hive_lib}/*"))
+
+    else:
+      Execute(('cp', '--remove-destination', params.downloaded_custom_connector, target),
+            #creates=target, TODO: uncomment after ranger_hive_plugin will not provide jdbc
+            path=["/bin", "/usr/bin/"],
+            sudo = True)
+
+  else:
+    #for default hive db (Mysql)
+    Execute(('cp', '--remove-destination', format('/usr/share/java/{jdbc_jar_name}'), target),
+            #creates=target, TODO: uncomment after ranger_hive_plugin will not provide jdbc
+            path=["/bin", "/usr/bin/"],
+            sudo=True
+    )
+  pass
+
+  File(target,
+       mode = 0644,
+  )

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_client.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_client.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_client.py
new file mode 100755
index 0000000..3d9bfd7
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_client.py
@@ -0,0 +1,68 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+import sys
+from resource_management import *
+from resource_management.libraries.functions import conf_select
+from resource_management.libraries.functions import stack_select
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from hive import hive
+from ambari_commons.os_family_impl import OsFamilyImpl
+from ambari_commons import OSConst
+from resource_management.core.exceptions import ClientComponentHasNoStatus
+
+class HiveClient(Script):
+  def install(self, env):
+    import params
+    self.install_packages(env)
+    self.configure(env)
+
+  def status(self, env):
+    raise ClientComponentHasNoStatus()
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    hive(name='client')
+
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class HiveClientWindows(HiveClient):
+  pass
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class HiveClientDefault(HiveClient):
+  def get_component_name(self):
+    return "hadoop-client"
+
+  def pre_upgrade_restart(self, env, upgrade_type=None):
+    Logger.info("Executing Hive client Stack Upgrade pre-restart")
+
+    import params
+    env.set_params(params)
+    if params.version and check_stack_feature(StackFeature.ROLLING_UPGRADE, params.version):
+      conf_select.select(params.stack_name, "hive", params.version)
+      conf_select.select(params.stack_name, "hadoop", params.version)
+      stack_select.select("hadoop-client", params.version)
+
+
+if __name__ == "__main__":
+  HiveClient().execute()

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_interactive.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_interactive.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_interactive.py
new file mode 100755
index 0000000..74c67fc
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_interactive.py
@@ -0,0 +1,302 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+# Python Imports
+import os
+import glob
+from urlparse import urlparse
+
+# Resource Management and Common Imports
+from resource_management.libraries.script.script import Script
+from resource_management.libraries.resources.hdfs_resource import HdfsResource
+from resource_management.libraries.functions.copy_tarball import copy_to_hdfs
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.functions.version import compare_versions
+from resource_management.core.resources.service import ServiceConfig
+from resource_management.core.resources.system import File, Execute, Directory
+from resource_management.core.source import StaticFile, Template, DownloadSource, InlineTemplate
+from resource_management.core.shell import as_user
+from resource_management.libraries.functions.is_empty import is_empty
+from resource_management.libraries.resources.xml_config import XmlConfig
+from resource_management.libraries.functions.format import format
+from resource_management.core.exceptions import Fail
+from resource_management.core.shell import as_sudo
+from resource_management.core.shell import quote_bash_args
+from resource_management.core.logger import Logger
+from resource_management.core import utils
+
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+from hive import fill_conf_dir, jdbc_connector
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def hive_interactive(name=None):
+  pass
+
+"""
+Sets up the configs, jdbc connection and tarball copy to HDFS for Hive Server Interactive.
+"""
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def hive_interactive(name=None):
+  import params
+
+  # list of properties that should be excluded from the config
+  # this approach is a compromise against adding a dedicated config
+  # type for hive_server_interactive or needed config groups on a
+  # per component basis
+  exclude_list = ['hive.enforce.bucketing',
+                  'hive.enforce.sorting']
+
+  # List of configs to be excluded from hive2 client, but present in Hive2 server.
+  exclude_list_for_hive2_client = ['javax.jdo.option.ConnectionPassword']
+
+  # Copy Tarballs in HDFS.
+  if params.stack_version_formatted_major and check_stack_feature(StackFeature.ROLLING_UPGRADE, params.stack_version_formatted_major):
+    resource_created = copy_to_hdfs("tez_hive2",
+                 params.user_group,
+                 params.hdfs_user,
+                 file_mode=params.tarballs_mode,
+                 host_sys_prepped=params.host_sys_prepped)
+
+    if resource_created:
+      params.HdfsResource(None, action="execute")
+
+  Directory(params.hive_interactive_etc_dir_prefix,
+            mode=0755
+            )
+
+  Logger.info("Directories to fill with configs: %s" % str(params.hive_conf_dirs_list))
+  for conf_dir in params.hive_conf_dirs_list:
+    fill_conf_dir(conf_dir)
+
+  '''
+  As hive2/hive-site.xml only contains the new + the changed props compared to hive/hive-site.xml,
+  we need to merge hive/hive-site.xml and hive2/hive-site.xml and store it in hive2/hive-site.xml.
+  '''
+  merged_hive_interactive_site = {}
+  merged_hive_interactive_site.update(params.config['configurations']['hive-site'])
+  merged_hive_interactive_site.update(params.config['configurations']['hive-interactive-site'])
+  for item in exclude_list:
+    if item in merged_hive_interactive_site.keys():
+      del merged_hive_interactive_site[item]
+
+  '''
+  Hive2 doesn't have support for Atlas, we need to remove the Hook 'org.apache.atlas.hive.hook.HiveHook',
+  which would have come in config 'hive.exec.post.hooks' during the site merge logic, if Atlas is installed.
+  '''
+  remove_atlas_hook_if_exists(merged_hive_interactive_site)
+
+  '''
+  As tez_hive2/tez-site.xml only contains the new + the changed props compared to tez/tez-site.xml,
+  we need to merge tez/tez-site.xml and tez_hive2/tez-site.xml and store it in tez_hive2/tez-site.xml.
+  '''
+  merged_tez_interactive_site = {}
+  if 'tez-site' in params.config['configurations']:
+    merged_tez_interactive_site.update(params.config['configurations']['tez-site'])
+    Logger.info("Retrieved 'tez/tez-site' for merging with 'tez_hive2/tez-interactive-site'.")
+  else:
+    Logger.error("Tez's 'tez-site' couldn't be retrieved from passed-in configurations.")
+
+  merged_tez_interactive_site.update(params.config['configurations']['tez-interactive-site'])
+  XmlConfig("tez-site.xml",
+            conf_dir = params.tez_interactive_config_dir,
+            configurations = merged_tez_interactive_site,
+            configuration_attributes=params.config['configuration_attributes']['tez-interactive-site'],
+            owner = params.tez_interactive_user,
+            group = params.user_group,
+            mode = 0664)
+
+  '''
+  Merge properties from hiveserver2-interactive-site into hiveserver2-site
+  '''
+  merged_hiveserver2_interactive_site = {}
+  if 'hiveserver2-site' in params.config['configurations']:
+    merged_hiveserver2_interactive_site.update(params.config['configurations']['hiveserver2-site'])
+    Logger.info("Retrieved 'hiveserver2-site' for merging with 'hiveserver2-interactive-site'.")
+  else:
+    Logger.error("'hiveserver2-site' couldn't be retrieved from passed-in configurations.")
+  merged_hiveserver2_interactive_site.update(params.config['configurations']['hiveserver2-interactive-site'])
+
+
+  # Create config files under /etc/hive2/conf and /etc/hive2/conf/conf.server:
+  #   hive-site.xml
+  #   hive-env.sh
+  #   llap-daemon-log4j2.properties
+  #   llap-cli-log4j2.properties
+  #   hive-log4j2.properties
+  #   hive-exec-log4j2.properties
+  #   beeline-log4j2.properties
+
+  hive2_conf_dirs_list = params.hive_conf_dirs_list
+  hive2_client_conf_path = format("{stack_root}/current/{component_directory}/conf")
+
+  # Making copy of 'merged_hive_interactive_site' in 'merged_hive_interactive_site_copy', and deleting 'javax.jdo.option.ConnectionPassword'
+  # config from there, as Hive2 client shouldn't have that config.
+  merged_hive_interactive_site_copy = merged_hive_interactive_site.copy()
+  for item in exclude_list_for_hive2_client:
+    if item in merged_hive_interactive_site.keys():
+      del merged_hive_interactive_site_copy[item]
+
+  for conf_dir in hive2_conf_dirs_list:
+      if conf_dir == hive2_client_conf_path:
+        XmlConfig("hive-site.xml",
+                  conf_dir=conf_dir,
+                  configurations=merged_hive_interactive_site_copy,
+                  configuration_attributes=params.config['configuration_attributes']['hive-interactive-site'],
+                  owner=params.hive_user,
+                  group=params.user_group,
+                  mode=0644)
+      else:
+        XmlConfig("hive-site.xml",
+                  conf_dir=conf_dir,
+                  configurations=merged_hive_interactive_site,
+                  configuration_attributes=params.config['configuration_attributes']['hive-interactive-site'],
+                  owner=params.hive_user,
+                  group=params.user_group,
+                  mode=0644)
+
+      XmlConfig("hiveserver2-site.xml",
+                conf_dir=conf_dir,
+                configurations=merged_hiveserver2_interactive_site,
+                configuration_attributes=params.config['configuration_attributes']['hiveserver2-interactive-site'],
+                owner=params.hive_user,
+                group=params.user_group,
+                mode=0644)
+
+      hive_server_interactive_conf_dir = conf_dir
+
+      File(format("{hive_server_interactive_conf_dir}/hive-env.sh"),
+           owner=params.hive_user,
+           group=params.user_group,
+           content=InlineTemplate(params.hive_interactive_env_sh_template))
+
+      llap_daemon_log4j_filename = 'llap-daemon-log4j2.properties'
+      File(format("{hive_server_interactive_conf_dir}/{llap_daemon_log4j_filename}"),
+           mode=0644,
+           group=params.user_group,
+           owner=params.hive_user,
+           content=params.llap_daemon_log4j)
+
+      llap_cli_log4j2_filename = 'llap-cli-log4j2.properties'
+      File(format("{hive_server_interactive_conf_dir}/{llap_cli_log4j2_filename}"),
+           mode=0644,
+           group=params.user_group,
+           owner=params.hive_user,
+           content=params.llap_cli_log4j2)
+
+      hive_log4j2_filename = 'hive-log4j2.properties'
+      File(format("{hive_server_interactive_conf_dir}/{hive_log4j2_filename}"),
+         mode=0644,
+         group=params.user_group,
+         owner=params.hive_user,
+         content=params.hive_log4j2)
+
+      hive_exec_log4j2_filename = 'hive-exec-log4j2.properties'
+      File(format("{hive_server_interactive_conf_dir}/{hive_exec_log4j2_filename}"),
+         mode=0644,
+         group=params.user_group,
+         owner=params.hive_user,
+         content=params.hive_exec_log4j2)
+
+      beeline_log4j2_filename = 'beeline-log4j2.properties'
+      File(format("{hive_server_interactive_conf_dir}/{beeline_log4j2_filename}"),
+         mode=0644,
+         group=params.user_group,
+         owner=params.hive_user,
+         content=params.beeline_log4j2)
+
+      File(os.path.join(hive_server_interactive_conf_dir, "hadoop-metrics2-hiveserver2.properties"),
+           owner=params.hive_user,
+           group=params.user_group,
+           content=Template("hadoop-metrics2-hiveserver2.properties.j2")
+           )
+
+      File(format("{hive_server_interactive_conf_dir}/hadoop-metrics2-llapdaemon.properties"),
+           owner=params.hive_user,
+           group=params.user_group,
+           content=Template("hadoop-metrics2-llapdaemon.j2"))
+
+      File(format("{hive_server_interactive_conf_dir}/hadoop-metrics2-llaptaskscheduler.properties"),
+           owner=params.hive_user,
+           group=params.user_group,
+           content=Template("hadoop-metrics2-llaptaskscheduler.j2"))
+
+
+  # On some OS this folder could be not exists, so we will create it before pushing there files
+  Directory(params.limits_conf_dir,
+            create_parents = True,
+            owner='root',
+            group='root')
+
+  File(os.path.join(params.limits_conf_dir, 'hive.conf'),
+       owner='root',
+       group='root',
+       mode=0644,
+       content=Template("hive.conf.j2"))
+
+  if not os.path.exists(params.target_hive_interactive):
+    jdbc_connector(params.target_hive_interactive, params.hive_intaractive_previous_jdbc_jar)
+
+  File(format("/usr/lib/ambari-agent/{check_db_connection_jar_name}"),
+       content = DownloadSource(format("{jdk_location}{check_db_connection_jar_name}")),
+       mode = 0644)
+  File(params.start_hiveserver2_interactive_path,
+       mode=0755,
+       content=Template(format('{start_hiveserver2_interactive_script}')))
+
+  Directory(params.hive_pid_dir,
+            create_parents=True,
+            cd_access='a',
+            owner=params.hive_user,
+            group=params.user_group,
+            mode=0755)
+  Directory(params.hive_log_dir,
+            create_parents=True,
+            cd_access='a',
+            owner=params.hive_user,
+            group=params.user_group,
+            mode=0755)
+  Directory(params.hive_interactive_var_lib,
+            create_parents=True,
+            cd_access='a',
+            owner=params.hive_user,
+            group=params.user_group,
+            mode=0755)
+
+"""
+Remove 'org.apache.atlas.hive.hook.HiveHook' value from Hive2/hive-site.xml config 'hive.exec.post.hooks', if exists.
+"""
+def remove_atlas_hook_if_exists(merged_hive_interactive_site):
+  if 'hive.exec.post.hooks' in merged_hive_interactive_site.keys():
+    existing_hive_exec_post_hooks = merged_hive_interactive_site.get('hive.exec.post.hooks')
+    if existing_hive_exec_post_hooks:
+      hook_splits = existing_hive_exec_post_hooks.split(",")
+      updated_hook_splits = [hook for hook in hook_splits if not hook.strip() == 'org.apache.atlas.hive.hook.HiveHook']
+      updated_hooks_str = ",".join((str(hook)).strip() for hook in updated_hook_splits)
+      if updated_hooks_str != existing_hive_exec_post_hooks:
+        merged_hive_interactive_site['hive.exec.post.hooks'] = updated_hooks_str
+        Logger.info("Updated Hive2/hive-site.xml 'hive.exec.post.hooks' value from : '{0}' to : '{1}'"
+                    .format(existing_hive_exec_post_hooks, updated_hooks_str))
+      else:
+        Logger.info("No change done to Hive2/hive-site.xml 'hive.exec.post.hooks' value.")
+  else:
+      Logger.debug("'hive.exec.post.hooks' doesn't exist in Hive2/hive-site.xml")

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_metastore.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_metastore.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_metastore.py
new file mode 100755
index 0000000..17bf581
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_metastore.py
@@ -0,0 +1,259 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+import os
+
+from resource_management.core.logger import Logger
+from resource_management.core.resources.system import Execute, Directory
+from resource_management.libraries.script import Script
+from resource_management.libraries.functions import conf_select
+from resource_management.libraries.functions import stack_select
+from resource_management.libraries.functions.constants import Direction
+from resource_management.libraries.functions.format import format
+from resource_management.libraries.functions.version import format_stack_version
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.functions.security_commons import build_expectations
+from resource_management.libraries.functions.security_commons import cached_kinit_executor
+from resource_management.libraries.functions.security_commons import get_params_from_filesystem
+from resource_management.libraries.functions.security_commons import validate_security_config_properties
+from resource_management.libraries.functions.security_commons import FILE_TYPE_XML
+from resource_management.core.resources.system import File
+
+from hive import hive
+from hive import jdbc_connector
+from hive_service import hive_service
+from ambari_commons.os_family_impl import OsFamilyImpl
+from ambari_commons import OSConst
+
+# the legacy conf.server location in previous stack versions
+LEGACY_HIVE_SERVER_CONF = "/etc/hive/conf.server"
+
+class HiveMetastore(Script):
+  def install(self, env):
+    import params
+    self.install_packages(env)
+
+
+  def start(self, env, upgrade_type=None):
+    import params
+    env.set_params(params)
+
+    # writing configurations on start required for securtity
+    self.configure(env)
+
+    hive_service('metastore', action='start', upgrade_type=upgrade_type)
+
+
+  def stop(self, env, upgrade_type=None):
+    import params
+    env.set_params(params)
+    hive_service('metastore', action='stop', upgrade_type=upgrade_type)
+
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    hive(name = 'metastore')
+
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class HiveMetastoreWindows(HiveMetastore):
+  def status(self, env):
+    import status_params
+    from resource_management.libraries.functions import check_windows_service_status
+    check_windows_service_status(status_params.hive_metastore_win_service_name)
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class HiveMetastoreDefault(HiveMetastore):
+  def get_component_name(self):
+    return "hive-metastore"
+
+
+  def status(self, env):
+    import status_params
+    from resource_management.libraries.functions import check_process_status
+
+    env.set_params(status_params)
+    pid_file = format("{hive_pid_dir}/{hive_metastore_pid}")
+    # Recursively check all existing gmetad pid files
+    check_process_status(pid_file)
+
+
+  def pre_upgrade_restart(self, env, upgrade_type=None):
+    Logger.info("Executing Metastore Stack Upgrade pre-restart")
+    import params
+
+    env.set_params(params)
+
+    is_upgrade = params.upgrade_direction == Direction.UPGRADE
+
+    if params.version and check_stack_feature(StackFeature.ROLLING_UPGRADE, params.version):
+      conf_select.select(params.stack_name, "hive", params.version)
+      stack_select.select("hive-metastore", params.version)
+
+    if is_upgrade and params.stack_version_formatted_major and \
+            check_stack_feature(StackFeature.HIVE_METASTORE_UPGRADE_SCHEMA, params.stack_version_formatted_major):
+      self.upgrade_schema(env)
+
+
+  def security_status(self, env):
+    import status_params
+    env.set_params(status_params)
+    if status_params.security_enabled:
+      props_value_check = {"hive.server2.authentication": "KERBEROS",
+                           "hive.metastore.sasl.enabled": "true",
+                           "hive.security.authorization.enabled": "true"}
+      props_empty_check = ["hive.metastore.kerberos.keytab.file",
+                           "hive.metastore.kerberos.principal"]
+
+      props_read_check = ["hive.metastore.kerberos.keytab.file"]
+      hive_site_props = build_expectations('hive-site', props_value_check, props_empty_check,
+                                            props_read_check)
+
+      hive_expectations ={}
+      hive_expectations.update(hive_site_props)
+
+      security_params = get_params_from_filesystem(status_params.hive_conf_dir,
+                                                   {'hive-site.xml': FILE_TYPE_XML})
+      result_issues = validate_security_config_properties(security_params, hive_expectations)
+      if not result_issues: # If all validations passed successfully
+        try:
+          # Double check the dict before calling execute
+          if 'hive-site' not in security_params \
+            or 'hive.metastore.kerberos.keytab.file' not in security_params['hive-site'] \
+            or 'hive.metastore.kerberos.principal' not in security_params['hive-site']:
+            self.put_structured_out({"securityState": "UNSECURED"})
+            self.put_structured_out({"securityIssuesFound": "Keytab file or principal are not set property."})
+            return
+
+          cached_kinit_executor(status_params.kinit_path_local,
+                                status_params.hive_user,
+                                security_params['hive-site']['hive.metastore.kerberos.keytab.file'],
+                                security_params['hive-site']['hive.metastore.kerberos.principal'],
+                                status_params.hostname,
+                                status_params.tmp_dir)
+
+          self.put_structured_out({"securityState": "SECURED_KERBEROS"})
+        except Exception as e:
+          self.put_structured_out({"securityState": "ERROR"})
+          self.put_structured_out({"securityStateErrorInfo": str(e)})
+      else:
+        issues = []
+        for cf in result_issues:
+          issues.append("Configuration file %s did not pass the validation. Reason: %s" % (cf, result_issues[cf]))
+        self.put_structured_out({"securityIssuesFound": ". ".join(issues)})
+        self.put_structured_out({"securityState": "UNSECURED"})
+    else:
+      self.put_structured_out({"securityState": "UNSECURED"})
+
+
+  def upgrade_schema(self, env):
+    """
+    Executes the schema upgrade binary.  This is its own function because it could
+    be called as a standalone task from the upgrade pack, but is safe to run it for each
+    metastore instance. The schema upgrade on an already upgraded metastore is a NOOP.
+
+    The metastore schema upgrade requires a database driver library for most
+    databases. During an upgrade, it's possible that the library is not present,
+    so this will also attempt to copy/download the appropriate driver.
+
+    This function will also ensure that configurations are written out to disk before running
+    since the new configs will most likely not yet exist on an upgrade.
+
+    Should not be invoked for a DOWNGRADE; Metastore only supports schema upgrades.
+    """
+    Logger.info("Upgrading Hive Metastore Schema")
+    import status_params
+    import params
+    env.set_params(params)
+
+    # ensure that configurations are written out before trying to upgrade the schema
+    # since the schematool needs configs and doesn't know how to use the hive conf override
+    self.configure(env)
+
+    if params.security_enabled:
+      cached_kinit_executor(status_params.kinit_path_local,
+        status_params.hive_user,
+        params.hive_metastore_keytab_path,
+        params.hive_metastore_principal,
+        status_params.hostname,
+        status_params.tmp_dir)
+      
+    # ensure that the JDBC drive is present for the schema tool; if it's not
+    # present, then download it first
+    if params.hive_jdbc_driver in params.hive_jdbc_drivers_list:
+      target_directory = format("{stack_root}/{version}/hive/lib")
+
+      # download it if it does not exist
+      if not os.path.exists(params.source_jdbc_file):
+        jdbc_connector(params.hive_jdbc_target, params.hive_previous_jdbc_jar)
+
+      target_directory_and_filename = os.path.join(target_directory, os.path.basename(params.source_jdbc_file))
+
+      if params.sqla_db_used:
+        target_native_libs_directory = format("{target_directory}/native/lib64")
+
+        Execute(format("yes | {sudo} cp {jars_in_hive_lib} {target_directory}"))
+
+        Directory(target_native_libs_directory, create_parents = True)
+
+        Execute(format("yes | {sudo} cp {libs_in_hive_lib} {target_native_libs_directory}"))
+
+        Execute(format("{sudo} chown -R {hive_user}:{user_group} {hive_lib}/*"))
+      else:
+        # copy the JDBC driver from the older metastore location to the new location only
+        # if it does not already exist
+        if not os.path.exists(target_directory_and_filename):
+          Execute(('cp', params.source_jdbc_file, target_directory),
+            path=["/bin", "/usr/bin/"], sudo = True)
+
+      File(target_directory_and_filename, mode = 0644)
+
+    # build the schema tool command
+    binary = format("{hive_schematool_ver_bin}/schematool")
+
+    # the conf.server directory changed locations between stack versions
+    # since the configurations have not been written out yet during an upgrade
+    # we need to choose the original legacy location
+    schematool_hive_server_conf_dir = params.hive_server_conf_dir
+    if params.current_version is not None:
+      current_version = format_stack_version(params.current_version)
+      if not(check_stack_feature(StackFeature.CONFIG_VERSIONING, current_version)):
+        schematool_hive_server_conf_dir = LEGACY_HIVE_SERVER_CONF
+
+    env_dict = {
+      'HIVE_CONF_DIR': schematool_hive_server_conf_dir
+    }
+
+    command = format("{binary} -dbType {hive_metastore_db_type} -upgradeSchema")
+    Execute(command, user=params.hive_user, tries=1, environment=env_dict, logoutput=True)
+    
+  def get_log_folder(self):
+    import params
+    return params.hive_log_dir
+
+  def get_user(self):
+    import params
+    return params.hive_user
+
+
+if __name__ == "__main__":
+  HiveMetastore().execute()

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server.py
new file mode 100755
index 0000000..31b083b
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server.py
@@ -0,0 +1,211 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+
+from resource_management.libraries.script.script import Script
+from resource_management.libraries.resources.hdfs_resource import HdfsResource
+from resource_management.libraries.functions import conf_select
+from resource_management.libraries.functions import stack_select
+from resource_management.libraries.functions import format
+from resource_management.libraries.functions.copy_tarball import copy_to_hdfs
+from resource_management.libraries.functions.get_stack_version import get_stack_version
+from resource_management.libraries.functions.check_process_status import check_process_status
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.functions.security_commons import build_expectations, \
+  cached_kinit_executor, get_params_from_filesystem, validate_security_config_properties, \
+  FILE_TYPE_XML
+from ambari_commons import OSCheck, OSConst
+if OSCheck.is_windows_family():
+  from resource_management.libraries.functions.windows_service_utils import check_windows_service_status
+from setup_ranger_hive import setup_ranger_hive
+from ambari_commons.os_family_impl import OsFamilyImpl
+from ambari_commons.constants import UPGRADE_TYPE_ROLLING
+from resource_management.core.logger import Logger
+
+import hive_server_upgrade
+from hive import hive
+from hive_service import hive_service
+
+
+class HiveServer(Script):
+  def install(self, env):
+    import params
+    self.install_packages(env)
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    hive(name='hiveserver2')
+
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class HiveServerWindows(HiveServer):
+  def start(self, env):
+    import params
+    env.set_params(params)
+    self.configure(env) # FOR SECURITY
+    hive_service('hiveserver2', action='start')
+
+  def stop(self, env):
+    import params
+    env.set_params(params)
+    hive_service('hiveserver2', action='stop')
+
+  def status(self, env):
+    import status_params
+    check_windows_service_status(status_params.hive_server_win_service_name)
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class HiveServerDefault(HiveServer):
+  def get_component_name(self):
+    return "hive-server2"
+
+  def start(self, env, upgrade_type=None):
+    import params
+    env.set_params(params)
+    self.configure(env) # FOR SECURITY
+
+    setup_ranger_hive(upgrade_type=upgrade_type)
+    hive_service('hiveserver2', action = 'start', upgrade_type=upgrade_type)
+
+    # only perform this if upgrading and rolling; a non-rolling upgrade doesn't need
+    # to do this since hive is already down
+    if upgrade_type == UPGRADE_TYPE_ROLLING:
+      hive_server_upgrade.post_upgrade_deregister()
+
+
+  def stop(self, env, upgrade_type=None):
+    import params
+    env.set_params(params)
+
+    # During rolling upgrade, HiveServer2 should not be stopped before new server is available.
+    # Once new server is started, old one is stopped by the --deregister command which is 
+    # invoked by the 'hive_server_upgrade.post_upgrade_deregister()' method
+    if upgrade_type != UPGRADE_TYPE_ROLLING:
+      hive_service( 'hiveserver2', action = 'stop' )
+
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    pid_file = format("{hive_pid_dir}/{hive_pid}")
+
+    # Recursively check all existing gmetad pid files
+    check_process_status(pid_file)
+
+
+  def pre_upgrade_restart(self, env, upgrade_type=None):
+    Logger.info("Executing Hive Server Stack Upgrade pre-restart")
+    import params
+    env.set_params(params)
+
+    if params.version and check_stack_feature(StackFeature.ROLLING_UPGRADE, params.version):
+      conf_select.select(params.stack_name, "hive", params.version)
+      stack_select.select("hive-server2", params.version)
+
+      # Copy mapreduce.tar.gz and tez.tar.gz to HDFS
+      resource_created = copy_to_hdfs(
+        "mapreduce",
+        params.user_group,
+        params.hdfs_user,
+        host_sys_prepped=params.host_sys_prepped)
+
+      resource_created = copy_to_hdfs(
+        "tez",
+        params.user_group,
+        params.hdfs_user,
+        host_sys_prepped=params.host_sys_prepped) or resource_created
+
+      if resource_created:
+        params.HdfsResource(None, action="execute")
+
+
+  def security_status(self, env):
+    import status_params
+    env.set_params(status_params)
+    if status_params.security_enabled:
+      props_value_check = {"hive.server2.authentication": "KERBEROS",
+                           "hive.metastore.sasl.enabled": "true",
+                           "hive.security.authorization.enabled": "true"}
+      props_empty_check = ["hive.server2.authentication.kerberos.keytab",
+                           "hive.server2.authentication.kerberos.principal",
+                           "hive.server2.authentication.spnego.principal",
+                           "hive.server2.authentication.spnego.keytab"]
+
+      props_read_check = ["hive.server2.authentication.kerberos.keytab",
+                          "hive.server2.authentication.spnego.keytab"]
+      hive_site_props = build_expectations('hive-site', props_value_check, props_empty_check,
+                                            props_read_check)
+
+      hive_expectations ={}
+      hive_expectations.update(hive_site_props)
+
+      security_params = get_params_from_filesystem(status_params.hive_conf_dir,
+                                                   {'hive-site.xml': FILE_TYPE_XML})
+      result_issues = validate_security_config_properties(security_params, hive_expectations)
+      if not result_issues: # If all validations passed successfully
+        try:
+          # Double check the dict before calling execute
+          if 'hive-site' not in security_params \
+            or 'hive.server2.authentication.kerberos.keytab' not in security_params['hive-site'] \
+            or 'hive.server2.authentication.kerberos.principal' not in security_params['hive-site']  \
+            or 'hive.server2.authentication.spnego.keytab' not in security_params['hive-site'] \
+            or 'hive.server2.authentication.spnego.principal' not in security_params['hive-site']:
+            self.put_structured_out({"securityState": "UNSECURED"})
+            self.put_structured_out({"securityIssuesFound": "Keytab file or principal are not set property."})
+            return
+
+          cached_kinit_executor(status_params.kinit_path_local,
+                                status_params.hive_user,
+                                security_params['hive-site']['hive.server2.authentication.kerberos.keytab'],
+                                security_params['hive-site']['hive.server2.authentication.kerberos.principal'],
+                                status_params.hostname,
+                                status_params.tmp_dir)
+          cached_kinit_executor(status_params.kinit_path_local,
+                                status_params.hive_user,
+                                security_params['hive-site']['hive.server2.authentication.spnego.keytab'],
+                                security_params['hive-site']['hive.server2.authentication.spnego.principal'],
+                                status_params.hostname,
+                                status_params.tmp_dir)
+          self.put_structured_out({"securityState": "SECURED_KERBEROS"})
+        except Exception as e:
+          self.put_structured_out({"securityState": "ERROR"})
+          self.put_structured_out({"securityStateErrorInfo": str(e)})
+      else:
+        issues = []
+        for cf in result_issues:
+          issues.append("Configuration file %s did not pass the validation. Reason: %s" % (cf, result_issues[cf]))
+        self.put_structured_out({"securityIssuesFound": ". ".join(issues)})
+        self.put_structured_out({"securityState": "UNSECURED"})
+    else:
+      self.put_structured_out({"securityState": "UNSECURED"})
+
+  def get_log_folder(self):
+    import params
+    return params.hive_log_dir
+  
+  def get_user(self):
+    import params
+    return params.hive_user
+
+if __name__ == "__main__":
+  HiveServer().execute()

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_interactive.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_interactive.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_interactive.py
new file mode 100755
index 0000000..2df001c
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_interactive.py
@@ -0,0 +1,535 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+# Python Imports
+import subprocess
+import os
+import re
+import time
+import shutil
+from datetime import datetime
+import json
+
+# Ambari Commons & Resource Management imports
+from resource_management.libraries.script.script import Script
+from resource_management.libraries.functions import format
+from resource_management.libraries.functions.check_process_status import check_process_status
+from resource_management.core.source import InlineTemplate
+from resource_management.core.resources.system import Execute
+
+# Imports needed for Rolling/Express Upgrade
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.functions import conf_select
+from resource_management.libraries.functions import stack_select
+from resource_management.libraries.functions.copy_tarball import copy_to_hdfs
+
+from resource_management.core import shell
+from resource_management.core.exceptions import Fail
+from resource_management.core.logger import Logger
+from ambari_commons import OSCheck, OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
+
+from resource_management.core.exceptions import ComponentIsNotRunning
+from resource_management.libraries.functions.decorator import retry
+from resource_management.libraries.functions.security_commons import build_expectations, \
+  cached_kinit_executor, get_params_from_filesystem, validate_security_config_properties, \
+  FILE_TYPE_XML
+
+# Local Imports
+from setup_ranger_hive import setup_ranger_hive
+from hive_service_interactive import hive_service_interactive
+from hive_interactive import hive_interactive
+from hive_server import HiveServerDefault
+from setup_ranger_hive_interactive import setup_ranger_hive_interactive
+
+import traceback
+
+class HiveServerInteractive(Script):
+  pass
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class HiveServerInteractiveDefault(HiveServerInteractive):
+
+    def get_component_name(self):
+      return "hive-server2-hive2"
+
+    def install(self, env):
+      import params
+      self.install_packages(env)
+
+    def configure(self, env):
+      import params
+      env.set_params(params)
+      hive_interactive(name='hiveserver2')
+
+    def pre_upgrade_restart(self, env, upgrade_type=None):
+      Logger.info("Executing Hive Server Interactive Stack Upgrade pre-restart")
+      import params
+      env.set_params(params)
+
+      if params.version and check_stack_feature(StackFeature.ROLLING_UPGRADE, params.version):
+        stack_select.select("hive-server2-hive2", params.version)
+        conf_select.select(params.stack_name, "hive2", params.version)
+
+        # Copy hive.tar.gz and tez.tar.gz used by Hive Interactive to HDFS
+        resource_created = copy_to_hdfs(
+          "hive2",
+          params.user_group,
+          params.hdfs_user,
+          host_sys_prepped=params.host_sys_prepped)
+
+        resource_created = copy_to_hdfs(
+          "tez_hive2",
+          params.user_group,
+          params.hdfs_user,
+          host_sys_prepped=params.host_sys_prepped) or resource_created
+
+        if resource_created:
+          params.HdfsResource(None, action="execute")
+
+    def start(self, env, upgrade_type=None):
+      import params
+      env.set_params(params)
+      self.configure(env)
+
+      if params.security_enabled:
+        # Do the security setup, internally calls do_kinit()
+        self.setup_security()
+
+      # TODO : We need have conditional [re]start of LLAP once "status check command" for LLAP is ready.
+      # Check status and based on that decide on [re]starting.
+
+      # Start LLAP before Hive Server Interactive start.
+      status = self._llap_start(env)
+      if not status:
+        raise Fail("Skipping START of Hive Server Interactive since LLAP app couldn't be STARTED.")
+
+      # TODO : test the workability of Ranger and Hive2 during upgrade
+      setup_ranger_hive_interactive(upgrade_type=upgrade_type)
+      hive_service_interactive('hiveserver2', action='start', upgrade_type=upgrade_type)
+
+
+    def stop(self, env, upgrade_type=None):
+      import params
+      env.set_params(params)
+
+      if params.security_enabled:
+        self.do_kinit()
+
+      # Stop Hive Interactive Server first
+      hive_service_interactive('hiveserver2', action='stop')
+
+      self._llap_stop(env)
+
+    def status(self, env):
+      import status_params
+      env.set_params(status_params)
+
+      # We are not doing 'llap' status check done here as part of status check for 'HSI', as 'llap' status
+      # check is a heavy weight operation.
+
+      pid_file = format("{hive_pid_dir}/{hive_interactive_pid}")
+      # Recursively check all existing gmetad pid files
+      check_process_status(pid_file)
+
+    def security_status(self, env):
+      import status_params
+      env.set_params(status_params)
+
+      if status_params.security_enabled:
+        props_value_check = {"hive.server2.authentication": "KERBEROS",
+                             "hive.metastore.sasl.enabled": "true",
+                             "hive.security.authorization.enabled": "true"}
+        props_empty_check = ["hive.server2.authentication.kerberos.keytab",
+                             "hive.server2.authentication.kerberos.principal",
+                             "hive.server2.authentication.spnego.principal",
+                             "hive.server2.authentication.spnego.keytab"]
+
+        props_read_check = ["hive.server2.authentication.kerberos.keytab",
+                            "hive.server2.authentication.spnego.keytab"]
+        hive_site_props = build_expectations('hive-site', props_value_check, props_empty_check,
+                                             props_read_check)
+
+        hive_expectations ={}
+        hive_expectations.update(hive_site_props)
+
+        security_params = get_params_from_filesystem(status_params.hive_server_interactive_conf_dir,
+                                                     {'hive-site.xml': FILE_TYPE_XML})
+        result_issues = validate_security_config_properties(security_params, hive_expectations)
+        if not result_issues: # If all validations passed successfully
+          try:
+            # Double check the dict before calling execute
+            if 'hive-site' not in security_params \
+              or 'hive.server2.authentication.kerberos.keytab' not in security_params['hive-site'] \
+              or 'hive.server2.authentication.kerberos.principal' not in security_params['hive-site'] \
+              or 'hive.server2.authentication.spnego.keytab' not in security_params['hive-site'] \
+              or 'hive.server2.authentication.spnego.principal' not in security_params['hive-site']:
+              self.put_structured_out({"securityState": "UNSECURED"})
+              self.put_structured_out({"securityIssuesFound": "Keytab file or principal are not set property."})
+              return
+
+            cached_kinit_executor(status_params.kinit_path_local,
+                                  status_params.hive_user,
+                                  security_params['hive-site']['hive.server2.authentication.kerberos.keytab'],
+                                  security_params['hive-site']['hive.server2.authentication.kerberos.principal'],
+                                  status_params.hostname,
+                                  status_params.tmp_dir)
+            cached_kinit_executor(status_params.kinit_path_local,
+                                  status_params.hive_user,
+                                  security_params['hive-site']['hive.server2.authentication.spnego.keytab'],
+                                  security_params['hive-site']['hive.server2.authentication.spnego.principal'],
+                                  status_params.hostname,
+                                  status_params.tmp_dir)
+            self.put_structured_out({"securityState": "SECURED_KERBEROS"})
+          except Exception as e:
+            self.put_structured_out({"securityState": "ERROR"})
+            self.put_structured_out({"securityStateErrorInfo": str(e)})
+        else:
+          issues = []
+          for cf in result_issues:
+            issues.append("Configuration file %s did not pass the validation. Reason: %s" % (cf, result_issues[cf]))
+          self.put_structured_out({"securityIssuesFound": ". ".join(issues)})
+          self.put_structured_out({"securityState": "UNSECURED"})
+      else:
+        self.put_structured_out({"securityState": "UNSECURED"})
+
+    def restart_llap(self, env):
+      """
+      Custom command to Restart LLAP
+      """
+      Logger.info("Custom Command to retart LLAP")
+      import params
+      env.set_params(params)
+
+      if params.security_enabled:
+        self.do_kinit()
+
+      self._llap_stop(env)
+      self._llap_start(env)
+
+    def _llap_stop(self, env):
+      import params
+      Logger.info("Stopping LLAP")
+      SLIDER_APP_NAME = "llap0"
+
+      stop_cmd = ["slider", "stop", SLIDER_APP_NAME]
+
+      code, output, error = shell.call(stop_cmd, user=params.hive_user, stderr=subprocess.PIPE, logoutput=True)
+      if code == 0:
+        Logger.info(format("Stopped {SLIDER_APP_NAME} application on Slider successfully"))
+      elif code == 69 and output is not None and "Unknown application instance" in output:
+        Logger.info(format("Application {SLIDER_APP_NAME} was already stopped on Slider"))
+      else:
+        raise Fail(format("Could not stop application {SLIDER_APP_NAME} on Slider. {error}\n{output}"))
+
+      # Will exit with code 4 if need to run with "--force" to delete directories and registries.
+      Execute(('slider', 'destroy', SLIDER_APP_NAME, "--force"),
+              user=params.hive_user,
+              timeout=30,
+              ignore_failures=True,
+      )
+
+    """
+    Controls the start of LLAP.
+    """
+    def _llap_start(self, env, cleanup=False):
+      import params
+      env.set_params(params)
+      Logger.info("Starting LLAP")
+      LLAP_PACKAGE_CREATION_PATH = Script.get_tmp_dir()
+      LLAP_APP_NAME = 'llap0'
+
+      unique_name = "llap-slider%s" % datetime.utcnow().strftime('%Y-%m-%d_%H-%M-%S')
+
+      cmd = format("{stack_root}/current/hive-server2-hive2/bin/hive --service llap --instances {params.num_llap_nodes}"
+                   " --slider-am-container-mb {params.slider_am_container_mb} --size {params.llap_daemon_container_size}m "
+                   " --cache {params.hive_llap_io_mem_size}m --xmx {params.llap_heap_size}m --loglevel {params.llap_log_level}"
+                   " --output {LLAP_PACKAGE_CREATION_PATH}/{unique_name}")
+      if params.security_enabled:
+        llap_keytab_splits = params.hive_llap_keytab_file.split("/")
+        Logger.debug("llap_keytab_splits : {0}".format(llap_keytab_splits))
+        cmd += format(" --slider-keytab-dir .slider/keytabs/{params.hive_user}/ --slider-keytab "
+                      "{llap_keytab_splits[4]} --slider-principal {params.hive_llap_principal}")
+
+      # Append args.
+      llap_java_args = InlineTemplate(params.llap_app_java_opts).get_content()
+      cmd += format(" --args \" {llap_java_args}\"")
+
+      run_file_path = None
+      try:
+        Logger.info(format("Command: {cmd}"))
+        code, output, error = shell.checked_call(cmd, user=params.hive_user, stderr=subprocess.PIPE, logoutput=True)
+
+        if code != 0 or output is None:
+          raise Fail("Command failed with either non-zero return code or no output.")
+
+        # E.g., output:
+        # Prepared llap-slider-05Apr2016/run.sh for running LLAP on Slider
+        exp = r"Prepared (.*?run.sh) for running LLAP"
+        run_file_path = None
+        out_splits = output.split("\n")
+        for line in out_splits:
+          line = line.strip()
+          m = re.match(exp, line, re.I)
+          if m and len(m.groups()) == 1:
+            run_file_name = m.group(1)
+            run_file_path = os.path.join(params.hive_user_home_dir, run_file_name)
+            break
+        if not run_file_path:
+          raise Fail("Did not find run.sh file in output: " + str(output))
+
+        Logger.info(format("Run file path: {run_file_path}"))
+        Execute(run_file_path, user=params.hive_user)
+        Logger.info("Submitted LLAP app name : {0}".format(LLAP_APP_NAME))
+
+        # We need to check the status of LLAP app to figure out it got
+        # launched properly and is in running state. Then go ahead with Hive Interactive Server start.
+        status = self.check_llap_app_status(LLAP_APP_NAME, params.num_retries_for_checking_llap_status)
+        if status:
+          Logger.info("LLAP app '{0}' deployed successfully.".format(LLAP_APP_NAME))
+          return True
+        else:
+          Logger.error("LLAP app '{0}' deployment unsuccessful.".format(LLAP_APP_NAME))
+          return False
+      except:
+        # Attempt to clean up the packaged application, or potentially rename it with a .bak
+        if run_file_path is not None and cleanup:
+          try:
+            parent_dir = os.path.dirname(run_file_path)
+            if os.path.isdir(parent_dir):
+              shutil.rmtree(parent_dir)
+          except Exception, e:
+            Logger.error("Could not cleanup LLAP app package. Error: " + str(e))
+
+        # throw the original exception
+        raise
+
+    """
+    Does kinit and copies keytab for Hive/LLAP to HDFS.
+    """
+    def setup_security(self):
+      import params
+
+      self.do_kinit()
+
+      # Copy params.hive_llap_keytab_file to hdfs://<host>:<port>/user/<hive_user>/.slider/keytabs/<hive_user> , required by LLAP
+      slider_keytab_install_cmd = format("slider install-keytab --keytab {params.hive_llap_keytab_file} --folder {params.hive_user} --overwrite")
+      Execute(slider_keytab_install_cmd, user=params.hive_user)
+
+    def do_kinit(self):
+      import params
+
+      hive_interactive_kinit_cmd = format("{kinit_path_local} -kt {params.hive_server2_keytab} {params.hive_principal}; ")
+      Execute(hive_interactive_kinit_cmd, user=params.hive_user)
+
+      llap_kinit_cmd = format("{kinit_path_local} -kt {params.hive_llap_keytab_file} {params.hive_llap_principal}; ")
+      Execute(llap_kinit_cmd, user=params.hive_user)
+
+    """
+    Get llap app status data.
+    """
+    def _get_llap_app_status_info(self, app_name):
+      import status_params
+      LLAP_APP_STATUS_CMD_TIMEOUT = 0
+
+      llap_status_cmd = format("{stack_root}/current/hive-server2-hive2/bin/hive --service llapstatus --name {app_name} --findAppTimeout {LLAP_APP_STATUS_CMD_TIMEOUT}")
+      code, output, error = shell.checked_call(llap_status_cmd, user=status_params.hive_user, stderr=subprocess.PIPE,
+                                               logoutput=False)
+      Logger.info("Received 'llapstatus' command 'output' : {0}".format(output))
+      return self._make_valid_json(output)
+
+
+    """
+    Remove extra lines from 'llapstatus' status output (eg: because of MOTD logging) so as to have a valid JSON data to be passed in
+    to JSON converter.
+    """
+    def _make_valid_json(self, output):
+      '''
+
+      Note: It is assumed right now that extra lines will be only at the start and not at the end.
+
+      Sample expected JSON to be passed for 'loads' is either of the form :
+
+      Case 'A':
+      {
+          "amInfo" : {
+          "appName" : "llap0",
+          "appType" : "org-apache-slider",
+          "appId" : "APP1",
+          "containerId" : "container_1466036628595_0010_01_000001",
+          "hostname" : "hostName",
+          "amWebUrl" : "http://hostName:port/"
+        },
+        "state" : "LAUNCHING",
+        ....
+        "desiredInstances" : 1,
+        "liveInstances" : 0,
+        ....
+        ....
+      }
+
+      or
+
+      Case 'B':
+      {
+        "state" : "APP_NOT_FOUND"
+      }
+
+      '''
+      splits = output.split("\n")
+
+      len_splits = len(splits)
+      if (len_splits < 3):
+        raise Fail ("Malformed JSON data received from 'llapstatus' command. Exiting ....")
+
+      marker_idx = None # To detect where from to start reading for JSON data
+      for idx, split in enumerate(splits):
+        curr_elem = split.strip()
+        if idx+2 > len_splits:
+          raise Fail("Iterated over the received 'llapstatus' comamnd. Couldn't validate the received output for JSON parsing.")
+        next_elem = (splits[(idx + 1)]).strip()
+        if curr_elem == "{":
+          if next_elem == "\"amInfo\" : {" and (splits[len_splits-1]).strip() == '}':
+            # For Case 'A'
+            marker_idx = idx
+            break;
+          elif idx+3 == len_splits and next_elem.startswith('"state" : ') and (splits[idx + 2]).strip() == '}':
+              # For Case 'B'
+              marker_idx = idx
+              break;
+
+      Logger.info("Marker index for start of JSON data for 'llapsrtatus' comamnd : {0}".format(marker_idx))
+
+      # Remove extra logging from possible JSON output
+      if marker_idx is None:
+        raise Fail("Couldn't validate the received output for JSON parsing.")
+      else:
+        if marker_idx != 0:
+          del splits[0:marker_idx]
+          Logger.info("Removed lines: '1-{0}' from the received 'llapstatus' output to make it valid for JSON parsing.".format(marker_idx))
+
+      scanned_output = '\n'.join(splits)
+      llap_app_info = json.loads(scanned_output)
+      return llap_app_info
+
+
+    """
+    Checks llap app status. The states can be : 'COMPLETE', 'APP_NOT_FOUND', 'RUNNING_PARTIAL', 'RUNNING_ALL' & 'LAUNCHING'.
+
+    if app is in 'APP_NOT_FOUND', 'RUNNING_PARTIAL' and 'LAUNCHING' state:
+       we wait for 'num_times_to_wait' to have app in (1). 'RUNNING_ALL' or (2). 'RUNNING_PARTIAL'
+       state with 80% or more 'desiredInstances' running and Return True
+    else :
+       Return False
+
+    Parameters: llap_app_name : deployed llap app name.
+                num_retries :   Number of retries to check the LLAP app status.
+    """
+    def check_llap_app_status(self, llap_app_name, num_retries):
+      # counters based on various states.
+      curr_time = time.time()
+
+      if num_retries <= 0:
+        num_retries = 2
+      if num_retries > 20:
+        num_retries = 20
+      @retry(times=num_retries, sleep_time=2, err_class=Fail)
+      def do_retries():
+        live_instances = 0
+        desired_instances = 0
+
+        percent_desired_instances_to_be_up = 80 # Used in 'RUNNING_PARTIAL' state.
+        llap_app_info = self._get_llap_app_status_info(llap_app_name)
+        if llap_app_info is None or 'state' not in llap_app_info:
+          Logger.error("Malformed JSON data received for LLAP app. Exiting ....")
+          return False
+
+        if llap_app_info['state'].upper() == 'RUNNING_ALL':
+          Logger.info(
+            "LLAP app '{0}' in '{1}' state.".format(llap_app_name, llap_app_info['state']))
+          return True
+        elif llap_app_info['state'].upper() == 'RUNNING_PARTIAL':
+          # Check how many instances were up.
+          if 'liveInstances' in llap_app_info and 'desiredInstances' in llap_app_info:
+            live_instances = llap_app_info['liveInstances']
+            desired_instances = llap_app_info['desiredInstances']
+          else:
+            Logger.info(
+              "LLAP app '{0}' is in '{1}' state, but 'instances' information not available in JSON received. " \
+              "Exiting ....".format(llap_app_name, llap_app_info['state']))
+            Logger.info(llap_app_info)
+            return False
+          if desired_instances == 0:
+            Logger.info("LLAP app '{0}' desired instance are set to 0. Exiting ....".format(llap_app_name))
+            return False
+
+          percentInstancesUp = 0
+          if live_instances > 0:
+            percentInstancesUp = float(live_instances) / desired_instances * 100
+          if percentInstancesUp >= percent_desired_instances_to_be_up:
+            Logger.info("LLAP app '{0}' in '{1}' state. Live Instances : '{2}'  >= {3}% of Desired Instances : " \
+                        "'{4}'.".format(llap_app_name, llap_app_info['state'],
+                                       llap_app_info['liveInstances'],
+                                       percent_desired_instances_to_be_up,
+                                       llap_app_info['desiredInstances']))
+            return True
+          else:
+            Logger.info("LLAP app '{0}' in '{1}' state. Live Instances : '{2}'. Desired Instances : " \
+                        "'{3}' after {4} secs.".format(llap_app_name, llap_app_info['state'],
+                                                       llap_app_info['liveInstances'],
+                                                       llap_app_info['desiredInstances'],
+                                                       time.time() - curr_time))
+            raise Fail("App state is RUNNING_PARTIAL. Live Instances : '{0}', Desired Instance : '{1}'".format(llap_app_info['liveInstances'],
+                                                                                                           llap_app_info['desiredInstances']))
+        elif llap_app_info['state'].upper() in ['APP_NOT_FOUND', 'LAUNCHING', 'COMPLETE']:
+          status_str = format("LLAP app '{0}' current state is {1}.".format(llap_app_name, llap_app_info['state']))
+          Logger.info(status_str)
+          raise Fail(status_str)
+        else:  # Covers any unknown that we get.
+          Logger.info(
+            "LLAP app '{0}' current state is '{1}'. Expected : 'RUNNING'.".format(llap_app_name, llap_app_info['state']))
+          return False
+
+      try:
+        status = do_retries()
+        return status
+      except Exception, e:
+        Logger.info("LLAP app '{0}' did not come up after a wait of {1} seconds.".format(llap_app_name,
+                                                                                          time.time() - curr_time))
+        traceback.print_exc()
+        return False
+
+    def get_log_folder(self):
+      import params
+      return params.hive_log_dir
+
+    def get_user(self):
+      import params
+      return params.hive_user
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class HiveServerInteractiveWindows(HiveServerInteractive):
+
+  def status(self, env):
+    pass
+
+if __name__ == "__main__":
+  HiveServerInteractive().execute()
\ No newline at end of file


[38/50] [abbrv] bigtop git commit: ODPI-196. Adding tests to test the doas feature of httpfs.

Posted by rv...@apache.org.
ODPI-196. Adding tests to test the doas feature of httpfs.


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

Branch: refs/heads/BIGTOP-2666
Commit: bebdd0997a3dd05c1f867fcdf6f2a92ae3e6fc33
Parents: 5678bcf
Author: Clint Edwards <cl...@sas.com>
Authored: Fri Oct 21 15:05:58 2016 -0400
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:06 2017 -0800

----------------------------------------------------------------------
 .../bigtop/itest/httpfs/TestHttpFs.groovy       | 118 ++++++++++++++-----
 1 file changed, 87 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/bebdd099/bigtop-tests/test-artifacts/httpfs/src/main/groovy/org/apache/bigtop/itest/httpfs/TestHttpFs.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/httpfs/src/main/groovy/org/apache/bigtop/itest/httpfs/TestHttpFs.groovy b/bigtop-tests/test-artifacts/httpfs/src/main/groovy/org/apache/bigtop/itest/httpfs/TestHttpFs.groovy
index bcec657..a4134a2 100644
--- a/bigtop-tests/test-artifacts/httpfs/src/main/groovy/org/apache/bigtop/itest/httpfs/TestHttpFs.groovy
+++ b/bigtop-tests/test-artifacts/httpfs/src/main/groovy/org/apache/bigtop/itest/httpfs/TestHttpFs.groovy
@@ -32,6 +32,7 @@ public class TestHttpFs {
   private static Log LOG = LogFactory.getLog(TestHttpFs.class)
 
   private static final String USERNAME = System.getProperty("user.name");
+  private static final String DOASUSER = 'hdfs';
   private static final String HTTPFS_PROXY = System.getenv('HTTPFS_PROXY');
   static {
     assertNotNull("HTTPFS_PROXY has to be set to run this test",
@@ -96,56 +97,59 @@ public class TestHttpFs {
     assertTrue(expected + " NOT found!", exists == true);
   }
 
-  private void createDir(String dirname) {
-    sh.exec("curl -i -X PUT '$HTTPFS_PREFIX$dirname?user.name=$USERNAME&op=MKDIRS'");
-  }
-
-  @Test
-  public void testCreateDir() {
-    createDir(testHttpFsFolder)
+  private void createDir(String dirname, String doasUser='') {
+    def doasStr = ''
+    if(doasUser.length()) {
+        doasStr = "doas=$doasUser&"
+    }
+    sh.exec("curl -i -X PUT '$HTTPFS_PREFIX$dirname?user.name=$USERNAME&${doasStr}op=MKDIRS'");
     assertTrue("curl command to create a dir failed", sh.getRet() == 0);
     assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
   }
 
-  @Test
-  public void testRenameDir() {
-    createDir(testHttpFsFolder);
-    assertTrue("curl command to create a dir failed", sh.getRet() == 0);
-    assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
-    sh.exec("curl -i -X PUT '$HTTPFS_PREFIX$testHttpFsFolder?user.name=$USERNAME&op=RENAME&destination=$testHttpFsFolderRenamed'");
+  private void renameDir(String dirname, String doasUser='') {
+    createDir(dirname, doasUser);
+    def doasStr = ''
+    if(doasUser.length()) {
+        doasStr = "doas=$doasUser&"
+    }
+    sh.exec("curl -i -X PUT '$HTTPFS_PREFIX$testHttpFsFolder?user.name=$USERNAME&${doasStr}op=RENAME&destination=$testHttpFsFolderRenamed'");
     assertTrue("curl command to rename a dir failed", sh.getRet() == 0);
     assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
   }
 
-  @Test
-  public void testDeleteDir() {
-    createDir(testHttpFsFolder);
-    assertTrue("curl command to create a dir failed", sh.getRet() == 0);
-    assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
-    sh.exec("curl -i -X DELETE '$HTTPFS_PREFIX$testHttpFsFolder?user.name=$USERNAME&op=DELETE'");
+  private void deleteDir(String dirname, String doasUser='') {
+    createDir(dirname, doasUser);
+    def doasStr = ''
+    if(doasUser.length()) {
+        doasStr = "doas=$doasUser&"
+    }
+    sh.exec("curl -i -X DELETE '$HTTPFS_PREFIX$testHttpFsFolder?user.name=$USERNAME&${doasStr}op=DELETE'");
     assertTrue("curl command to delete a dir failed", sh.getRet() == 0);
     assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
   }
 
-  @Test
-  public void testStatusDir() {
-    createDir(testHttpFsFolder);
-    assertTrue("curl command to create a dir failed", sh.getRet() == 0);
-    assertValueExists(sh.getOut(), HTTPFS_SUCCESS);
-    sh.exec("curl -i '$HTTPFS_PREFIX$testHttpFsFolder?user.name=$USERNAME&op=GETFILESTATUS'");
+  private void statusDir(String dirname, String doasUser='') {
+    createDir(dirname, doasUser);
+    def doasStr = ''
+    if(doasUser.length()) {
+        doasStr = "doas=$doasUser&"
+    }
+    sh.exec("curl -i '$HTTPFS_PREFIX$testHttpFsFolder?user.name=$USERNAME&${doasStr}op=GETFILESTATUS'");
     assertTrue("curl command to create a dir failed", sh.getRet() == 0);
     assertValueContains(sh.getOut(), "DIRECTORY");
     assertValueExists(sh.getOut(), HTTP_OK);
   }
 
-  @Test
-  public void testCreateFile() {
-    String filename = "helloworld.txt";
+  private void createFile(String filename, String doasUser='') {
     String filenameContent = 'Hello World!';
+    def doasStr = ''
+    if(doasUser.length()) {
+        doasStr = "doas=$doasUser&"
+    }
 
-    createDir(testHttpFsFolder);
-    assertTrue("curl command to create a dir failed", sh.getRet() == 0);
-    sh.exec("curl -i -X PUT '$HTTPFS_PREFIX$testHttpFsFolder/$filename?user.name=$USERNAME&op=CREATE'");
+    createDir(testHttpFsFolder, doasUser);
+    sh.exec("curl -i -X PUT '$HTTPFS_PREFIX$testHttpFsFolder/$filename?user.name=$USERNAME&${doasStr}op=CREATE'");
     assertTrue("curl command to create a file failed", sh.getRet() == 0);
     String datanodeLocation = null;
     sh.getOut().each {
@@ -165,5 +169,57 @@ public class TestHttpFs {
     assertValueExists(sh.getOut(), HTTP_OK);
     assertValueExists(sh.getOut(), filenameContent);
   }
+
+  @Test
+  public void testCreateDir() {
+    createDir(testHttpFsFolder)
+  }
+
+  @Test
+  public void testCreateDirAsUser() {
+    createDir(testHttpFsFolder, DOASUSER)
+  }
+
+  @Test
+  public void testRenameDir() {
+    renameDir(testHttpFsFolder);
+  }
+
+  @Test
+  public void testRenameDirAsUser() {
+    renameDir(testHttpFsFolder, DOASUSER);
+  }
+
+  @Test
+  public void testDeleteDir() {
+    deleteDir(testHttpFsFolder);
+  }
+
+  @Test
+  public void testDeleteDirAsUser() {
+    deleteDir(testHttpFsFolder, DOASUSER);
+  }
+
+  @Test
+  public void testStatusDir() {
+    statusDir(testHttpFsFolder);
+  }
+
+  @Test
+  public void testStatusDirAsUser() {
+    statusDir(testHttpFsFolder, DOASUSER);
+  }
+
+  @Test
+  public void testCreateFile() {
+    String filename = "helloworld.txt";
+    createFile(filename)
+  }
+
+  @Test
+  public void testCreateFileAsUser() {
+    String filename = "helloworld.txt";
+    createFile(filename, DOASUSER)
+  }
 }
 


[09/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_upgrade.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_upgrade.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_upgrade.py
new file mode 100755
index 0000000..478c240
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_upgrade.py
@@ -0,0 +1,141 @@
+"""
+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.
+
+"""
+
+import os
+import re
+from resource_management.core.logger import Logger
+from resource_management.core.exceptions import Fail
+from resource_management.core.resources.system import Execute
+from resource_management.core import shell
+from resource_management.libraries.functions import format
+from resource_management.libraries.functions import stack_select
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.functions.version import format_stack_version
+
+
+def post_upgrade_deregister():
+  """
+  Runs the "hive --service hiveserver2 --deregister <version>" command to
+  de-provision the server in preparation for an upgrade. This will contact
+  ZooKeeper to remove the server so that clients that attempt to connect
+  will be directed to other servers automatically. Once all
+  clients have drained, the server will shutdown automatically; this process
+  could take a very long time.
+  This function will obtain the Kerberos ticket if security is enabled.
+  :return:
+  """
+  import params
+
+  Logger.info('HiveServer2 executing "deregister" command to complete upgrade...')
+
+  if params.security_enabled:
+    kinit_command=format("{kinit_path_local} -kt {smoke_user_keytab} {smokeuser_principal}; ")
+    Execute(kinit_command,user=params.smokeuser)
+
+  # calculate the current hive server version
+  current_hiveserver_version = _get_current_hiveserver_version()
+  if current_hiveserver_version is None:
+    raise Fail('Unable to determine the current HiveServer2 version to deregister.')
+
+  # fallback when upgrading because <stack-root>/current/hive-server2/conf/conf.server may not exist
+  hive_server_conf_dir = params.hive_server_conf_dir
+  if not os.path.exists(hive_server_conf_dir):
+    hive_server_conf_dir = "/etc/hive/conf.server"
+
+  # deregister
+  hive_execute_path = params.execute_path
+  # If upgrading, the upgrade-target hive binary should be used to call the --deregister command.
+  # If downgrading, the downgrade-source hive binary should be used to call the --deregister command.
+  # By now <stack-selector-tool> has been called to set 'current' to target-stack
+  if "downgrade" == params.upgrade_direction:
+    # hive_bin
+    downgrade_version = params.current_version
+    if params.downgrade_from_version:
+      downgrade_version = params.downgrade_from_version
+    hive_execute_path = _get_hive_execute_path(downgrade_version)
+
+  command = format('hive --config {hive_server_conf_dir} --service hiveserver2 --deregister ' + current_hiveserver_version)
+  Execute(command, user=params.hive_user, path=hive_execute_path, tries=1 )
+
+
+def _get_hive_execute_path(stack_version_formatted):
+  """
+  Returns the exact execute path to use for the given stack-version.
+  This method does not return the "current" path
+  :param stack_version_formatted: Exact stack-version to use in the new path
+  :return: Hive execute path for the exact stack-version
+  """
+  import params
+
+  hive_execute_path = params.execute_path
+  formatted_stack_version = format_stack_version(stack_version_formatted)
+  if formatted_stack_version and check_stack_feature(StackFeature.ROLLING_UPGRADE, formatted_stack_version):
+    # hive_bin
+    new_hive_bin = format('{stack_root}/{stack_version_formatted}/hive/bin')
+    if (os.pathsep + params.hive_bin) in hive_execute_path:
+      hive_execute_path = hive_execute_path.replace(os.pathsep + params.hive_bin, os.pathsep + new_hive_bin)
+    # hadoop_bin_dir
+    new_hadoop_bin = stack_select.get_hadoop_dir_for_stack_version("bin", stack_version_formatted)
+    old_hadoop_bin = params.hadoop_bin_dir
+    if new_hadoop_bin and len(new_hadoop_bin) > 0 and (os.pathsep + old_hadoop_bin) in hive_execute_path:
+      hive_execute_path = hive_execute_path.replace(os.pathsep + old_hadoop_bin, os.pathsep + new_hadoop_bin)
+  return hive_execute_path
+
+
+def _get_current_hiveserver_version():
+  """
+  Runs "hive --version" and parses the result in order
+  to obtain the current version of hive.
+
+  :return:  the hiveserver2 version, returned by "hive --version"
+  """
+  import params
+
+  try:
+    # When downgrading the source version should be the version we are downgrading from
+    if "downgrade" == params.upgrade_direction:
+      if not params.downgrade_from_version:
+        raise Fail('The version from which we are downgrading from should be provided in \'downgrade_from_version\'')
+      source_version = params.downgrade_from_version
+    else:
+      source_version = params.current_version
+    hive_execute_path = _get_hive_execute_path(source_version)
+    version_hive_bin = params.hive_bin
+    formatted_source_version = format_stack_version(source_version)
+    if formatted_source_version and check_stack_feature(StackFeature.ROLLING_UPGRADE, formatted_source_version):
+      version_hive_bin = format('{stack_root}/{source_version}/hive/bin')
+    command = format('{version_hive_bin}/hive --version')
+    return_code, output = shell.call(command, user=params.hive_user, path=hive_execute_path)
+  except Exception, e:
+    Logger.error(str(e))
+    raise Fail('Unable to execute hive --version command to retrieve the hiveserver2 version.')
+
+  if return_code != 0:
+    raise Fail('Unable to determine the current HiveServer2 version because of a non-zero return code of {0}'.format(str(return_code)))
+
+  match = re.search('^(Hive) ([0-9]+.[0-9]+.\S+)', output, re.MULTILINE)
+
+  if match:
+    current_hive_server_version = match.group(2)
+    return current_hive_server_version
+  else:
+    raise Fail('The extracted hiveserver2 version "{0}" does not matching any known pattern'.format(output))
+
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service.py
new file mode 100755
index 0000000..22b4061
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service.py
@@ -0,0 +1,187 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import os
+import time
+
+from ambari_commons.constants import UPGRADE_TYPE_ROLLING
+from resource_management.core.logger import Logger
+from resource_management.core import shell
+from resource_management.libraries.functions.format import format
+from resource_management.core.resources.system import File, Execute
+from resource_management.core.resources.service import Service
+from resource_management.core.exceptions import Fail
+from resource_management.core.shell import as_user
+from resource_management.libraries.functions.hive_check import check_thrift_port_sasl
+from resource_management.libraries.functions import get_user_call_output
+from resource_management.libraries.functions.show_logs import show_logs
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def hive_service(name, action='start', upgrade_type=None):
+  import params
+  if name == 'metastore':
+    if action == 'start' or action == 'stop':
+      Service(params.hive_metastore_win_service_name, action=action)
+
+  if name == 'hiveserver2':
+    if action == 'start' or action == 'stop':
+      Service(params.hive_server_win_service_name, action=action)
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def hive_service(name, action='start', upgrade_type=None):
+
+  import params
+
+  if name == 'metastore':
+    pid_file = format("{hive_pid_dir}/{hive_metastore_pid}")
+    cmd = format("{start_metastore_path} {hive_log_dir}/hive.out {hive_log_dir}/hive.err {pid_file} {hive_server_conf_dir} {hive_log_dir}")
+  elif name == 'hiveserver2':
+    pid_file = format("{hive_pid_dir}/{hive_pid}")
+    cmd = format("{start_hiveserver2_path} {hive_log_dir}/hive-server2.out {hive_log_dir}/hive-server2.err {pid_file} {hive_server_conf_dir} {hive_log_dir}")
+
+
+    if params.security_enabled and params.current_version and check_stack_feature(StackFeature.HIVE_SERVER2_KERBERIZED_ENV, params.current_version):
+      hive_kinit_cmd = format("{kinit_path_local} -kt {hive_server2_keytab} {hive_principal}; ")
+      Execute(hive_kinit_cmd, user=params.hive_user)
+
+  pid = get_user_call_output.get_user_call_output(format("cat {pid_file}"), user=params.hive_user, is_checked_call=False)[1]
+  process_id_exists_command = format("ls {pid_file} >/dev/null 2>&1 && ps -p {pid} >/dev/null 2>&1")
+
+  if action == 'start':
+    if name == 'hiveserver2':
+      check_fs_root(params.hive_server_conf_dir, params.execute_path)
+
+    daemon_cmd = cmd
+    hadoop_home = params.hadoop_home
+    hive_bin = "hive"
+
+    # upgrading hiveserver2 (rolling_restart) means that there is an existing,
+    # de-registering hiveserver2; the pid will still exist, but the new
+    # hiveserver is spinning up on a new port, so the pid will be re-written
+    if upgrade_type == UPGRADE_TYPE_ROLLING:
+      process_id_exists_command = None
+
+      if params.version and params.stack_root:
+        hadoop_home = format("{stack_root}/{version}/hadoop")
+        hive_bin = os.path.join(params.hive_bin, hive_bin)
+      
+    Execute(daemon_cmd, 
+      user = params.hive_user,
+      environment = { 'HADOOP_HOME': hadoop_home, 'JAVA_HOME': params.java64_home, 'HIVE_BIN': hive_bin },
+      path = params.execute_path,
+      not_if = process_id_exists_command)
+
+    if params.hive_jdbc_driver == "com.mysql.jdbc.Driver" or \
+       params.hive_jdbc_driver == "org.postgresql.Driver" or \
+       params.hive_jdbc_driver == "oracle.jdbc.driver.OracleDriver":
+
+      validation_called = False
+
+      if params.hive_jdbc_target is not None:
+        validation_called = True
+        validate_connection(params.hive_jdbc_target, params.hive_lib)
+      if params.hive2_jdbc_target is not None:
+        validation_called = True
+        validate_connection(params.hive2_jdbc_target, params.hive_server2_hive2_lib)
+
+      if not validation_called:
+        emessage = "ERROR! DB connection check should be executed at least one time!"
+        Logger.error(emessage)
+
+  elif action == 'stop':
+
+    daemon_kill_cmd = format("{sudo} kill {pid}")
+    daemon_hard_kill_cmd = format("{sudo} kill -9 {pid}")
+
+    Execute(daemon_kill_cmd,
+      not_if = format("! ({process_id_exists_command})")
+    )
+
+    wait_time = 5
+    Execute(daemon_hard_kill_cmd,
+      not_if = format("! ({process_id_exists_command}) || ( sleep {wait_time} && ! ({process_id_exists_command}) )"),
+      ignore_failures = True
+    )
+
+    try:
+      # check if stopped the process, else fail the task
+      Execute(format("! ({process_id_exists_command})"),
+        tries=20,
+        try_sleep=3,
+      )
+    except:
+      show_logs(params.hive_log_dir, params.hive_user)
+      raise
+
+    File(pid_file,
+         action = "delete"
+    )
+
+def validate_connection(target_path_to_jdbc, hive_lib_path):
+  import params
+
+  path_to_jdbc = target_path_to_jdbc
+  if not params.jdbc_jar_name:
+    path_to_jdbc = format("{hive_lib_path}/") + \
+                   params.default_connectors_map[params.hive_jdbc_driver] if params.hive_jdbc_driver in params.default_connectors_map else None
+    if not os.path.isfile(path_to_jdbc):
+      path_to_jdbc = format("{hive_lib_path}/") + "*"
+      error_message = "Error! Sorry, but we can't find jdbc driver with default name " + params.default_connectors_map[params.hive_jdbc_driver] + \
+                      " in hive lib dir. So, db connection check can fail. Please run 'ambari-server setup --jdbc-db={db_name} --jdbc-driver={path_to_jdbc} on server host.'"
+      Logger.error(error_message)
+
+  db_connection_check_command = format(
+    "{java64_home}/bin/java -cp {check_db_connection_jar}:{path_to_jdbc} org.apache.ambari.server.DBConnectionVerification '{hive_jdbc_connection_url}' {hive_metastore_user_name} {hive_metastore_user_passwd!p} {hive_jdbc_driver}")
+
+  try:
+    Execute(db_connection_check_command,
+            path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin', tries=5, try_sleep=10)
+  except:
+    show_logs(params.hive_log_dir, params.hive_user)
+    raise
+
+
+def check_fs_root(conf_dir, execution_path):
+  import params
+
+  if not params.fs_root.startswith("hdfs://"):
+    Logger.info("Skipping fs root check as fs_root does not start with hdfs://")
+    return
+
+  metatool_cmd = format("hive --config {conf_dir} --service metatool")
+  cmd = as_user(format("{metatool_cmd} -listFSRoot", env={'PATH': execution_path}), params.hive_user) \
+        + format(" 2>/dev/null | grep hdfs:// | cut -f1,2,3 -d '/' | grep -v '{fs_root}' | head -1")
+  code, out = shell.call(cmd)
+
+  if code == 0 and out.strip() != "" and params.fs_root.strip() != out.strip():
+    out = out.strip()
+    cmd = format("{metatool_cmd} -updateLocation {fs_root} {out}")
+    Execute(cmd,
+            user=params.hive_user,
+            environment={'PATH': execution_path}
+    )
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service_interactive.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service_interactive.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service_interactive.py
new file mode 100755
index 0000000..eaf95ad
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service_interactive.py
@@ -0,0 +1,109 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+# Python Imports
+
+# Ambari Commons & Resource Management imports
+import os
+from resource_management.core.logger import Logger
+from resource_management.libraries.functions.format import format
+from resource_management.core.resources.system import File, Execute
+from resource_management.libraries.functions import get_user_call_output
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+
+# Local Imports
+from hive_service import check_fs_root
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def hive_service_interactive(name, action='start', upgrade_type=None):
+  pass
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def hive_service_interactive(name, action='start', upgrade_type=None):
+  import params
+
+  pid_file = format("{hive_pid_dir}/{hive_interactive_pid}")
+  cmd = format("{start_hiveserver2_interactive_path} {hive_pid_dir}/hive-server2-interactive.out {hive_log_dir}/hive-server2-interactive.err {pid_file} {hive_server_interactive_conf_dir} {hive_log_dir}")
+
+  pid = get_user_call_output.get_user_call_output(format("cat {pid_file}"), user=params.hive_user, is_checked_call=False)[1]
+  process_id_exists_command = format("ls {pid_file} >/dev/null 2>&1 && ps -p {pid} >/dev/null 2>&1")
+
+  if action == 'start':
+    check_fs_root(params.hive_server_interactive_conf_dir, params.execute_path_hive_interactive)
+    daemon_cmd = cmd
+    hadoop_home = params.hadoop_home
+    hive_interactive_bin = "hive2"
+
+    Execute(daemon_cmd,
+            user = params.hive_user,
+            environment = { 'HADOOP_HOME': hadoop_home, 'JAVA_HOME': params.java64_home, 'HIVE_BIN': hive_interactive_bin },
+            path = params.execute_path,
+            not_if = process_id_exists_command)
+
+    if params.hive_jdbc_driver == "com.mysql.jdbc.Driver" or \
+        params.hive_jdbc_driver == "org.postgresql.Driver" or \
+        params.hive_jdbc_driver == "oracle.jdbc.driver.OracleDriver":
+
+      path_to_jdbc = params.target_hive_interactive
+      if not params.jdbc_jar_name:
+        path_to_jdbc = format("{hive_interactive_lib}/") + \
+                       params.default_connectors_map[params.hive_jdbc_driver] if params.hive_jdbc_driver in params.default_connectors_map else None
+        if not os.path.isfile(path_to_jdbc):
+          path_to_jdbc = format("{hive_interactive_lib}/") + "*"
+          error_message = "Error! Sorry, but we can't find jdbc driver with default name " + params.default_connectors_map[params.hive_jdbc_driver] + \
+                " in hive lib dir. So, db connection check can fail. Please run 'ambari-server setup --jdbc-db={db_name} --jdbc-driver={path_to_jdbc} on server host.'"
+          Logger.error(error_message)
+
+      db_connection_check_command = format(
+        "{java64_home}/bin/java -cp {check_db_connection_jar}:{path_to_jdbc} org.apache.ambari.server.DBConnectionVerification '{hive_jdbc_connection_url}' {hive_metastore_user_name} {hive_metastore_user_passwd!p} {hive_jdbc_driver}")
+      Execute(db_connection_check_command,
+              path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin', tries=5, try_sleep=10)
+  elif action == 'stop':
+
+    daemon_kill_cmd = format("{sudo} kill {pid}")
+    daemon_hard_kill_cmd = format("{sudo} kill -9 {pid}")
+
+    Execute(daemon_kill_cmd,
+            not_if = format("! ({process_id_exists_command})")
+            )
+
+    # check if stopped the process, otherwise send hard kill command.
+    try:
+      Execute(format("! ({process_id_exists_command})"),
+              tries=10,
+              try_sleep=3,
+              )
+    except:
+      Execute(daemon_hard_kill_cmd,
+              not_if = format("! ({process_id_exists_command}) ")
+              )
+
+    # check if stopped the process, else fail the task
+    Execute(format("! ({process_id_exists_command})"),
+            tries=20,
+            try_sleep=3,
+            )
+
+    File(pid_file,
+         action = "delete"
+         )

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_server.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_server.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_server.py
new file mode 100755
index 0000000..851dc02
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_server.py
@@ -0,0 +1,64 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import sys
+import os
+import mysql_users
+from resource_management import *
+
+from mysql_service import mysql_service
+from mysql_utils import mysql_configure
+
+
+class MysqlServer(Script):
+  def install(self, env):
+    import params
+    self.install_packages(env)
+    self.configure(env)
+
+  def clean(self, env):
+    import params
+    env.set_params(params)
+    mysql_users.mysql_deluser()
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    mysql_configure()
+
+  def start(self, env, rolling_restart=False):
+    import params
+    env.set_params(params)
+    mysql_service(daemon_name=params.daemon_name, action='start')
+
+  def stop(self, env, rolling_restart=False):
+    import params
+    env.set_params(params)
+    mysql_service(daemon_name=params.daemon_name, action='stop')
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+
+    mysql_service(daemon_name=status_params.daemon_name, action='status')
+
+
+if __name__ == "__main__":
+  MysqlServer().execute()

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_service.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_service.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_service.py
new file mode 100755
index 0000000..8b98ed1
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_service.py
@@ -0,0 +1,49 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+from resource_management import *
+
+
+def mysql_service(daemon_name=None, action='start'): 
+  status_cmd = format("pgrep -l '^{process_name}$'")
+  cmd = ('service', daemon_name, action)
+
+  if action == 'status':
+    try:
+      Execute(status_cmd)
+    except Fail:
+      raise ComponentIsNotRunning()
+  elif action == 'stop':
+    import params
+    Execute(cmd,
+            logoutput = True,
+            only_if = status_cmd,
+            sudo = True,
+    )
+  elif action == 'start':
+    import params   
+    Execute(cmd,
+      logoutput = True,
+      not_if = status_cmd,
+      sudo = True,
+    )
+
+
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_users.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_users.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_users.py
new file mode 100755
index 0000000..c023548
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_users.py
@@ -0,0 +1,70 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+from resource_management import *
+
+# Used to add hive access to the needed components
+def mysql_adduser():
+  import params
+  
+  File(params.mysql_adduser_path,
+       mode=0755,
+       content=StaticFile('addMysqlUser.sh')
+  )
+  hive_server_host = format("{hive_server_host}")
+  hive_metastore_host = format("{hive_metastore_host}")
+
+  add_metastore_cmd = "bash -x {mysql_adduser_path} {daemon_name} {hive_metastore_user_name} {hive_metastore_user_passwd!p} {hive_metastore_host}"
+  add_hiveserver_cmd = "bash -x {mysql_adduser_path} {daemon_name} {hive_metastore_user_name} {hive_metastore_user_passwd!p} {hive_server_host}"
+  if (hive_server_host == hive_metastore_host):
+    cmd = format(add_hiveserver_cmd)
+  else:
+    cmd = format(add_hiveserver_cmd + ";" + add_metastore_cmd)
+  Execute(cmd,
+          tries=3,
+          try_sleep=5,
+          logoutput=False,
+          path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin'
+  )
+
+# Removes hive access from components
+def mysql_deluser():
+  import params
+  
+  File(params.mysql_deluser_path,
+       mode=0755,
+       content=StaticFile('removeMysqlUser.sh')
+  )
+  hive_server_host = format("{hive_server_host}")
+  hive_metastore_host = format("{hive_metastore_host}")
+
+  del_hiveserver_cmd = "bash -x {mysql_deluser_path} {daemon_name} {hive_metastore_user_name} {hive_server_host}"
+  del_metastore_cmd = "bash -x {mysql_deluser_path} {daemon_name} {hive_metastore_user_name} {hive_metastore_host}"
+  if (hive_server_host == hive_metastore_host):
+    cmd = format(del_hiveserver_cmd)
+  else:
+    cmd = format(
+      del_hiveserver_cmd + ";" + del_metastore_cmd)
+  Execute(cmd,
+          tries=3,
+          try_sleep=5,
+          path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin',
+  )
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_utils.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_utils.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_utils.py
new file mode 100755
index 0000000..5006b56
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_utils.py
@@ -0,0 +1,35 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+from resource_management import *
+import mysql_users
+
+def mysql_configure():
+  import params
+
+  # required for running hive
+  replace_bind_address = ('sed','-i','s|^bind-address[ \t]*=.*|bind-address = 0.0.0.0|',params.mysql_configname)
+  Execute(replace_bind_address,
+          sudo = True,
+  )
+  
+  # this also will start mysql-server
+  mysql_users.mysql_adduser()
+  
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params.py
new file mode 100755
index 0000000..f10a3f3
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params.py
@@ -0,0 +1,29 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+from ambari_commons import OSCheck
+from resource_management.libraries.functions.default import default
+
+if OSCheck.is_windows_family():
+  from params_windows import *
+else:
+  from params_linux import *
+
+host_sys_prepped = default("/hostLevelParams/host_sys_prepped", False)
+retryAble = default("/commandParams/command_retry_enabled", False)

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
new file mode 100755
index 0000000..1cd58c3
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
@@ -0,0 +1,736 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import status_params
+import ambari_simplejson as json # simplejson is much faster comparing to Python 2.6 json module and has the same functions set.
+import os
+
+from urlparse import urlparse
+
+from ambari_commons.constants import AMBARI_SUDO_BINARY
+from ambari_commons.os_check import OSCheck
+
+from resource_management.libraries.resources.hdfs_resource import HdfsResource
+from resource_management.libraries.functions.default import default
+from resource_management.libraries.functions.format import format
+from resource_management.libraries.functions.is_empty import is_empty
+from resource_management.libraries.functions.copy_tarball import STACK_ROOT_PATTERN, STACK_NAME_PATTERN, STACK_VERSION_PATTERN
+from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
+from resource_management.libraries.script.script import Script
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.functions.stack_features import get_stack_feature_version
+from resource_management.libraries.functions.get_port_from_url import get_port_from_url
+from resource_management.libraries.functions.expect import expect
+from resource_management.libraries import functions
+from resource_management.libraries.functions.setup_atlas_hook import has_atlas_in_cluster
+
+# Default log4j version; put config files under /etc/hive/conf
+log4j_version = '1'
+
+# server configurations
+config = Script.get_config()
+tmp_dir = Script.get_tmp_dir()
+sudo = AMBARI_SUDO_BINARY
+
+stack_root = status_params.stack_root
+stack_name = status_params.stack_name
+stack_name_uppercase = stack_name.upper()
+agent_stack_retry_on_unavailability = config['hostLevelParams']['agent_stack_retry_on_unavailability']
+agent_stack_retry_count = expect("/hostLevelParams/agent_stack_retry_count", int)
+
+# Needed since this is an Atlas Hook service.
+cluster_name = config['clusterName']
+
+# node hostname
+hostname = config["hostname"]
+
+# This is expected to be of the form #.#.#.#
+stack_version_unformatted = status_params.stack_version_unformatted
+stack_version_formatted_major = status_params.stack_version_formatted_major
+
+# this is not available on INSTALL action because <stack-selector-tool> is not available
+stack_version_formatted = functions.get_stack_version('hive-server2')
+
+# New Cluster Stack Version that is defined during the RESTART of a Rolling Upgrade.
+# It cannot be used during the initial Cluser Install because the version is not yet known.
+version = default("/commandParams/version", None)
+
+# current host stack version
+current_version = default("/hostLevelParams/current_version", None)
+
+# When downgrading the 'version' and 'current_version' are both pointing to the downgrade-target version
+# downgrade_from_version provides the source-version the downgrade is happening from
+downgrade_from_version = default("/commandParams/downgrade_from_version", None)
+
+# get the correct version to use for checking stack features
+version_for_stack_feature_checks = get_stack_feature_version(config)
+
+# Upgrade direction
+upgrade_direction = default("/commandParams/upgrade_direction", None)
+stack_supports_ranger_kerberos = check_stack_feature(StackFeature.RANGER_KERBEROS_SUPPORT, version_for_stack_feature_checks)
+stack_supports_ranger_audit_db = check_stack_feature(StackFeature.RANGER_AUDIT_DB_SUPPORT, version_for_stack_feature_checks)
+
+# component ROLE directory (like hive-metastore or hive-server2-hive2)
+component_directory = status_params.component_directory
+component_directory_interactive = status_params.component_directory_interactive
+
+hadoop_home = format('{stack_root}/current/hadoop-client')
+hive_bin = format('{stack_root}/current/{component_directory}/bin')
+hive_schematool_ver_bin = format('{stack_root}/{version}/hive/bin')
+hive_schematool_bin = format('{stack_root}/current/{component_directory}/bin')
+hive_lib = format('{stack_root}/current/{component_directory}/lib')
+hive_version_lib = format('{stack_root}/{version}/hive/lib')
+hive_var_lib = '/var/lib/hive'
+hive_user_home_dir = "/home/hive"
+
+# starting on stacks where HSI is supported, we need to begin using the 'hive2' schematool
+hive_server2_hive2_dir = None
+hive_server2_hive2_lib = None
+
+version = default("/commandParams/version", None)
+
+if check_stack_feature(StackFeature.HIVE_SERVER_INTERACTIVE, version_for_stack_feature_checks):
+  # the name of the hiveserver2-hive2 component
+  hive_server2_hive2_component = status_params.SERVER_ROLE_DIRECTORY_MAP["HIVE_SERVER_INTERACTIVE"]
+
+  # when using the version, we can just specify the component as "hive2"
+  hive_schematool_ver_bin = format('{stack_root}/{version}/hive2/bin')
+
+  # use the schematool which ships with hive2
+  hive_schematool_bin = format('{stack_root}/current/{hive_server2_hive2_component}/bin')
+
+  # <stack-root>/<version>/hive2 (as opposed to <stack-root>/<version>/hive)
+  hive_server2_hive2_dir = format('{stack_root}/current/{hive_server2_hive2_component}')
+
+  # <stack-root>/<version>/hive2 (as opposed to <stack-root>/<version>/hive)
+  hive_server2_hive2_version_dir = format('{stack_root}/{version}/hive2')
+
+  # <stack-root>/current/hive-server2-hive2/lib -> <stack-root>/<version>/hive2/lib
+  hive_server2_hive2_lib = format('{hive_server2_hive2_dir}/lib')
+
+  # <stack-root>/<version>/hive2/lib
+  hive_server2_hive2_version_lib = format('{hive_server2_hive2_version_dir}/lib')
+
+
+hive_interactive_bin = format('{stack_root}/current/{component_directory_interactive}/bin')
+hive_interactive_lib = format('{stack_root}/current/{component_directory_interactive}/lib')
+
+# Hive Interactive related paths
+hive_interactive_var_lib = '/var/lib/hive2'
+
+# These tar folders were used in previous stack versions, e.g., HDP 2.1
+hadoop_streaming_jars = '/usr/lib/hadoop-mapreduce/hadoop-streaming-*.jar'
+pig_tar_file = format('/usr/share/{stack_name_uppercase}-webhcat/pig.tar.gz')
+hive_tar_file = format('/usr/share/{stack_name_uppercase}-webhcat/hive.tar.gz')
+sqoop_tar_file = format('/usr/share/{stack_name_uppercase}-webhcat/sqoop*.tar.gz')
+
+hive_metastore_site_supported = False
+hive_etc_dir_prefix = "/etc/hive"
+hive_interactive_etc_dir_prefix = "/etc/hive2"
+limits_conf_dir = "/etc/security/limits.d"
+
+hive_user_nofile_limit = default("/configurations/hive-env/hive_user_nofile_limit", "32000")
+hive_user_nproc_limit = default("/configurations/hive-env/hive_user_nproc_limit", "16000")
+
+# use the directories from status_params as they are already calculated for
+# the correct stack version
+hadoop_conf_dir = status_params.hadoop_conf_dir
+hadoop_bin_dir = status_params.hadoop_bin_dir
+webhcat_conf_dir = status_params.webhcat_conf_dir
+hive_conf_dir = status_params.hive_conf_dir
+hive_home_dir = status_params.hive_home_dir
+hive_config_dir = status_params.hive_config_dir
+hive_client_conf_dir = status_params.hive_client_conf_dir
+hive_server_conf_dir = status_params.hive_server_conf_dir
+
+hcat_conf_dir = '/etc/hive-hcatalog/conf'
+config_dir = '/etc/hive-webhcat/conf'
+hcat_lib = '/usr/lib/hive-hcatalog/share/hcatalog'
+webhcat_bin_dir = '/usr/lib/hive-hcatalog/sbin'
+
+# there are no client versions of these, use server versions directly
+hcat_lib = format('{stack_root}/current/hive-webhcat/share/hcatalog')
+webhcat_bin_dir = format('{stack_root}/current/hive-webhcat/sbin')
+
+# --- Tarballs ---
+# DON'T CHANGE THESE VARIABLE NAMES
+# Values don't change from those in copy_tarball.py
+webhcat_apps_dir = "/apps/webhcat"
+hive_tar_source = "{0}/{1}/hive/hive.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN)
+pig_tar_source = "{0}/{1}/pig/pig.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN)
+hive_tar_dest_file = "/{0}/apps/{1}/hive/hive.tar.gz".format(STACK_NAME_PATTERN,STACK_VERSION_PATTERN)
+pig_tar_dest_file = "/{0}/apps/{1}/pig/pig.tar.gz".format(STACK_NAME_PATTERN, STACK_VERSION_PATTERN)
+
+hadoop_streaming_tar_source = "{0}/{1}/hadoop-mapreduce/hadoop-streaming.jar".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN)
+sqoop_tar_source = "{0}/{1}/sqoop/sqoop.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN)
+hadoop_streaming_tar_dest_dir = "/{0}/apps/{1}/mapreduce/".format(STACK_NAME_PATTERN,STACK_VERSION_PATTERN)
+sqoop_tar_dest_dir = "/{0}/apps/{1}/sqoop/".format(STACK_NAME_PATTERN, STACK_VERSION_PATTERN)
+
+tarballs_mode = 0444
+
+purge_tables = "false"
+# Starting from stack version for feature hive_purge_table drop should be executed with purge
+if check_stack_feature(StackFeature.HIVE_PURGE_TABLE, version_for_stack_feature_checks):
+  purge_tables = 'true'
+
+if check_stack_feature(StackFeature.HIVE_WEBHCAT_SPECIFIC_CONFIGS, version_for_stack_feature_checks):
+  # this is NOT a typo.  Configs for hcatalog/webhcat point to a
+  # specific directory which is NOT called 'conf'
+  hcat_conf_dir = format('{stack_root}/current/hive-webhcat/etc/hcatalog')
+  config_dir = format('{stack_root}/current/hive-webhcat/etc/webhcat')
+
+if check_stack_feature(StackFeature.HIVE_METASTORE_SITE_SUPPORT, version_for_stack_feature_checks):
+  hive_metastore_site_supported = True
+
+execute_path = os.environ['PATH'] + os.pathsep + hive_bin + os.pathsep + hadoop_bin_dir
+
+hive_metastore_user_name = config['configurations']['hive-site']['javax.jdo.option.ConnectionUserName']
+hive_jdbc_connection_url = config['configurations']['hive-site']['javax.jdo.option.ConnectionURL']
+
+hive_metastore_user_passwd = config['configurations']['hive-site']['javax.jdo.option.ConnectionPassword']
+hive_metastore_user_passwd = unicode(hive_metastore_user_passwd) if not is_empty(hive_metastore_user_passwd) else hive_metastore_user_passwd
+hive_metastore_db_type = config['configurations']['hive-env']['hive_database_type']
+
+#HACK Temporarily use dbType=azuredb while invoking schematool
+if hive_metastore_db_type == "mssql":
+  hive_metastore_db_type = "azuredb"
+
+#users
+hive_user = config['configurations']['hive-env']['hive_user']
+
+#JDBC driver jar name
+hive_jdbc_driver = config['configurations']['hive-site']['javax.jdo.option.ConnectionDriverName']
+jdk_location = config['hostLevelParams']['jdk_location']
+java_share_dir = '/usr/share/java'
+hive_database_name = config['configurations']['hive-env']['hive_database_name']
+hive_database = config['configurations']['hive-env']['hive_database']
+hive_use_existing_db = hive_database.startswith('Existing')
+
+default_connectors_map = { "com.microsoft.sqlserver.jdbc.SQLServerDriver":"sqljdbc4.jar",
+                           "com.mysql.jdbc.Driver":"mysql-connector-java.jar",
+                           "org.postgresql.Driver":"postgresql-jdbc.jar",
+                           "oracle.jdbc.driver.OracleDriver":"ojdbc.jar",
+                           "sap.jdbc4.sqlanywhere.IDriver":"sajdbc4.jar"}
+
+# NOT SURE THAT IT'S A GOOD IDEA TO USE PATH TO CLASS IN DRIVER, MAYBE IT WILL BE BETTER TO USE DB TYPE.
+# BECAUSE PATH TO CLASSES COULD BE CHANGED
+sqla_db_used = False
+hive_previous_jdbc_jar_name = None
+if hive_jdbc_driver == "com.microsoft.sqlserver.jdbc.SQLServerDriver":
+  jdbc_jar_name = default("/hostLevelParams/custom_mssql_jdbc_name", None)
+  hive_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_mssql_jdbc_name", None)
+elif hive_jdbc_driver == "com.mysql.jdbc.Driver":
+  jdbc_jar_name = default("/hostLevelParams/custom_mysql_jdbc_name", None)
+  hive_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_mysql_jdbc_name", None)
+elif hive_jdbc_driver == "org.postgresql.Driver":
+  jdbc_jar_name = default("/hostLevelParams/custom_postgres_jdbc_name", None)
+  hive_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_postgres_jdbc_name", None)
+elif hive_jdbc_driver == "oracle.jdbc.driver.OracleDriver":
+  jdbc_jar_name = default("/hostLevelParams/custom_oracle_jdbc_name", None)
+  hive_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_oracle_jdbc_name", None)
+elif hive_jdbc_driver == "sap.jdbc4.sqlanywhere.IDriver":
+  jdbc_jar_name = default("/hostLevelParams/custom_sqlanywhere_jdbc_name", None)
+  hive_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_sqlanywhere_jdbc_name", None)
+  sqla_db_used = True
+
+default_mysql_jar_name = "mysql-connector-java.jar"
+default_mysql_target = format("{hive_lib}/{default_mysql_jar_name}")
+hive_previous_jdbc_jar = format("{hive_lib}/{hive_previous_jdbc_jar_name}")
+if not hive_use_existing_db:
+  jdbc_jar_name = default_mysql_jar_name
+
+
+downloaded_custom_connector = format("{tmp_dir}/{jdbc_jar_name}")
+
+hive_jdbc_target = format("{hive_lib}/{jdbc_jar_name}")
+hive2_jdbc_target = None
+if hive_server2_hive2_dir:
+  hive2_jdbc_target = format("{hive_server2_hive2_lib}/{jdbc_jar_name}")
+
+# during upgrade / downgrade, use the specific version to copy the JDBC JAR to
+if upgrade_direction:
+  hive_jdbc_target = format("{hive_version_lib}/{jdbc_jar_name}")
+  hive2_jdbc_target = format("{hive_server2_hive2_version_lib}/{jdbc_jar_name}") if hive2_jdbc_target is not None else None
+
+
+hive2_previous_jdbc_jar = format("{hive_server2_hive2_lib}/{hive_previous_jdbc_jar_name}") if hive_server2_hive2_lib is not None else None
+driver_curl_source = format("{jdk_location}/{jdbc_jar_name}")
+
+# normally, the JDBC driver would be referenced by <stack-root>/current/.../foo.jar
+# but in RU if <stack-selector-tool> is called and the restart fails, then this means that current pointer
+# is now pointing to the upgraded version location; that's bad for the cp command
+source_jdbc_file = format("{stack_root}/{current_version}/hive/lib/{jdbc_jar_name}")
+
+check_db_connection_jar_name = "DBConnectionVerification.jar"
+check_db_connection_jar = format("/usr/lib/ambari-agent/{check_db_connection_jar_name}")
+hive_jdbc_drivers_list = ["com.microsoft.sqlserver.jdbc.SQLServerDriver","com.mysql.jdbc.Driver",
+                          "org.postgresql.Driver","oracle.jdbc.driver.OracleDriver","sap.jdbc4.sqlanywhere.IDriver"]
+
+prepackaged_jdbc_name = "ojdbc6.jar"
+prepackaged_ojdbc_symlink = format("{hive_lib}/{prepackaged_jdbc_name}")
+templeton_port = config['configurations']['webhcat-site']['templeton.port']
+
+#constants for type2 jdbc
+jdbc_libs_dir = format("{hive_lib}/native/lib64")
+lib_dir_available = os.path.exists(jdbc_libs_dir)
+
+if sqla_db_used:
+  jars_path_in_archive = format("{tmp_dir}/sqla-client-jdbc/java/*")
+  libs_path_in_archive = format("{tmp_dir}/sqla-client-jdbc/native/lib64/*")
+  downloaded_custom_connector = format("{tmp_dir}/{jdbc_jar_name}")
+  libs_in_hive_lib = format("{jdbc_libs_dir}/*")
+
+
+# Start, Common Hosts and Ports
+ambari_server_hostname = config['clusterHostInfo']['ambari_server_host'][0]
+
+hive_metastore_hosts = default('/clusterHostInfo/hive_metastore_host', [])
+hive_metastore_host = hive_metastore_hosts[0] if len(hive_metastore_hosts) > 0 else None
+hive_metastore_port = get_port_from_url(config['configurations']['hive-site']['hive.metastore.uris'])
+
+hive_server_hosts = default("/clusterHostInfo/hive_server_host", [])
+hive_server_host = hive_server_hosts[0] if len(hive_server_hosts) > 0 else None
+
+hive_server_interactive_hosts = default('/clusterHostInfo/hive_server_interactive_hosts', [])
+hive_server_interactive_host = hive_server_interactive_hosts[0] if len(hive_server_interactive_hosts) > 0 else None
+# End, Common Hosts and Ports
+
+hive_transport_mode = config['configurations']['hive-site']['hive.server2.transport.mode']
+
+if hive_transport_mode.lower() == "http":
+  hive_server_port = config['configurations']['hive-site']['hive.server2.thrift.http.port']
+else:
+  hive_server_port = default('/configurations/hive-site/hive.server2.thrift.port',"10000")
+
+hive_url = format("jdbc:hive2://{hive_server_host}:{hive_server_port}")
+hive_http_endpoint = default('/configurations/hive-site/hive.server2.thrift.http.path', "cliservice")
+hive_server_principal = config['configurations']['hive-site']['hive.server2.authentication.kerberos.principal']
+hive_server2_authentication = config['configurations']['hive-site']['hive.server2.authentication']
+
+# ssl options
+hive_ssl = default('/configurations/hive-site/hive.server2.use.SSL', False)
+hive_ssl_keystore_path = default('/configurations/hive-site/hive.server2.keystore.path', None)
+hive_ssl_keystore_password = default('/configurations/hive-site/hive.server2.keystore.password', None)
+
+smokeuser = config['configurations']['cluster-env']['smokeuser']
+smoke_test_sql = format("{tmp_dir}/hiveserver2.sql")
+smoke_test_path = format("{tmp_dir}/hiveserver2Smoke.sh")
+smoke_user_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
+smokeuser_principal = config['configurations']['cluster-env']['smokeuser_principal_name']
+
+fs_root = config['configurations']['core-site']['fs.defaultFS']
+security_enabled = config['configurations']['cluster-env']['security_enabled']
+
+kinit_path_local = get_kinit_path(default('/configurations/kerberos-env/executable_search_paths', None))
+hive_metastore_keytab_path = config['configurations']['hive-site']['hive.metastore.kerberos.keytab.file']
+hive_metastore_principal = config['configurations']['hive-site']['hive.metastore.kerberos.principal']
+
+hive_server2_keytab = config['configurations']['hive-site']['hive.server2.authentication.kerberos.keytab']
+
+#hive_env
+hive_log_dir = config['configurations']['hive-env']['hive_log_dir']
+hive_pid_dir = status_params.hive_pid_dir
+hive_pid = status_params.hive_pid
+hive_interactive_pid = status_params.hive_interactive_pid
+
+#Default conf dir for client
+hive_conf_dirs_list = [hive_client_conf_dir]
+
+# These are the folders to which the configs will be written to.
+ranger_hive_component = status_params.SERVER_ROLE_DIRECTORY_MAP['HIVE_SERVER']
+if status_params.role == "HIVE_METASTORE" and hive_metastore_hosts is not None and hostname in hive_metastore_hosts:
+  hive_conf_dirs_list.append(hive_server_conf_dir)
+elif status_params.role == "HIVE_SERVER" and hive_server_hosts is not None and hostname in hive_server_host:
+  hive_conf_dirs_list.append(hive_server_conf_dir)
+elif status_params.role == "HIVE_SERVER_INTERACTIVE" and hive_server_interactive_hosts is not None and hostname in hive_server_interactive_hosts:
+  hive_conf_dirs_list.append(status_params.hive_server_interactive_conf_dir)
+  ranger_hive_component = status_params.SERVER_ROLE_DIRECTORY_MAP['HIVE_SERVER_INTERACTIVE']
+# log4j version is 2 for hive2; put config files under /etc/hive2/conf
+if status_params.role == "HIVE_SERVER_INTERACTIVE":
+  log4j_version = '2'
+
+#Starting hiveserver2
+start_hiveserver2_script = 'startHiveserver2.sh.j2'
+
+##Starting metastore
+start_metastore_script = 'startMetastore.sh'
+hive_metastore_pid = status_params.hive_metastore_pid
+
+# Hive Server Interactive
+slider_am_container_mb = default("/configurations/hive-interactive-env/slider_am_container_mb", 341)
+
+hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
+yarn_user = config['configurations']['yarn-env']['yarn_user']
+user_group = config['configurations']['cluster-env']['user_group']
+artifact_dir = format("{tmp_dir}/AMBARI-artifacts/")
+# Need this for yarn.nodemanager.recovery.dir in yarn-site
+yarn_log_dir_prefix = config['configurations']['yarn-env']['yarn_log_dir_prefix']
+
+target_hive_interactive = format("{hive_interactive_lib}/{jdbc_jar_name}")
+hive_intaractive_previous_jdbc_jar = format("{hive_interactive_lib}/{hive_previous_jdbc_jar_name}")
+jars_in_hive_lib = format("{hive_lib}/*.jar")
+
+start_hiveserver2_path = format("{tmp_dir}/start_hiveserver2_script")
+start_metastore_path = format("{tmp_dir}/start_metastore_script")
+
+hadoop_heapsize = config['configurations']['hadoop-env']['hadoop_heapsize']
+
+if 'role' in config and config['role'] in ["HIVE_SERVER", "HIVE_METASTORE"]:
+  if check_stack_feature(StackFeature.HIVE_ENV_HEAPSIZE, version_for_stack_feature_checks):
+    hive_heapsize = config['configurations']['hive-env']['hive.heapsize']
+  else:
+    hive_heapsize = config['configurations']['hive-site']['hive.heapsize']
+else:
+  hive_heapsize = config['configurations']['hive-env']['hive.client.heapsize']
+
+hive_metastore_heapsize = config['configurations']['hive-env']['hive.metastore.heapsize']
+
+java64_home = config['hostLevelParams']['java_home']
+java_version = expect("/hostLevelParams/java_version", int)
+
+##### MYSQL
+db_name = config['configurations']['hive-env']['hive_database_name']
+mysql_group = 'mysql'
+mysql_host = config['clusterHostInfo']['hive_mysql_host']
+
+mysql_adduser_path = format("{tmp_dir}/addMysqlUser.sh")
+mysql_deluser_path = format("{tmp_dir}/removeMysqlUser.sh")
+
+#### Metastore
+# initialize the schema only if not in an upgrade/downgrade
+init_metastore_schema = upgrade_direction is None
+
+########## HCAT
+hcat_dbroot = hcat_lib
+
+hcat_user = config['configurations']['hive-env']['hcat_user']
+webhcat_user = config['configurations']['hive-env']['webhcat_user']
+
+hcat_pid_dir = status_params.hcat_pid_dir
+hcat_log_dir = config['configurations']['hive-env']['hcat_log_dir']
+hcat_env_sh_template = config['configurations']['hcat-env']['content']
+
+#hive-log4j.properties.template
+if (('hive-log4j' in config['configurations']) and ('content' in config['configurations']['hive-log4j'])):
+  log4j_props = config['configurations']['hive-log4j']['content']
+else:
+  log4j_props = None
+
+#webhcat-log4j.properties.template
+if (('webhcat-log4j' in config['configurations']) and ('content' in config['configurations']['webhcat-log4j'])):
+  log4j_webhcat_props = config['configurations']['webhcat-log4j']['content']
+else:
+  log4j_webhcat_props = None
+
+#hive-exec-log4j.properties.template
+if (('hive-exec-log4j' in config['configurations']) and ('content' in config['configurations']['hive-exec-log4j'])):
+  log4j_exec_props = config['configurations']['hive-exec-log4j']['content']
+else:
+  log4j_exec_props = None
+
+daemon_name = status_params.daemon_name
+process_name = status_params.process_name
+hive_env_sh_template = config['configurations']['hive-env']['content']
+
+hive_hdfs_user_dir = format("/user/{hive_user}")
+hive_hdfs_user_mode = 0755
+hive_apps_whs_dir = config['configurations']['hive-site']["hive.metastore.warehouse.dir"]
+whs_dir_protocol = urlparse(hive_apps_whs_dir).scheme
+hive_exec_scratchdir = config['configurations']['hive-site']["hive.exec.scratchdir"]
+#for create_hdfs_directory
+hdfs_user_keytab = config['configurations']['hadoop-env']['hdfs_user_keytab']
+hdfs_principal_name = default('/configurations/hadoop-env/hdfs_principal_name', 'missing_principal').replace("_HOST", hostname)
+
+# Tez-related properties
+tez_user = config['configurations']['tez-env']['tez_user']
+
+# Tez jars
+tez_local_api_jars = '/usr/lib/tez/tez*.jar'
+tez_local_lib_jars = '/usr/lib/tez/lib/*.jar'
+
+# Tez libraries
+tez_lib_uris = default("/configurations/tez-site/tez.lib.uris", None)
+
+if OSCheck.is_ubuntu_family():
+  mysql_configname = '/etc/mysql/my.cnf'
+else:
+  mysql_configname = '/etc/my.cnf'
+
+mysql_user = 'mysql'
+
+# Hive security
+hive_authorization_enabled = config['configurations']['hive-site']['hive.security.authorization.enabled']
+
+mysql_jdbc_driver_jar = "/usr/share/java/mysql-connector-java.jar"
+
+hive_site_config = dict(config['configurations']['hive-site'])
+
+########################################################
+############# AMS related params #####################
+########################################################
+ams_collector_hosts = default("/clusterHostInfo/metrics_collector_hosts", [])
+has_metric_collector = not len(ams_collector_hosts) == 0
+if has_metric_collector:
+  if 'cluster-env' in config['configurations'] and \
+      'metrics_collector_vip_host' in config['configurations']['cluster-env']:
+    metric_collector_host = config['configurations']['cluster-env']['metrics_collector_vip_host']
+  else:
+    metric_collector_host = ams_collector_hosts[0]
+  if 'cluster-env' in config['configurations'] and \
+      'metrics_collector_vip_port' in config['configurations']['cluster-env']:
+    metric_collector_port = config['configurations']['cluster-env']['metrics_collector_vip_port']
+  else:
+    metric_collector_web_address = default("/configurations/ams-site/timeline.metrics.service.webapp.address", "localhost:6188")
+    if metric_collector_web_address.find(':') != -1:
+      metric_collector_port = metric_collector_web_address.split(':')[1]
+    else:
+      metric_collector_port = '6188'
+  if default("/configurations/ams-site/timeline.metrics.service.http.policy", "HTTP_ONLY") == "HTTPS_ONLY":
+    metric_collector_protocol = 'https'
+  else:
+    metric_collector_protocol = 'http'
+  metric_truststore_path= default("/configurations/ams-ssl-client/ssl.client.truststore.location", "")
+  metric_truststore_type= default("/configurations/ams-ssl-client/ssl.client.truststore.type", "")
+  metric_truststore_password= default("/configurations/ams-ssl-client/ssl.client.truststore.password", "")
+
+metrics_report_interval = default("/configurations/ams-site/timeline.metrics.sink.report.interval", 60)
+metrics_collection_period = default("/configurations/ams-site/timeline.metrics.sink.collection.period", 10)
+
+########################################################
+############# Atlas related params #####################
+########################################################
+#region Atlas Hooks
+hive_atlas_application_properties = default('/configurations/hive-atlas-application.properties', {})
+
+if has_atlas_in_cluster():
+  atlas_hook_filename = default('/configurations/atlas-env/metadata_conf_file', 'atlas-application.properties')
+#endregion
+
+########################################################
+########### WebHCat related params #####################
+########################################################
+
+webhcat_env_sh_template = config['configurations']['webhcat-env']['content']
+templeton_log_dir = config['configurations']['hive-env']['hcat_log_dir']
+templeton_pid_dir = status_params.hcat_pid_dir
+
+webhcat_pid_file = status_params.webhcat_pid_file
+
+templeton_jar = config['configurations']['webhcat-site']['templeton.jar']
+
+
+webhcat_server_host = config['clusterHostInfo']['webhcat_server_host']
+
+hcat_hdfs_user_dir = format("/user/{hcat_user}")
+hcat_hdfs_user_mode = 0755
+webhcat_hdfs_user_dir = format("/user/{webhcat_user}")
+webhcat_hdfs_user_mode = 0755
+#for create_hdfs_directory
+security_param = "true" if security_enabled else "false"
+
+
+
+hdfs_site = config['configurations']['hdfs-site']
+default_fs = config['configurations']['core-site']['fs.defaultFS']
+
+dfs_type = default("/commandParams/dfs_type", "")
+
+import functools
+#create partial functions with common arguments for every HdfsResource call
+#to create hdfs directory we need to call params.HdfsResource in code
+HdfsResource = functools.partial(
+ HdfsResource,
+  user = hdfs_user,
+  hdfs_resource_ignore_file = "/var/lib/ambari-agent/data/.hdfs_resource_ignore",
+  security_enabled = security_enabled,
+  keytab = hdfs_user_keytab,
+  kinit_path_local = kinit_path_local,
+  hadoop_bin_dir = hadoop_bin_dir,
+  hadoop_conf_dir = hadoop_conf_dir,
+  principal_name = hdfs_principal_name,
+  hdfs_site = hdfs_site,
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources(),
+  dfs_type = dfs_type
+ )
+
+# Hive Interactive related
+hive_interactive_hosts = default('/clusterHostInfo/hive_server_interactive_hosts', [])
+has_hive_interactive = len(hive_interactive_hosts) > 0
+if has_hive_interactive:
+  llap_daemon_log4j = config['configurations']['llap-daemon-log4j']['content']
+  llap_cli_log4j2 = config['configurations']['llap-cli-log4j2']['content']
+  hive_log4j2 = config['configurations']['hive-log4j2']['content']
+  hive_exec_log4j2 = config['configurations']['hive-exec-log4j2']['content']
+  beeline_log4j2 = config['configurations']['beeline-log4j2']['content']
+
+  hive_server_interactive_conf_dir = status_params.hive_server_interactive_conf_dir
+  execute_path_hive_interactive = os.path.join(os.environ['PATH'], hive_interactive_bin, hadoop_bin_dir)
+  start_hiveserver2_interactive_script = 'startHiveserver2Interactive.sh.j2'
+  start_hiveserver2_interactive_path = format("{tmp_dir}/start_hiveserver2_interactive_script")
+  hive_interactive_env_sh_template = config['configurations']['hive-interactive-env']['content']
+  hive_interactive_enabled = default('/configurations/hive-interactive-env/enable_hive_interactive', False)
+  llap_app_java_opts = default('/configurations/hive-interactive-env/llap_java_opts', '-XX:+AlwaysPreTouch {% if java_version > 7 %}-XX:+UseG1GC -XX:TLABSize=8m -XX:+ResizeTLAB -XX:+UseNUMA -XX:+AggressiveOpts -XX:MetaspaceSize=1024m -XX:InitiatingHeapOccupancyPercent=80 -XX:MaxGCPauseMillis=200{% else %}-XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC{% endif %}')
+
+  # Service check related
+  if hive_transport_mode.lower() == "http":
+    hive_server_interactive_port = config['configurations']['hive-interactive-site']['hive.server2.thrift.http.port']
+  else:
+    hive_server_interactive_port = default('/configurations/hive-interactive-site/hive.server2.thrift.port',"10500")
+  # Tez for Hive interactive related
+  tez_interactive_config_dir = "/etc/tez_hive2/conf"
+  tez_interactive_user = config['configurations']['tez-env']['tez_user']
+  num_retries_for_checking_llap_status = default('/configurations/hive-interactive-env/num_retries_for_checking_llap_status', 10)
+  # Used in LLAP slider package creation
+  num_llap_nodes = config['configurations']['hive-interactive-env']['num_llap_nodes']
+  llap_daemon_container_size = config['configurations']['hive-interactive-site']['hive.llap.daemon.yarn.container.mb']
+  llap_log_level = config['configurations']['hive-interactive-env']['llap_log_level']
+  hive_llap_io_mem_size = config['configurations']['hive-interactive-site']['hive.llap.io.memory.size']
+  llap_heap_size = config['configurations']['hive-interactive-env']['llap_heap_size']
+  llap_app_name = config['configurations']['hive-interactive-env']['llap_app_name']
+  hive_llap_principal = None
+  if security_enabled:
+    hive_llap_keytab_file = config['configurations']['hive-interactive-site']['hive.llap.zk.sm.keytab.file']
+    hive_llap_principal = (config['configurations']['hive-interactive-site']['hive.llap.zk.sm.principal']).replace('_HOST',hostname.lower())
+  pass
+
+# ranger host
+ranger_admin_hosts = default("/clusterHostInfo/ranger_admin_hosts", [])
+has_ranger_admin = not len(ranger_admin_hosts) == 0
+xml_configurations_supported = config['configurations']['ranger-env']['xml_configurations_supported']
+
+#ranger hive properties
+policymgr_mgr_url = config['configurations']['admin-properties']['policymgr_external_url']
+if 'admin-properties' in config['configurations'] and 'policymgr_external_url' in config['configurations']['admin-properties'] and policymgr_mgr_url.endswith('/'):
+  policymgr_mgr_url = policymgr_mgr_url.rstrip('/')
+xa_audit_db_name = default('/configurations/admin-properties/audit_db_name', 'ranger_audits')
+xa_audit_db_user = default('/configurations/admin-properties/audit_db_user', 'rangerlogger')
+xa_db_host = config['configurations']['admin-properties']['db_host']
+repo_name = str(config['clusterName']) + '_hive'
+
+jdbc_driver_class_name = config['configurations']['ranger-hive-plugin-properties']['jdbc.driverClassName']
+common_name_for_certificate = config['configurations']['ranger-hive-plugin-properties']['common.name.for.certificate']
+
+repo_config_username = config['configurations']['ranger-hive-plugin-properties']['REPOSITORY_CONFIG_USERNAME']
+
+ranger_env = config['configurations']['ranger-env']
+ranger_plugin_properties = config['configurations']['ranger-hive-plugin-properties']
+policy_user = config['configurations']['ranger-hive-plugin-properties']['policy_user']
+
+if security_enabled:
+  hive_principal = hive_server_principal.replace('_HOST',hostname.lower())
+  hive_keytab = config['configurations']['hive-site']['hive.server2.authentication.kerberos.keytab']
+
+#For curl command in ranger plugin to get db connector
+if has_ranger_admin:
+  enable_ranger_hive = (config['configurations']['hive-env']['hive_security_authorization'].lower() == 'ranger')
+  repo_config_password = unicode(config['configurations']['ranger-hive-plugin-properties']['REPOSITORY_CONFIG_PASSWORD'])
+  xa_audit_db_flavor = (config['configurations']['admin-properties']['DB_FLAVOR']).lower()
+  ranger_previous_jdbc_jar_name = None
+
+  if stack_supports_ranger_audit_db:
+    if xa_audit_db_flavor and xa_audit_db_flavor == 'mysql':
+      ranger_jdbc_jar_name = default("/hostLevelParams/custom_mysql_jdbc_name", None)
+      ranger_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_mysql_jdbc_name", None)
+      audit_jdbc_url = format('jdbc:mysql://{xa_db_host}/{xa_audit_db_name}')
+      jdbc_driver = "com.mysql.jdbc.Driver"
+    elif xa_audit_db_flavor and xa_audit_db_flavor == 'oracle':
+      ranger_jdbc_jar_name = default("/hostLevelParams/custom_oracle_jdbc_name", None)
+      ranger_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_oracle_jdbc_name", None)
+      colon_count = xa_db_host.count(':')
+      if colon_count == 2 or colon_count == 0:
+        audit_jdbc_url = format('jdbc:oracle:thin:@{xa_db_host}')
+      else:
+        audit_jdbc_url = format('jdbc:oracle:thin:@//{xa_db_host}')
+      jdbc_driver = "oracle.jdbc.OracleDriver"
+    elif xa_audit_db_flavor and xa_audit_db_flavor == 'postgres':
+      ranger_jdbc_jar_name = default("/hostLevelParams/custom_postgres_jdbc_name", None)
+      ranger_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_postgres_jdbc_name", None)
+      audit_jdbc_url = format('jdbc:postgresql://{xa_db_host}/{xa_audit_db_name}')
+      jdbc_driver = "org.postgresql.Driver"
+    elif xa_audit_db_flavor and xa_audit_db_flavor == 'mssql':
+      ranger_jdbc_jar_name = default("/hostLevelParams/custom_mssql_jdbc_name", None)
+      ranger_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_mssql_jdbc_name", None)
+      audit_jdbc_url = format('jdbc:sqlserver://{xa_db_host};databaseName={xa_audit_db_name}')
+      jdbc_driver = "com.microsoft.sqlserver.jdbc.SQLServerDriver"
+    elif xa_audit_db_flavor and xa_audit_db_flavor == 'sqla':
+      ranger_jdbc_jar_name = default("/hostLevelParams/custom_sqlanywhere_jdbc_name", None)
+      ranger_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_sqlanywhere_jdbc_name", None)
+      audit_jdbc_url = format('jdbc:sqlanywhere:database={xa_audit_db_name};host={xa_db_host}')
+      jdbc_driver = "sap.jdbc4.sqlanywhere.IDriver"
+
+  ranger_downloaded_custom_connector = format("{tmp_dir}/{ranger_jdbc_jar_name}") if stack_supports_ranger_audit_db else None
+  ranger_driver_curl_source = format("{jdk_location}/{ranger_jdbc_jar_name}") if stack_supports_ranger_audit_db else None
+  ranger_driver_curl_target = format("{hive_lib}/{ranger_jdbc_jar_name}") if stack_supports_ranger_audit_db else None
+  ranger_previous_jdbc_jar = format("{hive_lib}/{ranger_previous_jdbc_jar_name}") if stack_supports_ranger_audit_db else None
+  sql_connector_jar = ''
+
+  hive_ranger_plugin_config = {
+    'username': repo_config_username,
+    'password': repo_config_password,
+    'jdbc.driverClassName': jdbc_driver_class_name,
+    'jdbc.url': format("{hive_url}/default;principal={hive_principal}") if security_enabled else hive_url,
+    'commonNameForCertificate': common_name_for_certificate
+  }
+
+  hive_ranger_plugin_repo = {
+    'isActive': 'true',
+    'config': json.dumps(hive_ranger_plugin_config),
+    'description': 'hive repo',
+    'name': repo_name,
+    'repositoryType': 'hive',
+    'assetType': '3'
+  }
+
+  if stack_supports_ranger_kerberos and security_enabled:
+    hive_ranger_plugin_config['policy.download.auth.users'] = hive_user
+    hive_ranger_plugin_config['tag.download.auth.users'] = hive_user
+    hive_ranger_plugin_config['policy.grantrevoke.auth.users'] = hive_user
+
+  if stack_supports_ranger_kerberos:
+    hive_ranger_plugin_config['ambari.service.check.user'] = policy_user
+
+    hive_ranger_plugin_repo = {
+      'isEnabled': 'true',
+      'configs': hive_ranger_plugin_config,
+      'description': 'hive repo',
+      'name': repo_name,
+      'type': 'hive'
+    }
+
+
+  xa_audit_db_is_enabled = False
+  xa_audit_db_password = ''
+  if not is_empty(config['configurations']['admin-properties']['audit_db_password']) and stack_supports_ranger_audit_db:
+    xa_audit_db_password = unicode(config['configurations']['admin-properties']['audit_db_password'])
+  ranger_audit_solr_urls = config['configurations']['ranger-admin-site']['ranger.audit.solr.urls']
+  if xml_configurations_supported and stack_supports_ranger_audit_db:
+    xa_audit_db_is_enabled = config['configurations']['ranger-hive-audit']['xasecure.audit.destination.db']
+  xa_audit_hdfs_is_enabled = config['configurations']['ranger-hive-audit']['xasecure.audit.destination.hdfs'] if xml_configurations_supported else None
+  ssl_keystore_password = unicode(config['configurations']['ranger-hive-policymgr-ssl']['xasecure.policymgr.clientssl.keystore.password']) if xml_configurations_supported else None
+  ssl_truststore_password = unicode(config['configurations']['ranger-hive-policymgr-ssl']['xasecure.policymgr.clientssl.truststore.password']) if xml_configurations_supported else None
+  credential_file = format('/etc/ranger/{repo_name}/cred.jceks') if xml_configurations_supported else None
+
+  #For SQLA explicitly disable audit to DB for Ranger
+  if xa_audit_db_flavor == 'sqla':
+    xa_audit_db_is_enabled = False
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_windows.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_windows.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_windows.py
new file mode 100755
index 0000000..880fdb5
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_windows.py
@@ -0,0 +1,74 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+from resource_management import *
+from status_params import *
+
+# server configurations
+config = Script.get_config()
+
+# This is expected to be of the form #.#.#.#
+stack_version_unformatted = config['hostLevelParams']['stack_version']
+stack_version_formatted = format_stack_version(stack_version_unformatted)
+
+stack_root = None
+hive_conf_dir = None
+hive_home = None
+hive_lib_dir = None
+hive_log_dir = None
+hive_opts = None
+hcat_home = None
+hcat_config_dir = None
+hive_bin = None
+
+try:
+  stack_root = os.path.abspath(os.path.join(os.environ["HADOOP_HOME"],".."))
+  hive_conf_dir = os.environ["HIVE_CONF_DIR"]
+  hive_home = os.environ["HIVE_HOME"]
+  hive_lib_dir = os.environ["HIVE_LIB_DIR"]
+  hive_log_dir = os.environ["HIVE_LOG_DIR"]
+  hive_opts = os.environ["HIVE_OPTS"]
+  hcat_home = os.environ["HCAT_HOME"]
+  hcat_config_dir = os.environ["WEBHCAT_CONF_DIR"]
+  hive_bin = os.path.join(hive_home, "bin")
+except:
+  pass
+
+hive_env_sh_template = config['configurations']['hive-env']['content']
+hive_warehouse_dir = config['configurations']['hive-site']['hive.metastore.warehouse.dir']
+hadoop_user = config["configurations"]["cluster-env"]["hadoop.user.name"]
+hive_user = hadoop_user
+hcat_user = hadoop_user
+
+hive_metastore_db_type = config['configurations']['hive-env']['hive_database_type']
+hive_metastore_user_name = config['configurations']['hive-site']['javax.jdo.option.ConnectionUserName']
+hive_metastore_user_passwd = config['configurations']['hive-site']['javax.jdo.option.ConnectionPassword']
+
+hive_execution_engine = config["configurations"]["hive-site"]["hive.execution.engine"]
+
+######## Metastore Schema
+init_metastore_schema = not config['configurations']['hive-site']['datanucleus.autoCreateSchema']
+
+service_map = {
+  "metastore" : hive_metastore_win_service_name,
+  "client" : hive_client_win_service_name,
+  "hiveserver2" : hive_server_win_service_name,
+  "templeton" : webhcat_server_win_service_name
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/service_check.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/service_check.py
new file mode 100755
index 0000000..1836d0f
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/service_check.py
@@ -0,0 +1,190 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+from resource_management import *
+import socket
+import sys
+import time
+import subprocess
+
+from hcat_service_check import hcat_service_check
+from webhcat_service_check import webhcat_service_check
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
+from resource_management.core import shell
+from resource_management.core.logger import Logger
+from resource_management.libraries.functions import get_unique_id_and_date
+
+class HiveServiceCheck(Script):
+  pass
+
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class HiveServiceCheckWindows(HiveServiceCheck):
+  def service_check(self, env):
+    import params
+    env.set_params(params)
+    smoke_cmd = os.path.join(params.stack_root,"Run-SmokeTests.cmd")
+    service = "HIVE"
+    Execute(format("cmd /C {smoke_cmd} {service}"), user=params.hive_user, logoutput=True)
+
+    hcat_service_check()
+    webhcat_service_check()
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class HiveServiceCheckDefault(HiveServiceCheck):
+
+  def __init__(self):
+    super(HiveServiceCheckDefault, self).__init__()
+    Logger.initialize_logger()
+
+  def service_check(self, env):
+    import params
+    env.set_params(params)
+
+    if params.security_enabled:
+      kinit_cmd = format(
+        "{kinit_path_local} -kt {smoke_user_keytab} {smokeuser_principal}; ")
+    else:
+      kinit_cmd = ""
+
+    # Check HiveServer
+    Logger.info("Running Hive Server checks")
+    Logger.info("--------------------------\n")
+    self.check_hive_server(env, 'Hive Server', kinit_cmd, params.hive_server_hosts,
+                           int(format("{hive_server_port}")))
+
+
+    if params.has_hive_interactive  and params.hive_interactive_enabled:
+      Logger.info("Running Hive Server2 checks")
+      Logger.info("--------------------------\n")
+
+      self.check_hive_server(env, 'Hive Server2', kinit_cmd, params.hive_interactive_hosts,
+                             int(format("{hive_server_interactive_port}")))
+
+      Logger.info("Running LLAP checks")
+      Logger.info("-------------------\n")
+      self.check_llap(env, kinit_cmd, params.hive_interactive_hosts, int(format("{hive_server_interactive_port}")),
+                      params.hive_llap_principal, params.hive_server2_authentication, params.hive_transport_mode,
+                      params.hive_http_endpoint)
+
+
+    Logger.info("Running HCAT checks")
+    Logger.info("-------------------\n")
+    hcat_service_check()
+
+    Logger.info("Running WEBHCAT checks")
+    Logger.info("---------------------\n")
+    webhcat_service_check()
+
+  def check_hive_server(self, env, server_component_name, kinit_cmd, address_list, server_port):
+    import params
+    env.set_params(params)
+    Logger.info("Server Address List : {0}, Port : {1}".format(address_list, server_port))
+
+    if not address_list:
+      raise Fail("Can not find any "+server_component_name+" ,host. Please check configuration.")
+
+    SOCKET_WAIT_SECONDS = 290
+
+    start_time = time.time()
+    end_time = start_time + SOCKET_WAIT_SECONDS
+
+    Logger.info("Waiting for the {0} to start...".format(server_component_name))
+
+    workable_server_available = False
+    i = 0
+    while time.time() < end_time and not workable_server_available:
+      address = address_list[i]
+      try:
+        check_thrift_port_sasl(address, server_port, params.hive_server2_authentication,
+                               params.hive_server_principal, kinit_cmd, params.smokeuser,
+                               transport_mode=params.hive_transport_mode, http_endpoint=params.hive_http_endpoint,
+                               ssl=params.hive_ssl, ssl_keystore=params.hive_ssl_keystore_path,
+                               ssl_password=params.hive_ssl_keystore_password)
+        Logger.info("Successfully connected to {0} on port {1}".format(address, server_port))
+        workable_server_available = True
+      except:
+        Logger.info("Connection to {0} on port {1} failed".format(address, server_port))
+        time.sleep(5)
+
+      i += 1
+      if i == len(address_list):
+        i = 0
+
+    elapsed_time = time.time() - start_time
+
+    if not workable_server_available:
+      raise Fail("Connection to '{0}' on host: {1} and port {2} failed after {3} seconds"
+                 .format(server_component_name, params.hostname, server_port, elapsed_time))
+
+    Logger.info("Successfully stayed connected to '{0}' on host: {1} and port {2} after {3} seconds"
+                .format(server_component_name, params.hostname, server_port, elapsed_time))
+
+  """
+  Performs Service check for LLAP app
+  """
+  def check_llap(self, env, kinit_cmd, address, port, key, hive_auth="NOSASL", transport_mode="binary", http_endpoint="cliservice"):
+    import params
+    env.set_params(params)
+
+    unique_id = get_unique_id_and_date()
+
+    beeline_url = ['jdbc:hive2://{address}:{port}/', "transportMode={transport_mode}"]
+
+    # Currently, HSI is supported on a single node only. The address list should be of size 1,
+    # thus picking the 1st node value.
+    address = address[0]
+
+    # append url according to used transport
+    if transport_mode == "http":
+      beeline_url.append('httpPath={http_endpoint}')
+
+    # append url according to used auth
+    if hive_auth == "NOSASL":
+      beeline_url.append('auth=noSasl')
+
+    # append url according to principal
+    if kinit_cmd:
+      beeline_url.append('principal={key}')
+
+    exec_path = params.execute_path
+    if params.version and params.stack_root:
+      upgrade_hive_bin = format("{stack_root}/{version}/hive2/bin")
+      exec_path =  os.environ['PATH'] + os.pathsep + params.hadoop_bin_dir + os.pathsep + upgrade_hive_bin
+
+    # beeline path
+    llap_cmd = "! beeline -u '%s'" % format(";".join(beeline_url))
+    # Append LLAP SQL script path
+    llap_cmd += format(" --hiveconf \"hiveLlapServiceCheck={unique_id}\" -f {stack_root}/current/hive-server2-hive2/scripts/llap/sql/serviceCheckScript.sql")
+    # Append grep patterns for detecting failure
+    llap_cmd += " -e '' 2>&1| awk '{print}'|grep -i -e 'Invalid status\|Invalid URL\|command not found\|Connection refused'"
+
+    Execute(llap_cmd,
+            user=params.hive_user,
+            path=['/usr/sbin', '/usr/local/bin', '/bin', '/usr/bin', exec_path],
+            tries=1,
+            wait_for_finish=True,
+            stderr=subprocess.PIPE,
+            logoutput=True)
+
+if __name__ == "__main__":
+  HiveServiceCheck().execute()
\ No newline at end of file


[34/50] [abbrv] bigtop git commit: Made changes to work with our reference implementation.

Posted by rv...@apache.org.
Made changes to work with our reference implementation.


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

Branch: refs/heads/BIGTOP-2666
Commit: 1792d1e8755b9937b721b3dd12987bd66844af98
Parents: ad84696
Author: Raj Desai <rd...@us.ibm.com>
Authored: Tue Nov 1 16:17:57 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:04 2017 -0800

----------------------------------------------------------------------
 .../org/odpi/specs/runtime/hive/TestCLI.java    | 88 ++++++++++----------
 1 file changed, 44 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/1792d1e8/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestCLI.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestCLI.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestCLI.java
index 18ee81d..2b70909 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestCLI.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestCLI.java
@@ -30,13 +30,13 @@ import org.junit.Assert;
 public class TestCLI {
 	
 	static Map<String, String> results;
+	static String db = "javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=odpi_metastore_db;create=true";
 	
 	@BeforeClass
 	public static void setup(){
 		
 		results = HiveHelper.execCommand(new CommandLine("which").addArgument("hive"));
 		Assert.assertEquals("Hive is not in the current path.", 0, Integer.parseInt(results.get("exitValue")));
-
 	}
 	
 	@Test
@@ -55,17 +55,17 @@ public class TestCLI {
 	@Test
 	public void sqlFromCmdLine(){
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("SHOW DATABASES command failed to execute.", 0, Integer.parseInt(results.get("exitValue")));
 		if(!results.get("outputStream").contains("odpi_runtime_hive")){
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 			Assert.assertEquals("Could not create database odpi_runtime_hive.", 0, Integer.parseInt(results.get("exitValue")));
 		}else{
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 			Assert.assertEquals("Could not create database odpi_runtime_hive.", 0, Integer.parseInt(results.get("exitValue")));
 		}
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 	}
 	
 	@Test
@@ -74,33 +74,33 @@ public class TestCLI {
 		try(PrintWriter out = new PrintWriter("hive-f2.sql")){ out.println("CREATE DATABASE odpi_runtime_hive;"); }
 		try(PrintWriter out = new PrintWriter("hive-f3.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); out.println("CREATE DATABASE odpi_runtime_hive;"); }
 		try(PrintWriter out = new PrintWriter("hive-f4.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); }
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f1.sql"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f1.sql").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("SHOW DATABASES command failed to execute.", 0, Integer.parseInt(results.get("exitValue")));
 		if(!results.get("outputStream").contains("odpi_runtime_hive")){
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f2.sql"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f2.sql").addArgument("--hiveconf").addArgument(db));
 			Assert.assertEquals("Could not create database odpi_runtime_hive.", 0, Integer.parseInt(results.get("exitValue")));
 		}else{
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f3.sql"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f3.sql").addArgument("--hiveconf").addArgument(db));
 			Assert.assertEquals("Could not create database odpi_runtime_hive.", 0, Integer.parseInt(results.get("exitValue")));
 		}
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f4.sql"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f4.sql").addArgument("--hiveconf").addArgument(db));
 	}
 	
 	@Test
 	public void silent() {
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("-S"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("-S").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("-S option did not work.", new Boolean(false), results.get("outputStream").contains("Time taken:"));
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--silent"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--silent").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("--silent option did not work.", new Boolean(false), results.get("outputStream").contains("Time taken:"));
 	}
 	
 	@Test
 	public void verbose(){
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("-v"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("-v").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("-v option did not work.", new Boolean(true), results.get("outputStream").contains("SHOW DATABASES"));
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--verbose"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--verbose").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("--verbose option did not work.", new Boolean(true), results.get("outputStream").contains("SHOW DATABASES"));		
 	}
 	
@@ -109,100 +109,100 @@ public class TestCLI {
 		try(PrintWriter out = new PrintWriter("hive-init1.sql")){ out.println("CREATE DATABASE odpi_runtime_hive;"); }
 		try(PrintWriter out = new PrintWriter("hive-init2.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); out.println("CREATE DATABASE odpi_runtime_hive;"); }
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("SHOW DATABASES command failed to execute.", 0, Integer.parseInt(results.get("exitValue")));
 		if(!results.get("outputStream").contains("odpi_runtime_hive")){
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-i").addArgument("hive-init1.sql").addArgument("-e").addArgument("SHOW DATABASES"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-i").addArgument("hive-init1.sql").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--hiveconf").addArgument(db));
 			Assert.assertEquals("Could not create database odpi_runtime_hive using the init -i option.", 0, Integer.parseInt(results.get("exitValue")));
 			Assert.assertEquals("Could not create database odpi_runtime_hive using the init -i option.", true, results.get("outputStream").contains("odpi_runtime_hive"));
 		}else{
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-i").addArgument("hive-init2.sql").addArgument("-e").addArgument("SHOW DATABASES"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-i").addArgument("hive-init2.sql").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--hiveconf").addArgument(db));
 			Assert.assertEquals("Could not create database odpi_runtime_hive.", 0, Integer.parseInt(results.get("exitValue")));
 			Assert.assertEquals("Could not create database odpi_runtime_hive using the init -i option.", true, results.get("outputStream").contains("odpi_runtime_hive"));
 		}
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 	}
 	
 	@Test
 	public void database(){
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--hiveconf").addArgument(db));
 		if(!results.get("outputStream").contains("odpi_runtime_hive")){
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 		}else{
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 		}
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive_1234").addArgument("-e").addArgument("CREATE TABLE odpi ( MYID INT );"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive_1234").addArgument("-e").addArgument("CREATE TABLE odpi ( MYID INT );").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("Non-existent database returned with wrong exit code: "+Integer.parseInt(results.get("exitValue")), 88, Integer.parseInt(results.get("exitValue")));
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive").addArgument("-e").addArgument("CREATE TABLE odpi ( MYID INT );"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive").addArgument("-e").addArgument("CREATE TABLE odpi ( MYID INT );").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("Failed to create table using --database argument.", 0, Integer.parseInt(results.get("exitValue")));
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive").addArgument("-e").addArgument("DESCRIBE odpi"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive").addArgument("-e").addArgument("DESCRIBE odpi").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("Failed to get expected column after creating odpi table using --database argument.", true, results.get("outputStream").contains("myid"));
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive").addArgument("-e").addArgument("DROP TABLE odpi"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive").addArgument("-e").addArgument("DROP TABLE odpi").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("Failed to create table using --database argument.", 0, Integer.parseInt(results.get("exitValue")));
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 	}
 	
 	@Test
 	public void hiveConf(){
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--hiveconf").addArgument("hive.root.logger=INFO,console").addArgument("-e").addArgument("SHOW DATABASES"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--hiveconf").addArgument("hive.root.logger=INFO,console").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("The --hiveconf option did not work in setting hive.root.logger=INFO,console.", true, results.get("outputStream").contains("INFO parse.ParseDriver: Parsing command: SHOW DATABASES"));
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-hiveconf").addArgument("hive.root.logger=INFO,console").addArgument("-e").addArgument("SHOW DATABASES"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-hiveconf").addArgument("hive.root.logger=INFO,console").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--hiveconf").addArgument(db));
 		Assert.assertEquals("The -hiveconf variant option did not work in setting hive.root.logger=INFO,console.", true, results.get("outputStream").contains("INFO parse.ParseDriver: Parsing command: SHOW DATABASES"));
 	}
 	
 	@Test
 	public void variableSubsitution() throws FileNotFoundException{
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--hiveconf").addArgument(db));
 		if(!results.get("outputStream").contains("odpi_runtime_hive")){
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 		}else{
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 		}
 		try(PrintWriter out = new PrintWriter("hive-define.sql")){ out.println("show ${A};"); out.println("quit;"); }
-		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive -d A=DATABASES < hive-define.sql", false));		
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive -d A=DATABASES --hiveconf '"+db+"' < hive-define.sql", false));		
 		Assert.assertEquals("The hive -d A=DATABASES option did not work.", 0, Integer.parseInt(results.get("exitValue")));
 		Assert.assertEquals("The hive -d A=DATABASES option did not work.", true, results.get("outputStream").contains("odpi_runtime_hive"));
 		
-		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive --define A=DATABASES < hive-define.sql", false));		
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive --define A=DATABASES --hiveconf '"+db+"' < hive-define.sql", false));		
 		Assert.assertEquals("The hive --define A=DATABASES option did not work.", 0, Integer.parseInt(results.get("exitValue")));
 		Assert.assertEquals("The hive --define A=DATABASES option did not work.", true, results.get("outputStream").contains("odpi_runtime_hive"));
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 	}
 	
 	@Test
 	public void hiveVar() throws FileNotFoundException{
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--hiveconf").addArgument(db));
 		if(!results.get("outputStream").contains("odpi_runtime_hive")){
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 		}else{
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
-			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 		}
 		try(PrintWriter out = new PrintWriter("hive-var.sql")){ out.println("show ${A};"); out.println("quit;"); }
-		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive --hivevar A=DATABASES < hive-var.sql", false));		
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive --hivevar A=DATABASES --hiveconf '"+db+"' < hive-var.sql", false));		
 		Assert.assertEquals("The hive --hivevar A=DATABASES option did not work.", 0, Integer.parseInt(results.get("exitValue")));
 		Assert.assertEquals("The hive --hivevar A=DATABASES option did not work.", true, results.get("outputStream").contains("odpi_runtime_hive"));
 		
 		try(PrintWriter out = new PrintWriter("hiveconf-var.sql")){ out.println("show ${hiveconf:A};"); out.println("quit;"); }
-		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive --hiveconf A=DATABASES < hiveconf-var.sql", false));		
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive --hiveconf A=DATABASES --hiveconf '"+db+"' < hiveconf-var.sql", false));		
 		Assert.assertEquals("The hive --hiveconf A=DATABASES option did not work.", 0, Integer.parseInt(results.get("exitValue")));
 		Assert.assertEquals("The hive --hiveconf A=DATABASES option did not work.", true, results.get("outputStream").contains("odpi_runtime_hive"));
 		
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 	}
 	
 	@AfterClass
 	public static void cleanup(){
-		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive").addArgument("--hiveconf").addArgument(db));
 		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf hive-f*.sql", false));
 		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf hive-init*.sql", false));
 		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf hive-define.sql", false));


[16/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.mysql.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.mysql.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.mysql.sql
new file mode 100755
index 0000000..b0415b1
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.mysql.sql
@@ -0,0 +1,777 @@
+-- MySQL dump 10.13  Distrib 5.5.25, for osx10.6 (i386)
+--
+-- Host: localhost    Database: test
+-- ------------------------------------------------------
+-- Server version	5.5.25
+
+/*!40101 SET @OLD_CHARACTER_SET_CLIENT=@@CHARACTER_SET_CLIENT */;
+/*!40101 SET @OLD_CHARACTER_SET_RESULTS=@@CHARACTER_SET_RESULTS */;
+/*!40101 SET @OLD_COLLATION_CONNECTION=@@COLLATION_CONNECTION */;
+/*!40101 SET NAMES utf8 */;
+/*!40103 SET @OLD_TIME_ZONE=@@TIME_ZONE */;
+/*!40103 SET TIME_ZONE='+00:00' */;
+/*!40014 SET @OLD_UNIQUE_CHECKS=@@UNIQUE_CHECKS, UNIQUE_CHECKS=0 */;
+/*!40014 SET @OLD_FOREIGN_KEY_CHECKS=@@FOREIGN_KEY_CHECKS, FOREIGN_KEY_CHECKS=0 */;
+/*!40101 SET @OLD_SQL_MODE=@@SQL_MODE, SQL_MODE='NO_AUTO_VALUE_ON_ZERO' */;
+/*!40111 SET @OLD_SQL_NOTES=@@SQL_NOTES, SQL_NOTES=0 */;
+
+--
+-- Table structure for table `BUCKETING_COLS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `BUCKETING_COLS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `BUCKET_COL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
+  KEY `BUCKETING_COLS_N49` (`SD_ID`),
+  CONSTRAINT `BUCKETING_COLS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `CDS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `CDS` (
+  `CD_ID` bigint(20) NOT NULL,
+  PRIMARY KEY (`CD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `COLUMNS_V2`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `COLUMNS_V2` (
+  `CD_ID` bigint(20) NOT NULL,
+  `COMMENT` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TYPE_NAME` varchar(4000) DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`CD_ID`,`COLUMN_NAME`),
+  KEY `COLUMNS_V2_N49` (`CD_ID`),
+  CONSTRAINT `COLUMNS_V2_FK1` FOREIGN KEY (`CD_ID`) REFERENCES `CDS` (`CD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `DATABASE_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` (
+  `DB_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(180) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`DB_ID`,`PARAM_KEY`),
+  KEY `DATABASE_PARAMS_N49` (`DB_ID`),
+  CONSTRAINT `DATABASE_PARAMS_FK1` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `DBS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `DBS` (
+  `DB_ID` bigint(20) NOT NULL,
+  `DESC` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `DB_LOCATION_URI` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`DB_ID`),
+  UNIQUE KEY `UNIQUE_DATABASE` (`NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `DB_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `DB_PRIVS` (
+  `DB_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `DB_ID` bigint(20) DEFAULT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `DB_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`DB_GRANT_ID`),
+  UNIQUE KEY `DBPRIVILEGEINDEX` (`DB_ID`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`DB_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `DB_PRIVS_N49` (`DB_ID`),
+  CONSTRAINT `DB_PRIVS_FK1` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `GLOBAL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `GLOBAL_PRIVS` (
+  `USER_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `USER_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`USER_GRANT_ID`),
+  UNIQUE KEY `GLOBALPRIVILEGEINDEX` (`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`USER_PRIV`,`GRANTOR`,`GRANTOR_TYPE`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `IDXS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `IDXS` (
+  `INDEX_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `DEFERRED_REBUILD` bit(1) NOT NULL,
+  `INDEX_HANDLER_CLASS` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INDEX_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INDEX_TBL_ID` bigint(20) DEFAULT NULL,
+  `LAST_ACCESS_TIME` int(11) NOT NULL,
+  `ORIG_TBL_ID` bigint(20) DEFAULT NULL,
+  `SD_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`INDEX_ID`),
+  UNIQUE KEY `UNIQUEINDEX` (`INDEX_NAME`,`ORIG_TBL_ID`),
+  KEY `IDXS_N51` (`SD_ID`),
+  KEY `IDXS_N50` (`INDEX_TBL_ID`),
+  KEY `IDXS_N49` (`ORIG_TBL_ID`),
+  CONSTRAINT `IDXS_FK1` FOREIGN KEY (`ORIG_TBL_ID`) REFERENCES `TBLS` (`TBL_ID`),
+  CONSTRAINT `IDXS_FK2` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`),
+  CONSTRAINT `IDXS_FK3` FOREIGN KEY (`INDEX_TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `INDEX_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `INDEX_PARAMS` (
+  `INDEX_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`),
+  KEY `INDEX_PARAMS_N49` (`INDEX_ID`),
+  CONSTRAINT `INDEX_PARAMS_FK1` FOREIGN KEY (`INDEX_ID`) REFERENCES `IDXS` (`INDEX_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `NUCLEUS_TABLES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `NUCLEUS_TABLES` (
+  `CLASS_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TABLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TYPE` varchar(4) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `OWNER` varchar(2) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `VERSION` varchar(20) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `INTERFACE_NAME` varchar(255) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`CLASS_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITIONS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITIONS` (
+  `PART_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `LAST_ACCESS_TIME` int(11) NOT NULL,
+  `PART_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SD_ID` bigint(20) DEFAULT NULL,
+  `TBL_ID` bigint(20) DEFAULT NULL,
+  `LINK_TARGET_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`PART_ID`),
+  UNIQUE KEY `UNIQUEPARTITION` (`PART_NAME`,`TBL_ID`),
+  KEY `PARTITIONS_N49` (`TBL_ID`),
+  KEY `PARTITIONS_N50` (`SD_ID`),
+  KEY `PARTITIONS_N51` (`LINK_TARGET_ID`),
+  CONSTRAINT `PARTITIONS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`),
+  CONSTRAINT `PARTITIONS_FK2` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`),
+  CONSTRAINT `PARTITIONS_FK3` FOREIGN KEY (`LINK_TARGET_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_EVENTS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_EVENTS` (
+  `PART_NAME_ID` bigint(20) NOT NULL,
+  `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `EVENT_TIME` bigint(20) NOT NULL,
+  `EVENT_TYPE` int(11) NOT NULL,
+  `PARTITION_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_NAME_ID`),
+  KEY `PARTITIONEVENTINDEX` (`PARTITION_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_KEYS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_KEYS` (
+  `TBL_ID` bigint(20) NOT NULL,
+  `PKEY_COMMENT` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PKEY_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PKEY_TYPE` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`TBL_ID`,`PKEY_NAME`),
+  KEY `PARTITION_KEYS_N49` (`TBL_ID`),
+  CONSTRAINT `PARTITION_KEYS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_KEY_VALS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_KEY_VALS` (
+  `PART_ID` bigint(20) NOT NULL,
+  `PART_KEY_VAL` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`PART_ID`,`INTEGER_IDX`),
+  KEY `PARTITION_KEY_VALS_N49` (`PART_ID`),
+  CONSTRAINT `PARTITION_KEY_VALS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_PARAMS` (
+  `PART_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_ID`,`PARAM_KEY`),
+  KEY `PARTITION_PARAMS_N49` (`PART_ID`),
+  CONSTRAINT `PARTITION_PARAMS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PART_COL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PART_COL_PRIVS` (
+  `PART_COLUMN_GRANT_ID` bigint(20) NOT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_ID` bigint(20) DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_COL_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_COLUMN_GRANT_ID`),
+  KEY `PART_COL_PRIVS_N49` (`PART_ID`),
+  KEY `PARTITIONCOLUMNPRIVILEGEINDEX` (`PART_ID`,`COLUMN_NAME`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`PART_COL_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  CONSTRAINT `PART_COL_PRIVS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PART_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PART_PRIVS` (
+  `PART_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_ID` bigint(20) DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_GRANT_ID`),
+  KEY `PARTPRIVILEGEINDEX` (`PART_ID`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`PART_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `PART_PRIVS_N49` (`PART_ID`),
+  CONSTRAINT `PART_PRIVS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `ROLES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `ROLES` (
+  `ROLE_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `OWNER_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `ROLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`ROLE_ID`),
+  UNIQUE KEY `ROLEENTITYINDEX` (`ROLE_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `ROLE_MAP`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `ROLE_MAP` (
+  `ROLE_GRANT_ID` bigint(20) NOT NULL,
+  `ADD_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `ROLE_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`ROLE_GRANT_ID`),
+  UNIQUE KEY `USERROLEMAPINDEX` (`PRINCIPAL_NAME`,`ROLE_ID`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `ROLE_MAP_N49` (`ROLE_ID`),
+  CONSTRAINT `ROLE_MAP_FK1` FOREIGN KEY (`ROLE_ID`) REFERENCES `ROLES` (`ROLE_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SDS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SDS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `CD_ID` bigint(20) DEFAULT NULL,
+  `INPUT_FORMAT` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `IS_COMPRESSED` bit(1) NOT NULL,
+  `IS_STOREDASSUBDIRECTORIES` bit(1) NOT NULL,
+  `LOCATION` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `NUM_BUCKETS` int(11) NOT NULL,
+  `OUTPUT_FORMAT` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SERDE_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`SD_ID`),
+  KEY `SDS_N49` (`SERDE_ID`),
+  KEY `SDS_N50` (`CD_ID`),
+  CONSTRAINT `SDS_FK1` FOREIGN KEY (`SERDE_ID`) REFERENCES `SERDES` (`SERDE_ID`),
+  CONSTRAINT `SDS_FK2` FOREIGN KEY (`CD_ID`) REFERENCES `CDS` (`CD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SD_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SD_PARAMS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SD_ID`,`PARAM_KEY`),
+  KEY `SD_PARAMS_N49` (`SD_ID`),
+  CONSTRAINT `SD_PARAMS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SEQUENCE_TABLE`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SEQUENCE_TABLE` (
+  `SEQUENCE_NAME` varchar(255) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `NEXT_VAL` bigint(20) NOT NULL,
+  PRIMARY KEY (`SEQUENCE_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SERDES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SERDES` (
+  `SERDE_ID` bigint(20) NOT NULL,
+  `NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SLIB` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SERDE_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SERDE_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SERDE_PARAMS` (
+  `SERDE_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`),
+  KEY `SERDE_PARAMS_N49` (`SERDE_ID`),
+  CONSTRAINT `SERDE_PARAMS_FK1` FOREIGN KEY (`SERDE_ID`) REFERENCES `SERDES` (`SERDE_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_COL_NAMES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_COL_NAMES` (
+  `SD_ID` bigint(20) NOT NULL,
+  `SKEWED_COL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
+  KEY `SKEWED_COL_NAMES_N49` (`SD_ID`),
+  CONSTRAINT `SKEWED_COL_NAMES_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_COL_VALUE_LOC_MAP`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_COL_VALUE_LOC_MAP` (
+  `SD_ID` bigint(20) NOT NULL,
+  `STRING_LIST_ID_KID` bigint(20) NOT NULL,
+  `LOCATION` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`),
+  KEY `SKEWED_COL_VALUE_LOC_MAP_N49` (`STRING_LIST_ID_KID`),
+  KEY `SKEWED_COL_VALUE_LOC_MAP_N50` (`SD_ID`),
+  CONSTRAINT `SKEWED_COL_VALUE_LOC_MAP_FK2` FOREIGN KEY (`STRING_LIST_ID_KID`) REFERENCES `SKEWED_STRING_LIST` (`STRING_LIST_ID`),
+  CONSTRAINT `SKEWED_COL_VALUE_LOC_MAP_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_STRING_LIST`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST` (
+  `STRING_LIST_ID` bigint(20) NOT NULL,
+  PRIMARY KEY (`STRING_LIST_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_STRING_LIST_VALUES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST_VALUES` (
+  `STRING_LIST_ID` bigint(20) NOT NULL,
+  `STRING_LIST_VALUE` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`),
+  KEY `SKEWED_STRING_LIST_VALUES_N49` (`STRING_LIST_ID`),
+  CONSTRAINT `SKEWED_STRING_LIST_VALUES_FK1` FOREIGN KEY (`STRING_LIST_ID`) REFERENCES `SKEWED_STRING_LIST` (`STRING_LIST_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_VALUES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_VALUES` (
+  `SD_ID_OID` bigint(20) NOT NULL,
+  `STRING_LIST_ID_EID` bigint(20) NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`),
+  KEY `SKEWED_VALUES_N50` (`SD_ID_OID`),
+  KEY `SKEWED_VALUES_N49` (`STRING_LIST_ID_EID`),
+  CONSTRAINT `SKEWED_VALUES_FK2` FOREIGN KEY (`STRING_LIST_ID_EID`) REFERENCES `SKEWED_STRING_LIST` (`STRING_LIST_ID`),
+  CONSTRAINT `SKEWED_VALUES_FK1` FOREIGN KEY (`SD_ID_OID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SORT_COLS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SORT_COLS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `ORDER` int(11) NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
+  KEY `SORT_COLS_N49` (`SD_ID`),
+  CONSTRAINT `SORT_COLS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TABLE_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` (
+  `TBL_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`TBL_ID`,`PARAM_KEY`),
+  KEY `TABLE_PARAMS_N49` (`TBL_ID`),
+  CONSTRAINT `TABLE_PARAMS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TBLS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TBLS` (
+  `TBL_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `DB_ID` bigint(20) DEFAULT NULL,
+  `LAST_ACCESS_TIME` int(11) NOT NULL,
+  `OWNER` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `RETENTION` int(11) NOT NULL,
+  `SD_ID` bigint(20) DEFAULT NULL,
+  `TBL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `VIEW_EXPANDED_TEXT` mediumtext,
+  `VIEW_ORIGINAL_TEXT` mediumtext,
+  `LINK_TARGET_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`TBL_ID`),
+  UNIQUE KEY `UNIQUETABLE` (`TBL_NAME`,`DB_ID`),
+  KEY `TBLS_N50` (`SD_ID`),
+  KEY `TBLS_N49` (`DB_ID`),
+  KEY `TBLS_N51` (`LINK_TARGET_ID`),
+  CONSTRAINT `TBLS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`),
+  CONSTRAINT `TBLS_FK2` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`),
+  CONSTRAINT `TBLS_FK3` FOREIGN KEY (`LINK_TARGET_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TBL_COL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` (
+  `TBL_COLUMN_GRANT_ID` bigint(20) NOT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_COL_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`TBL_COLUMN_GRANT_ID`),
+  KEY `TABLECOLUMNPRIVILEGEINDEX` (`TBL_ID`,`COLUMN_NAME`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`TBL_COL_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `TBL_COL_PRIVS_N49` (`TBL_ID`),
+  CONSTRAINT `TBL_COL_PRIVS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TBL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TBL_PRIVS` (
+  `TBL_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`TBL_GRANT_ID`),
+  KEY `TBL_PRIVS_N49` (`TBL_ID`),
+  KEY `TABLEPRIVILEGEINDEX` (`TBL_ID`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`TBL_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  CONSTRAINT `TBL_PRIVS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TAB_COL_STATS`
+--
+CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` (
+ `CS_ID` bigint(20) NOT NULL,
+ `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `TABLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `TBL_ID` bigint(20) NOT NULL,
+ `LONG_LOW_VALUE` bigint(20),
+ `LONG_HIGH_VALUE` bigint(20),
+ `DOUBLE_HIGH_VALUE` double(53,4),
+ `DOUBLE_LOW_VALUE` double(53,4),
+ `BIG_DECIMAL_LOW_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `BIG_DECIMAL_HIGH_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `NUM_NULLS` bigint(20) NOT NULL,
+ `NUM_DISTINCTS` bigint(20),
+ `AVG_COL_LEN` double(53,4),
+ `MAX_COL_LEN` bigint(20),
+ `NUM_TRUES` bigint(20),
+ `NUM_FALSES` bigint(20),
+ `LAST_ANALYZED` bigint(20) NOT NULL,
+  PRIMARY KEY (`CS_ID`),
+  CONSTRAINT `TAB_COL_STATS_FK` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for table `PART_COL_STATS`
+--
+CREATE TABLE IF NOT EXISTS `PART_COL_STATS` (
+ `CS_ID` bigint(20) NOT NULL,
+ `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `TABLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `PARTITION_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `PART_ID` bigint(20) NOT NULL,
+ `LONG_LOW_VALUE` bigint(20),
+ `LONG_HIGH_VALUE` bigint(20),
+ `DOUBLE_HIGH_VALUE` double(53,4),
+ `DOUBLE_LOW_VALUE` double(53,4),
+ `BIG_DECIMAL_LOW_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `BIG_DECIMAL_HIGH_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `NUM_NULLS` bigint(20) NOT NULL,
+ `NUM_DISTINCTS` bigint(20),
+ `AVG_COL_LEN` double(53,4),
+ `MAX_COL_LEN` bigint(20),
+ `NUM_TRUES` bigint(20),
+ `NUM_FALSES` bigint(20),
+ `LAST_ANALYZED` bigint(20) NOT NULL,
+  PRIMARY KEY (`CS_ID`),
+  CONSTRAINT `PART_COL_STATS_FK` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for table `TYPES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TYPES` (
+  `TYPES_ID` bigint(20) NOT NULL,
+  `TYPE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TYPE1` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TYPE2` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`TYPES_ID`),
+  UNIQUE KEY `UNIQUE_TYPE` (`TYPE_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TYPE_FIELDS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TYPE_FIELDS` (
+  `TYPE_NAME` bigint(20) NOT NULL,
+  `COMMENT` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `FIELD_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `FIELD_TYPE` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`TYPE_NAME`,`FIELD_NAME`),
+  KEY `TYPE_FIELDS_N49` (`TYPE_NAME`),
+  CONSTRAINT `TYPE_FIELDS_FK1` FOREIGN KEY (`TYPE_NAME`) REFERENCES `TYPES` (`TYPES_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+-- Table `MASTER_KEYS` for classes [org.apache.hadoop.hive.metastore.model.MMasterKey]
+CREATE TABLE IF NOT EXISTS `MASTER_KEYS` 
+(
+    `KEY_ID` INTEGER NOT NULL AUTO_INCREMENT,
+    `MASTER_KEY` VARCHAR(767) BINARY NULL,
+    PRIMARY KEY (`KEY_ID`)
+) ENGINE=INNODB DEFAULT CHARSET=latin1;
+
+-- Table `DELEGATION_TOKENS` for classes [org.apache.hadoop.hive.metastore.model.MDelegationToken]
+CREATE TABLE IF NOT EXISTS `DELEGATION_TOKENS`
+(
+    `TOKEN_IDENT` VARCHAR(767) BINARY NOT NULL,
+    `TOKEN` VARCHAR(767) BINARY NULL,
+    PRIMARY KEY (`TOKEN_IDENT`)
+) ENGINE=INNODB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for VERSION
+--
+CREATE TABLE IF NOT EXISTS `VERSION` (
+  `VER_ID` BIGINT NOT NULL,
+  `SCHEMA_VERSION` VARCHAR(127) NOT NULL,
+  `VERSION_COMMENT` VARCHAR(255),
+  PRIMARY KEY (`VER_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.12.0', 'Hive release version 0.12.0');
+
+/*!40101 SET character_set_client = @saved_cs_client */;
+/*!40103 SET TIME_ZONE=@OLD_TIME_ZONE */;
+
+/*!40101 SET SQL_MODE=@OLD_SQL_MODE */;
+/*!40014 SET FOREIGN_KEY_CHECKS=@OLD_FOREIGN_KEY_CHECKS */;
+/*!40014 SET UNIQUE_CHECKS=@OLD_UNIQUE_CHECKS */;
+/*!40101 SET CHARACTER_SET_CLIENT=@OLD_CHARACTER_SET_CLIENT */;
+/*!40101 SET CHARACTER_SET_RESULTS=@OLD_CHARACTER_SET_RESULTS */;
+/*!40101 SET COLLATION_CONNECTION=@OLD_COLLATION_CONNECTION */;
+/*!40111 SET SQL_NOTES=@OLD_SQL_NOTES */;
+
+-- Dump completed on 2012-08-23  0:56:31

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.oracle.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.oracle.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.oracle.sql
new file mode 100755
index 0000000..812b897
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/etc/hive-schema-0.12.0.oracle.sql
@@ -0,0 +1,718 @@
+-- Table SEQUENCE_TABLE is an internal table required by DataNucleus.
+-- NOTE: Some versions of SchemaTool do not automatically generate this table.
+-- See http://www.datanucleus.org/servlet/jira/browse/NUCRDBMS-416
+CREATE TABLE SEQUENCE_TABLE
+(
+   SEQUENCE_NAME VARCHAR2(255) NOT NULL,
+   NEXT_VAL NUMBER NOT NULL
+);
+
+ALTER TABLE SEQUENCE_TABLE ADD CONSTRAINT PART_TABLE_PK PRIMARY KEY (SEQUENCE_NAME);
+
+-- Table NUCLEUS_TABLES is an internal table required by DataNucleus.
+-- This table is required if datanucleus.autoStartMechanism=SchemaTable
+-- NOTE: Some versions of SchemaTool do not automatically generate this table.
+-- See http://www.datanucleus.org/servlet/jira/browse/NUCRDBMS-416
+CREATE TABLE NUCLEUS_TABLES
+(
+   CLASS_NAME VARCHAR2(128) NOT NULL,
+   TABLE_NAME VARCHAR2(128) NOT NULL,
+   TYPE VARCHAR2(4) NOT NULL,
+   OWNER VARCHAR2(2) NOT NULL,
+   VERSION VARCHAR2(20) NOT NULL,
+   INTERFACE_NAME VARCHAR2(255) NULL
+);
+
+ALTER TABLE NUCLEUS_TABLES ADD CONSTRAINT NUCLEUS_TABLES_PK PRIMARY KEY (CLASS_NAME);
+
+-- Table PART_COL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege]
+CREATE TABLE PART_COL_PRIVS
+(
+    PART_COLUMN_GRANT_ID NUMBER NOT NULL,
+    "COLUMN_NAME" VARCHAR2(128) NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PART_ID NUMBER NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    PART_COL_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE PART_COL_PRIVS ADD CONSTRAINT PART_COL_PRIVS_PK PRIMARY KEY (PART_COLUMN_GRANT_ID);
+
+-- Table CDS.
+CREATE TABLE CDS
+(
+    CD_ID NUMBER NOT NULL
+);
+
+ALTER TABLE CDS ADD CONSTRAINT CDS_PK PRIMARY KEY (CD_ID);
+
+-- Table COLUMNS_V2 for join relationship
+CREATE TABLE COLUMNS_V2
+(
+    CD_ID NUMBER NOT NULL,
+    "COMMENT" VARCHAR2(256) NULL,
+    "COLUMN_NAME" VARCHAR2(128) NOT NULL,
+    TYPE_NAME VARCHAR2(4000) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE COLUMNS_V2 ADD CONSTRAINT COLUMNS_V2_PK PRIMARY KEY (CD_ID,"COLUMN_NAME");
+
+-- Table PARTITION_KEY_VALS for join relationship
+CREATE TABLE PARTITION_KEY_VALS
+(
+    PART_ID NUMBER NOT NULL,
+    PART_KEY_VAL VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE PARTITION_KEY_VALS ADD CONSTRAINT PARTITION_KEY_VALS_PK PRIMARY KEY (PART_ID,INTEGER_IDX);
+
+-- Table DBS for classes [org.apache.hadoop.hive.metastore.model.MDatabase]
+CREATE TABLE DBS
+(
+    DB_ID NUMBER NOT NULL,
+    "DESC" VARCHAR2(4000) NULL,
+    DB_LOCATION_URI VARCHAR2(4000) NOT NULL,
+    "NAME" VARCHAR2(128) NULL
+);
+
+ALTER TABLE DBS ADD CONSTRAINT DBS_PK PRIMARY KEY (DB_ID);
+
+-- Table PARTITION_PARAMS for join relationship
+CREATE TABLE PARTITION_PARAMS
+(
+    PART_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE PARTITION_PARAMS ADD CONSTRAINT PARTITION_PARAMS_PK PRIMARY KEY (PART_ID,PARAM_KEY);
+
+-- Table SERDES for classes [org.apache.hadoop.hive.metastore.model.MSerDeInfo]
+CREATE TABLE SERDES
+(
+    SERDE_ID NUMBER NOT NULL,
+    "NAME" VARCHAR2(128) NULL,
+    SLIB VARCHAR2(4000) NULL
+);
+
+ALTER TABLE SERDES ADD CONSTRAINT SERDES_PK PRIMARY KEY (SERDE_ID);
+
+-- Table TYPES for classes [org.apache.hadoop.hive.metastore.model.MType]
+CREATE TABLE TYPES
+(
+    TYPES_ID NUMBER NOT NULL,
+    TYPE_NAME VARCHAR2(128) NULL,
+    TYPE1 VARCHAR2(767) NULL,
+    TYPE2 VARCHAR2(767) NULL
+);
+
+ALTER TABLE TYPES ADD CONSTRAINT TYPES_PK PRIMARY KEY (TYPES_ID);
+
+-- Table PARTITION_KEYS for join relationship
+CREATE TABLE PARTITION_KEYS
+(
+    TBL_ID NUMBER NOT NULL,
+    PKEY_COMMENT VARCHAR2(4000) NULL,
+    PKEY_NAME VARCHAR2(128) NOT NULL,
+    PKEY_TYPE VARCHAR2(767) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE PARTITION_KEYS ADD CONSTRAINT PARTITION_KEY_PK PRIMARY KEY (TBL_ID,PKEY_NAME);
+
+-- Table ROLES for classes [org.apache.hadoop.hive.metastore.model.MRole]
+CREATE TABLE ROLES
+(
+    ROLE_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    OWNER_NAME VARCHAR2(128) NULL,
+    ROLE_NAME VARCHAR2(128) NULL
+);
+
+ALTER TABLE ROLES ADD CONSTRAINT ROLES_PK PRIMARY KEY (ROLE_ID);
+
+-- Table PARTITIONS for classes [org.apache.hadoop.hive.metastore.model.MPartition]
+CREATE TABLE PARTITIONS
+(
+    PART_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    LAST_ACCESS_TIME NUMBER (10) NOT NULL,
+    PART_NAME VARCHAR2(767) NULL,
+    SD_ID NUMBER NULL,
+    TBL_ID NUMBER NULL
+);
+
+ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_PK PRIMARY KEY (PART_ID);
+
+-- Table INDEX_PARAMS for join relationship
+CREATE TABLE INDEX_PARAMS
+(
+    INDEX_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE INDEX_PARAMS ADD CONSTRAINT INDEX_PARAMS_PK PRIMARY KEY (INDEX_ID,PARAM_KEY);
+
+-- Table TBL_COL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege]
+CREATE TABLE TBL_COL_PRIVS
+(
+    TBL_COLUMN_GRANT_ID NUMBER NOT NULL,
+    "COLUMN_NAME" VARCHAR2(128) NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    TBL_COL_PRIV VARCHAR2(128) NULL,
+    TBL_ID NUMBER NULL
+);
+
+ALTER TABLE TBL_COL_PRIVS ADD CONSTRAINT TBL_COL_PRIVS_PK PRIMARY KEY (TBL_COLUMN_GRANT_ID);
+
+-- Table IDXS for classes [org.apache.hadoop.hive.metastore.model.MIndex]
+CREATE TABLE IDXS
+(
+    INDEX_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    DEFERRED_REBUILD NUMBER(1) NOT NULL CHECK (DEFERRED_REBUILD IN (1,0)),
+    INDEX_HANDLER_CLASS VARCHAR2(4000) NULL,
+    INDEX_NAME VARCHAR2(128) NULL,
+    INDEX_TBL_ID NUMBER NULL,
+    LAST_ACCESS_TIME NUMBER (10) NOT NULL,
+    ORIG_TBL_ID NUMBER NULL,
+    SD_ID NUMBER NULL
+);
+
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_PK PRIMARY KEY (INDEX_ID);
+
+-- Table BUCKETING_COLS for join relationship
+CREATE TABLE BUCKETING_COLS
+(
+    SD_ID NUMBER NOT NULL,
+    BUCKET_COL_NAME VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE BUCKETING_COLS ADD CONSTRAINT BUCKETING_COLS_PK PRIMARY KEY (SD_ID,INTEGER_IDX);
+
+-- Table TYPE_FIELDS for join relationship
+CREATE TABLE TYPE_FIELDS
+(
+    TYPE_NAME NUMBER NOT NULL,
+    "COMMENT" VARCHAR2(256) NULL,
+    FIELD_NAME VARCHAR2(128) NOT NULL,
+    FIELD_TYPE VARCHAR2(767) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE TYPE_FIELDS ADD CONSTRAINT TYPE_FIELDS_PK PRIMARY KEY (TYPE_NAME,FIELD_NAME);
+
+-- Table SD_PARAMS for join relationship
+CREATE TABLE SD_PARAMS
+(
+    SD_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE SD_PARAMS ADD CONSTRAINT SD_PARAMS_PK PRIMARY KEY (SD_ID,PARAM_KEY);
+
+-- Table GLOBAL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MGlobalPrivilege]
+CREATE TABLE GLOBAL_PRIVS
+(
+    USER_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    USER_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE GLOBAL_PRIVS ADD CONSTRAINT GLOBAL_PRIVS_PK PRIMARY KEY (USER_GRANT_ID);
+
+-- Table SDS for classes [org.apache.hadoop.hive.metastore.model.MStorageDescriptor]
+CREATE TABLE SDS
+(
+    SD_ID NUMBER NOT NULL,
+    CD_ID NUMBER NULL,
+    INPUT_FORMAT VARCHAR2(4000) NULL,
+    IS_COMPRESSED NUMBER(1) NOT NULL CHECK (IS_COMPRESSED IN (1,0)),
+    LOCATION VARCHAR2(4000) NULL,
+    NUM_BUCKETS NUMBER (10) NOT NULL,
+    OUTPUT_FORMAT VARCHAR2(4000) NULL,
+    SERDE_ID NUMBER NULL,
+    IS_STOREDASSUBDIRECTORIES NUMBER(1) NOT NULL CHECK (IS_STOREDASSUBDIRECTORIES IN (1,0))
+);
+
+ALTER TABLE SDS ADD CONSTRAINT SDS_PK PRIMARY KEY (SD_ID);
+
+-- Table TABLE_PARAMS for join relationship
+CREATE TABLE TABLE_PARAMS
+(
+    TBL_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE TABLE_PARAMS ADD CONSTRAINT TABLE_PARAMS_PK PRIMARY KEY (TBL_ID,PARAM_KEY);
+
+-- Table SORT_COLS for join relationship
+CREATE TABLE SORT_COLS
+(
+    SD_ID NUMBER NOT NULL,
+    "COLUMN_NAME" VARCHAR2(128) NULL,
+    "ORDER" NUMBER (10) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SORT_COLS ADD CONSTRAINT SORT_COLS_PK PRIMARY KEY (SD_ID,INTEGER_IDX);
+
+-- Table TBL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MTablePrivilege]
+CREATE TABLE TBL_PRIVS
+(
+    TBL_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    TBL_PRIV VARCHAR2(128) NULL,
+    TBL_ID NUMBER NULL
+);
+
+ALTER TABLE TBL_PRIVS ADD CONSTRAINT TBL_PRIVS_PK PRIMARY KEY (TBL_GRANT_ID);
+
+-- Table DATABASE_PARAMS for join relationship
+CREATE TABLE DATABASE_PARAMS
+(
+    DB_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(180) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE DATABASE_PARAMS ADD CONSTRAINT DATABASE_PARAMS_PK PRIMARY KEY (DB_ID,PARAM_KEY);
+
+-- Table ROLE_MAP for classes [org.apache.hadoop.hive.metastore.model.MRoleMap]
+CREATE TABLE ROLE_MAP
+(
+    ROLE_GRANT_ID NUMBER NOT NULL,
+    ADD_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    ROLE_ID NUMBER NULL
+);
+
+ALTER TABLE ROLE_MAP ADD CONSTRAINT ROLE_MAP_PK PRIMARY KEY (ROLE_GRANT_ID);
+
+-- Table SERDE_PARAMS for join relationship
+CREATE TABLE SERDE_PARAMS
+(
+    SERDE_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE SERDE_PARAMS ADD CONSTRAINT SERDE_PARAMS_PK PRIMARY KEY (SERDE_ID,PARAM_KEY);
+
+-- Table PART_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MPartitionPrivilege]
+CREATE TABLE PART_PRIVS
+(
+    PART_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PART_ID NUMBER NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    PART_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE PART_PRIVS ADD CONSTRAINT PART_PRIVS_PK PRIMARY KEY (PART_GRANT_ID);
+
+-- Table DB_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MDBPrivilege]
+CREATE TABLE DB_PRIVS
+(
+    DB_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    DB_ID NUMBER NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    DB_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE DB_PRIVS ADD CONSTRAINT DB_PRIVS_PK PRIMARY KEY (DB_GRANT_ID);
+
+-- Table TBLS for classes [org.apache.hadoop.hive.metastore.model.MTable]
+CREATE TABLE TBLS
+(
+    TBL_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    DB_ID NUMBER NULL,
+    LAST_ACCESS_TIME NUMBER (10) NOT NULL,
+    OWNER VARCHAR2(767) NULL,
+    RETENTION NUMBER (10) NOT NULL,
+    SD_ID NUMBER NULL,
+    TBL_NAME VARCHAR2(128) NULL,
+    TBL_TYPE VARCHAR2(128) NULL,
+    VIEW_EXPANDED_TEXT CLOB NULL,
+    VIEW_ORIGINAL_TEXT CLOB NULL
+);
+
+ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);
+
+-- Table PARTITION_EVENTS for classes [org.apache.hadoop.hive.metastore.model.MPartitionEvent]
+CREATE TABLE PARTITION_EVENTS
+(
+    PART_NAME_ID NUMBER NOT NULL,
+    DB_NAME VARCHAR2(128) NULL,
+    EVENT_TIME NUMBER NOT NULL,
+    EVENT_TYPE NUMBER (10) NOT NULL,
+    PARTITION_NAME VARCHAR2(767) NULL,
+    TBL_NAME VARCHAR2(128) NULL
+);
+
+ALTER TABLE PARTITION_EVENTS ADD CONSTRAINT PARTITION_EVENTS_PK PRIMARY KEY (PART_NAME_ID);
+
+-- Table SKEWED_STRING_LIST for classes [org.apache.hadoop.hive.metastore.model.MStringList]
+CREATE TABLE SKEWED_STRING_LIST
+(
+    STRING_LIST_ID NUMBER NOT NULL
+);
+
+ALTER TABLE SKEWED_STRING_LIST ADD CONSTRAINT SKEWED_STRING_LIST_PK PRIMARY KEY (STRING_LIST_ID);
+
+CREATE TABLE SKEWED_STRING_LIST_VALUES
+(
+    STRING_LIST_ID NUMBER NOT NULL,
+    "STRING_LIST_VALUE" VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SKEWED_STRING_LIST_VALUES ADD CONSTRAINT SKEWED_STRING_LIST_VALUES_PK PRIMARY KEY (STRING_LIST_ID,INTEGER_IDX);
+
+ALTER TABLE SKEWED_STRING_LIST_VALUES ADD CONSTRAINT SKEWED_STRING_LIST_VALUES_FK1 FOREIGN KEY (STRING_LIST_ID) REFERENCES SKEWED_STRING_LIST (STRING_LIST_ID) INITIALLY DEFERRED ;
+
+CREATE TABLE SKEWED_COL_NAMES
+(
+    SD_ID NUMBER NOT NULL,
+    "SKEWED_COL_NAME" VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SKEWED_COL_NAMES ADD CONSTRAINT SKEWED_COL_NAMES_PK PRIMARY KEY (SD_ID,INTEGER_IDX);
+
+ALTER TABLE SKEWED_COL_NAMES ADD CONSTRAINT SKEWED_COL_NAMES_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE TABLE SKEWED_COL_VALUE_LOC_MAP
+(
+    SD_ID NUMBER NOT NULL,
+    STRING_LIST_ID_KID NUMBER NOT NULL,
+    "LOCATION" VARCHAR2(4000) NULL
+);
+
+CREATE TABLE MASTER_KEYS
+(
+    KEY_ID NUMBER (10) NOT NULL,
+    MASTER_KEY VARCHAR2(767) NULL
+);
+
+CREATE TABLE DELEGATION_TOKENS
+(
+    TOKEN_IDENT VARCHAR2(767) NOT NULL,
+    TOKEN VARCHAR2(767) NULL
+);
+
+ALTER TABLE SKEWED_COL_VALUE_LOC_MAP ADD CONSTRAINT SKEWED_COL_VALUE_LOC_MAP_PK PRIMARY KEY (SD_ID,STRING_LIST_ID_KID);
+
+ALTER TABLE SKEWED_COL_VALUE_LOC_MAP ADD CONSTRAINT SKEWED_COL_VALUE_LOC_MAP_FK1 FOREIGN KEY (STRING_LIST_ID_KID) REFERENCES SKEWED_STRING_LIST (STRING_LIST_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE SKEWED_COL_VALUE_LOC_MAP ADD CONSTRAINT SKEWED_COL_VALUE_LOC_MAP_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE TABLE SKEWED_VALUES
+(
+    SD_ID_OID NUMBER NOT NULL,
+    STRING_LIST_ID_EID NUMBER NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_PK PRIMARY KEY (SD_ID_OID,INTEGER_IDX);
+
+ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_FK1 FOREIGN KEY (STRING_LIST_ID_EID) REFERENCES SKEWED_STRING_LIST (STRING_LIST_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_FK2 FOREIGN KEY (SD_ID_OID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+-- column statistics
+
+CREATE TABLE TAB_COL_STATS (
+ CS_ID NUMBER NOT NULL,
+ DB_NAME VARCHAR2(128) NOT NULL,
+ TABLE_NAME VARCHAR2(128) NOT NULL, 
+ COLUMN_NAME VARCHAR2(128) NOT NULL,
+ COLUMN_TYPE VARCHAR2(128) NOT NULL,
+ TBL_ID NUMBER NOT NULL,
+ LONG_LOW_VALUE NUMBER,
+ LONG_HIGH_VALUE NUMBER,
+ DOUBLE_LOW_VALUE NUMBER,
+ DOUBLE_HIGH_VALUE NUMBER,
+ BIG_DECIMAL_LOW_VALUE VARCHAR2(4000),
+ BIG_DECIMAL_HIGH_VALUE VARCHAR2(4000),
+ NUM_NULLS NUMBER NOT NULL,
+ NUM_DISTINCTS NUMBER,
+ AVG_COL_LEN NUMBER,
+ MAX_COL_LEN NUMBER,
+ NUM_TRUES NUMBER,
+ NUM_FALSES NUMBER,
+ LAST_ANALYZED NUMBER NOT NULL
+);
+
+CREATE TABLE VERSION (
+  VER_ID NUMBER NOT NULL,
+  SCHEMA_VERSION VARCHAR(127) NOT NULL,
+  VERSION_COMMENT VARCHAR(255)
+);
+ALTER TABLE VERSION ADD CONSTRAINT VERSION_PK PRIMARY KEY (VER_ID);
+
+ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_PKEY PRIMARY KEY (CS_ID);
+
+ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_FK FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TAB_COL_STATS_N49 ON TAB_COL_STATS(TBL_ID);
+
+CREATE TABLE PART_COL_STATS (
+ CS_ID NUMBER NOT NULL,
+ DB_NAME VARCHAR2(128) NOT NULL,
+ TABLE_NAME VARCHAR2(128) NOT NULL,
+ PARTITION_NAME VARCHAR2(767) NOT NULL,
+ COLUMN_NAME VARCHAR2(128) NOT NULL,
+ COLUMN_TYPE VARCHAR2(128) NOT NULL,
+ PART_ID NUMBER NOT NULL,
+ LONG_LOW_VALUE NUMBER,
+ LONG_HIGH_VALUE NUMBER,
+ DOUBLE_LOW_VALUE NUMBER,
+ DOUBLE_HIGH_VALUE NUMBER,
+ BIG_DECIMAL_LOW_VALUE VARCHAR2(4000),
+ BIG_DECIMAL_HIGH_VALUE VARCHAR2(4000),
+ NUM_NULLS NUMBER NOT NULL,
+ NUM_DISTINCTS NUMBER,
+ AVG_COL_LEN NUMBER,
+ MAX_COL_LEN NUMBER,
+ NUM_TRUES NUMBER,
+ NUM_FALSES NUMBER,
+ LAST_ANALYZED NUMBER NOT NULL
+);
+
+ALTER TABLE PART_COL_STATS ADD CONSTRAINT PART_COL_STATS_PKEY PRIMARY KEY (CS_ID);
+
+ALTER TABLE PART_COL_STATS ADD CONSTRAINT PART_COL_STATS_FK FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED;
+
+CREATE INDEX PART_COL_STATS_N49 ON PART_COL_STATS (PART_ID);
+
+-- Constraints for table PART_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege]
+ALTER TABLE PART_COL_PRIVS ADD CONSTRAINT PART_COL_PRIVS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PART_COL_PRIVS_N49 ON PART_COL_PRIVS (PART_ID);
+
+CREATE INDEX PARTITIONCOLUMNPRIVILEGEINDEX ON PART_COL_PRIVS (PART_ID,"COLUMN_NAME",PRINCIPAL_NAME,PRINCIPAL_TYPE,PART_COL_PRIV,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table COLUMNS_V2
+ALTER TABLE COLUMNS_V2 ADD CONSTRAINT COLUMNS_V2_FK1 FOREIGN KEY (CD_ID) REFERENCES CDS (CD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX COLUMNS_V2_N49 ON COLUMNS_V2 (CD_ID);
+
+
+-- Constraints for table PARTITION_KEY_VALS
+ALTER TABLE PARTITION_KEY_VALS ADD CONSTRAINT PARTITION_KEY_VALS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITION_KEY_VALS_N49 ON PARTITION_KEY_VALS (PART_ID);
+
+
+-- Constraints for table DBS for class(es) [org.apache.hadoop.hive.metastore.model.MDatabase]
+CREATE UNIQUE INDEX UNIQUE_DATABASE ON DBS ("NAME");
+
+
+-- Constraints for table PARTITION_PARAMS
+ALTER TABLE PARTITION_PARAMS ADD CONSTRAINT PARTITION_PARAMS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITION_PARAMS_N49 ON PARTITION_PARAMS (PART_ID);
+
+
+-- Constraints for table SERDES for class(es) [org.apache.hadoop.hive.metastore.model.MSerDeInfo]
+
+-- Constraints for table TYPES for class(es) [org.apache.hadoop.hive.metastore.model.MType]
+CREATE UNIQUE INDEX UNIQUE_TYPE ON TYPES (TYPE_NAME);
+
+
+-- Constraints for table PARTITION_KEYS
+ALTER TABLE PARTITION_KEYS ADD CONSTRAINT PARTITION_KEYS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITION_KEYS_N49 ON PARTITION_KEYS (TBL_ID);
+
+
+-- Constraints for table ROLES for class(es) [org.apache.hadoop.hive.metastore.model.MRole]
+CREATE UNIQUE INDEX ROLEENTITYINDEX ON ROLES (ROLE_NAME);
+
+
+-- Constraints for table PARTITIONS for class(es) [org.apache.hadoop.hive.metastore.model.MPartition]
+ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITIONS_N49 ON PARTITIONS (SD_ID);
+
+CREATE INDEX PARTITIONS_N50 ON PARTITIONS (TBL_ID);
+
+CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (PART_NAME,TBL_ID);
+
+
+-- Constraints for table INDEX_PARAMS
+ALTER TABLE INDEX_PARAMS ADD CONSTRAINT INDEX_PARAMS_FK1 FOREIGN KEY (INDEX_ID) REFERENCES IDXS (INDEX_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX INDEX_PARAMS_N49 ON INDEX_PARAMS (INDEX_ID);
+
+
+-- Constraints for table TBL_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege]
+ALTER TABLE TBL_COL_PRIVS ADD CONSTRAINT TBL_COL_PRIVS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TABLECOLUMNPRIVILEGEINDEX ON TBL_COL_PRIVS (TBL_ID,"COLUMN_NAME",PRINCIPAL_NAME,PRINCIPAL_TYPE,TBL_COL_PRIV,GRANTOR,GRANTOR_TYPE);
+
+CREATE INDEX TBL_COL_PRIVS_N49 ON TBL_COL_PRIVS (TBL_ID);
+
+
+-- Constraints for table IDXS for class(es) [org.apache.hadoop.hive.metastore.model.MIndex]
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK1 FOREIGN KEY (ORIG_TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK3 FOREIGN KEY (INDEX_TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE UNIQUE INDEX UNIQUEINDEX ON IDXS (INDEX_NAME,ORIG_TBL_ID);
+
+CREATE INDEX IDXS_N50 ON IDXS (INDEX_TBL_ID);
+
+CREATE INDEX IDXS_N51 ON IDXS (SD_ID);
+
+CREATE INDEX IDXS_N49 ON IDXS (ORIG_TBL_ID);
+
+
+-- Constraints for table BUCKETING_COLS
+ALTER TABLE BUCKETING_COLS ADD CONSTRAINT BUCKETING_COLS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX BUCKETING_COLS_N49 ON BUCKETING_COLS (SD_ID);
+
+
+-- Constraints for table TYPE_FIELDS
+ALTER TABLE TYPE_FIELDS ADD CONSTRAINT TYPE_FIELDS_FK1 FOREIGN KEY (TYPE_NAME) REFERENCES TYPES (TYPES_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TYPE_FIELDS_N49 ON TYPE_FIELDS (TYPE_NAME);
+
+
+-- Constraints for table SD_PARAMS
+ALTER TABLE SD_PARAMS ADD CONSTRAINT SD_PARAMS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SD_PARAMS_N49 ON SD_PARAMS (SD_ID);
+
+
+-- Constraints for table GLOBAL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MGlobalPrivilege]
+CREATE UNIQUE INDEX GLOBALPRIVILEGEINDEX ON GLOBAL_PRIVS (PRINCIPAL_NAME,PRINCIPAL_TYPE,USER_PRIV,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table SDS for class(es) [org.apache.hadoop.hive.metastore.model.MStorageDescriptor]
+ALTER TABLE SDS ADD CONSTRAINT SDS_FK1 FOREIGN KEY (SERDE_ID) REFERENCES SERDES (SERDE_ID) INITIALLY DEFERRED ;
+ALTER TABLE SDS ADD CONSTRAINT SDS_FK2 FOREIGN KEY (CD_ID) REFERENCES CDS (CD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SDS_N49 ON SDS (SERDE_ID);
+CREATE INDEX SDS_N50 ON SDS (CD_ID);
+
+
+-- Constraints for table TABLE_PARAMS
+ALTER TABLE TABLE_PARAMS ADD CONSTRAINT TABLE_PARAMS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TABLE_PARAMS_N49 ON TABLE_PARAMS (TBL_ID);
+
+
+-- Constraints for table SORT_COLS
+ALTER TABLE SORT_COLS ADD CONSTRAINT SORT_COLS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SORT_COLS_N49 ON SORT_COLS (SD_ID);
+
+
+-- Constraints for table TBL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MTablePrivilege]
+ALTER TABLE TBL_PRIVS ADD CONSTRAINT TBL_PRIVS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TBL_PRIVS_N49 ON TBL_PRIVS (TBL_ID);
+
+CREATE INDEX TABLEPRIVILEGEINDEX ON TBL_PRIVS (TBL_ID,PRINCIPAL_NAME,PRINCIPAL_TYPE,TBL_PRIV,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table DATABASE_PARAMS
+ALTER TABLE DATABASE_PARAMS ADD CONSTRAINT DATABASE_PARAMS_FK1 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX DATABASE_PARAMS_N49 ON DATABASE_PARAMS (DB_ID);
+
+
+-- Constraints for table ROLE_MAP for class(es) [org.apache.hadoop.hive.metastore.model.MRoleMap]
+ALTER TABLE ROLE_MAP ADD CONSTRAINT ROLE_MAP_FK1 FOREIGN KEY (ROLE_ID) REFERENCES ROLES (ROLE_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX ROLE_MAP_N49 ON ROLE_MAP (ROLE_ID);
+
+CREATE UNIQUE INDEX USERROLEMAPINDEX ON ROLE_MAP (PRINCIPAL_NAME,ROLE_ID,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table SERDE_PARAMS
+ALTER TABLE SERDE_PARAMS ADD CONSTRAINT SERDE_PARAMS_FK1 FOREIGN KEY (SERDE_ID) REFERENCES SERDES (SERDE_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SERDE_PARAMS_N49 ON SERDE_PARAMS (SERDE_ID);
+
+
+-- Constraints for table PART_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionPrivilege]
+ALTER TABLE PART_PRIVS ADD CONSTRAINT PART_PRIVS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTPRIVILEGEINDEX ON PART_PRIVS (PART_ID,PRINCIPAL_NAME,PRINCIPAL_TYPE,PART_PRIV,GRANTOR,GRANTOR_TYPE);
+
+CREATE INDEX PART_PRIVS_N49 ON PART_PRIVS (PART_ID);
+
+
+-- Constraints for table DB_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MDBPrivilege]
+ALTER TABLE DB_PRIVS ADD CONSTRAINT DB_PRIVS_FK1 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED ;
+
+CREATE UNIQUE INDEX DBPRIVILEGEINDEX ON DB_PRIVS (DB_ID,PRINCIPAL_NAME,PRINCIPAL_TYPE,DB_PRIV,GRANTOR,GRANTOR_TYPE);
+
+CREATE INDEX DB_PRIVS_N49 ON DB_PRIVS (DB_ID);
+
+
+-- Constraints for table TBLS for class(es) [org.apache.hadoop.hive.metastore.model.MTable]
+ALTER TABLE TBLS ADD CONSTRAINT TBLS_FK2 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE TBLS ADD CONSTRAINT TBLS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TBLS_N49 ON TBLS (DB_ID);
+
+CREATE UNIQUE INDEX UNIQUETABLE ON TBLS (TBL_NAME,DB_ID);
+
+CREATE INDEX TBLS_N50 ON TBLS (SD_ID);
+
+
+-- Constraints for table PARTITION_EVENTS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionEvent]
+CREATE INDEX PARTITIONEVENTINDEX ON PARTITION_EVENTS (PARTITION_NAME);
+
+INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.12.0', 'Hive release version 0.12.0');
+


[26/50] [abbrv] bigtop git commit: Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
Add Hive 1.2 to ODPi ambari reference implementation


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

Branch: refs/heads/BIGTOP-2666
Commit: 623e0332ffd81402ea009c8577218fc15d8df622
Parents: c67dfbc
Author: Sumit Mohanty <su...@gmail.com>
Authored: Wed Oct 26 17:46:24 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:00 2017 -0800

----------------------------------------------------------------------
 .../ambari/ODPi/1.0/role_command_order.json     |   1 +
 .../HIVE/configuration/beeline-log4j2.xml       |  62 --
 .../hive-atlas-application.properties.xml       |  67 --
 .../HIVE/configuration/hive-exec-log4j2.xml     |  83 --
 .../HIVE/configuration/hive-interactive-env.xml | 373 --------
 .../configuration/hive-interactive-site.xml     | 909 -------------------
 .../services/HIVE/configuration/hive-log4j2.xml |  90 --
 .../hiveserver2-interactive-site.xml            |  56 --
 .../HIVE/configuration/llap-cli-log4j2.xml      |  91 --
 .../HIVE/configuration/llap-daemon-log4j.xml    | 158 ----
 .../HIVE/configuration/ranger-hive-audit.xml    | 136 ---
 .../ranger-hive-plugin-properties.xml           |  63 --
 .../configuration/ranger-hive-policymgr-ssl.xml |  71 --
 .../HIVE/configuration/ranger-hive-security.xml |  81 --
 .../HIVE/configuration/tez-interactive-site.xml | 144 ---
 .../ambari/ODPi/1.0/services/HIVE/metainfo.xml  | 158 +---
 .../HIVE/package/scripts/params_linux.py        |  42 +-
 .../HIVE/package/scripts/status_params.py       |  14 +-
 .../services/YARN/configuration/yarn-site.xml   | 207 ++++-
 .../ambari/ODPi/1.0/services/YARN/metainfo.xml  |  14 +
 20 files changed, 230 insertions(+), 2590 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/role_command_order.json
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/role_command_order.json b/bigtop-packages/src/common/ambari/ODPi/1.0/role_command_order.json
index 31f26e3..05beb76 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/role_command_order.json
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/role_command_order.json
@@ -5,6 +5,7 @@
     "_comment" : "dependencies for all cases",
     "HBASE_MASTER-START": ["ZOOKEEPER_SERVER-START"],
     "HBASE_REGIONSERVER-START": ["HBASE_MASTER-START"],
+    "APP_TIMELINE_SERVER-START": ["NAMENODE-START", "DATANODE-START"],
     "OOZIE_SERVER-START": ["NODEMANAGER-START", "RESOURCEMANAGER-START"],
     "WEBHCAT_SERVER-START": ["NODEMANAGER-START", "HIVE_SERVER-START"],
     "WEBHCAT_SERVER-RESTART": ["NODEMANAGER-RESTART", "HIVE_SERVER-RESTART"],

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/beeline-log4j2.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/beeline-log4j2.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/beeline-log4j2.xml
deleted file mode 100755
index 03de64e..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/beeline-log4j2.xml
+++ /dev/null
@@ -1,62 +0,0 @@
-<configuration><property require-input="false">
-    <name>content</name>
-    <value>
-# 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.
-
-status = INFO
-name = BeelineLog4j2
-packages = org.apache.hadoop.hive.ql.log
-
-# list of properties
-property.hive.log.level = WARN
-property.hive.root.logger = console
-
-# list of all appenders
-appenders = console
-
-# console appender
-appender.console.type = Console
-appender.console.name = console
-appender.console.target = SYSTEM_ERR
-appender.console.layout.type = PatternLayout
-appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n
-
-# list of all loggers
-loggers = HiveConnection
-
-# HiveConnection logs useful info for dynamic service discovery
-logger.HiveConnection.name = org.apache.hive.jdbc.HiveConnection
-logger.HiveConnection.level = INFO
-
-# root logger
-rootLogger.level = ${sys:hive.log.level}
-rootLogger.appenderRefs = root
-rootLogger.appenderRef.root.ref = ${sys:hive.root.logger}
-  </value>
-    <description>Custom beeline-log4j2.properties</description>
-    <display-name>beeline-log4j template</display-name>
-    <filename>beeline-log4j2.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>content</type>
-        <show-property-name>false</show-property-name>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-atlas-application.properties.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-atlas-application.properties.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-atlas-application.properties.xml
deleted file mode 100755
index 7eb72ef..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-atlas-application.properties.xml
+++ /dev/null
@@ -1,67 +0,0 @@
-<configuration><property require-input="false">
-    <name>atlas.hook.hive.synchronous</name>
-    <value>false</value>
-    <description></description>
-    <filename>hive-atlas-application.properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>atlas.hook.hive.numRetries</name>
-    <value>3</value>
-    <description></description>
-    <filename>hive-atlas-application.properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>atlas.hook.hive.minThreads</name>
-    <value>5</value>
-    <description></description>
-    <filename>hive-atlas-application.properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>atlas.hook.hive.maxThreads</name>
-    <value>5</value>
-    <description></description>
-    <filename>hive-atlas-application.properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>atlas.hook.hive.keepAliveTime</name>
-    <value>10</value>
-    <description></description>
-    <filename>hive-atlas-application.properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>atlas.hook.hive.queueSize</name>
-    <value>1000</value>
-    <description></description>
-    <filename>hive-atlas-application.properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j2.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j2.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j2.xml
deleted file mode 100755
index c818d43..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-exec-log4j2.xml
+++ /dev/null
@@ -1,83 +0,0 @@
-<configuration><property require-input="false">
-    <name>content</name>
-    <value>
-# 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.
-
-status = INFO
-name = HiveExecLog4j2
-packages = org.apache.hadoop.hive.ql.log
-
-# list of properties
-property.hive.log.level = INFO
-property.hive.root.logger = FA
-property.hive.query.id = hadoop
-property.hive.log.dir = ${sys:java.io.tmpdir}/${sys:user.name}
-property.hive.log.file = ${sys:hive.query.id}.log
-
-# list of all appenders
-appenders = console, FA
-
-# console appender
-appender.console.type = Console
-appender.console.name = console
-appender.console.target = SYSTEM_ERR
-appender.console.layout.type = PatternLayout
-appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n
-
-# simple file appender
-appender.FA.type = File
-appender.FA.name = FA
-appender.FA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
-appender.FA.layout.type = PatternLayout
-appender.FA.layout.pattern = %d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n
-
-# list of all loggers
-loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX
-
-logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn
-logger.NIOServerCnxn.level = WARN
-
-logger.ClientCnxnSocketNIO.name = org.apache.zookeeper.ClientCnxnSocketNIO
-logger.ClientCnxnSocketNIO.level = WARN
-
-logger.DataNucleus.name = DataNucleus
-logger.DataNucleus.level = ERROR
-
-logger.Datastore.name = Datastore
-logger.Datastore.level = ERROR
-
-logger.JPOX.name = JPOX
-logger.JPOX.level = ERROR
-
-# root logger
-rootLogger.level = ${sys:hive.log.level}
-rootLogger.appenderRefs = root
-rootLogger.appenderRef.root.ref = ${sys:hive.root.logger}
-  </value>
-    <description>Custom hive-exec-log4j2.properties</description>
-    <display-name>hive-exec-log4j2 template</display-name>
-    <filename>hive-exec-log4j2.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>content</type>
-        <show-property-name>false</show-property-name>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-env.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-env.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-env.xml
deleted file mode 100755
index 7035283..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-env.xml
+++ /dev/null
@@ -1,373 +0,0 @@
-<configuration><property require-input="false">
-    <name>enable_hive_interactive</name>
-    <value>false</value>
-    <description>Enable or disable interactive query in this cluster.</description>
-    <display-name>Enable Interactive Query (Tech Preview)</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>value-list</type>
-        <overridable>false</overridable>
-        <entries>
-            <entry>
-                <value>true</value>
-                <label>Yes</label>
-            </entry>
-            <entry>
-                <value>false</value>
-                <label>No</label>
-            </entry>
-        </entries>
-        <selection-cardinality>1</selection-cardinality>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by>
-        <dependedByProperties>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.llap.daemon.num.executors</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>llap_queue_capacity</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>llap_heap_size</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>tez.am.resource.memory.mb</name>
-            <type>tez-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.llap.daemon.yarn.container.mb</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>capacity-scheduler</name>
-            <type>capacity-scheduler</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.server2.tez.sessions.per.default.queue</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.llap.io.memory.size</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>num_llap_nodes</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>slider_am_container_mb</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-    </property_depended_by>
-</property><property require-input="false">
-    <name>hive_server_interactive_host</name>
-    <value>localhost</value>
-    <description>HiveServer2 Interactive Host</description>
-    <display-name>HiveServer2 Interactive Host</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <overridable>false</overridable>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>llap_queue_capacity</name>
-    <value>0</value>
-    <description>Percentage of the cluster dedicated to interactive query.</description>
-    <display-name>% of Cluster Capacity</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-        <maximum>100</maximum>
-        <minimum>20</minimum>
-        <increment-step>1</increment-step>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </property>
-        <property>
-            <name>enable_hive_interactive</name>
-            <type>hive-interactive-env</type>
-        </property>
-    </depends-on>
-    <property_depended_by>
-        <dependedByProperties>
-            <name>hive.llap.daemon.num.executors</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>llap_heap_size</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>tez.am.resource.memory.mb</name>
-            <type>tez-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.llap.daemon.yarn.container.mb</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>capacity-scheduler</name>
-            <type>capacity-scheduler</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.server2.tez.sessions.per.default.queue</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.llap.io.memory.size</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>num_llap_nodes</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>slider_am_container_mb</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-    </property_depended_by>
-</property><property require-input="false">
-    <name>num_llap_nodes</name>
-    <value>1</value>
-    <description>The number of Hive LLAP daemons to run.</description>
-    <display-name>Number of LLAP Daemons</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-        <minimum>1</minimum>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </property>
-        <property>
-            <name>llap_queue_capacity</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>enable_hive_interactive</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>hive.server2.tez.sessions.per.default.queue</name>
-            <type>hive-interactive-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>num_retries_for_checking_llap_status</name>
-    <value>10</value>
-    <description>After starting LLAP app, retry count to check LLAP status before starting HiveServer2.</description>
-    <display-name>Number of retries while checking LLAP app status</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-        <maximum>20</maximum>
-        <minimum>0</minimum>
-        <increment-step>1</increment-step>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>llap_heap_size</name>
-    <value>0</value>
-    <description>Heap Size used by LLAP app.</description>
-    <display-name>LLAP heap size</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>tez.am.resource.memory.mb</name>
-            <type>tez-site</type>
-        </property>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </property>
-        <property>
-            <name>llap_queue_capacity</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>enable_hive_interactive</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>hive.server2.tez.sessions.per.default.queue</name>
-            <type>hive-interactive-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>slider_am_container_mb</name>
-    <value>341</value>
-    <description>Slider's app master container size in MB.</description>
-    <display-name>Slider AM container size</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-        <unit>MB</unit>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </property>
-        <property>
-            <name>llap_queue_capacity</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>enable_hive_interactive</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>hive.server2.tez.sessions.per.default.queue</name>
-            <type>hive-interactive-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>llap_log_level</name>
-    <value>INFO</value>
-    <description>LLAP app logging level (WARN/INFO/DEBUG/TRACE)</description>
-    <display-name>LLAP app logging level (WARN/INFO/DEBUG/TRACE)</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>llap_app_name</name>
-    <value>llap0</value>
-    <description>LLAP app name</description>
-    <display-name>LLAP app name</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>llap_java_opts</name>
-    <value>-XX:+AlwaysPreTouch {% if java_version &gt; 7 %}-XX:+UseG1GC -XX:TLABSize=8m -XX:+ResizeTLAB -XX:+UseNUMA -XX:+AggressiveOpts -XX:MetaspaceSize=1024m -XX:InitiatingHeapOccupancyPercent=80 -XX:MaxGCPauseMillis=200{% else %}-XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC{% endif %}</value>
-    <description>Java opts for llap application</description>
-    <display-name>LLAP app java opts</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>content</name>
-    <value>
-      if [ "$SERVICE" = "cli" ]; then
-      if [ -z "$DEBUG" ]; then
-      export HADOOP_OPTS="$HADOOP_OPTS -XX:NewRatio=12 -XX:MaxHeapFreeRatio=40 -XX:MinHeapFreeRatio=15 -XX:+UseParNewGC -XX:-UseGCOverheadLimit"
-      else
-      export HADOOP_OPTS="$HADOOP_OPTS -XX:NewRatio=12 -XX:MaxHeapFreeRatio=40 -XX:MinHeapFreeRatio=15 -XX:-UseGCOverheadLimit"
-      fi
-      fi
-
-      # The heap size of the jvm stared by hive shell script can be controlled via:
-
-      if [ "$SERVICE" = "metastore" ]; then
-      export HADOOP_HEAPSIZE={{hive_metastore_heapsize}} # Setting for HiveMetastore
-      else
-      export HADOOP_HEAPSIZE={{hive_heapsize}} # Setting for HiveServer2 and Client
-      fi
-
-      export HADOOP_CLIENT_OPTS="$HADOOP_CLIENT_OPTS  -Xmx${HADOOP_HEAPSIZE}m"
-
-      # Larger heap size may be required when running queries over large number of files or partitions.
-      # By default hive shell scripts use a heap size of 256 (MB).  Larger heap size would also be
-      # appropriate for hive server (hwi etc).
-
-
-      # Set HADOOP_HOME to point to a specific hadoop install directory
-      HADOOP_HOME=${HADOOP_HOME:-{{hadoop_home}}}
-
-      # Hive Configuration Directory can be controlled by:
-      export HIVE_CONF_DIR={{hive_server_interactive_conf_dir}}
-
-      # Add additional hcatalog jars
-      if [ "${HIVE_AUX_JARS_PATH}" != "" ]; then
-      export HIVE_AUX_JARS_PATH=${HIVE_AUX_JARS_PATH}
-      else
-      export HIVE_AUX_JARS_PATH=/usr/hdp/current/hive-server2-hive2/lib/hive-hcatalog-core.jar
-      fi
-
-      export METASTORE_PORT={{hive_metastore_port}}
-
-      # Spark assembly contains a conflicting copy of HiveConf from hive-1.2
-      export HIVE_SKIP_SPARK_ASSEMBLY=true
-
-    </value>
-    <description>This is the jinja template for hive-env.sh file</description>
-    <display-name>hive-interactive-env template</display-name>
-    <filename>hive-interactive-env.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>content</type>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-site.xml
deleted file mode 100755
index 27d3541..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-interactive-site.xml
+++ /dev/null
@@ -1,909 +0,0 @@
-<configuration><property require-input="false">
-    <name>hive.server2.thrift.port</name>
-    <value>10500</value>
-    <description>
-      TCP port number to listen on, default 10500.
-    </description>
-    <display-name>HiveServer2 Port</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-        <overridable>false</overridable>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.server2.thrift.http.port</name>
-    <value>10501</value>
-    <description>Port number of HiveServer2 Thrift interface when hive.server2.transport.mode is 'http'.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.server2.tez.sessions.per.default.queue</name>
-    <value>1</value>
-    <description>
-      The maximum number of queries the Hive Interactive cluster will be able to handle concurrently.
-    </description>
-    <display-name>Maximum Total Concurrent Queries</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-        <maximum>10</maximum>
-        <minimum>1</minimum>
-        <increment-step>1</increment-step>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </property>
-        <property>
-            <name>llap_queue_capacity</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>enable_hive_interactive</name>
-            <type>hive-interactive-env</type>
-        </property>
-    </depends-on>
-    <property_depended_by>
-        <dependedByProperties>
-            <name>hive.llap.daemon.num.executors</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>llap_heap_size</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>tez.am.resource.memory.mb</name>
-            <type>tez-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.llap.daemon.yarn.container.mb</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.llap.io.memory.size</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>num_llap_nodes</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>slider_am_container_mb</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-    </property_depended_by>
-</property><property require-input="false">
-    <name>hive.metastore.uris</name>
-    <value></value>
-    <description>Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <empty-value-valid>true</empty-value-valid>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.server2.enable.doAs</name>
-    <value>false</value>
-    <description>
-      Setting this property to true will have HiveServer2 execute
-      Hive operations as the user making the calls to it.
-    </description>
-    <display-name>Run as end user instead of Hive user</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>value-list</type>
-        <entries>
-            <entry>
-                <value>true</value>
-                <label>True</label>
-            </entry>
-            <entry>
-                <value>false</value>
-                <label>False</label>
-            </entry>
-        </entries>
-        <selection-cardinality>1</selection-cardinality>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>hive_security_authorization</name>
-            <type>hive-env</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.prewarm.enabled</name>
-    <value>false</value>
-    <description>Enables container prewarm for Tez (Hadoop 2 only)</description>
-    <display-name>Hold Containers to Reduce Latency</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>value-list</type>
-        <entries>
-            <entry>
-                <value>true</value>
-                <label>True</label>
-            </entry>
-            <entry>
-                <value>false</value>
-                <label>False</label>
-            </entry>
-        </entries>
-        <selection-cardinality>1</selection-cardinality>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.vectorized.execution.reduce.enabled</name>
-    <value>true</value>
-    <description>
-      This flag should be set to true to enable vectorized mode of the reduce-side of
-      query execution.
-    </description>
-    <display-name>Enable Reduce Vectorization</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>value-list</type>
-        <entries>
-            <entry>
-                <value>true</value>
-                <label>True</label>
-            </entry>
-            <entry>
-                <value>false</value>
-                <label>False</label>
-            </entry>
-        </entries>
-        <selection-cardinality>1</selection-cardinality>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.server2.tez.default.queues</name>
-    <value>default</value>
-    <description>
-      A list of comma separated values corresponding to YARN queues of the same name.
-      When HiveServer2 is launched in Tez mode, this configuration needs to be set
-      for multiple Tez sessions to run in parallel on the cluster.
-    </description>
-    <display-name>Default query queues</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>combo</type>
-        <entries>
-            <entry>
-                <value>default</value>
-                <label>Default</label>
-            </entry>
-        </entries>
-        <selection-cardinality>1+</selection-cardinality>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.server2.tez.initialize.default.sessions</name>
-    <value>true</value>
-    <description>
-      This flag is used in HiveServer2 to enable a user to use HiveServer2 without
-      turning on Tez for HiveServer2. The user could potentially want to run queries
-      over Tez without the pool of sessions.
-    </description>
-    <display-name>Start Tez session at Initialization</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>value-list</type>
-        <entries>
-            <entry>
-                <value>true</value>
-                <label>True</label>
-            </entry>
-            <entry>
-                <value>false</value>
-                <label>False</label>
-            </entry>
-        </entries>
-        <selection-cardinality>1</selection-cardinality>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.driver.parallel.compilation</name>
-    <value>true</value>
-    <description>
-      This flag allows HiveServer2 to compile queries in parallel.
-    </description>
-    <display-name>Compile queries in parallel</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>value-list</type>
-        <entries>
-            <entry>
-                <value>true</value>
-                <label>True</label>
-            </entry>
-            <entry>
-                <value>false</value>
-                <label>False</label>
-            </entry>
-        </entries>
-        <selection-cardinality>1</selection-cardinality>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.server2.webui.port</name>
-    <value>10502</value>
-    <description>Web UI port address</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.server2.webui.use.ssl</name>
-    <value>false</value>
-    <description>Enable SSL for HiveServer2 Interactive</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.server2.zookeeper.namespace</name>
-    <value>hiveserver2-hive2</value>
-    <description>The parent node in ZooKeeper used by HiveServer2 when supporting dynamic service discovery.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.daemon.queue.name</name>
-    <value>default</value>
-    <description>Choose the YARN queue in this cluster that is dedicated to interactive query.</description>
-    <display-name>Interactive Query Queue</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>combo</type>
-        <entries>
-            <entry>
-                <value>default</value>
-                <label>Default</label>
-            </entry>
-        </entries>
-        <selection-cardinality>1</selection-cardinality>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>enable_hive_interactive</name>
-            <type>hive-interactive-env</type>
-        </property>
-    </depends-on>
-    <property_depended_by>
-        <dependedByProperties>
-            <name>hive.server2.tez.default.queues</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.llap.daemon.num.executors</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>llap_queue_capacity</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>llap_heap_size</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>tez.am.resource.memory.mb</name>
-            <type>tez-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.llap.daemon.yarn.container.mb</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.server2.tez.sessions.per.default.queue</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>hive.llap.io.memory.size</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>num_llap_nodes</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-        <dependedByProperties>
-            <name>slider_am_container_mb</name>
-            <type>hive-interactive-env</type>
-        </dependedByProperties>
-    </property_depended_by>
-</property><property require-input="false">
-    <name>hive.llap.daemon.yarn.shuffle.port</name>
-    <value>15551</value>
-    <description>YARN shuffle port for LLAP-daemon-hosted shuffle.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.execution.engine</name>
-    <value>tez</value>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.execution.mode</name>
-    <value>llap</value>
-    <description>Chooses whether query fragments will run in container or in llap</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.execution.mode</name>
-    <value>all</value>
-    <description>Chooses which fragments of a query will run in llap</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.io.enabled</name>
-    <value>true</value>
-    <description>Whether the LLAP IO layer is enabled.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on>
-        <property>
-            <name>hive.llap.io.memory.size</name>
-            <type>hive-interactive-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.io.use.lrfu</name>
-    <value>true</value>
-    <description>Whether ORC low-level cache should use LRFU cache policy instead of default (FIFO).</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.auto.allow.uber</name>
-    <value>false</value>
-    <description>Whether or not to allow the planner to run vertices in the AM.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.object.cache.enabled</name>
-    <value>true</value>
-    <description>Cache objects (plans, hashtables, etc) in llap</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.tez.input.generate.consistent.splits</name>
-    <value>true</value>
-    <description>Whether to generate consistent split locations when generating splits in the AM</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.client.consistent.splits</name>
-    <value>true</value>
-    <description>
-      Whether to setup split locations to match nodes on which llap daemons are running,
-      instead of using the locations provided by the split itself.
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.task.scheduler.locality.delay</name>
-    <value>-1</value>
-    <description>
-      Amount of time to wait before allocating a request which contains location information,
-      to a location other than the ones requested. Set to -1 for an infinite delay, 0
-      for no delay.
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.exec.orc.split.strategy</name>
-    <value>HYBRID</value>
-    <description>
-      This is not a user level config. BI strategy is used when the requirement is to spend less time in split generation
-      as opposed to query execution (split generation does not read or cache file footers).
-      ETL strategy is used when spending little more time in split generation is acceptable
-      (split generation reads and caches file footers). HYBRID chooses between the above strategies
-      based on heuristics.
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.daemon.service.hosts</name>
-    <value>@llap0</value>
-    <description>
-      Explicitly specified hosts to use for LLAP scheduling. Useful for testing. By default,
-      YARN registry is used.
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.daemon.allow.permanent.fns</name>
-    <value>false</value>
-    <description>Whether LLAP daemon should localize the resources for permanent UDFs.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.io.memory.size</name>
-    <value>0</value>
-    <description>The amount of memory reserved for Hive's optimized in-memory cache.</description>
-    <display-name>In-Memory Cache per Daemon</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-        <unit>MB</unit>
-        <overridable>false</overridable>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>tez.am.resource.memory.mb</name>
-            <type>tez-site</type>
-        </property>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </property>
-        <property>
-            <name>llap_queue_capacity</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>enable_hive_interactive</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>hive.server2.tez.sessions.per.default.queue</name>
-            <type>hive-interactive-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by>
-        <dependedByProperties>
-            <name>hive.llap.io.enabled</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-    </property_depended_by>
-</property><property require-input="false">
-    <name>hive.llap.daemon.num.executors</name>
-    <value>1</value>
-    <description>The maximum number of CPUs a single LLAP daemon will use. Usually this should be equal to the number of available CPUs.</description>
-    <display-name>Maximum CPUs per Daemon</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>tez.am.resource.memory.mb</name>
-            <type>tez-site</type>
-        </property>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </property>
-        <property>
-            <name>llap_queue_capacity</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>enable_hive_interactive</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>hive.server2.tez.sessions.per.default.queue</name>
-            <type>hive-interactive-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by>
-        <dependedByProperties>
-            <name>hive.llap.io.threadpool.size</name>
-            <type>hive-interactive-site</type>
-        </dependedByProperties>
-    </property_depended_by>
-</property><property require-input="false">
-    <name>hive.llap.daemon.vcpus.per.instance</name>
-    <value>${hive.llap.daemon.num.executors}</value>
-    <description>The total number of vcpus to use for the executors inside LLAP.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.daemon.yarn.container.mb</name>
-    <value>341</value>
-    <description>Total memory used by individual LLAP daemons. This includes memory for the cache as well as for the query execution.</description>
-    <display-name>Memory per daemon</display-name>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-        <unit>MB</unit>
-        <overridable>false</overridable>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </property>
-        <property>
-            <name>llap_queue_capacity</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>enable_hive_interactive</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>hive.server2.tez.sessions.per.default.queue</name>
-            <type>hive-interactive-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>llap.shuffle.connection-keep-alive.enable</name>
-    <value>true</value>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>llap.shuffle.connection-keep-alive.timeout</name>
-    <value>60</value>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.io.threadpool.size</name>
-    <value>2</value>
-    <description>Specify the number of threads to use for low-level IO thread pool.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on>
-        <property>
-            <name>hive.llap.daemon.num.executors</name>
-            <type>hive-interactive-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.daemon.rpc.port</name>
-    <value>15001</value>
-    <description>The LLAP daemon RPC port.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.management.rpc.port</name>
-    <value>15004</value>
-    <description>RPC port for LLAP daemon management service.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.daemon.task.scheduler.enable.preemption</name>
-    <value>true</value>
-    <description>hive.llap.daemon.task.scheduler.enable.preemption</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.tez.exec.print.summary</name>
-    <value>true</value>
-    <description>Display breakdown of execution steps, for every query executed by the shell.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.vectorized.execution.mapjoin.native.enabled</name>
-    <value>true</value>
-    <description>
-      This flag should be set to true to enable native (i.e. non-pass through) vectorization
-      of queries using MapJoin.
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.vectorized.execution.mapjoin.minmax.enabled</name>
-    <value>true</value>
-    <description>
-      This flag should be set to true to enable vector map join hash tables to
-      use max / max filtering for integer join queries using MapJoin.
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.vectorized.execution.mapjoin.native.fast.hashtable.enabled</name>
-    <value>true</value>
-    <description>
-      This flag should be set to true to enable use of native fast vector map join hash tables in
-      queries using MapJoin.
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.mapjoin.hybridgrace.hashtable</name>
-    <value>false</value>
-    <description>Whether to use hybrid grace hash join as the join method for mapjoin. Tez only.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.tez.bucket.pruning</name>
-    <value>true</value>
-    <description>
-      When pruning is enabled, filters on bucket columns will be processed by
-      filtering the splits against a bitset of included buckets. This needs predicates
-      produced by hive.optimize.ppd and hive.optimize.index.filters.
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.optimize.dynamic.partition.hashjoin</name>
-    <value>true</value>
-    <description>
-      Whether to enable dynamically partitioned hash join optimization.
-      This setting is also dependent on enabling hive.auto.convert.join
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.zk.sm.connectionString</name>
-    <value>localhost:2181</value>
-    <description>ZooKeeper connection string for ZooKeeper SecretManager.</description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on>
-        <property>
-            <name>clientPort</name>
-            <type>zoo.cfg</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.llap.io.memory.mode</name>
-    <value></value>
-    <description>
-      LLAP IO memory usage; 'cache' (the default) uses data and metadata cache with a
-      custom off-heap allocator, 'allocator' uses the custom allocator without the caches,
-      'none' doesn't use either (this mode may result in significant performance degradation)
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <empty-value-valid>true</empty-value-valid>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.metastore.event.listeners</name>
-    <value></value>
-    <description>
-      Listeners for metastore events
-    </description>
-    <filename>hive-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <empty-value-valid>true</empty-value-valid>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j2.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j2.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j2.xml
deleted file mode 100755
index 798063b..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-log4j2.xml
+++ /dev/null
@@ -1,90 +0,0 @@
-<configuration><property require-input="false">
-    <name>content</name>
-    <value>
-# 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.
-
-status = INFO
-name = HiveLog4j2
-packages = org.apache.hadoop.hive.ql.log
-
-# list of properties
-property.hive.log.level = INFO
-property.hive.root.logger = DRFA
-property.hive.log.dir = ${sys:java.io.tmpdir}/${sys:user.name}
-property.hive.log.file = hive.log
-
-# list of all appenders
-appenders = console, DRFA
-
-# console appender
-appender.console.type = Console
-appender.console.name = console
-appender.console.target = SYSTEM_ERR
-appender.console.layout.type = PatternLayout
-appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n
-
-# daily rolling file appender
-appender.DRFA.type = RollingFile
-appender.DRFA.name = DRFA
-appender.DRFA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
-# Use %pid in the filePattern to append process-id@host-name to the filename if you want separate log files for different CLI session
-appender.DRFA.filePattern = ${sys:hive.log.dir}/${sys:hive.log.file}.%d{yyyy-MM-dd}.gz
-appender.DRFA.layout.type = PatternLayout
-appender.DRFA.layout.pattern = %d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n
-appender.DRFA.policies.type = Policies
-appender.DRFA.policies.time.type = TimeBasedTriggeringPolicy
-appender.DRFA.policies.time.interval = 1
-appender.DRFA.policies.time.modulate = true
-appender.DRFA.strategy.type = DefaultRolloverStrategy
-appender.DRFA.strategy.max = 30
-
-# list of all loggers
-loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX
-
-logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn
-logger.NIOServerCnxn.level = WARN
-
-logger.ClientCnxnSocketNIO.name = org.apache.zookeeper.ClientCnxnSocketNIO
-logger.ClientCnxnSocketNIO.level = WARN
-
-logger.DataNucleus.name = DataNucleus
-logger.DataNucleus.level = ERROR
-
-logger.Datastore.name = Datastore
-logger.Datastore.level = ERROR
-
-logger.JPOX.name = JPOX
-logger.JPOX.level = ERROR
-
-# root logger
-rootLogger.level = ${sys:hive.log.level}
-rootLogger.appenderRefs = root
-rootLogger.appenderRef.root.ref = ${sys:hive.root.logger}
-  </value>
-    <description>Custom hive-log4j2.properties</description>
-    <display-name>hive-log4j2 template</display-name>
-    <filename>hive-log4j2.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>content</type>
-        <show-property-name>false</show-property-name>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-interactive-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-interactive-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-interactive-site.xml
deleted file mode 100755
index 6954e56..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-interactive-site.xml
+++ /dev/null
@@ -1,56 +0,0 @@
-<configuration><property require-input="false">
-    <name>hive.metastore.metrics.enabled</name>
-    <value>true</value>
-    <filename>hiveserver2-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>boolean</type>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.service.metrics.reporter</name>
-    <value>JSON_FILE, JMX, HADOOP2</value>
-    <filename>hiveserver2-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.service.metrics.hadoop2.component</name>
-    <value>hiveserver2</value>
-    <filename>hiveserver2-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.service.metrics.file.location</name>
-    <value>/var/log/hive/hiveserver2Interactive-report.json</value>
-    <filename>hiveserver2-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>hive.async.log.enabled</name>
-    <value>false</value>
-    <description>Whether to enable Log4j2's asynchronous logging. Asynchronous logging can give significant performance improvement as logging will be handled in separate thread that uses LMAX disruptor queue for buffering log messages. Refer https://logging.apache.org/log4j/2.x/manual/async.html for benefits and drawbacks.</description>
-    <filename>hiveserver2-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>boolean</type>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-cli-log4j2.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-cli-log4j2.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-cli-log4j2.xml
deleted file mode 100755
index b7f6523..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-cli-log4j2.xml
+++ /dev/null
@@ -1,91 +0,0 @@
-<configuration><property require-input="false">
-    <name>content</name>
-    <value>
-# 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.
-
-status = WARN
-name = LlapCliLog4j2
-packages = org.apache.hadoop.hive.ql.log
-
-# list of properties
-property.hive.log.level = INFO
-property.hive.root.logger = console
-property.hive.log.dir = ${sys:java.io.tmpdir}/${sys:user.name}
-property.hive.log.file = llap-cli.log
-
-# list of all appenders
-appenders = console, DRFA
-
-# console appender
-appender.console.type = Console
-appender.console.name = console
-appender.console.target = SYSTEM_ERR
-appender.console.layout.type = PatternLayout
-appender.console.layout.pattern = %p %c{2}: %m%n
-
-# daily rolling file appender
-appender.DRFA.type = RollingRandomAccessFile
-appender.DRFA.name = DRFA
-appender.DRFA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
-# Use %pid in the filePattern to append process-id@host-name to the filename if you want separate log files for different CLI session
-appender.DRFA.filePattern = ${sys:hive.log.dir}/${sys:hive.log.file}.%d{yyyy-MM-dd}
-appender.DRFA.layout.type = PatternLayout
-appender.DRFA.layout.pattern = %d{ISO8601} %5p [%t] %c{2}: %m%n
-appender.DRFA.policies.type = Policies
-appender.DRFA.policies.time.type = TimeBasedTriggeringPolicy
-appender.DRFA.policies.time.interval = 1
-appender.DRFA.policies.time.modulate = true
-appender.DRFA.strategy.type = DefaultRolloverStrategy
-appender.DRFA.strategy.max = 30
-
-# list of all loggers
-loggers = ZooKeeper, DataNucleus, Datastore, JPOX, HadoopConf
-
-logger.ZooKeeper.name = org.apache.zookeeper
-logger.ZooKeeper.level = WARN
-
-logger.DataNucleus.name = DataNucleus
-logger.DataNucleus.level = ERROR
-
-logger.Datastore.name = Datastore
-logger.Datastore.level = ERROR
-
-logger.JPOX.name = JPOX
-logger.JPOX.level = ERROR
-
-logger.HadoopConf.name = org.apache.hadoop.conf.Configuration
-logger.HadoopConf.level = ERROR
-
-# root logger
-rootLogger.level = ${sys:hive.log.level}
-rootLogger.appenderRefs = root, DRFA
-rootLogger.appenderRef.root.ref = ${sys:hive.root.logger}
-rootLogger.appenderRef.DRFA.ref = DRFA
-  </value>
-    <description>Custom llap-cli-log4j2.properties</description>
-    <display-name>llap-cli-log4j2 template</display-name>
-    <filename>llap-cli-log4j2.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>content</type>
-        <show-property-name>false</show-property-name>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-daemon-log4j.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-daemon-log4j.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-daemon-log4j.xml
deleted file mode 100755
index 30c31be..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-daemon-log4j.xml
+++ /dev/null
@@ -1,158 +0,0 @@
-<configuration><property require-input="false">
-    <name>content</name>
-    <value>
-# 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.
-
-
-# This is the log4j2 properties file used by llap-daemons. There's several loggers defined, which
-# can be selected while configuring LLAP.
-# Based on the one selected - UI links etc need to be manipulated in the system.
-# Note: Some names and logic is common to this file and llap LogHelpers. Make sure to change that
-# as well, if changing this file.
-
-status = INFO
-name = LlapDaemonLog4j2
-packages = org.apache.hadoop.hive.ql.log
-
-# list of properties
-property.llap.daemon.log.level = INFO
-property.llap.daemon.root.logger = console
-property.llap.daemon.log.dir = .
-property.llap.daemon.log.file = llapdaemon.log
-property.llap.daemon.historylog.file = llapdaemon_history.log
-property.llap.daemon.log.maxfilesize = 256MB
-property.llap.daemon.log.maxbackupindex = 240
-
-# list of all appenders
-appenders = console, RFA, HISTORYAPPENDER, query-routing
-
-# console appender
-appender.console.type = Console
-appender.console.name = console
-appender.console.target = SYSTEM_ERR
-appender.console.layout.type = PatternLayout
-appender.console.layout.pattern = %d{ISO8601} %5p [%t (%X{fragmentId})] %c{2}: %m%n
-
-# rolling file appender
-appender.RFA.type = RollingRandomAccessFile
-appender.RFA.name = RFA
-appender.RFA.fileName = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.log.file}
-appender.RFA.filePattern = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.log.file}_%d{yyyy-MM-dd-HH}_%i.done
-appender.RFA.layout.type = PatternLayout
-appender.RFA.layout.pattern = %d{ISO8601} %-5p [%t (%X{fragmentId})] %c: %m%n
-appender.RFA.policies.type = Policies
-appender.RFA.policies.time.type = TimeBasedTriggeringPolicy
-appender.RFA.policies.time.interval = 1
-appender.RFA.policies.time.modulate = true
-appender.RFA.policies.size.type = SizeBasedTriggeringPolicy
-appender.RFA.policies.size.size = ${sys:llap.daemon.log.maxfilesize}
-appender.RFA.strategy.type = DefaultRolloverStrategy
-appender.RFA.strategy.max = ${sys:llap.daemon.log.maxbackupindex}
-
-# history file appender
-appender.HISTORYAPPENDER.type = RollingRandomAccessFile
-appender.HISTORYAPPENDER.name = HISTORYAPPENDER
-appender.HISTORYAPPENDER.fileName = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.historylog.file}
-appender.HISTORYAPPENDER.filePattern = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.historylog.file}_%d{yyyy-MM-dd}_%i.done
-appender.HISTORYAPPENDER.layout.type = PatternLayout
-appender.HISTORYAPPENDER.layout.pattern = %m%n
-appender.HISTORYAPPENDER.policies.type = Policies
-appender.HISTORYAPPENDER.policies.size.type = SizeBasedTriggeringPolicy
-appender.HISTORYAPPENDER.policies.size.size = ${sys:llap.daemon.log.maxfilesize}
-appender.HISTORYAPPENDER.policies.time.type = TimeBasedTriggeringPolicy
-appender.HISTORYAPPENDER.policies.time.interval = 1
-appender.HISTORYAPPENDER.policies.time.modulate = true
-appender.HISTORYAPPENDER.strategy.type = DefaultRolloverStrategy
-appender.HISTORYAPPENDER.strategy.max = ${sys:llap.daemon.log.maxbackupindex}
-
-# queryId based routing file appender
-appender.query-routing.type = Routing
-appender.query-routing.name = query-routing
-appender.query-routing.routes.type = Routes
-appender.query-routing.routes.pattern = $${ctx:queryId}
-#Purge polciy for query-based Routing Appender
-appender.query-routing.purgePolicy.type = LlapRoutingAppenderPurgePolicy
-# Note: Do not change this name without changing the corresponding entry in LlapConstants
-appender.query-routing.purgePolicy.name = llapLogPurgerQueryRouting
-# default route
-appender.query-routing.routes.route-default.type = Route
-appender.query-routing.routes.route-default.key = $${ctx:queryId}
-appender.query-routing.routes.route-default.ref = RFA
-# queryId based route
-appender.query-routing.routes.route-mdc.type = Route
-appender.query-routing.routes.route-mdc.file-mdc.type = LlapWrappedAppender
-appender.query-routing.routes.route-mdc.file-mdc.name = IrrelevantName-query-routing
-appender.query-routing.routes.route-mdc.file-mdc.app.type = RandomAccessFile
-appender.query-routing.routes.route-mdc.file-mdc.app.name = file-mdc
-appender.query-routing.routes.route-mdc.file-mdc.app.fileName = ${sys:llap.daemon.log.dir}/${ctx:queryId}-${ctx:dagId}.log
-appender.query-routing.routes.route-mdc.file-mdc.app.layout.type = PatternLayout
-appender.query-routing.routes.route-mdc.file-mdc.app.layout.pattern = %d{ISO8601} %5p [%t (%X{fragmentId})] %c{2}: %m%n
-
-# list of all loggers
-loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX, HistoryLogger, LlapIoImpl, LlapIoOrc, LlapIoCache, LlapIoLocking
-
-logger.LlapIoImpl.name = LlapIoImpl
-logger.LlapIoImpl.level = INFO
-
-logger.LlapIoOrc.name = LlapIoOrc
-logger.LlapIoOrc.level = WARN
-
-logger.LlapIoCache.name = LlapIoCache
-logger.LlapIOCache.level = WARN
-
-logger.LlapIoLocking.name = LlapIoLocking
-logger.LlapIoLocking.level = WARN
-
-logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn
-logger.NIOServerCnxn.level = WARN
-
-logger.ClientCnxnSocketNIO.name = org.apache.zookeeper.ClientCnxnSocketNIO
-logger.ClientCnxnSocketNIO.level = WARN
-
-logger.DataNucleus.name = DataNucleus
-logger.DataNucleus.level = ERROR
-
-logger.Datastore.name = Datastore
-logger.Datastore.level = ERROR
-
-logger.JPOX.name = JPOX
-logger.JPOX.level = ERROR
-
-logger.HistoryLogger.name = org.apache.hadoop.hive.llap.daemon.HistoryLogger
-logger.HistoryLogger.level = INFO
-logger.HistoryLogger.additivity = false
-logger.HistoryLogger.appenderRefs = HistoryAppender
-logger.HistoryLogger.appenderRef.HistoryAppender.ref = HISTORYAPPENDER
-
-# root logger
-rootLogger.level = ${sys:llap.daemon.log.level}
-rootLogger.appenderRefs = root
-rootLogger.appenderRef.root.ref = ${sys:llap.daemon.root.logger}
-  </value>
-    <description>Custom llap-daemon-log4j2.properties</description>
-    <display-name>llap-deamon-log4j template</display-name>
-    <filename>llap-daemon-log4j.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>content</type>
-        <show-property-name>false</show-property-name>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-audit.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-audit.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-audit.xml
deleted file mode 100755
index 9e74aa0..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-audit.xml
+++ /dev/null
@@ -1,136 +0,0 @@
-<configuration><property require-input="false">
-    <name>xasecure.audit.is.enabled</name>
-    <value>true</value>
-    <description>Is Audit enabled?</description>
-    <filename>ranger-hive-audit.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.audit.destination.hdfs</name>
-    <value>true</value>
-    <description>Is Audit to HDFS enabled?</description>
-    <display-name>Audit to HDFS</display-name>
-    <filename>ranger-hive-audit.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>boolean</type>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>xasecure.audit.destination.hdfs</name>
-            <type>ranger-env</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.audit.destination.hdfs.dir</name>
-    <value>hdfs://NAMENODE_HOSTNAME:8020/ranger/audit</value>
-    <description>HDFS folder to write audit to, make sure the service user has requried permissions</description>
-    <filename>ranger-hive-audit.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on>
-        <property>
-            <name>xasecure.audit.destination.hdfs.dir</name>
-            <type>ranger-env</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.audit.destination.hdfs.batch.filespool.dir</name>
-    <value>/var/log/hive/audit/hdfs/spool</value>
-    <description>/var/log/hive/audit/hdfs/spool</description>
-    <filename>ranger-hive-audit.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.audit.destination.solr</name>
-    <value>false</value>
-    <description>Is Solr audit enabled?</description>
-    <display-name>Audit to SOLR</display-name>
-    <filename>ranger-hive-audit.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>boolean</type>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>xasecure.audit.destination.solr</name>
-            <type>ranger-env</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.audit.destination.solr.urls</name>
-    <value></value>
-    <description>Solr URL</description>
-    <filename>ranger-hive-audit.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <empty-value-valid>true</empty-value-valid>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>ranger.audit.solr.urls</name>
-            <type>ranger-admin-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.audit.destination.solr.zookeepers</name>
-    <value>NONE</value>
-    <description>Solr Zookeeper string</description>
-    <filename>ranger-hive-audit.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on>
-        <property>
-            <name>ranger.audit.solr.zookeepers</name>
-            <type>ranger-admin-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.audit.destination.solr.batch.filespool.dir</name>
-    <value>/var/log/hive/audit/solr/spool</value>
-    <description>/var/log/hive/audit/solr/spool</description>
-    <filename>ranger-hive-audit.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.audit.provider.summary.enabled</name>
-    <value>false</value>
-    <description>Enable Summary audit?</description>
-    <display-name>Audit provider summary enabled</display-name>
-    <filename>ranger-hive-audit.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>boolean</type>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file


[33/50] [abbrv] bigtop git commit: ODPI-187. ODPi reference implementation Ambari deployment fails for extra services

Posted by rv...@apache.org.
ODPI-187. ODPi reference implementation Ambari deployment fails for extra services


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

Branch: refs/heads/BIGTOP-2666
Commit: 80fe9cbd84a0a2930f1edfc63e9be7f59775cf36
Parents: 3bc96f8
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Tue Nov 1 08:59:40 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:03 2017 -0800

----------------------------------------------------------------------
 .../ODPi/1.0/configuration/cluster-env.xml      |   2 +-
 .../ODPi/1.0/properties/stack_features.json     | 257 -------------------
 .../ambari/ODPi/1.0/properties/stack_tools.json |   4 +-
 .../ambari/ODPi/1.0/services/YARN/metainfo.xml  |  12 +-
 odpi.bom                                        |   2 +-
 5 files changed, 10 insertions(+), 267 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/80fe9cbd/bigtop-packages/src/common/ambari/ODPi/1.0/configuration/cluster-env.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/configuration/cluster-env.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/configuration/cluster-env.xml
index 81cb175..61274b6 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/configuration/cluster-env.xml
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/configuration/cluster-env.xml
@@ -196,7 +196,7 @@ gpgcheck=0</value>
   </property>
   <property>
     <name>stack_root</name>
-    <value>/usr/hdp</value>
+    <value>/usr/odpi</value>
     <description>Stack root folder</description>
     <value-attributes>
       <read-only>true</read-only>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/80fe9cbd/bigtop-packages/src/common/ambari/ODPi/1.0/properties/stack_features.json
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/properties/stack_features.json b/bigtop-packages/src/common/ambari/ODPi/1.0/properties/stack_features.json
index 8c838db..0c3e305 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/properties/stack_features.json
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/properties/stack_features.json
@@ -12,175 +12,16 @@
       "min_version": "2.2.1.0"
     },
     {
-      "name": "express_upgrade",
-      "description": "Express upgrade support",
-      "min_version": "2.1.0.0"
-    },
-    {
-      "name": "rolling_upgrade",
-      "description": "Rolling upgrade support",
-      "min_version": "2.2.0.0"
-    },
-    {
-      "name": "config_versioning",
-      "description": "Configurable versions support",
-      "min_version": "2.3.0.0"
-    },
-    {
-      "name": "datanode_non_root",
-      "description": "DataNode running as non-root support (AMBARI-7615)",
-      "min_version": "2.2.0.0"
-    },
-    {
-      "name": "remove_ranger_hdfs_plugin_env",
-      "description": "HDFS removes Ranger env files (AMBARI-14299)",
-      "min_version": "2.3.0.0"
-    },
-    {
-      "name": "ranger",
-      "description": "Ranger Service support",
-      "min_version": "2.2.0.0"
-    },
-    {
-      "name": "ranger_tagsync_component",
-      "description": "Ranger Tagsync component support (AMBARI-14383)",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "phoenix",
-      "description": "Phoenix Service support",
-      "min_version": "2.3.0.0"
-    },
-    {
-      "name": "nfs",
-      "description": "NFS support",
-      "min_version": "2.3.0.0"
-    },
-    {
-      "name": "tez_for_spark",
-      "description": "Tez dependency for Spark",
-      "min_version": "2.2.0.0",
-      "max_version": "2.3.0.0"
-    },
-    {
-      "name": "timeline_state_store",
-      "description": "Yarn application timeline-service supports state store property (AMBARI-11442)",
-      "min_version": "2.2.0.0"
-    },
-    {
       "name": "copy_tarball_to_hdfs",
       "description": "Copy tarball to HDFS support (AMBARI-12113)",
       "min_version": "2.2.0.0"
     },
     {
-      "name": "spark_16plus",
-      "description": "Spark 1.6+",
-      "min_version": "2.4.0.0"
-    },
-    {
-      "name": "spark_thriftserver",
-      "description": "Spark Thrift Server",
-      "min_version": "2.3.2.0"
-    },
-    {
-      "name": "storm_kerberos",
-      "description": "Storm Kerberos support (AMBARI-7570)",
-      "min_version": "2.2.0.0"
-    },
-    {
-      "name": "storm_ams",
-      "description": "Storm AMS integration (AMBARI-10710)",
-      "min_version": "2.2.0.0"
-    },
-    {
-      "name": "create_kafka_broker_id",
-      "description": "Ambari should create Kafka Broker Id (AMBARI-12678)",
-      "min_version": "2.2.0.0",
-      "max_version": "2.3.0.0"
-    },
-    {
-      "name": "kafka_listeners",
-      "description": "Kafka listeners (AMBARI-10984)",
-      "min_version": "2.3.0.0"
-    },
-    {
-      "name": "kafka_kerberos",
-      "description": "Kafka Kerberos support (AMBARI-10984)",
-      "min_version": "2.3.0.0"
-    },
-    {
-      "name": "pig_on_tez",
-      "description": "Pig on Tez support (AMBARI-7863)",
-      "min_version": "2.2.0.0"
-    },
-    {
-      "name": "ranger_usersync_non_root",
-      "description": "Ranger Usersync as non-root user (AMBARI-10416)",
-      "min_version": "2.3.0.0"
-    },
-    {
-      "name": "ranger_audit_db_support",
-      "description": "Ranger Audit to DB support",
-      "min_version": "2.2.0.0",
-      "max_version": "2.5.0.0"
-    },
-    {
-      "name": "accumulo_kerberos_user_auth",
-      "description": "Accumulo Kerberos User Auth (AMBARI-10163)",
-      "min_version": "2.3.0.0"
-    },
-    {
-      "name": "knox_versioned_data_dir",
-      "description": "Use versioned data dir for Knox (AMBARI-13164)",
-      "min_version": "2.3.2.0"
-    },
-    {
-      "name": "knox_sso_topology",
-      "description": "Knox SSO Topology support (AMBARI-13975)",
-      "min_version": "2.3.8.0"
-    },
-    {
-      "name": "atlas_rolling_upgrade",
-      "description": "Rolling upgrade support for Atlas",
-      "min_version": "2.3.0.0"
-    },
-    {
-      "name": "oozie_admin_user",
-      "description": "Oozie install user as an Oozie admin user (AMBARI-7976)",
-      "min_version": "2.2.0.0"
-    },
-    {
-      "name": "oozie_create_hive_tez_configs",
-      "description": "Oozie create configs for Ambari Hive and Tez deployments (AMBARI-8074)",
-      "min_version": "2.2.0.0"
-    },
-    {
-      "name": "oozie_setup_shared_lib",
-      "description": "Oozie setup tools used to shared Oozie lib to HDFS (AMBARI-7240)",
-      "min_version": "2.2.0.0"
-    },
-    {
-      "name": "oozie_host_kerberos",
-      "description": "Oozie in secured clusters uses _HOST in Kerberos principal (AMBARI-9775)",
-      "min_version": "2.0.0.0",
-      "max_version": "2.2.0.0"
-    },
-    {
-      "name": "falcon_extensions",
-      "description": "Falcon Extension",
-      "min_version": "2.5.0.0"
-    },
-    {
       "name": "hive_metastore_upgrade_schema",
       "description": "Hive metastore upgrade schema support (AMBARI-11176)",
       "min_version": "2.3.0.0"
      },
     {
-      "name": "hive_server_interactive",
-      "description": "Hive server interactive support (AMBARI-15573)",
-      "min_version": "2.5.0.0"
-     },
-    {
       "name": "hive_webhcat_specific_configs",
       "description": "Hive webhcat specific configurations support (AMBARI-12364)",
       "min_version": "2.3.0.0"
@@ -202,107 +43,9 @@
       "min_version": "2.2.0.0"
     },
     {
-      "name": "ranger_kms_hsm_support",
-      "description": "Ranger KMS HSM support (AMBARI-15752)",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "ranger_log4j_support",
-      "description": "Ranger supporting log-4j properties (AMBARI-15681)",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "ranger_kerberos_support",
-      "description": "Ranger Kerberos support",
-      "min_version": "2.5.0.0"
-    },
-    {
       "name": "hive_metastore_site_support",
       "description": "Hive Metastore site support",
       "min_version": "2.5.0.0"
-    },
-    {
-      "name": "ranger_usersync_password_jceks",
-      "description": "Saving Ranger Usersync credentials in jceks",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "ranger_install_infra_client",
-      "description": "Ambari Infra Service support",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "falcon_atlas_support_2_3",
-      "description": "Falcon Atlas integration support for 2.3 stack",
-      "min_version": "2.3.99.0",
-      "max_version": "2.4.0.0"
-    },
-    {
-      "name": "falcon_atlas_support",
-      "description": "Falcon Atlas integration",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "hbase_home_directory",
-      "description": "Hbase home directory in HDFS needed for HBASE backup",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "spark_livy",
-      "description": "Livy as slave component of spark",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "atlas_ranger_plugin_support",
-      "description": "Atlas Ranger plugin support",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "atlas_conf_dir_in_path",
-      "description": "Prepend the Atlas conf dir (/etc/atlas/conf) to the classpath of Storm and Falcon",
-      "min_version": "2.3.0.0",
-      "max_version": "2.4.99.99"
-    },
-    {
-      "name": "atlas_upgrade_support",
-      "description": "Atlas supports express and rolling upgrades",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "atlas_hook_support",
-      "description": "Atlas support for hooks in Hive, Storm, Falcon, and Sqoop",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "ranger_pid_support",
-      "description": "Ranger Service support pid generation AMBARI-16756",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "ranger_kms_pid_support",
-      "description": "Ranger KMS Service support pid generation",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "ranger_admin_password_change",
-      "description": "Allow ranger admin credentials to be specified during cluster creation (AMBARI-17000)",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "storm_metrics_apache_classes",
-      "description": "Metrics sink for Storm that uses Apache class names",
-      "min_version": "2.5.0.0"
-    },
-    {
-      "name": "spark_java_opts_support",
-      "description": "Allow Spark to generate java-opts file",
-      "min_version": "2.2.0.0",
-      "max_version": "2.4.0.0"
-    },
-    {
-      "name": "atlas_hbase_setup",
-      "description": "Use script to create Atlas tables in Hbase and set permissions for Atlas user.",
-      "min_version": "2.5.0.0"
     }
   ]
 }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/80fe9cbd/bigtop-packages/src/common/ambari/ODPi/1.0/properties/stack_tools.json
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/properties/stack_tools.json b/bigtop-packages/src/common/ambari/ODPi/1.0/properties/stack_tools.json
index d1aab4b..c3df235 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/properties/stack_tools.json
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/properties/stack_tools.json
@@ -1,4 +1,4 @@
 {
-  "stack_selector": ["hdp-select", "/usr/bin/hdp-select", "hdp-select"],
+  "stack_selector": ["distro-select", "/usr/bin/distro-select", "distro-select"],
   "conf_selector": ["conf-select", "/usr/bin/conf-select", "conf-select"]
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/80fe9cbd/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/metainfo.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/metainfo.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/metainfo.xml
index 35da7fd..3e79bc5 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/metainfo.xml
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/metainfo.xml
@@ -31,7 +31,7 @@
           <displayName>App Timeline Server</displayName>
           <category>MASTER</category>
           <cardinality>0-1</cardinality>
-          <versionAdvertised>true</versionAdvertised>
+          <versionAdvertised>false</versionAdvertised>
           <reassignAllowed>true</reassignAllowed>
           <commandScript>
             <script>scripts/application_timeline_server.py</script>
@@ -45,7 +45,7 @@
           <displayName>ResourceManager</displayName>
           <category>MASTER</category>
           <cardinality>1</cardinality>
-          <versionAdvertised>true</versionAdvertised>
+          <versionAdvertised>false</versionAdvertised>
           <reassignAllowed>true</reassignAllowed>
           <commandScript>
             <script>scripts/resourcemanager.py</script>
@@ -93,7 +93,7 @@
           <displayName>NodeManager</displayName>
           <category>SLAVE</category>
           <cardinality>1+</cardinality>
-          <versionAdvertised>true</versionAdvertised>
+          <versionAdvertised>false</versionAdvertised>
           <decommissionAllowed>true</decommissionAllowed>
           <commandScript>
             <script>scripts/nodemanager.py</script>
@@ -117,7 +117,7 @@
           <displayName>YARN Client</displayName>
           <category>CLIENT</category>
           <cardinality>1+</cardinality>
-          <versionAdvertised>true</versionAdvertised>
+          <versionAdvertised>false</versionAdvertised>
           <commandScript>
             <script>scripts/yarn_client.py</script>
             <scriptType>PYTHON</scriptType>
@@ -210,7 +210,7 @@
           <displayName>History Server</displayName>
           <category>MASTER</category>
           <cardinality>1</cardinality>
-          <versionAdvertised>true</versionAdvertised>
+          <versionAdvertised>false</versionAdvertised>
           <reassignAllowed>true</reassignAllowed>
           <auto-deploy>
             <enabled>true</enabled>
@@ -250,7 +250,7 @@
           <displayName>MapReduce2 Client</displayName>
           <category>CLIENT</category>
           <cardinality>0+</cardinality>
-          <versionAdvertised>true</versionAdvertised>
+          <versionAdvertised>false</versionAdvertised>
           <commandScript>
             <script>scripts/mapreduce2_client.py</script>
             <scriptType>PYTHON</scriptType>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/80fe9cbd/odpi.bom
----------------------------------------------------------------------
diff --git a/odpi.bom b/odpi.bom
index 2ec14b9..27c50a2 100644
--- a/odpi.bom
+++ b/odpi.bom
@@ -135,7 +135,7 @@ bigtop {
     'ambari' {
       name    = 'ambari'
       relNotes = 'Apache Ambari'
-      version { base = '2.4.0'; pkg = base; release = 1 }
+      version { base = '2.4.1'; pkg = base; release = 1 }
       tarball { destination = "apache-$name-${version.base}-src.tar.gz"
                 source      = destination }
       url     { download_path = "/$name/$name-${version.base}/"


[43/50] [abbrv] bigtop git commit: A few other last minute changes to make it sort of work with hard wired values.

Posted by rv...@apache.org.
A few other last minute changes to make it sort of work with hard wired values.


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

Branch: refs/heads/BIGTOP-2666
Commit: 1b4f286be14e6868df2b3f83243df42a26dfd1c1
Parents: 246939a
Author: Alan Gates <ga...@hortonworks.com>
Authored: Mon Nov 7 16:39:01 2016 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:24 2017 -0800

----------------------------------------------------------------------
 .../org/odpi/specs/runtime/hive/HiveHelper.java | 12 ++-
 .../odpi/specs/runtime/hive/TestHCatalog.java   | 86 ++------------------
 2 files changed, 19 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/1b4f286b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
index a4477ff..3e56224 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
@@ -36,8 +36,13 @@ import org.apache.commons.logging.LogFactory;
 public class HiveHelper {
 	
 	private static final Log LOG = LogFactory.getLog(HiveHelper.class.getName());
-		
+
 	public static Map<String, String> execCommand(CommandLine commandline) {
+		return execCommand(commandline, null);
+	}
+
+	public static Map<String, String> execCommand(CommandLine commandline,
+																								Map<String, String> envVars) {
 		
 		System.out.println("Executing command:");
 		System.out.println(commandline.toString());
@@ -50,6 +55,11 @@ public class HiveHelper {
 			LOG.debug("Failed to get process environment: "+ e1.getMessage());
 			e1.printStackTrace();
 		}
+		if (envVars != null) {
+			for (String key : envVars.keySet()) {
+				env.put(key, envVars.get(key));
+			}
+		}
 
 		DefaultExecuteResultHandler resultHandler = new DefaultExecuteResultHandler();
 		ByteArrayOutputStream outputStream = new ByteArrayOutputStream();

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1b4f286b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
index 4b61131..bb237d8 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
@@ -32,20 +32,8 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hive.hcatalog.data.DefaultHCatRecord;
-import org.apache.hive.hcatalog.data.HCatRecord;
 import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
-import org.apache.hive.hcatalog.data.schema.HCatSchemaUtils;
-import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
 import org.apache.thrift.TException;
 import org.junit.Assert;
 import org.junit.Assume;
@@ -54,14 +42,12 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
-import java.util.StringTokenizer;
 
 
 public class TestHCatalog {
@@ -136,10 +122,6 @@ public class TestHCatalog {
     // Write some stuff into a file in the location of the table
     table = client.getTable("default", inputTable);
     String inputFile = table.getSd().getLocation() + "/input";
-    /*
-    String inputFile = JdbcConnector.getProperty(JdbcConnector.LOCATION,
-        "Directory to write a file in ") + "/odpi_hcat_input_" + rand.nextInt(Integer.MAX_VALUE);
-        */
     Path inputPath = new Path(inputFile);
     FileSystem fs = FileSystem.get(conf);
     FSDataOutputStream out = fs.create(inputPath);
@@ -149,76 +131,24 @@ public class TestHCatalog {
     out.writeChars("the lamb was sure to go\n");
     out.close();
 
-    Map<String, String> results = HiveHelper.execCommand(new CommandLine("hadoop")
+    Map<String, String> env = new HashMap<>();
+    env.put("HADOOP_HOME","/Users/gates/grid/odpi-testing/hadoop-2.7.3");
+    env.put("HADOOP_CLASSPATH", "/Users/gates/grid/odpi-testing/apache-hive-1.2.1-bin/hcatalog/share/hcatalog/hive-hcatalog-core-1.2.1.jar");
+    env.put("HIVE_HOME", "/Users/gates/grid/odpi-testing/apache-hive-1.2.1-bin");
+    Map<String, String> results = HiveHelper.execCommand(new CommandLine("/Users/gates/grid/odpi-testing/apache-hive-1.2.1-bin/bin/hive")
+        .addArgument("--service")
         .addArgument("jar")
         .addArgument("/Users/gates/git/bigtop/runtime-1.2.0-SNAPSHOT.jar")
         .addArgument(HCatalogMR.class.getName())
         .addArgument(inputTable)
         .addArgument(outputTable)
         .addArgument(inputSchema.getSchemaAsTypeString())
-        .addArgument(outputSchema.getSchemaAsTypeString()));
+        .addArgument(outputSchema.getSchemaAsTypeString()), env);
+    LOG.info(results.toString());
     Assert.assertEquals("HCat job failed", 0, Integer.parseInt(results.get("exitValue")));
 
-
-
-    /*
-    Job job = new Job(conf, "odpi_hcat_test");
-    HCatInputFormat.setInput(job, "default", inputTable);
-
-    job.setInputFormatClass(HCatInputFormat.class);
-    job.setJarByClass(TestHCatalog.class);
-    job.setMapperClass(Map.class);
-    job.setReducerClass(Reduce.class);
-    job.setMapOutputKeyClass(Text.class);
-    job.setMapOutputValueClass(IntWritable.class);
-    job.setOutputKeyClass(WritableComparable.class);
-    job.setOutputValueClass(HCatRecord.class);
-    HCatOutputFormat.setOutput(job, OutputJobInfo.create("default", outputTable, null));
-    HCatOutputFormat.setSchema(job, outputSchema);
-    job.setOutputFormatClass(HCatOutputFormat.class);
-
-    job.addCacheArchive(new URI("hdfs:/user/gates/hive-hcatalog-core-1.2.1.jar"));
-    job.addCacheArchive(new URI("hdfs:/user/gates/hive-metastore-1.2.1.jar"));
-    job.addCacheArchive(new URI("hdfs:/user/gates/hive-exec-1.2.1.jar"));
-
-    Assert.assertTrue(job.waitForCompletion(true));
-    */
-
     client.dropTable("default", inputTable);
     client.dropTable("default", outputTable);
   }
 
-  /*
-  public static class Map extends Mapper<WritableComparable,
-        HCatRecord, Text, IntWritable> {
-    private final static IntWritable one = new IntWritable(1);
-    private Text word = new Text();
-
-    @Override
-    protected void map(WritableComparable key, HCatRecord value, Context context)
-        throws IOException, InterruptedException {
-      String line = value.getString("line", inputSchema);
-      StringTokenizer tokenizer = new StringTokenizer(line);
-      while (tokenizer.hasMoreTokens()) {
-        word.set(tokenizer.nextToken());
-        context.write(word, one);
-      }
-    }
-  }
-
-  public static class Reduce extends Reducer<Text, IntWritable, WritableComparable, HCatRecord> {
-    @Override
-    protected void reduce(Text key, Iterable<IntWritable> values, Context context) throws
-        IOException, InterruptedException {
-      int sum = 0;
-      for (IntWritable i : values) {
-        sum += i.get();
-      }
-      HCatRecord output = new DefaultHCatRecord(2);
-      output.set("word", outputSchema, key);
-      output.set("count", outputSchema, sum);
-      context.write(null, output);
-    }
-  }
-  */
 }


[17/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hivemetastore-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hivemetastore-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hivemetastore-site.xml
new file mode 100755
index 0000000..75be7f2
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hivemetastore-site.xml
@@ -0,0 +1,43 @@
+<configuration><property require-input="false">
+    <name>hive.metastore.metrics.enabled</name>
+    <value>true</value>
+    <filename>hivemetastore-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>boolean</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.service.metrics.reporter</name>
+    <value>JSON_FILE, JMX, HADOOP2</value>
+    <filename>hivemetastore-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.service.metrics.hadoop2.component</name>
+    <value>hivemetastore</value>
+    <filename>hivemetastore-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.service.metrics.file.location</name>
+    <value>/var/log/hive/hivemetastore-report.json</value>
+    <filename>hivemetastore-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-interactive-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-interactive-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-interactive-site.xml
new file mode 100755
index 0000000..6954e56
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-interactive-site.xml
@@ -0,0 +1,56 @@
+<configuration><property require-input="false">
+    <name>hive.metastore.metrics.enabled</name>
+    <value>true</value>
+    <filename>hiveserver2-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>boolean</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.service.metrics.reporter</name>
+    <value>JSON_FILE, JMX, HADOOP2</value>
+    <filename>hiveserver2-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.service.metrics.hadoop2.component</name>
+    <value>hiveserver2</value>
+    <filename>hiveserver2-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.service.metrics.file.location</name>
+    <value>/var/log/hive/hiveserver2Interactive-report.json</value>
+    <filename>hiveserver2-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.async.log.enabled</name>
+    <value>false</value>
+    <description>Whether to enable Log4j2's asynchronous logging. Asynchronous logging can give significant performance improvement as logging will be handled in separate thread that uses LMAX disruptor queue for buffering log messages. Refer https://logging.apache.org/log4j/2.x/manual/async.html for benefits and drawbacks.</description>
+    <filename>hiveserver2-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>boolean</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-site.xml
new file mode 100755
index 0000000..e78f176
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hiveserver2-site.xml
@@ -0,0 +1,122 @@
+<configuration><property require-input="false">
+    <name>hive.metastore.metrics.enabled</name>
+    <value>true</value>
+    <filename>hiveserver2-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>boolean</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.service.metrics.reporter</name>
+    <value>JSON_FILE, JMX, HADOOP2</value>
+    <filename>hiveserver2-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.service.metrics.hadoop2.component</name>
+    <value>hiveserver2</value>
+    <filename>hiveserver2-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.service.metrics.file.location</name>
+    <value>/var/log/hive/hiveserver2-report.json</value>
+    <filename>hiveserver2-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.security.authenticator.manager</name>
+    <value>org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator</value>
+    <description>Hive client authenticator manager class name. The user-defined authenticator class should implement interface org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider.  </description>
+    <filename>hiveserver2-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive_security_authorization</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.security.authorization.manager</name>
+    <value>org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory</value>
+    <description>the hive client authorization manager class name.
+    The user defined authorization class should implement interface org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider.  </description>
+    <filename>hiveserver2-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive_security_authorization</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.security.authorization.enabled</name>
+    <value>false</value>
+    <description>enable or disable the Hive client authorization</description>
+    <display-name>Enable Authorization</display-name>
+    <filename>hiveserver2-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>value-list</type>
+        <entries>
+            <entry>
+                <value>true</value>
+                <label>True</label>
+            </entry>
+            <entry>
+                <value>false</value>
+                <label>False</label>
+            </entry>
+        </entries>
+        <selection-cardinality>1</selection-cardinality>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>hive_security_authorization</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>hive.conf.restricted.list</name>
+    <value>hive.security.authenticator.manager,hive.security.authorization.manager,hive.users.in.admin.role</value>
+    <description></description>
+    <filename>hiveserver2-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>hive_security_authorization</name>
+            <type>hive-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-cli-log4j2.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-cli-log4j2.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-cli-log4j2.xml
new file mode 100755
index 0000000..b7f6523
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-cli-log4j2.xml
@@ -0,0 +1,91 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+# 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.
+
+status = WARN
+name = LlapCliLog4j2
+packages = org.apache.hadoop.hive.ql.log
+
+# list of properties
+property.hive.log.level = INFO
+property.hive.root.logger = console
+property.hive.log.dir = ${sys:java.io.tmpdir}/${sys:user.name}
+property.hive.log.file = llap-cli.log
+
+# list of all appenders
+appenders = console, DRFA
+
+# console appender
+appender.console.type = Console
+appender.console.name = console
+appender.console.target = SYSTEM_ERR
+appender.console.layout.type = PatternLayout
+appender.console.layout.pattern = %p %c{2}: %m%n
+
+# daily rolling file appender
+appender.DRFA.type = RollingRandomAccessFile
+appender.DRFA.name = DRFA
+appender.DRFA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
+# Use %pid in the filePattern to append process-id@host-name to the filename if you want separate log files for different CLI session
+appender.DRFA.filePattern = ${sys:hive.log.dir}/${sys:hive.log.file}.%d{yyyy-MM-dd}
+appender.DRFA.layout.type = PatternLayout
+appender.DRFA.layout.pattern = %d{ISO8601} %5p [%t] %c{2}: %m%n
+appender.DRFA.policies.type = Policies
+appender.DRFA.policies.time.type = TimeBasedTriggeringPolicy
+appender.DRFA.policies.time.interval = 1
+appender.DRFA.policies.time.modulate = true
+appender.DRFA.strategy.type = DefaultRolloverStrategy
+appender.DRFA.strategy.max = 30
+
+# list of all loggers
+loggers = ZooKeeper, DataNucleus, Datastore, JPOX, HadoopConf
+
+logger.ZooKeeper.name = org.apache.zookeeper
+logger.ZooKeeper.level = WARN
+
+logger.DataNucleus.name = DataNucleus
+logger.DataNucleus.level = ERROR
+
+logger.Datastore.name = Datastore
+logger.Datastore.level = ERROR
+
+logger.JPOX.name = JPOX
+logger.JPOX.level = ERROR
+
+logger.HadoopConf.name = org.apache.hadoop.conf.Configuration
+logger.HadoopConf.level = ERROR
+
+# root logger
+rootLogger.level = ${sys:hive.log.level}
+rootLogger.appenderRefs = root, DRFA
+rootLogger.appenderRef.root.ref = ${sys:hive.root.logger}
+rootLogger.appenderRef.DRFA.ref = DRFA
+  </value>
+    <description>Custom llap-cli-log4j2.properties</description>
+    <display-name>llap-cli-log4j2 template</display-name>
+    <filename>llap-cli-log4j2.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+        <show-property-name>false</show-property-name>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-daemon-log4j.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-daemon-log4j.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-daemon-log4j.xml
new file mode 100755
index 0000000..30c31be
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/llap-daemon-log4j.xml
@@ -0,0 +1,158 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+# 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.
+
+
+# This is the log4j2 properties file used by llap-daemons. There's several loggers defined, which
+# can be selected while configuring LLAP.
+# Based on the one selected - UI links etc need to be manipulated in the system.
+# Note: Some names and logic is common to this file and llap LogHelpers. Make sure to change that
+# as well, if changing this file.
+
+status = INFO
+name = LlapDaemonLog4j2
+packages = org.apache.hadoop.hive.ql.log
+
+# list of properties
+property.llap.daemon.log.level = INFO
+property.llap.daemon.root.logger = console
+property.llap.daemon.log.dir = .
+property.llap.daemon.log.file = llapdaemon.log
+property.llap.daemon.historylog.file = llapdaemon_history.log
+property.llap.daemon.log.maxfilesize = 256MB
+property.llap.daemon.log.maxbackupindex = 240
+
+# list of all appenders
+appenders = console, RFA, HISTORYAPPENDER, query-routing
+
+# console appender
+appender.console.type = Console
+appender.console.name = console
+appender.console.target = SYSTEM_ERR
+appender.console.layout.type = PatternLayout
+appender.console.layout.pattern = %d{ISO8601} %5p [%t (%X{fragmentId})] %c{2}: %m%n
+
+# rolling file appender
+appender.RFA.type = RollingRandomAccessFile
+appender.RFA.name = RFA
+appender.RFA.fileName = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.log.file}
+appender.RFA.filePattern = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.log.file}_%d{yyyy-MM-dd-HH}_%i.done
+appender.RFA.layout.type = PatternLayout
+appender.RFA.layout.pattern = %d{ISO8601} %-5p [%t (%X{fragmentId})] %c: %m%n
+appender.RFA.policies.type = Policies
+appender.RFA.policies.time.type = TimeBasedTriggeringPolicy
+appender.RFA.policies.time.interval = 1
+appender.RFA.policies.time.modulate = true
+appender.RFA.policies.size.type = SizeBasedTriggeringPolicy
+appender.RFA.policies.size.size = ${sys:llap.daemon.log.maxfilesize}
+appender.RFA.strategy.type = DefaultRolloverStrategy
+appender.RFA.strategy.max = ${sys:llap.daemon.log.maxbackupindex}
+
+# history file appender
+appender.HISTORYAPPENDER.type = RollingRandomAccessFile
+appender.HISTORYAPPENDER.name = HISTORYAPPENDER
+appender.HISTORYAPPENDER.fileName = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.historylog.file}
+appender.HISTORYAPPENDER.filePattern = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.historylog.file}_%d{yyyy-MM-dd}_%i.done
+appender.HISTORYAPPENDER.layout.type = PatternLayout
+appender.HISTORYAPPENDER.layout.pattern = %m%n
+appender.HISTORYAPPENDER.policies.type = Policies
+appender.HISTORYAPPENDER.policies.size.type = SizeBasedTriggeringPolicy
+appender.HISTORYAPPENDER.policies.size.size = ${sys:llap.daemon.log.maxfilesize}
+appender.HISTORYAPPENDER.policies.time.type = TimeBasedTriggeringPolicy
+appender.HISTORYAPPENDER.policies.time.interval = 1
+appender.HISTORYAPPENDER.policies.time.modulate = true
+appender.HISTORYAPPENDER.strategy.type = DefaultRolloverStrategy
+appender.HISTORYAPPENDER.strategy.max = ${sys:llap.daemon.log.maxbackupindex}
+
+# queryId based routing file appender
+appender.query-routing.type = Routing
+appender.query-routing.name = query-routing
+appender.query-routing.routes.type = Routes
+appender.query-routing.routes.pattern = $${ctx:queryId}
+#Purge polciy for query-based Routing Appender
+appender.query-routing.purgePolicy.type = LlapRoutingAppenderPurgePolicy
+# Note: Do not change this name without changing the corresponding entry in LlapConstants
+appender.query-routing.purgePolicy.name = llapLogPurgerQueryRouting
+# default route
+appender.query-routing.routes.route-default.type = Route
+appender.query-routing.routes.route-default.key = $${ctx:queryId}
+appender.query-routing.routes.route-default.ref = RFA
+# queryId based route
+appender.query-routing.routes.route-mdc.type = Route
+appender.query-routing.routes.route-mdc.file-mdc.type = LlapWrappedAppender
+appender.query-routing.routes.route-mdc.file-mdc.name = IrrelevantName-query-routing
+appender.query-routing.routes.route-mdc.file-mdc.app.type = RandomAccessFile
+appender.query-routing.routes.route-mdc.file-mdc.app.name = file-mdc
+appender.query-routing.routes.route-mdc.file-mdc.app.fileName = ${sys:llap.daemon.log.dir}/${ctx:queryId}-${ctx:dagId}.log
+appender.query-routing.routes.route-mdc.file-mdc.app.layout.type = PatternLayout
+appender.query-routing.routes.route-mdc.file-mdc.app.layout.pattern = %d{ISO8601} %5p [%t (%X{fragmentId})] %c{2}: %m%n
+
+# list of all loggers
+loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX, HistoryLogger, LlapIoImpl, LlapIoOrc, LlapIoCache, LlapIoLocking
+
+logger.LlapIoImpl.name = LlapIoImpl
+logger.LlapIoImpl.level = INFO
+
+logger.LlapIoOrc.name = LlapIoOrc
+logger.LlapIoOrc.level = WARN
+
+logger.LlapIoCache.name = LlapIoCache
+logger.LlapIOCache.level = WARN
+
+logger.LlapIoLocking.name = LlapIoLocking
+logger.LlapIoLocking.level = WARN
+
+logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn
+logger.NIOServerCnxn.level = WARN
+
+logger.ClientCnxnSocketNIO.name = org.apache.zookeeper.ClientCnxnSocketNIO
+logger.ClientCnxnSocketNIO.level = WARN
+
+logger.DataNucleus.name = DataNucleus
+logger.DataNucleus.level = ERROR
+
+logger.Datastore.name = Datastore
+logger.Datastore.level = ERROR
+
+logger.JPOX.name = JPOX
+logger.JPOX.level = ERROR
+
+logger.HistoryLogger.name = org.apache.hadoop.hive.llap.daemon.HistoryLogger
+logger.HistoryLogger.level = INFO
+logger.HistoryLogger.additivity = false
+logger.HistoryLogger.appenderRefs = HistoryAppender
+logger.HistoryLogger.appenderRef.HistoryAppender.ref = HISTORYAPPENDER
+
+# root logger
+rootLogger.level = ${sys:llap.daemon.log.level}
+rootLogger.appenderRefs = root
+rootLogger.appenderRef.root.ref = ${sys:llap.daemon.root.logger}
+  </value>
+    <description>Custom llap-daemon-log4j2.properties</description>
+    <display-name>llap-deamon-log4j template</display-name>
+    <filename>llap-daemon-log4j.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+        <show-property-name>false</show-property-name>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-audit.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-audit.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-audit.xml
new file mode 100755
index 0000000..9e74aa0
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-audit.xml
@@ -0,0 +1,136 @@
+<configuration><property require-input="false">
+    <name>xasecure.audit.is.enabled</name>
+    <value>true</value>
+    <description>Is Audit enabled?</description>
+    <filename>ranger-hive-audit.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.audit.destination.hdfs</name>
+    <value>true</value>
+    <description>Is Audit to HDFS enabled?</description>
+    <display-name>Audit to HDFS</display-name>
+    <filename>ranger-hive-audit.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>boolean</type>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>xasecure.audit.destination.hdfs</name>
+            <type>ranger-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.audit.destination.hdfs.dir</name>
+    <value>hdfs://NAMENODE_HOSTNAME:8020/ranger/audit</value>
+    <description>HDFS folder to write audit to, make sure the service user has requried permissions</description>
+    <filename>ranger-hive-audit.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>xasecure.audit.destination.hdfs.dir</name>
+            <type>ranger-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.audit.destination.hdfs.batch.filespool.dir</name>
+    <value>/var/log/hive/audit/hdfs/spool</value>
+    <description>/var/log/hive/audit/hdfs/spool</description>
+    <filename>ranger-hive-audit.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.audit.destination.solr</name>
+    <value>false</value>
+    <description>Is Solr audit enabled?</description>
+    <display-name>Audit to SOLR</display-name>
+    <filename>ranger-hive-audit.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>boolean</type>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>xasecure.audit.destination.solr</name>
+            <type>ranger-env</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.audit.destination.solr.urls</name>
+    <value></value>
+    <description>Solr URL</description>
+    <filename>ranger-hive-audit.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <empty-value-valid>true</empty-value-valid>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>ranger.audit.solr.urls</name>
+            <type>ranger-admin-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.audit.destination.solr.zookeepers</name>
+    <value>NONE</value>
+    <description>Solr Zookeeper string</description>
+    <filename>ranger-hive-audit.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>ranger.audit.solr.zookeepers</name>
+            <type>ranger-admin-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.audit.destination.solr.batch.filespool.dir</name>
+    <value>/var/log/hive/audit/solr/spool</value>
+    <description>/var/log/hive/audit/solr/spool</description>
+    <filename>ranger-hive-audit.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.audit.provider.summary.enabled</name>
+    <value>false</value>
+    <description>Enable Summary audit?</description>
+    <display-name>Audit provider summary enabled</display-name>
+    <filename>ranger-hive-audit.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>boolean</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-plugin-properties.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-plugin-properties.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-plugin-properties.xml
new file mode 100755
index 0000000..346baa9
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-plugin-properties.xml
@@ -0,0 +1,63 @@
+<configuration><property require-input="false">
+    <name>policy_user</name>
+    <value>ambari-qa</value>
+    <description>This user must be system user and also present at Ranger admin portal</description>
+    <display-name>Policy user for HIVE</display-name>
+    <filename>ranger-hive-plugin-properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>jdbc.driverClassName</name>
+    <value>org.apache.hive.jdbc.HiveDriver</value>
+    <description>Used for repository creation on ranger admin</description>
+    <filename>ranger-hive-plugin-properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>common.name.for.certificate</name>
+    <value></value>
+    <description>Common name for certificate, this value should match what is specified in repo within ranger admin</description>
+    <filename>ranger-hive-plugin-properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <empty-value-valid>true</empty-value-valid>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>REPOSITORY_CONFIG_USERNAME</name>
+    <value>hive</value>
+    <description>Used for repository creation on ranger admin</description>
+    <display-name>Ranger repository config user</display-name>
+    <filename>ranger-hive-plugin-properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>REPOSITORY_CONFIG_PASSWORD</name>
+    <value>hive</value>
+    <description>Used for repository creation on ranger admin</description>
+    <display-name>Ranger repository config password</display-name>
+    <filename>ranger-hive-plugin-properties.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type>PASSWORD</property-type>
+    <value-attributes>
+        <type>password</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-policymgr-ssl.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-policymgr-ssl.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-policymgr-ssl.xml
new file mode 100755
index 0000000..bf73d9f
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-policymgr-ssl.xml
@@ -0,0 +1,71 @@
+<configuration><property require-input="false">
+    <name>xasecure.policymgr.clientssl.keystore</name>
+    <value>/usr/hdp/current/hive-server2/conf/ranger-plugin-keystore.jks</value>
+    <description>Java Keystore files</description>
+    <filename>ranger-hive-policymgr-ssl.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.policymgr.clientssl.keystore.password</name>
+    <value>myKeyFilePassword</value>
+    <description>password for keystore</description>
+    <filename>ranger-hive-policymgr-ssl.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type>PASSWORD</property-type>
+    <value-attributes>
+        <type>password</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.policymgr.clientssl.truststore</name>
+    <value>/usr/hdp/current/hive-server2/conf/ranger-plugin-truststore.jks</value>
+    <description>java truststore file</description>
+    <filename>ranger-hive-policymgr-ssl.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.policymgr.clientssl.truststore.password</name>
+    <value>changeit</value>
+    <description>java truststore password</description>
+    <filename>ranger-hive-policymgr-ssl.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type>PASSWORD</property-type>
+    <value-attributes>
+        <type>password</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.policymgr.clientssl.keystore.credential.file</name>
+    <value>jceks://file{{credential_file}}</value>
+    <description>java keystore credential file</description>
+    <filename>ranger-hive-policymgr-ssl.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.policymgr.clientssl.truststore.credential.file</name>
+    <value>jceks://file{{credential_file}}</value>
+    <description>java truststore credential file</description>
+    <filename>ranger-hive-policymgr-ssl.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-security.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-security.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-security.xml
new file mode 100755
index 0000000..a29780f
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-security.xml
@@ -0,0 +1,81 @@
+<configuration><property require-input="false">
+    <name>ranger.plugin.hive.policy.rest.ssl.config.file</name>
+    <value>/usr/hdp/current/{{ranger_hive_component}}/conf/conf.server/ranger-policymgr-ssl.xml</value>
+    <description>Path to the file containing SSL details to contact Ranger Admin</description>
+    <filename>ranger-hive-security.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>ranger.plugin.hive.service.name</name>
+    <value>{{repo_name}}</value>
+    <description>Name of the Ranger service containing policies for this HIVE instance</description>
+    <filename>ranger-hive-security.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>ranger.plugin.hive.policy.source.impl</name>
+    <value>org.apache.ranger.admin.client.RangerAdminRESTClient</value>
+    <description>Class to retrieve policies from the source</description>
+    <filename>ranger-hive-security.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>ranger.plugin.hive.policy.rest.url</name>
+    <value>{{policymgr_mgr_url}}</value>
+    <description>URL to Ranger Admin</description>
+    <filename>ranger-hive-security.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>ranger.plugin.hive.policy.pollIntervalMs</name>
+    <value>30000</value>
+    <description>How often to poll for changes in policies?</description>
+    <filename>ranger-hive-security.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>ranger.plugin.hive.policy.cache.dir</name>
+    <value>/etc/ranger/{{repo_name}}/policycache</value>
+    <description>Directory where Ranger policies are cached after successful retrieval from the source</description>
+    <filename>ranger-hive-security.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>xasecure.hive.update.xapolicies.on.grant.revoke</name>
+    <value>true</value>
+    <description>Should Hive plugin update Ranger policies for updates to permissions done using GRANT/REVOKE?</description>
+    <display-name>Should Hive GRANT/REVOKE update XA policies</display-name>
+    <filename>ranger-hive-security.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>boolean</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/tez-interactive-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/tez-interactive-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/tez-interactive-site.xml
new file mode 100755
index 0000000..3865c36
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/tez-interactive-site.xml
@@ -0,0 +1,144 @@
+<configuration><property require-input="false">
+    <name>tez.runtime.shuffle.fetch.buffer.percent</name>
+    <value>0.6</value>
+    <description>Fraction (0-1) of the available memory which can be used to
+      retain shuffled data</description>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>tez.runtime.shuffle.memory.limit.percent</name>
+    <value>0.25</value>
+    <description>This property determines the maximum size of a shuffle segment
+      which can be fetched to memory. Fraction (0-1) of shuffle memory
+      (after applying tez.runtime.shuffle.fetch.buffer.percent)</description>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>tez.runtime.report.partition.stats</name>
+    <value>true</value>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>tez.runtime.pipelined-shuffle.enabled</name>
+    <value>false</value>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>tez.runtime.pipelined.sorter.lazy-allocate.memory</name>
+    <value>true</value>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>tez.grouping.node.local.only</name>
+    <value>true</value>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>tez.runtime.shuffle.fetch.verify-disk-checksum</name>
+    <value>false</value>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>tez.lib.uris</name>
+    <value>/hdp/apps/${hdp.version}/tez_hive2/tez.tar.gz</value>
+    <description>Comma-delimited list of the location of the Tez libraries which will be localized for DAGs.
+      Specifying a single .tar.gz or .tgz assumes that a compressed version of the tez libs is being used. This is uncompressed into a tezlibs directory when running containers, and tezlibs/;tezlibs/lib/ are added to the classpath (after . and .*).
+      If multiple files are specified - files are localized as regular files, contents of directories are localized as regular files (non-recursive).
+    </description>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>tez.dag.recovery.enabled</name>
+    <value>false</value>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>tez.am.resource.memory.mb</name>
+    <value>1536</value>
+    <description>The amount of memory to be used by the AppMaster</description>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>int</type>
+    </value-attributes>
+    <depends-on>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+        <property>
+            <name>hive.llap.daemon.queue.name</name>
+            <type>hive-interactive-site</type>
+        </property>
+        <property>
+            <name>llap_queue_capacity</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>enable_hive_interactive</name>
+            <type>hive-interactive-env</type>
+        </property>
+        <property>
+            <name>hive.server2.tez.sessions.per.default.queue</name>
+            <type>hive-interactive-site</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>tez.session.am.dag.submit.timeout.secs</name>
+    <value>3600</value>
+    <filename>tez-interactive-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-env.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-env.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-env.xml
new file mode 100755
index 0000000..fa62c78
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-env.xml
@@ -0,0 +1,38 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+# The file containing the running pid
+PID_FILE={{webhcat_pid_file}}
+
+TEMPLETON_LOG_DIR={{templeton_log_dir}}/
+
+
+WEBHCAT_LOG_DIR={{templeton_log_dir}}/
+
+# The console error log
+ERROR_LOG={{templeton_log_dir}}/webhcat-console-error.log
+
+# The console log
+CONSOLE_LOG={{templeton_log_dir}}/webhcat-console.log
+
+#TEMPLETON_JAR=templeton_jar_name
+
+#HADOOP_PREFIX=hadoop_prefix
+
+#HCAT_PREFIX=hive_prefix
+
+# Set HADOOP_HOME to point to a specific hadoop install directory
+export HADOOP_HOME=${HADOOP_HOME:-{{hadoop_home}}}
+    </value>
+    <description>webhcat-env.sh content</description>
+    <display-name>webhcat-env template</display-name>
+    <filename>webhcat-env.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-log4j.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-log4j.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-log4j.xml
new file mode 100755
index 0000000..541b1c9
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-log4j.xml
@@ -0,0 +1,63 @@
+<configuration><property require-input="false">
+    <name>content</name>
+    <value>
+# 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.
+
+# Define some default values that can be overridden by system properties
+webhcat.root.logger = INFO, standard
+webhcat.log.dir = .
+webhcat.log.file = webhcat.log
+
+log4j.rootLogger = ${webhcat.root.logger}
+
+# Logging Threshold
+log4j.threshhold = DEBUG
+
+log4j.appender.standard  =  org.apache.log4j.DailyRollingFileAppender
+log4j.appender.standard.File = ${webhcat.log.dir}/${webhcat.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern = .yyyy-MM-dd
+
+log4j.appender.DRFA.layout = org.apache.log4j.PatternLayout
+
+log4j.appender.standard.layout = org.apache.log4j.PatternLayout
+log4j.appender.standard.layout.conversionPattern = %-5p | %d{DATE} | %c | %m%n
+
+# Class logging settings
+log4j.logger.com.sun.jersey = DEBUG
+log4j.logger.com.sun.jersey.spi.container.servlet.WebComponent = ERROR
+log4j.logger.org.apache.hadoop = INFO
+log4j.logger.org.apache.hadoop.conf = WARN
+log4j.logger.org.apache.zookeeper = WARN
+log4j.logger.org.eclipse.jetty = INFO
+
+    </value>
+    <description>Custom webhcat-log4j.properties</description>
+    <display-name>webhcat-log4j template</display-name>
+    <filename>webhcat-log4j.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>content</type>
+        <show-property-name>false</show-property-name>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-site.xml
new file mode 100755
index 0000000..680ddc8
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/webhcat-site.xml
@@ -0,0 +1,287 @@
+<configuration><property require-input="false">
+    <name>templeton.libjars</name>
+    <value>/usr/hdp/${hdp.version}/zookeeper/zookeeper.jar,/usr/hdp/${hdp.version}/hive/lib/hive-common.jar</value>
+    <description>Jars to add the the classpath.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.hive.extra.files</name>
+    <value>/usr/hdp/${hdp.version}/tez/conf/tez-site.xml,/usr/hdp/${hdp.version}/tez,/usr/hdp/${hdp.version}/tez/lib</value>
+    <description>The resources in this list will be localized to the node running LaunchMapper and added to HADOOP_CLASSPTH
+      before launching 'hive' command.  If the path /foo/bar is a directory, the contents of the the entire dir will be localized
+      and ./foo/* will be added to HADOOP_CLASSPATH.  Note that since classpath path processing does not recurse into subdirectories,
+      the paths in this property may be overlapping.  In the example above, "./tez-site.xml:./tez-client/*:./lib/*" will be added to
+      HADOOP_CLASSPATH.
+      This can be used to specify config files, Tez artifacts, etc.  This will be sent -files option of hadoop jar command thus
+      each path is interpreted by Generic Option Parser.  It can be local or hdfs path.
+    </description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.jar</name>
+    <value>/usr/hdp/${hdp.version}/hive/share/webhcat/svr/lib/hive-webhcat-*.jar</value>
+    <description>The path to the Templeton jar file.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.hadoop</name>
+    <value>/usr/hdp/${hdp.version}/hadoop/bin/hadoop</value>
+    <description>The path to the Hadoop executable.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.pig.archive</name>
+    <value>hdfs:///hdp/apps/${hdp.version}/pig/pig.tar.gz</value>
+    <description>The path to the Pig archive in HDFS.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <empty-value-valid>true</empty-value-valid>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.pig.path</name>
+    <value>pig.tar.gz/pig/bin/pig</value>
+    <description>The path to the Pig executable.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.hcat</name>
+    <value>/usr/hdp/${hdp.version}/hive/bin/hcat</value>
+    <description>The path to the hcatalog executable.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.hive.archive</name>
+    <value>hdfs:///hdp/apps/${hdp.version}/hive/hive.tar.gz</value>
+    <description>The path to the Hive archive.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <empty-value-valid>true</empty-value-valid>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.hive.home</name>
+    <value>hive.tar.gz/hive</value>
+    <description>The path to the Hive home within the tar. Has no effect if templeton.hive.archive is not set.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.hcat.home</name>
+    <value>hive.tar.gz/hive/hcatalog</value>
+    <description>The path to the HCat home within the tar. Has no effect if templeton.hive.archive is not set.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.hive.path</name>
+    <value>hive.tar.gz/hive/bin/hive</value>
+    <description>The path to the Hive executable.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.sqoop.archive</name>
+    <value>hdfs:///hdp/apps/${hdp.version}/sqoop/sqoop.tar.gz</value>
+    <description>The path to the Sqoop archive in HDFS.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.sqoop.path</name>
+    <value>sqoop.tar.gz/sqoop/bin/sqoop</value>
+    <description>The path to the Sqoop executable.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.sqoop.home</name>
+    <value>sqoop.tar.gz/sqoop</value>
+    <description>The path to the Sqoop home within the tar. Has no effect if
+      templeton.sqoop.archive is not set.
+    </description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.streaming.jar</name>
+    <value>hdfs:///hdp/apps/${hdp.version}/mapreduce/hadoop-streaming.jar</value>
+    <description>The hdfs path to the Hadoop streaming jar file.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.port</name>
+    <value>50111</value>
+    <description>The HTTP port for the main server.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.hadoop.conf.dir</name>
+    <value>/etc/hadoop/conf</value>
+    <description>The path to the Hadoop configuration.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.python</name>
+    <value>${env.PYTHON_CMD}</value>
+    <description>The path to the Python executable.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.hive.properties</name>
+    <value>hive.metastore.local=false,hive.metastore.uris=thrift://localhost:9083,hive.metastore.sasl.enabled=false</value>
+    <description>Properties to set when running hive.</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.zookeeper.hosts</name>
+    <value>localhost:2181</value>
+    <description>ZooKeeper servers, as comma separated host:port pairs</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes>
+        <type>multiLine</type>
+    </value-attributes>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.storage.class</name>
+    <value>org.apache.hive.hcatalog.templeton.tool.ZooKeeperStorage</value>
+    <description>The class to use as storage</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.override.enabled</name>
+    <value>false</value>
+    <description>Enable the override path in templeton.override.jars</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.exec.timeout</name>
+    <value>60000</value>
+    <description>Time out for templeton api</description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="true" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on/>
+    <property_depended_by/>
+</property><property require-input="false">
+    <name>templeton.hadoop.queue.name</name>
+    <value>default</value>
+    <description>
+      MapReduce queue name where WebHCat map-only jobs will be submitted to. Can be used to avoid a deadlock where all map slots in the cluster are taken over by Templeton launcher tasks.
+    </description>
+    <filename>webhcat-site.xml</filename>
+    <deleted>false</deleted>
+    <on-ambari-upgrade add="false" delete="false" update="false"/>
+    <property-type></property-type>
+    <value-attributes/>
+    <depends-on>
+        <property>
+            <name>yarn.scheduler.capacity.root.queues</name>
+            <type>capacity-scheduler</type>
+        </property>
+    </depends-on>
+    <property_depended_by/>
+</property></configuration>
\ No newline at end of file


[50/50] [abbrv] bigtop git commit: With ODPi 2.0 release branch cut, the main is now at version 2.1-SNAPSHOT

Posted by rv...@apache.org.
With ODPi 2.0 release branch cut, the main is now at version 2.1-SNAPSHOT


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

Branch: refs/heads/BIGTOP-2666
Commit: 5966b079505d018a3ec52e9995961596b5333c13
Parents: 4e50242
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Fri Nov 11 13:20:01 2016 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:26 2017 -0800

----------------------------------------------------------------------
 build.gradle | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/5966b079/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index c0fc7e3..758d2d9 100644
--- a/build.gradle
+++ b/build.gradle
@@ -146,7 +146,7 @@ allprojects {
   apply plugin: 'maven'
 
   group = 'org.apache.bigtop'
-  version = '1.2.0'
+  version = '2.1.0-SNAPSHOT'
 
   description = """Bigtop"""
 


[03/50] [abbrv] bigtop git commit: Changed queries and inserts to explain. Still tests syntax, goes much faster and runs more reliably on a very small cluster.

Posted by rv...@apache.org.
Changed queries and inserts to explain.  Still tests syntax, goes much faster and runs more reliably on a very small cluster.


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

Branch: refs/heads/BIGTOP-2666
Commit: 56970d13d74ba651d787d1cab5b4bdbbc675c7dd
Parents: ff03afa
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Oct 12 17:21:08 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:45:55 2017 -0800

----------------------------------------------------------------------
 .../org/odpi/specs/runtime/hive/TestSql.java    | 30 +++++++++-----------
 1 file changed, 14 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/56970d13/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
index 71ca12c..f247841 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
@@ -226,14 +226,14 @@ public class TestSql extends JdbcConnector {
           " partitioned by (p1 string)");
 
       // insert with partition
-      stmt.execute("insert into " + table1 + " partition (p1 = 'a') values " +
+      stmt.execute("explain insert into " + table1 + " partition (p1 = 'a') values " +
           "(1, 2, 3, 4, 1.1, 2.2, 3.3, 'abcdef', 'ghi', true)," +
           "(5, 6, 7, 8, 9.9, 8.8, 7.7, 'jklmno', 'pqr', true)");
 
       stmt.execute("set hive.exec.dynamic.partition.mode=nonstrict");
 
       // dynamic partition
-      stmt.execute("insert into " + table1 + " partition (p1) values " +
+      stmt.execute("explain insert into " + table1 + " partition (p1) values " +
           "(1, 2, 3, 4, 1.1, 2.2, 3.3, 'abcdef', 'ghi', true, 'b')," +
           "(5, 6, 7, 8, 9.9, 8.8, 7.7, 'jklmno', 'pqr', true, 'b')");
 
@@ -251,12 +251,12 @@ public class TestSql extends JdbcConnector {
           " c9 char(10)," +
           " c10 boolean)");
 
-      stmt.execute("insert into " + table2 + " values " +
+      stmt.execute("explain insert into " + table2 + " values " +
           "(1, 2, 3, 4, 1.1, 2.2, 3.3, 'abcdef', 'ghi', true)," +
           "(5, 6, 7, 8, 9.9, 8.8, 7.7, 'jklmno', 'pqr', true)");
 
-      stmt.execute("insert overwrite table " + table2 + " select c1, c2, c3, c4, c5, c6, c7, c8, " +
-          "c9, c10 from " + table1);
+      stmt.execute("explain insert overwrite table " + table2 + " select c1, c2, c3, c4, c5, c6, " +
+          "c7, c8, c9, c10 from " + table1);
 
       // multi-insert
       stmt.execute("from " + table1 +
@@ -273,7 +273,6 @@ public class TestSql extends JdbcConnector {
     try (Statement stmt = conn.createStatement()) {
       stmt.execute("drop table if exists " + table1);
       stmt.execute("create table " + table1 + "(c1 int, c2 varchar(32))");
-      stmt.execute("insert into " + table1 + " values (1, 'abc'), (2, 'def')");
       stmt.execute("with cte1 as (select c1 from " + table1 + " where c1 < 10) " +
           " select c1 from cte1");
     }
@@ -290,37 +289,36 @@ public class TestSql extends JdbcConnector {
       for (int i = 0; i < tables.length; i++) {
         stmt.execute("drop table if exists " + tables[i]);
         stmt.execute("create table " + tables[i] + "(c1 int, c2 varchar(32))");
-        stmt.execute("insert into " + tables[i] + " values (1, 'abc'), (2, 'def')");
       }
 
       // single table queries tested above in several places
 
-      stmt.execute("select all a.c2, SUM(a.c1), SUM(b.c1) " +
+      stmt.execute("explain select all a.c2, SUM(a.c1), SUM(b.c1) " +
           "from " + tables[0] + " a join " + tables[1] + " b on (a.c2 = b.c2) " +
           "group by a.c2 " +
           "order by a.c2 asc " +
           "limit 10");
 
-      stmt.execute("select distinct a.c2 " +
+      stmt.execute("explain select distinct a.c2 " +
           "from " + tables[0] + " a left outer join " + tables[1] + " b on (a.c2 = b.c2) " +
           "order by a.c2 desc ");
 
-      stmt.execute("select a.c2, SUM(a.c1) " +
+      stmt.execute("explain select a.c2, SUM(a.c1) " +
           "from " + tables[0] + " a right outer join " + tables[1] + " b on (a.c2 = b.c2) " +
           "group by a.c2 " +
           "having SUM(b.c1) > 0 " +
           "order by a.c2 ");
 
-      stmt.execute("select a.c2, rank() over (partition by a.c1) " +
+      stmt.execute("explain select a.c2, rank() over (partition by a.c1) " +
           "from " + tables[0] + " a full outer join " + tables[1] + " b on (a.c2 = b.c2) ");
 
-      stmt.execute("select c2 from " + tables[0] + " union all select c2 from " + tables[1]);
+      stmt.execute("explain select c2 from " + tables[0] + " union all select c2 from " + tables[1]);
 
-      stmt.execute("select * from " + tables[0] + " distribute by c1 sort by c2");
-      stmt.execute("select * from " + tables[0] + " cluster by c1");
+      stmt.execute("explain select * from " + tables[0] + " distribute by c1 sort by c2");
+      stmt.execute("explain select * from " + tables[0] + " cluster by c1");
 
-      stmt.execute("select * from (select c1 from " + tables[0] + ") t");
-      stmt.execute("select * from " + tables[0] + " where c1 in (select c1 from " + tables[1] +
+      stmt.execute("explain select * from (select c1 from " + tables[0] + ") t");
+      stmt.execute("explain select * from " + tables[0] + " where c1 in (select c1 from " + tables[1] +
           ")");
 
     }


[37/50] [abbrv] bigtop git commit: changed url username and password strings

Posted by rv...@apache.org.
changed url username and password strings


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

Branch: refs/heads/BIGTOP-2666
Commit: 5678bcf5e4a724026294e507ac32cb0bc1662646
Parents: a03c50d
Author: roypradeep <ro...@us.ibm.com>
Authored: Wed Nov 2 11:40:36 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:06 2017 -0800

----------------------------------------------------------------------
 .../test/java/org/odpi/specs/runtime/hive/TestBeeline.java   | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/5678bcf5/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
index 682949d..578621a 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
@@ -31,9 +31,11 @@ import java.util.Map;
 public class TestBeeline {
 	
 	public static final Log LOG = LogFactory.getLog(TestBeeline.class.getName());
-	private static final String URL = "odpiHiveTestJdbcUrl";
-	private static final String USER = "odpiHiveTestJdbcUser";
-	private static final String PASSWD = "odpiHiveTestJdbcPassword";
+	
+	private static final String URL = "odpi.test.hive.jdbc.url";
+	private static final String USER = "odpi.test.hive.jdbc.user";
+	private static final String PASSWD = "odpi.test.hive.jdbc.password";
+	
 	private static Map<String, String> results;
 	private static String beelineUrl; 
 	private static String beelineUser;


[24/50] [abbrv] bigtop git commit: Swithing ODPi stack repos to trunk

Posted by rv...@apache.org.
Swithing ODPi stack repos to trunk


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

Branch: refs/heads/BIGTOP-2666
Commit: 066e1cffe73c292b9889f89c2d892f125c159e27
Parents: 623e033
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Wed Oct 26 17:48:52 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:00 2017 -0800

----------------------------------------------------------------------
 .../src/common/ambari/ODPi/1.0/repos/repoinfo.xml            | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/066e1cff/bigtop-packages/src/common/ambari/ODPi/1.0/repos/repoinfo.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/repos/repoinfo.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/repos/repoinfo.xml
index ab4f25f..60eae65 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/repos/repoinfo.xml
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/repos/repoinfo.xml
@@ -18,15 +18,15 @@
 <reposinfo>
   <os family="redhat6">
     <repo>
-      <baseurl>http://repo.odpi.org/ODPi/1.0/centos-6/</baseurl>
-      <repoid>ODPi-1.0</repoid>
+      <baseurl>http://repo.odpi.org/ODPi/trunk/centos-6/</baseurl>
+      <repoid>ODPi-trunk</repoid>
       <reponame>ODPi</reponame>
     </repo>
   </os>
   <os family="ubuntu14">
     <repo>
-      <baseurl>http://repo.odpi.org/ODPi/1.0/ubuntu-14.04/apt</baseurl>
-      <repoid>ODPi-1.0</repoid>
+      <baseurl>http://repo.odpi.org/ODPi/trunk/ubuntu-14.04/apt</baseurl>
+      <repoid>ODPi-trunk</repoid>
       <reponame>odpi</reponame>
     </repo>
   </os>


[35/50] [abbrv] bigtop git commit: ODPI-184 Pulled my head out. I had written the tests so that they were running on the client rather than connecting to the server. Fixed that so they will now talk to the metastore server.

Posted by rv...@apache.org.
ODPI-184 Pulled my head out.  I had written the tests so that they were running on the client rather than connecting to the server.  Fixed that so they will now talk to the metastore server.


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

Branch: refs/heads/BIGTOP-2666
Commit: 7d9763ccdae90f592dd3cb234b147dd36fca1563
Parents: 1792d1e
Author: Alan Gates <ga...@hortonworks.com>
Authored: Tue Nov 1 16:38:14 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:05 2017 -0800

----------------------------------------------------------------------
 .../org/odpi/specs/runtime/hive/TestThrift.java | 129 ++++++++-----------
 1 file changed, 57 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/7d9763cc/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
index 1aede96..5eaab95 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestThrift.java
@@ -17,29 +17,27 @@
  */
 package org.odpi.specs.runtime.hive;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.TableType;
-import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest;
 import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.DropPartitionsRequest;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.RequestPartsSpec;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
 import org.apache.thrift.TException;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -47,7 +45,9 @@ import java.util.Random;
 
 public class TestThrift {
 
-  private static ThriftHiveMetastore.Iface client = null;
+  private static final Log LOG = LogFactory.getLog(JdbcConnector.class.getName());
+
+  private static IMetaStoreClient client = null;
   private static HiveConf conf;
 
   private Random rand;
@@ -58,7 +58,8 @@ public class TestThrift {
       String url = JdbcConnector.getProperty(JdbcConnector.METASTORE_URL, "Thrift metastore URL");
       conf = new HiveConf();
       conf.setVar(HiveConf.ConfVars.METASTOREURIS, url);
-      client = new HiveMetaStore.HMSHandler("ODPi test", conf, true);
+      LOG.info("Set to test against metastore at " + url);
+      client = new HiveMetaStoreClient(conf);
     }
   }
 
@@ -72,22 +73,20 @@ public class TestThrift {
   public void db() throws TException {
     final String dbName = "odpi_thrift_db_" + rand.nextInt(Integer.MAX_VALUE);
 
-    String location = JdbcConnector.getProperty(JdbcConnector.LOCATION, " HDFS location we can " +
-        "write to");
-    Database db = new Database(dbName, "a db", location, new HashMap<String, String>());
-    client.create_database(db);
-    db = client.get_database(dbName);
+    Database db = new Database(dbName, "a db", null, new HashMap<String, String>());
+    client.createDatabase(db);
+    db = client.getDatabase(dbName);
     Assert.assertNotNull(db);
     db = new Database(db);
     db.getParameters().put("a", "b");
-    client.alter_database(dbName, db);
-    List<String> alldbs = client.get_databases("odpi_*");
+    client.alterDatabase(dbName, db);
+    List<String> alldbs = client.getDatabases("odpi_*");
     Assert.assertNotNull(alldbs);
     Assert.assertTrue(alldbs.size() > 0);
-    alldbs = client.get_all_databases();
+    alldbs = client.getAllDatabases();
     Assert.assertNotNull(alldbs);
     Assert.assertTrue(alldbs.size() > 0);
-    client.drop_database(dbName, true, true);
+    client.dropDatabase(dbName, true, true);
   }
 
   // Not testing types calls, as they aren't used AFAIK
@@ -95,138 +94,129 @@ public class TestThrift {
   @Test
   public void nonPartitionedTable() throws TException {
     final String tableName = "odpi_thrift_table_" + rand.nextInt(Integer.MAX_VALUE);
-    String location = JdbcConnector.getProperty(JdbcConnector.LOCATION, " HDFS location we can " +
-        "write to");
 
     // I don't test every operation related to tables, but only those that are frequently used.
     SerDeInfo serde = new SerDeInfo("default_serde",
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), new HashMap<String, String>());
     FieldSchema fs = new FieldSchema("a", "int", "no comment");
-    StorageDescriptor sd = new StorageDescriptor(Collections.singletonList(fs), location,
+    StorageDescriptor sd = new StorageDescriptor(Collections.singletonList(fs), null,
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT),
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT), false, 0, serde, null, null,
         new HashMap<String, String>());
     Table table = new Table(tableName, "default", "me", 0, 0, 0, sd, null,
         new HashMap<String, String>(), null, null, TableType.MANAGED_TABLE.toString());
-    EnvironmentContext envContext = new EnvironmentContext(new HashMap<String, String>());
-    client.create_table_with_environment_context(table, envContext);
+    client.createTable(table);
 
-    table = client.get_table("default", tableName);
+    table = client.getTable("default", tableName);
     Assert.assertNotNull(table);
 
     List<Table> tables =
-        client.get_table_objects_by_name("default", Collections.singletonList(tableName));
+        client.getTableObjectsByName("default", Collections.singletonList(tableName));
     Assert.assertNotNull(tables);
     Assert.assertEquals(1, tables.size());
 
-    List<String> tableNames = client.get_tables("default", "odpi_*");
+    List<String> tableNames = client.getTables("default", "odpi_*");
     Assert.assertNotNull(tableNames);
     Assert.assertTrue(tableNames.size() >= 1);
 
-    tableNames = client.get_all_tables("default");
+    tableNames = client.getAllTables("default");
     Assert.assertNotNull(tableNames);
     Assert.assertTrue(tableNames.size() >= 1);
 
-    List<FieldSchema> cols = client.get_fields("default", tableName);
+    List<FieldSchema> cols = client.getFields("default", tableName);
     Assert.assertNotNull(cols);
     Assert.assertEquals(1, cols.size());
 
-    cols = client.get_schema_with_environment_context("default", tableName, envContext);
+    cols = client.getSchema("default", tableName);
     Assert.assertNotNull(cols);
     Assert.assertEquals(1, cols.size());
 
     table = new Table(table);
     table.getParameters().put("a", "b");
-    client.alter_table_with_cascade("default", tableName, table, false);
+    client.alter_table("default", tableName, table, false);
 
     table.getParameters().put("c", "d");
-    client.alter_table_with_environment_context("default", tableName, table, envContext);
+    client.alter_table("default", tableName, table);
 
-    client.drop_table_with_environment_context("default", tableName, true, envContext);
+    client.dropTable("default", tableName, true, false);
   }
 
   @Test
   public void partitionedTable() throws TException {
     final String tableName = "odpi_thrift_partitioned_table_" + rand.nextInt(Integer.MAX_VALUE);
-    String location = JdbcConnector.getProperty(JdbcConnector.LOCATION, " HDFS location we can " +
-        "write to");
 
     // I don't test every operation related to tables, but only those that are frequently used.
     SerDeInfo serde = new SerDeInfo("default_serde",
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), new HashMap<String, String>());
     FieldSchema fs = new FieldSchema("a", "int", "no comment");
-    StorageDescriptor sd = new StorageDescriptor(Collections.singletonList(fs), location,
+    StorageDescriptor sd = new StorageDescriptor(Collections.singletonList(fs), null,
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT),
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT), false, 0, serde, null, null,
         new HashMap<String, String>());
     FieldSchema pk = new FieldSchema("pk", "string", "");
     Table table = new Table(tableName, "default", "me", 0, 0, 0, sd, Collections.singletonList(pk),
         new HashMap<String, String>(), null, null, TableType.MANAGED_TABLE.toString());
-    EnvironmentContext envContext = new EnvironmentContext(new HashMap<String, String>());
-    client.create_table_with_environment_context(table, envContext);
+    client.createTable(table);
 
-    sd = new StorageDescriptor(Collections.singletonList(fs), location + "/x",
+    sd = new StorageDescriptor(Collections.singletonList(fs), null,
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE),
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), false, 0, serde, null, null,
         new HashMap<String, String>());
     Partition partition = new Partition(Collections.singletonList("x"), "default", tableName, 0,
         0, sd, new HashMap<String, String>());
-    client.add_partition_with_environment_context(partition, envContext);
+    client.add_partition(partition);
 
-    sd = new StorageDescriptor(Collections.singletonList(fs), location + "/y",
+    List<Partition> partitions = new ArrayList<>(2);
+    sd = new StorageDescriptor(Collections.singletonList(fs), null,
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE),
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), false, 0, serde, null, null,
         new HashMap<String, String>());
-    partition = new Partition(Collections.singletonList("y"), "default", tableName, 0,
-        0, sd, new HashMap<String, String>());
-    client.add_partitions(Collections.singletonList(partition));
-
-    sd = new StorageDescriptor(Collections.singletonList(fs), location + "/z",
+    partitions.add(new Partition(Collections.singletonList("y"), "default", tableName, 0,
+        0, sd, new HashMap<String, String>()));
+    sd = new StorageDescriptor(Collections.singletonList(fs), null,
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE),
         conf.getVar(HiveConf.ConfVars.HIVEDEFAULTSERDE), false, 0, serde, null, null,
         new HashMap<String, String>());
-    partition = new Partition(Collections.singletonList("z"), "default", tableName, 0,
-        0, sd, new HashMap<String, String>());
-    AddPartitionsRequest rqst = new AddPartitionsRequest("default", tableName,
-        Collections.singletonList(partition), true);
-    client.add_partitions_req(rqst);
+    partitions.add(new Partition(Collections.singletonList("z"), "default", tableName, 0,
+        0, sd, new HashMap<String, String>()));
+    client.add_partitions(partitions);
 
-    List<Partition> parts = client.get_partitions("default", tableName, (short)-1);
+    List<Partition> parts = client.listPartitions("default", tableName, (short)-1);
     Assert.assertNotNull(parts);
     Assert.assertEquals(3, parts.size());
 
-    parts = client.get_partitions_with_auth("default", tableName, (short)-1, "me",
+    parts = client.listPartitions("default", tableName, Collections.singletonList("x"),
+        (short)-1);
+    Assert.assertNotNull(parts);
+    Assert.assertEquals(1, parts.size());
+
+    parts = client.listPartitionsWithAuthInfo("default", tableName, (short)-1, "me",
         Collections.<String>emptyList());
     Assert.assertNotNull(parts);
     Assert.assertEquals(3, parts.size());
 
-    parts = client.get_partitions_ps("default", tableName, Collections.singletonList("x"),
-        (short)-1);
-    Assert.assertNotNull(parts);
-    Assert.assertEquals(1, parts.size());
+    List<String> partNames = client.listPartitionNames("default", tableName, (short)-1);
+    Assert.assertNotNull(partNames);
+    Assert.assertEquals(3, partNames.size());
 
-    parts = client.get_partitions_by_filter("default", tableName, "pk = \"x\"", (short)-1);
+    parts = client.listPartitionsByFilter("default", tableName, "pk = \"x\"", (short)-1);
     Assert.assertNotNull(parts);
     Assert.assertEquals(1, parts.size());
 
-    parts = client.get_partitions_by_names("default", tableName, Collections.singletonList("pk=x"));
+    parts = client.getPartitionsByNames("default", tableName, Collections.singletonList("pk=x"));
     Assert.assertNotNull(parts);
     Assert.assertEquals(1, parts.size());
 
-    partition = client.get_partition("default", tableName, Collections.singletonList("x"));
+    partition = client.getPartition("default", tableName, Collections.singletonList("x"));
     Assert.assertNotNull(partition);
 
-    partition = client.get_partition_by_name("default", tableName, "pk=x");
+    partition = client.getPartition("default", tableName, "pk=x");
     Assert.assertNotNull(partition);
 
-    partition = client.get_partition_with_auth("default", tableName, Collections.singletonList("x"),
+    partition = client.getPartitionWithAuthInfo("default", tableName, Collections.singletonList("x"),
         "me", Collections.<String>emptyList());
     Assert.assertNotNull(partition);
 
-    List<String> partitionNames = client.get_partition_names("default", tableName, (short)-1);
-    Assert.assertNotNull(partitionNames);
-    Assert.assertEquals(3, partitionNames.size());
-
     partition = new Partition(partition);
     partition.getParameters().put("a", "b");
     client.alter_partition("default", tableName, partition);
@@ -240,13 +230,8 @@ public class TestThrift {
 
     // Not testing partition marking events, not used by anyone but Hive replication AFAIK
 
-    client.drop_partition_by_name_with_environment_context("default", tableName, "pk=x", true,
-        envContext);
-    client.drop_partition_with_environment_context("default", tableName,
-        Collections.singletonList("y"), true, envContext);
-    DropPartitionsRequest dropRequest = new DropPartitionsRequest("default", tableName,
-        RequestPartsSpec.names(Collections.singletonList("pk=z")));
-    client.drop_partitions_req(dropRequest);
+    client.dropPartition("default", tableName, "pk=x", true);
+    client.dropPartition("default", tableName, Collections.singletonList("y"), true);
   }
 
   // Not testing index calls, as no one uses indices


[21/50] [abbrv] bigtop git commit: Added dependencies for apache commons-exec.

Posted by rv...@apache.org.
Added dependencies for apache commons-exec.


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

Branch: refs/heads/BIGTOP-2666
Commit: c67dfbcbbc7daaadbe434be66b371757b2ab9bbd
Parents: e04f24f
Author: Raj Desai <rd...@us.ibm.com>
Authored: Mon Oct 24 17:18:16 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:45:59 2017 -0800

----------------------------------------------------------------------
 bigtop-tests/spec-tests/runtime/build.gradle | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/c67dfbcb/bigtop-tests/spec-tests/runtime/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/build.gradle b/bigtop-tests/spec-tests/runtime/build.gradle
index 4df0dae..5505550 100644
--- a/bigtop-tests/spec-tests/runtime/build.gradle
+++ b/bigtop-tests/spec-tests/runtime/build.gradle
@@ -25,6 +25,7 @@ repositories {
 dependencies {
   compile group: 'junit', name: 'junit', version: junitVersion, transitive: 'true'
   compile group: 'commons-logging', name: 'commons-logging', version: '1.1.3'
+  compile group: 'org.apache.commons', name: 'commons-exec', version: '1.3'
   compile group: 'org.apache.hive', name: 'hive-jdbc', version: '1.2.1'
   compile group: 'org.apache.hive', name: 'hive-metastore', version: '1.2.1'
   compile group: 'org.apache.hive', name: 'hive-common', version: '1.2.1'


[36/50] [abbrv] bigtop git commit: created base beelineUrl for better arguments add

Posted by rv...@apache.org.
created base beelineUrl for better arguments add

created base beelineUrl for better arguments add


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

Branch: refs/heads/BIGTOP-2666
Commit: a03c50db872c2e820e6758af183348767a4f1234
Parents: 7d9763c
Author: roypradeep <ro...@us.ibm.com>
Authored: Tue Nov 1 18:48:26 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:06 2017 -0800

----------------------------------------------------------------------
 .../odpi/specs/runtime/hive/TestBeeline.java    | 226 +++++--------------
 1 file changed, 60 insertions(+), 166 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/a03c50db/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
index 37c71cf..682949d 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestBeeline.java
@@ -1,5 +1,4 @@
 package org.odpi.specs.runtime.hive;
-
 import org.apache.commons.exec.CommandLine;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -11,7 +10,6 @@ import java.io.FileNotFoundException;
 import java.io.PrintWriter;
 import java.util.Map;
 
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -31,45 +29,36 @@ import java.util.Map;
  */
 
 public class TestBeeline {
-
+	
 	public static final Log LOG = LogFactory.getLog(TestBeeline.class.getName());
-
 	private static final String URL = "odpiHiveTestJdbcUrl";
 	private static final String USER = "odpiHiveTestJdbcUser";
 	private static final String PASSWD = "odpiHiveTestJdbcPassword";
-
 	private static Map<String, String> results;
-
 	private static String beelineUrl; 
 	private static String beelineUser;
 	private static String beelinePasswd;
-	private static String testUrl;
-
-	//flags to check if username and password should be added as argument in some tests
-	private static boolean bothUserPass = false;
-	private static boolean onlyUser = false;
+	
+	//creating beeline base command with username and password as per inputs
+	private static CommandLine beelineBaseCommand = new CommandLine("beeline");
 
 	@BeforeClass
 	public static void initialSetup(){
-
 		TestBeeline.beelineUrl = System.getProperty(URL);
 		TestBeeline.beelineUser = System.getProperty(USER);
 		TestBeeline.beelinePasswd =System.getProperty(PASSWD);
-		TestBeeline.testUrl = System.getProperty(URL);
 
-		// Create Url with username and/or passowrd to handle all ways to connect to beeline
 		if (beelineUser != null && beelineUser != "" && beelinePasswd != null && beelinePasswd != "") 
 		{ 
-			testUrl = beelineUrl+" -n "+beelineUser+" -p "+beelinePasswd; 
-			bothUserPass=true;
+			beelineBaseCommand.addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd);
 		}
 		else if (beelineUser != null && beelineUser != "") 
 		{ 
-			testUrl = beelineUrl+" -n "+beelineUser; 
-			onlyUser=true;
+			beelineBaseCommand.addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser);
+		}
+		else {
+			beelineBaseCommand.addArgument("-u").addArgument(beelineUrl);
 		}
-		System.out.println("Setting url"+testUrl); 
-
 		LOG.info("URL is " + beelineUrl); 
 		LOG.info("User is " + beelineUser);
 		LOG.info("Passwd is " + beelinePasswd); 
@@ -78,124 +67,67 @@ public class TestBeeline {
 
 	@Test
 	public void checkBeeline() {
-
-		System.out.println(beelineUrl);  
-
-		results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(testUrl));
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand));
 		String consoleMsg = results.get("outputStream").toLowerCase();
-		Assert.assertEquals("beeline -u FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("connecting to "+beelineUrl) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-
-
+		Assert.assertEquals("beeline -u FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("connecting to "+beelineUrl.toLowerCase()) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
 	}
-
+	
 	@Test
 	public void checkBeelineConnect(){
-		try(PrintWriter out = new PrintWriter("connect.url")){ out.println("!connect " + beelineUrl+" "+beelineUser+" "+beelinePasswd+";"); out.println("!quit;"); } 
+		try(PrintWriter out = new PrintWriter("connect.url")){ out.println("!connect " + beelineUrl+" "+beelineUser+" "+beelinePasswd); out.println("!quit"); } 
 		catch (FileNotFoundException e1) {
-			
 			e1.printStackTrace();
 		}
 		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -f connect.url",false));
 		String consoleMsg = results.get("outputStream").toLowerCase();
-
-
-		Assert.assertEquals("beeline !connect FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("connecting to "+beelineUrl) && !consoleMsg.contains("error") && !consoleMsg.contains("exception") );  
+		Assert.assertEquals("beeline !connect FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("connecting to "+beelineUrl.toLowerCase()) && !consoleMsg.contains("error") && !consoleMsg.contains("exception") );  
 	}
-
+	
 	@Test
 	public void checkBeelineHelp(){
 		results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("--help"));
 		String consoleMsg = results.get("outputStream").toLowerCase();
 		Assert.assertEquals("beeline --help FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("display this message" ) && consoleMsg.contains("usage: java org.apache.hive.cli.beeline.beeline") && !consoleMsg.contains("exception"));
-
 	}
-
+	
 	@Test
 	public void checkBeelineQueryExecFromCmdLine(){
-
-		if (bothUserPass) 
-		{ 
-			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("SHOW DATABASES;"));
-
-			if(!results.get("outputStream").contains("odpi_runtime_hive")){
-
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("SHOW DATABASES;"));
-			}else{
-
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive;"));
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("SHOW DATABASES;"));
-
-			}
-			String consoleMsg = results.get("outputStream").toLowerCase();
-			Assert.assertEquals("beeline -e FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-
-			HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
-		}
-		else if (onlyUser) 
-		{ 
-			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("SHOW DATABASES;"));
-
-			if(!results.get("outputStream").contains("odpi_runtime_hive")){
-
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("SHOW DATABASES;"));
-			}else{
-
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive;"));
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("SHOW DATABASES;"));
-
-			}
-			String consoleMsg = results.get("outputStream").toLowerCase();
-			Assert.assertEquals("beeline -e FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-
-			HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
-		}
-		else {
-			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("SHOW DATABASES;"));
-
-			if(!results.get("outputStream").contains("odpi_runtime_hive")){
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
-
-			}else{
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive;"));
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
-				results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("SHOW DATABASES;"));
-			
-
-			}
-			String consoleMsg = results.get("outputStream").toLowerCase();
-			Assert.assertEquals("beeline -e FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-
-			HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-e").addArgument("SHOW DATABASES;"));
+		if(!results.get("outputStream").contains("odpi_runtime_hive")){
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-e").addArgument("SHOW DATABASES;"));
+		}else{
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive;"));
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive;"));
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-e").addArgument("SHOW DATABASES;"));
 		}
+		String consoleMsg = results.get("outputStream").toLowerCase();
+		Assert.assertEquals("beeline -e FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+		HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
 	}
-
+	
 	@Test
 	public void checkBeelineQueryExecFromFile() throws FileNotFoundException{
-
+		
 		try(PrintWriter out = new PrintWriter("beeline-f1.sql")){ out.println("SHOW DATABASES;"); }
 		try(PrintWriter out = new PrintWriter("beeline-f2.sql")){ out.println("CREATE DATABASE odpi_runtime_hive;"); }
 		try(PrintWriter out = new PrintWriter("beeline-f3.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); out.println("CREATE DATABASE odpi_runtime_hive;"); }
 		try(PrintWriter out = new PrintWriter("beeline-f4.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); }
-		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -f beeline-f1.sql",false));
-
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-f").addArgument("beeline-f1.sql",false));
+		
 		if(!results.get("outputStream").contains("odpi_runtime_hive")){
-			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -f beeline-f2.sql",false));
-
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-f").addArgument("beeline-f2.sql",false));
 		}else{
-			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -f beeline-f3.sql",false));
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-f").addArgument("beeline-f3.sql",false));
 		}
-		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -f beeline-f1.sql",false));
+		
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-f").addArgument("beeline-f1.sql",false));
 
 		String consoleMsg = results.get("outputStream").toLowerCase();
 		Assert.assertEquals("beeline -f FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_hive" ) && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-
-		HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -f beeline-f4.sql",false));		    
+		HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-f").addArgument("beeline-f4.sql",false));	
 	}
-
+	
 	@Test
 	public void checkBeelineInitFile() throws FileNotFoundException{
 
@@ -203,22 +135,20 @@ public class TestBeeline {
 		try(PrintWriter out = new PrintWriter("beeline-i2.sql")){ out.println("CREATE DATABASE odpi_runtime_beeline_init;"); }
 		try(PrintWriter out = new PrintWriter("beeline-i3.sql")){ out.println("DROP DATABASE odpi_runtime_beeline_init;"); out.println("CREATE DATABASE odpi_runtime_beeline_init;"); }
 		try(PrintWriter out = new PrintWriter("beeline-i4.sql")){ out.println("DROP DATABASE odpi_runtime_beeline_init;"); }
-
-		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -i beeline-i1.sql",false));
-
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-i").addArgument("beeline-i1.sql",false));
+	
 		if(!results.get("outputStream").contains("odpi_runtime_beeline_init")){
-			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -i beeline-i2.sql",false));
-
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-i").addArgument("beeline-i2.sql",false));
 		}else{
-			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -i beeline-i3.sql",false));
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-i").addArgument("beeline-i3.sql",false));
 		}
-		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -i beeline-i1.sql",false));
+		
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-i").addArgument("beeline-i1.sql",false));
 		String consoleMsg = results.get("outputStream").toLowerCase();
 		Assert.assertEquals("beeline -i FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_beeline_init") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-
-		HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" -i beeline-i4.sql",false));		    
+		HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("-i").addArgument("beeline-i4.sql",false));	
 	}
-
+	
 	@Test
 	public void checkBeelineHiveVar() throws FileNotFoundException{
 
@@ -226,80 +156,44 @@ public class TestBeeline {
 		try(PrintWriter out = new PrintWriter("beeline-hv2.sql")){ out.println("CREATE DATABASE ${db};"); }
 		try(PrintWriter out = new PrintWriter("beeline-hv3.sql")){ out.println("DROP DATABASE ${db};"); out.println("CREATE DATABASE ${db};"); }
 		try(PrintWriter out = new PrintWriter("beeline-hv4.sql")){ out.println("DROP DATABASE ${db};"); }
-
-		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv1.sql",false));
-		String consoleMsg = results.get("outputStream");
-
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("--hivevar").addArgument("db=odpi_runtime_beeline_hivevar").addArgument("-i").addArgument("beeline-hv1.sql",false));
+	
 		if(!results.get("outputStream").contains("odpi_runtime_beeline_hivevar")){
-			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv2.sql",false));
-
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("--hivevar").addArgument("db=odpi_runtime_beeline_hivevar").addArgument("-i").addArgument("beeline-hv2.sql",false));
 		}else{
-			results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv3.sql",false));
+			results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("--hivevar").addArgument("db=odpi_runtime_beeline_hivevar").addArgument("-i").addArgument("beeline-hv3.sql",false));
 		}
-		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv1.sql",false));
-		consoleMsg = results.get("outputStream").toLowerCase();
-
+		
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("--hivevar").addArgument("db=odpi_runtime_beeline_hivevar").addArgument("-i").addArgument("beeline-hv1.sql",false));
+		String consoleMsg = results.get("outputStream").toLowerCase();
 		Assert.assertEquals("beeline --hivevar FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("odpi_runtime_beeline_hivevar") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-
-		HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("beeline -u "+testUrl+" --hivevar db=odpi_runtime_beeline_hivevar -i beeline-hv4.sql",false));		    
+		HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("--hivevar").addArgument("db=odpi_runtime_beeline_hivevar").addArgument("-i").addArgument("beeline-hv4.sql",false));		 
 	}
-
+	
 	@Test
 	public void checkBeelineFastConnect(){
-		results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(testUrl).addArgument("--fastConnect=false"));
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("--fastConnect=false"));
 		String consoleMsg = results.get("outputStream").toLowerCase();
-		Assert.assertEquals("beeline --fastConnect FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("set fastconnect to true to skip")); 
+		Assert.assertEquals("beeline --fastConnect FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("set fastconnect to true to skip"));
 	}
 
 	@Test
 	public void checkBeelineVerbose(){
-
-		//explicit check for username password again as url containing -u -p is not working in single addArgument function with testUrl
-
-		if (bothUserPass) 
-		{ 
-			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("--verbose=true"));
-		}
-		else if (onlyUser) 
-		{ 
-			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("--verbose=true"));
-		}
-		else {
-			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("--verbose=true"));
-		}
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("--verbose=true"));
 		String consoleMsg = results.get("outputStream").toLowerCase();
-		Assert.assertEquals("beeline --verbose FAILED using url "+testUrl+". \n" +results.get("outputStream"), true, consoleMsg.contains("issuing: !connect jdbc:hive2:") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
+		Assert.assertEquals("beeline --verbose FAILED." +results.get("outputStream"), true, consoleMsg.contains("issuing: !connect jdbc:hive2:") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
 	}
-
+	
 	@Test
 	public void checkBeelineShowHeader(){
-		
-		//explicit check for username password again as url containing -u -p is not working in single addArgument function with testUrl
-
-		if (bothUserPass) 
-		{ 
-			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("-p").addArgument(beelinePasswd).addArgument("--showHeader=false").addArgument("-e").addArgument("SHOW DATABASES;"));
-		}
-		else if (onlyUser) 
-		{ 
-			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("-n").addArgument(beelineUser).addArgument("--showHeader=false").addArgument("-e").addArgument("SHOW DATABASES;"));
-		}
-		else {
-			results = HiveHelper.execCommand(new CommandLine("beeline").addArgument("-u").addArgument(beelineUrl).addArgument("--showHeader=false").addArgument("-e").addArgument("SHOW DATABASES;"));
-		}
+		results = HiveHelper.execCommand(new CommandLine(beelineBaseCommand).addArgument("--showHeader=false").addArgument("-e").addArgument("SHOW DATABASES;"));
 		String consoleMsg = results.get("outputStream").toLowerCase();
 		Assert.assertEquals("beeline --showHeader FAILED. \n" +results.get("outputStream"), true, consoleMsg.contains("default")&&!consoleMsg.contains("database_name") && !consoleMsg.contains("error") && !consoleMsg.contains("exception"));
-
 	}
 
 	@AfterClass
 	public static void cleanup() throws FileNotFoundException {
-
 		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf beeline*.sql", false));
 		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf connect.url", false));
-
 	}
-
-
-
 }


[44/50] [abbrv] bigtop git commit: ODPI-185. Test [HIVE_HCATALOG]

Posted by rv...@apache.org.
ODPI-185. Test [HIVE_HCATALOG]


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

Branch: refs/heads/BIGTOP-2666
Commit: f8275749afa956f5e024ff5637419c075e846558
Parents: e5518cd
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Tue Nov 8 19:24:44 2016 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:24 2017 -0800

----------------------------------------------------------------------
 bigtop-tests/spec-tests/runtime/build.gradle         | 13 +++++++++++++
 .../java/org/odpi/specs/runtime/hive/HCatalogMR.java | 11 -----------
 .../org/odpi/specs/runtime/hive/TestHCatalog.java    | 15 +++++----------
 3 files changed, 18 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/f8275749/bigtop-tests/spec-tests/runtime/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/build.gradle b/bigtop-tests/spec-tests/runtime/build.gradle
index f0166c9..a88a3b6 100644
--- a/bigtop-tests/spec-tests/runtime/build.gradle
+++ b/bigtop-tests/spec-tests/runtime/build.gradle
@@ -43,7 +43,20 @@ dependencies {
   if (System.env.HADOOP_CONF_DIR) testRuntime files(System.env.HADOOP_CONF_DIR)
 }
 
+jar {
+    from {
+        (configurations.runtime).grep{it.toString() =~ /(hive|libfb303)-.*[jw]ar$/}.collect {
+              zipTree(it)
+        }
+    }
+
+    exclude 'META-INF/*.RSA', 'META-INF/*.SF','META-INF/*.DSA'
+}
+
 test {
   // Change the default location where test data is picked up
   systemProperty 'test.resources.dir', "${buildDir}/resources/test/"
+  systemProperty 'odpi.test.hive.hcat.job.jar', jar.archivePath
+  systemProperty 'odpi.test.hive.hcat.core.jar', (configurations.runtime).find { it.toString() =~ /hive-hcatalog-core-.*jar$/ }
 }
+test.dependsOn jar

http://git-wip-us.apache.org/repos/asf/bigtop/blob/f8275749/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java b/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
index a6ff375..7cb9bbe 100644
--- a/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
+++ b/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
@@ -72,17 +72,6 @@ public class HCatalogMR extends Configured implements Tool {
     HCatOutputFormat.setSchema(job, HCatSchemaUtils.getHCatSchema(outputSchemaStr));
     job.setOutputFormatClass(HCatOutputFormat.class);
 
-    // TODO All four of these jars need to be in the distributed cache of the job for the job to
-    // succeed.  I loaded them into a known location in HDFS to get them in the cache.  There may
-    // be a way to load them from a file on the gateway machine.  We could also put in a hdfs dfs
-    // -put operation into a gradle step as part of the build so that the jars are picked up from
-    // the distribution and put in a known location in HDFS from when they can be picked up in
-    // the distributed cache.
-    job.addCacheArchive(new URI("hdfs:/user/gates/hive-hcatalog-core-1.2.1.jar"));
-    job.addCacheArchive(new URI("hdfs:/user/gates/hive-metastore-1.2.1.jar"));
-    job.addCacheArchive(new URI("hdfs:/user/gates/hive-exec-1.2.1.jar"));
-    job.addCacheArchive(new URI("hdfs:/user/gates/libfb303-0.9.2.jar"));
-
     return job.waitForCompletion(true) ? 0 : 1;
 
 

http://git-wip-us.apache.org/repos/asf/bigtop/blob/f8275749/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
index 87e3eb0..b51db02 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestHCatalog.java
@@ -51,6 +51,8 @@ import java.util.Random;
 
 
 public class TestHCatalog {
+  private static final String JOBJAR = "odpi.test.hive.hcat.job.jar";
+  private static final String HCATCORE = "odpi.test.hive.hcat.core.jar";
 
   private static final Log LOG = LogFactory.getLog(TestHCatalog.class.getName());
 
@@ -132,18 +134,11 @@ public class TestHCatalog {
     out.close();
 
     Map<String, String> env = new HashMap<>();
-    // TODO These need to be set from the environment rather than hard wired
-    env.put("HADOOP_HOME","/Users/gates/grid/odpi-testing/hadoop-2.7.3");
-    env.put("HADOOP_CLASSPATH", "/Users/gates/grid/odpi-testing/apache-hive-1.2.1-bin/hcatalog/share/hcatalog/hive-hcatalog-core-1.2.1.jar");
-    env.put("HIVE_HOME", "/Users/gates/grid/odpi-testing/apache-hive-1.2.1-bin");
-    Map<String, String> results = HiveHelper.execCommand(new CommandLine("/Users/gates/grid/odpi-testing/apache-hive-1.2.1-bin/bin/hive")
+    env.put("HADOOP_CLASSPATH", System.getProperty(HCATCORE, ""));
+    Map<String, String> results = HiveHelper.execCommand(new CommandLine("hive")
         .addArgument("--service")
         .addArgument("jar")
-        // TODO This is the jar built by gradle, but I didn't know how to take the jar built in
-        // the build phase and reference it in the test phase.  Perhaps a move operation could be
-        // put in the middle so the jar is moved to a known location that can be referenced here,
-        // or maybe gradle can pass in its working directory so that we can reference it from there.
-        .addArgument("/Users/gates/git/bigtop/runtime-1.2.0-SNAPSHOT.jar")
+        .addArgument(System.getProperty(JOBJAR))
         .addArgument(HCatalogMR.class.getName())
         .addArgument(inputTable)
         .addArgument(outputTable)


[46/50] [abbrv] bigtop git commit: ODPI-200. Removed SNAPSHOT from version.

Posted by rv...@apache.org.
ODPI-200. Removed SNAPSHOT from version.


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

Branch: refs/heads/BIGTOP-2666
Commit: 4e502424d243e2eef2dbec198c0d5e860cc53892
Parents: cfc6685
Author: Raj Desai <rd...@us.ibm.com>
Authored: Thu Nov 10 15:48:53 2016 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:25 2017 -0800

----------------------------------------------------------------------
 build.gradle | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/4e502424/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 0aff57d..c0fc7e3 100644
--- a/build.gradle
+++ b/build.gradle
@@ -146,7 +146,7 @@ allprojects {
   apply plugin: 'maven'
 
   group = 'org.apache.bigtop'
-  version = '1.2.0-SNAPSHOT'
+  version = '1.2.0'
 
   description = """Bigtop"""
 


[25/50] [abbrv] bigtop git commit: Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-plugin-properties.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-plugin-properties.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-plugin-properties.xml
deleted file mode 100755
index 346baa9..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-plugin-properties.xml
+++ /dev/null
@@ -1,63 +0,0 @@
-<configuration><property require-input="false">
-    <name>policy_user</name>
-    <value>ambari-qa</value>
-    <description>This user must be system user and also present at Ranger admin portal</description>
-    <display-name>Policy user for HIVE</display-name>
-    <filename>ranger-hive-plugin-properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>jdbc.driverClassName</name>
-    <value>org.apache.hive.jdbc.HiveDriver</value>
-    <description>Used for repository creation on ranger admin</description>
-    <filename>ranger-hive-plugin-properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>common.name.for.certificate</name>
-    <value></value>
-    <description>Common name for certificate, this value should match what is specified in repo within ranger admin</description>
-    <filename>ranger-hive-plugin-properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <empty-value-valid>true</empty-value-valid>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>REPOSITORY_CONFIG_USERNAME</name>
-    <value>hive</value>
-    <description>Used for repository creation on ranger admin</description>
-    <display-name>Ranger repository config user</display-name>
-    <filename>ranger-hive-plugin-properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>REPOSITORY_CONFIG_PASSWORD</name>
-    <value>hive</value>
-    <description>Used for repository creation on ranger admin</description>
-    <display-name>Ranger repository config password</display-name>
-    <filename>ranger-hive-plugin-properties.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type>PASSWORD</property-type>
-    <value-attributes>
-        <type>password</type>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-policymgr-ssl.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-policymgr-ssl.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-policymgr-ssl.xml
deleted file mode 100755
index bf73d9f..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-policymgr-ssl.xml
+++ /dev/null
@@ -1,71 +0,0 @@
-<configuration><property require-input="false">
-    <name>xasecure.policymgr.clientssl.keystore</name>
-    <value>/usr/hdp/current/hive-server2/conf/ranger-plugin-keystore.jks</value>
-    <description>Java Keystore files</description>
-    <filename>ranger-hive-policymgr-ssl.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.policymgr.clientssl.keystore.password</name>
-    <value>myKeyFilePassword</value>
-    <description>password for keystore</description>
-    <filename>ranger-hive-policymgr-ssl.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type>PASSWORD</property-type>
-    <value-attributes>
-        <type>password</type>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.policymgr.clientssl.truststore</name>
-    <value>/usr/hdp/current/hive-server2/conf/ranger-plugin-truststore.jks</value>
-    <description>java truststore file</description>
-    <filename>ranger-hive-policymgr-ssl.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.policymgr.clientssl.truststore.password</name>
-    <value>changeit</value>
-    <description>java truststore password</description>
-    <filename>ranger-hive-policymgr-ssl.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type>PASSWORD</property-type>
-    <value-attributes>
-        <type>password</type>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.policymgr.clientssl.keystore.credential.file</name>
-    <value>jceks://file{{credential_file}}</value>
-    <description>java keystore credential file</description>
-    <filename>ranger-hive-policymgr-ssl.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.policymgr.clientssl.truststore.credential.file</name>
-    <value>jceks://file{{credential_file}}</value>
-    <description>java truststore credential file</description>
-    <filename>ranger-hive-policymgr-ssl.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-security.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-security.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-security.xml
deleted file mode 100755
index a29780f..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/ranger-hive-security.xml
+++ /dev/null
@@ -1,81 +0,0 @@
-<configuration><property require-input="false">
-    <name>ranger.plugin.hive.policy.rest.ssl.config.file</name>
-    <value>/usr/hdp/current/{{ranger_hive_component}}/conf/conf.server/ranger-policymgr-ssl.xml</value>
-    <description>Path to the file containing SSL details to contact Ranger Admin</description>
-    <filename>ranger-hive-security.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>ranger.plugin.hive.service.name</name>
-    <value>{{repo_name}}</value>
-    <description>Name of the Ranger service containing policies for this HIVE instance</description>
-    <filename>ranger-hive-security.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>ranger.plugin.hive.policy.source.impl</name>
-    <value>org.apache.ranger.admin.client.RangerAdminRESTClient</value>
-    <description>Class to retrieve policies from the source</description>
-    <filename>ranger-hive-security.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>ranger.plugin.hive.policy.rest.url</name>
-    <value>{{policymgr_mgr_url}}</value>
-    <description>URL to Ranger Admin</description>
-    <filename>ranger-hive-security.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>ranger.plugin.hive.policy.pollIntervalMs</name>
-    <value>30000</value>
-    <description>How often to poll for changes in policies?</description>
-    <filename>ranger-hive-security.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>ranger.plugin.hive.policy.cache.dir</name>
-    <value>/etc/ranger/{{repo_name}}/policycache</value>
-    <description>Directory where Ranger policies are cached after successful retrieval from the source</description>
-    <filename>ranger-hive-security.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>xasecure.hive.update.xapolicies.on.grant.revoke</name>
-    <value>true</value>
-    <description>Should Hive plugin update Ranger policies for updates to permissions done using GRANT/REVOKE?</description>
-    <display-name>Should Hive GRANT/REVOKE update XA policies</display-name>
-    <filename>ranger-hive-security.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>boolean</type>
-    </value-attributes>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/tez-interactive-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/tez-interactive-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/tez-interactive-site.xml
deleted file mode 100755
index 3865c36..0000000
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/tez-interactive-site.xml
+++ /dev/null
@@ -1,144 +0,0 @@
-<configuration><property require-input="false">
-    <name>tez.runtime.shuffle.fetch.buffer.percent</name>
-    <value>0.6</value>
-    <description>Fraction (0-1) of the available memory which can be used to
-      retain shuffled data</description>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>tez.runtime.shuffle.memory.limit.percent</name>
-    <value>0.25</value>
-    <description>This property determines the maximum size of a shuffle segment
-      which can be fetched to memory. Fraction (0-1) of shuffle memory
-      (after applying tez.runtime.shuffle.fetch.buffer.percent)</description>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>tez.runtime.report.partition.stats</name>
-    <value>true</value>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>tez.runtime.pipelined-shuffle.enabled</name>
-    <value>false</value>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>tez.runtime.pipelined.sorter.lazy-allocate.memory</name>
-    <value>true</value>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>tez.grouping.node.local.only</name>
-    <value>true</value>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>tez.runtime.shuffle.fetch.verify-disk-checksum</name>
-    <value>false</value>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>tez.lib.uris</name>
-    <value>/hdp/apps/${hdp.version}/tez_hive2/tez.tar.gz</value>
-    <description>Comma-delimited list of the location of the Tez libraries which will be localized for DAGs.
-      Specifying a single .tar.gz or .tgz assumes that a compressed version of the tez libs is being used. This is uncompressed into a tezlibs directory when running containers, and tezlibs/;tezlibs/lib/ are added to the classpath (after . and .*).
-      If multiple files are specified - files are localized as regular files, contents of directories are localized as regular files (non-recursive).
-    </description>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>tez.dag.recovery.enabled</name>
-    <value>false</value>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="false" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>tez.am.resource.memory.mb</name>
-    <value>1536</value>
-    <description>The amount of memory to be used by the AppMaster</description>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes>
-        <type>int</type>
-    </value-attributes>
-    <depends-on>
-        <property>
-            <name>yarn.scheduler.capacity.root.queues</name>
-            <type>capacity-scheduler</type>
-        </property>
-        <property>
-            <name>hive.llap.daemon.queue.name</name>
-            <type>hive-interactive-site</type>
-        </property>
-        <property>
-            <name>llap_queue_capacity</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>enable_hive_interactive</name>
-            <type>hive-interactive-env</type>
-        </property>
-        <property>
-            <name>hive.server2.tez.sessions.per.default.queue</name>
-            <type>hive-interactive-site</type>
-        </property>
-    </depends-on>
-    <property_depended_by/>
-</property><property require-input="false">
-    <name>tez.session.am.dag.submit.timeout.secs</name>
-    <value>3600</value>
-    <filename>tez-interactive-site.xml</filename>
-    <deleted>false</deleted>
-    <on-ambari-upgrade add="true" delete="false" update="false"/>
-    <property-type></property-type>
-    <value-attributes/>
-    <depends-on/>
-    <property_depended_by/>
-</property></configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/metainfo.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/metainfo.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/metainfo.xml
index d6ecbed..9e73118 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/metainfo.xml
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/metainfo.xml
@@ -3,7 +3,7 @@
   <services><service>
     <name>HIVE</name>
     <displayName>Hive</displayName>
-    <version>1.2.1</version>
+    <version>1.2.1+odpi</version>
     <comment>Data warehouse system for ad-hoc queries &amp; analysis of large datasets and table &amp; storage management service</comment>
     <components>
         <component>
@@ -12,7 +12,7 @@
             <category>CLIENT</category>
             <deleted>false</deleted>
             <cardinality>0+</cardinality>
-            <versionAdvertised>true</versionAdvertised>
+            <versionAdvertised>false</versionAdvertised>
             <versionAdvertisedInternal>false</versionAdvertisedInternal>
             <commandScript>
                 <script>scripts/hcat_client.py</script>
@@ -37,7 +37,7 @@
             <category>MASTER</category>
             <deleted>false</deleted>
             <cardinality>1+</cardinality>
-            <versionAdvertisedInternal>true</versionAdvertisedInternal>
+            <versionAdvertisedInternal>false</versionAdvertisedInternal>
             <commandScript>
                 <script>scripts/hive_server.py</script>
                 <scriptType>PYTHON</scriptType>
@@ -108,7 +108,7 @@
             <category>CLIENT</category>
             <deleted>false</deleted>
             <cardinality>1+</cardinality>
-            <versionAdvertised>true</versionAdvertised>
+            <versionAdvertised>false</versionAdvertised>
             <versionAdvertisedInternal>false</versionAdvertisedInternal>
             <commandScript>
                 <script>scripts/hive_client.py</script>
@@ -148,7 +148,7 @@
             <category>MASTER</category>
             <deleted>false</deleted>
             <cardinality>1+</cardinality>
-            <versionAdvertisedInternal>true</versionAdvertisedInternal>
+            <versionAdvertisedInternal>false</versionAdvertisedInternal>
             <commandScript>
                 <script>scripts/webhcat_server.py</script>
                 <scriptType>PYTHON</scriptType>
@@ -272,7 +272,7 @@
             <category>MASTER</category>
             <deleted>false</deleted>
             <cardinality>1+</cardinality>
-            <versionAdvertisedInternal>true</versionAdvertisedInternal>
+            <versionAdvertisedInternal>false</versionAdvertisedInternal>
             <commandScript>
                 <script>scripts/hive_metastore.py</script>
                 <scriptType>PYTHON</scriptType>
@@ -295,149 +295,8 @@
             <recovery_enabled>false</recovery_enabled>
             <reassignAllowed>true</reassignAllowed>
         </component>
-        <component>
-            <name>HIVE_SERVER_INTERACTIVE</name>
-            <displayName>HiveServer2 Interactive</displayName>
-            <category>MASTER</category>
-            <deleted>false</deleted>
-            <cardinality>0-1</cardinality>
-            <versionAdvertised>true</versionAdvertised>
-            <versionAdvertisedInternal>false</versionAdvertisedInternal>
-            <commandScript>
-                <script>scripts/hive_server_interactive.py</script>
-                <scriptType>PYTHON</scriptType>
-                <timeout>0</timeout>
-            </commandScript>
-            <customCommands>
-                <customCommand>
-                    <name>RESTART_LLAP</name>
-                    <commandScript>
-                        <script>scripts/hive_server_interactive.py</script>
-                        <scriptType>PYTHON</scriptType>
-                        <timeout>600</timeout>
-                    </commandScript>
-                    <background>false</background>
-                </customCommand>
-            </customCommands>
-            <dependencies>
-                <dependency>
-                    <auto-deploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </auto-deploy>
-                    <autoDeploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </autoDeploy>
-                    <name>ZOOKEEPER/ZOOKEEPER_SERVER</name>
-                    <scope>cluster</scope>
-                </dependency>
-                <dependency>
-                    <auto-deploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </auto-deploy>
-                    <autoDeploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </autoDeploy>
-                    <name>YARN/YARN_CLIENT</name>
-                    <scope>host</scope>
-                </dependency>
-                <dependency>
-                    <auto-deploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </auto-deploy>
-                    <autoDeploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </autoDeploy>
-                    <name>HDFS/HDFS_CLIENT</name>
-                    <scope>host</scope>
-                </dependency>
-                <dependency>
-                    <auto-deploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </auto-deploy>
-                    <autoDeploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </autoDeploy>
-                    <name>MAPREDUCE2/MAPREDUCE2_CLIENT</name>
-                    <scope>host</scope>
-                </dependency>
-                <dependency>
-                    <auto-deploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </auto-deploy>
-                    <autoDeploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </autoDeploy>
-                    <name>TEZ/TEZ_CLIENT</name>
-                    <scope>host</scope>
-                </dependency>
-                <dependency>
-                    <auto-deploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </auto-deploy>
-                    <autoDeploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </autoDeploy>
-                    <name>PIG/PIG</name>
-                    <scope>host</scope>
-                </dependency>
-                <dependency>
-                    <auto-deploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </auto-deploy>
-                    <autoDeploy>
-                        <co-locate>HIVE/HIVE_SERVER_INTERACTIVE</co-locate>
-                        <coLocate>HIVE/HIVE_SERVER_INTERACTIVE</coLocate>
-                        <enabled>true</enabled>
-                    </autoDeploy>
-                    <name>SLIDER/SLIDER</name>
-                    <scope>host</scope>
-                </dependency>
-            </dependencies>
-            <configuration-dependencies>
-                <config-type>beeline-log4j2</config-type>
-                <config-type>hive-exec-log4j2</config-type>
-                <config-type>hive-log4j2</config-type>
-                <config-type>hive-site</config-type>
-                <config-type>hive-interactive-site</config-type>
-                <config-type>tez-interactive-site</config-type>
-                <config-type>hiveserver2-interactive-site</config-type>
-                <config-type>hive-interactive-env</config-type>
-                <config-type>llap-cli-log4j2</config-type>
-                <config-type>llap-daemon-log4j</config-type>
-            </configuration-dependencies>
-            <recovery_enabled>false</recovery_enabled>
-        </component>
     </components>
-    <deleted>false</deleted>
     <configuration-dependencies>
-        <config-type>application-properties</config-type>
-        <config-type>hive-atlas-application.properties</config-type>
         <config-type>hive-log4j</config-type>
         <config-type>hive-exec-log4j</config-type>
         <config-type>tez-site</config-type>
@@ -445,12 +304,7 @@
         <config-type>hivemetastore-site.xml</config-type>
         <config-type>webhcat-site</config-type>
         <config-type>webhcat-env</config-type>
-        <config-type>ranger-hive-plugin-properties</config-type>
-        <config-type>ranger-hive-audit</config-type>
-        <config-type>ranger-hive-policymgr-ssl</config-type>
-        <config-type>ranger-hive-security</config-type>
         <config-type>mapred-site</config-type>
-        <config-type>application.properties</config-type>
     </configuration-dependencies>
     <widgetsFileName>widgets.json</widgetsFileName>
     <metricsFileName>metrics.json</metricsFileName>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
index 1cd58c3..b33d715 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
@@ -93,12 +93,18 @@ stack_supports_ranger_audit_db = check_stack_feature(StackFeature.RANGER_AUDIT_D
 component_directory = status_params.component_directory
 component_directory_interactive = status_params.component_directory_interactive
 
-hadoop_home = format('{stack_root}/current/hadoop-client')
-hive_bin = format('{stack_root}/current/{component_directory}/bin')
-hive_schematool_ver_bin = format('{stack_root}/{version}/hive/bin')
-hive_schematool_bin = format('{stack_root}/current/{component_directory}/bin')
-hive_lib = format('{stack_root}/current/{component_directory}/lib')
-hive_version_lib = format('{stack_root}/{version}/hive/lib')
+hadoop_home = '/usr/lib/hadoop'
+hive_bin = '/usr/lib/hive/bin'
+hive_schematool_ver_bin = '/usr/lib/hive/bin'
+hive_schematool_bin = '/usr/lib/hive/bin'
+hive_lib = '/usr/lib/hive/lib'
+hive_version_lib = '/usr/lib/hive/lib'
+#hadoop_home = format('{stack_root}/current/hadoop-client')
+#hive_bin = format('{stack_root}/current/{component_directory}/bin')
+#hive_schematool_ver_bin = format('{stack_root}/{version}/hive/bin')
+#hive_schematool_bin = format('{stack_root}/current/{component_directory}/bin')
+#hive_lib = format('{stack_root}/current/{component_directory}/lib')
+#hive_version_lib = format('{stack_root}/{version}/hive/lib')
 hive_var_lib = '/var/lib/hive'
 hive_user_home_dir = "/home/hive"
 
@@ -167,10 +173,6 @@ config_dir = '/etc/hive-webhcat/conf'
 hcat_lib = '/usr/lib/hive-hcatalog/share/hcatalog'
 webhcat_bin_dir = '/usr/lib/hive-hcatalog/sbin'
 
-# there are no client versions of these, use server versions directly
-hcat_lib = format('{stack_root}/current/hive-webhcat/share/hcatalog')
-webhcat_bin_dir = format('{stack_root}/current/hive-webhcat/sbin')
-
 # --- Tarballs ---
 # DON'T CHANGE THESE VARIABLE NAMES
 # Values don't change from those in copy_tarball.py
@@ -189,17 +191,14 @@ tarballs_mode = 0444
 
 purge_tables = "false"
 # Starting from stack version for feature hive_purge_table drop should be executed with purge
-if check_stack_feature(StackFeature.HIVE_PURGE_TABLE, version_for_stack_feature_checks):
-  purge_tables = 'true'
+purge_tables = 'true'
 
-if check_stack_feature(StackFeature.HIVE_WEBHCAT_SPECIFIC_CONFIGS, version_for_stack_feature_checks):
-  # this is NOT a typo.  Configs for hcatalog/webhcat point to a
-  # specific directory which is NOT called 'conf'
-  hcat_conf_dir = format('{stack_root}/current/hive-webhcat/etc/hcatalog')
-  config_dir = format('{stack_root}/current/hive-webhcat/etc/webhcat')
+# this is NOT a typo.  Configs for hcatalog/webhcat point to a
+# specific directory which is NOT called 'conf'
+hcat_conf_dir = format('{stack_root}/current/hive-webhcat/etc/hcatalog')
+config_dir = format('{stack_root}/current/hive-webhcat/etc/webhcat')
 
-if check_stack_feature(StackFeature.HIVE_METASTORE_SITE_SUPPORT, version_for_stack_feature_checks):
-  hive_metastore_site_supported = True
+hive_metastore_site_supported = True
 
 execute_path = os.environ['PATH'] + os.pathsep + hive_bin + os.pathsep + hadoop_bin_dir
 
@@ -395,10 +394,7 @@ start_metastore_path = format("{tmp_dir}/start_metastore_script")
 hadoop_heapsize = config['configurations']['hadoop-env']['hadoop_heapsize']
 
 if 'role' in config and config['role'] in ["HIVE_SERVER", "HIVE_METASTORE"]:
-  if check_stack_feature(StackFeature.HIVE_ENV_HEAPSIZE, version_for_stack_feature_checks):
-    hive_heapsize = config['configurations']['hive-env']['hive.heapsize']
-  else:
-    hive_heapsize = config['configurations']['hive-site']['hive.heapsize']
+  hive_heapsize = config['configurations']['hive-env']['hive.heapsize']
 else:
   hive_heapsize = config['configurations']['hive-env']['hive.client.heapsize']
 

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
index b7cb148..024f3df 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
@@ -92,10 +92,14 @@ else:
   hive_server_conf_dir = "/etc/hive/conf.server"
   hive_server_interactive_conf_dir = "/etc/hive2/conf.server"
 
-  webhcat_conf_dir = format("{stack_root}/current/hive-webhcat/conf")
-  hive_home_dir = format("{stack_root}/current/{component_directory}")
-  hive_conf_dir = format("{stack_root}/current/{component_directory}/conf")
-  hive_client_conf_dir = format("{stack_root}/current/{component_directory}/conf")
+#  webhcat_conf_dir = format("{stack_root}/current/hive-webhcat/conf")
+#  hive_home_dir = format("{stack_root}/current/{component_directory}")
+#  hive_conf_dir = format("{stack_root}/current/{component_directory}/conf")
+#  hive_client_conf_dir = format("{stack_root}/current/{component_directory}/conf")
+  webhcat_conf_dir = '/etc/hive/conf'
+  hive_home_dir = '/usr/lib/hive'
+  hive_conf_dir = '/usr/lib/hive/conf'
+  hive_client_conf_dir = '/etc/hive/conf'
 
   if check_stack_feature(StackFeature.CONFIG_VERSIONING, stack_version_formatted_major):
     hive_server_conf_dir = format("{stack_root}/current/{component_directory}/conf/conf.server")
@@ -115,4 +119,4 @@ else:
   if 'role' in config and config['role'] in ["HIVE_SERVER", "HIVE_METASTORE", "HIVE_SERVER_INTERACTIVE"]:
     hive_config_dir = hive_server_conf_dir
     
-stack_name = default("/hostLevelParams/stack_name", None)
\ No newline at end of file
+stack_name = default("/hostLevelParams/stack_name", None)

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/configuration/yarn-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/configuration/yarn-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/configuration/yarn-site.xml
index b20114c..59ff82b 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/configuration/yarn-site.xml
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/configuration/yarn-site.xml
@@ -19,13 +19,6 @@
 <!-- Put site-specific property overrides in this file. -->
 <configuration xmlns:xi="http://www.w3.org/2001/XInclude" supports_final="true">
   <property>
-    <name>yarn.timeline-service.leveldb-timeline-store.path</name>
-    <value>/tmp/dummy</value>
-    <description>This is a temporary workaround for ODPI-186</description>
-  </property>
-
-  <!-- ResourceManager -->
-  <property>
     <name>yarn.resourcemanager.hostname</name>
     <value>localhost</value>
     <description>The hostname of the RM.</description>
@@ -113,25 +106,6 @@
     <on-ambari-upgrade add="true"/>
   </property>
   <property>
-    <name>yarn.acl.enable</name>
-    <value>false</value>
-    <description> Are acls enabled. </description>
-    <value-attributes>
-      <type>boolean</type>
-    </value-attributes>
-    <on-ambari-upgrade add="true"/>
-  </property>
-  <property>
-    <name>yarn.admin.acl</name>
-    <value/>
-    <description> ACL of who can be admin of the YARN cluster. </description>
-    <value-attributes>
-      <empty-value-valid>true</empty-value-valid>
-    </value-attributes>
-    <on-ambari-upgrade add="true"/>
-  </property>
-  <!-- NodeManager -->
-  <property>
     <name>yarn.nodemanager.address</name>
     <value>0.0.0.0:45454</value>
     <description>The address of the container manager in the NM.</description>
@@ -421,4 +395,185 @@
     </description>
     <on-ambari-upgrade add="true"/>
   </property>
+====================
+  <property>
+    <name>yarn.timeline-service.enabled</name>
+    <value>true</value>
+    <description>Indicate to clients whether timeline service is enabled or not.
+      If enabled, clients will put entities and events to the timeline server.
+    </description>
+    <value-attributes>
+      <type>boolean</type>
+    </value-attributes>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.generic-application-history.store-class</name>
+    <value>org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore</value>
+    <description>
+      Store class name for history store, defaulting to file system store
+    </description>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.leveldb-timeline-store.path</name>
+    <value>/var/log/hadoop-yarn/timeline</value>
+    <description>
+      Store file name for leveldb timeline store
+    </description>
+    <value-attributes>
+      <type>directory</type>
+    </value-attributes>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.webapp.address</name>
+    <value>localhost:8188</value>
+    <description>
+      The http address of the timeline service web application.
+    </description>
+    <on-ambari-upgrade add="false"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.webapp.https.address</name>
+    <value>localhost:8190</value>
+    <description>
+      The http address of the timeline service web application.
+    </description>
+    <on-ambari-upgrade add="false"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.address</name>
+    <value>localhost:10200</value>
+    <description>
+      This is default address for the timeline server to start
+      the RPC server.
+    </description>
+    <on-ambari-upgrade add="false"/>
+  </property>
+  <property>
+    <description>Enable age off of timeline store data.</description>
+    <name>yarn.timeline-service.ttl-enable</name>
+    <value>true</value>
+    <value-attributes>
+      <type>boolean</type>
+    </value-attributes>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <description>Time to live for timeline store data in milliseconds.</description>
+    <name>yarn.timeline-service.ttl-ms</name>
+    <value>2678400000</value>
+    <value-attributes>
+      <type>int</type>
+    </value-attributes>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <description>Length of time to wait between deletion cycles of leveldb timeline store in milliseconds.</description>
+    <name>yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms</name>
+    <value>300000</value>
+    <value-attributes>
+      <type>int</type>
+    </value-attributes>
+    <on-ambari-upgrade add="true"/>
+  </property>
+=============================
+  <property>
+    <name>yarn.timeline-service.recovery.enabled</name>
+    <description>
+      Enable timeline server to recover state after starting. If
+      true, then yarn.timeline-service.state-store-class must be specified.
+    </description>
+    <value>true</value>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.acl.enable</name>
+    <value>false</value>
+    <description> Are acls enabled. </description>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.authorization-provider</name>
+    <description> Yarn authorization provider class. </description>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.admin.acl</name>
+    <value>yarn</value>
+    <description> ACL of who can be admin of the YARN cluster. </description>
+    <value-attributes>
+      <empty-value-valid>true</empty-value-valid>
+    </value-attributes>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <!--ats v1.5 properties-->
+  <property>
+    <name>yarn.timeline-service.store-class</name>
+    <value>org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore</value>
+    <description>Main storage class for YARN timeline server.</description>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.entity-group-fs-store.active-dir</name>
+    <value>/ats/active/</value>
+    <description>DFS path to store active application&#x2019;s timeline data</description>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.entity-group-fs-store.done-dir</name>
+    <value>/ats/done/</value>
+    <description>DFS path to store done application&#x2019;s timeline data</description>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.entity-group-fs-store.group-id-plugin-classes</name>
+    <value/>
+    <description>Plugins that can translate a timeline entity read request into a list of timeline cache ids, separated by commas. </description>
+    <value-attributes>
+      <empty-value-valid>true</empty-value-valid>
+    </value-attributes>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <!-- advanced ats v1.5 properties-->
+  <property>
+    <name>yarn.timeline-service.entity-group-fs-store.summary-store</name>
+    <description>Summary storage for ATS v1.5</description>
+    <!-- Use rolling leveldb, advanced -->
+    <value>org.apache.hadoop.yarn.server.timeline.RollingLevelDBTimelineStore</value>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.entity-group-fs-store.scan-interval-seconds</name>
+    <description>
+      Scan interval for ATS v1.5 entity group file system storage reader.This
+      value controls how frequent the reader will scan the HDFS active directory
+      for application status.
+    </description>
+    <!-- Default is 60 seconds, advanced -->
+    <value>60</value>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds</name>
+    <description>
+      Scan interval for ATS v1.5 entity group file system storage cleaner.This
+      value controls how frequent the reader will scan the HDFS done directory
+      for stale application data.
+    </description>
+    <!-- 3600 is default, advanced -->
+    <value>3600</value>
+    <on-ambari-upgrade add="true"/>
+  </property>
+  <property>
+    <name>yarn.timeline-service.entity-group-fs-store.retain-seconds</name>
+    <description>
+      How long the ATS v1.5 entity group file system storage will keep an
+      application's data in the done directory.
+    </description>
+    <!-- 7 days is default, advanced -->
+    <value>604800</value>
+    <on-ambari-upgrade add="true"/>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/623e0332/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/metainfo.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/metainfo.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/metainfo.xml
index 18117f3..35da7fd 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/metainfo.xml
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/YARN/metainfo.xml
@@ -26,6 +26,20 @@
       <version>2.7.1+odpi</version>
       <components>
 
+      <component>
+          <name>APP_TIMELINE_SERVER</name>
+          <displayName>App Timeline Server</displayName>
+          <category>MASTER</category>
+          <cardinality>0-1</cardinality>
+          <versionAdvertised>true</versionAdvertised>
+          <reassignAllowed>true</reassignAllowed>
+          <commandScript>
+            <script>scripts/application_timeline_server.py</script>
+            <scriptType>PYTHON</scriptType>
+            <timeout>1200</timeout>
+          </commandScript>
+        </component>
+
         <component>
           <name>RESOURCEMANAGER</name>
           <displayName>ResourceManager</displayName>


[28/50] [abbrv] bigtop git commit: Making ODPi Ambari stack compatible with ODPi packaging

Posted by rv...@apache.org.
Making ODPi Ambari stack compatible with ODPi packaging


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

Branch: refs/heads/BIGTOP-2666
Commit: b1cc82ce5a370f77d5c3f8e982c92140f02474ad
Parents: 307c4d0
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Fri Oct 28 10:54:16 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:46:01 2017 -0800

----------------------------------------------------------------------
 .../services/HIVE/configuration/hive-site.xml   |  2 +-
 .../HIVE/package/files/templetonSmoke.sh        | 37 --------------------
 2 files changed, 1 insertion(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/b1cc82ce/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-site.xml b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-site.xml
index cddb624..c1f2a98 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-site.xml
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/configuration/hive-site.xml
@@ -2167,7 +2167,7 @@
     <property_depended_by/>
 </property><property require-input="false">
     <name>hive.execution.engine</name>
-    <value>tez</value>
+    <value>mr</value>
     <description>
       Expects one of [mr, tez].
       Chooses execution engine. Options are: mr (Map reduce, default) or tez (hadoop 2 only)

http://git-wip-us.apache.org/repos/asf/bigtop/blob/b1cc82ce/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/templetonSmoke.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/templetonSmoke.sh b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/templetonSmoke.sh
index dd61631..0ab94fe 100755
--- a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/templetonSmoke.sh
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/templetonSmoke.sh
@@ -55,41 +55,4 @@ if [[ "$httpExitCode" -ne "200" ]] ; then
   exit 1
 fi
 
-#try hcat ddl command
-/var/lib/ambari-agent/ambari-sudo.sh rm -f ${tmp_dir}/show_db.post.txt
-echo "user.name=${smoke_test_user}&exec=show databases;" > ${tmp_dir}/show_db.post.txt
-/var/lib/ambari-agent/ambari-sudo.sh chown ${smoke_test_user} ${tmp_dir}/show_db.post.txt
-cmd="${kinitcmd}curl --negotiate -u : -s -w 'http_code <%{http_code}>' -d  @${tmp_dir}/show_db.post.txt  $ttonurl/ddl 2>&1"
-retVal=`/var/lib/ambari-agent/ambari-sudo.sh su ${smoke_test_user} -s /bin/bash - -c "$cmd"`
-httpExitCode=`echo $retVal |sed 's/.*http_code <\([0-9]*\)>.*/\1/'`
-
-if [[ "$httpExitCode" -ne "200" ]] ; then
-  echo "Templeton Smoke Test (ddl cmd): Failed. : $retVal"
-  export TEMPLETON_EXIT_CODE=1
-  exit  1
-fi
-
-# NOT SURE?? SUHAS
-if [[ $security_enabled == "true" ]]; then
-  echo "Templeton Pig Smoke Tests not run in secure mode"
-  exit 0
-fi
-
-#try pig query
-
-#create, copy post args file
-/var/lib/ambari-agent/ambari-sudo.sh rm -f ${tmp_dir}/pig_post.txt
-echo -n "user.name=${smoke_test_user}&file=/tmp/$ttonTestScript" > ${tmp_dir}/pig_post.txt
-/var/lib/ambari-agent/ambari-sudo.sh chown ${smoke_test_user} ${tmp_dir}/pig_post.txt
-
-#submit pig query
-cmd="curl --negotiate -u : -s -w 'http_code <%{http_code}>' -d  @${tmp_dir}/pig_post.txt  $ttonurl/pig 2>&1"
-retVal=`/var/lib/ambari-agent/ambari-sudo.sh su ${smoke_test_user} -s /bin/bash - -c "$cmd"`
-httpExitCode=`echo $retVal |sed 's/.*http_code <\([0-9]*\)>.*/\1/'`
-if [[ "$httpExitCode" -ne "200" ]] ; then
-  echo "Templeton Smoke Test (pig cmd): Failed. : $retVal"
-  export TEMPLETON_EXIT_CODE=1
-  exit 1
-fi
-
 exit 0


[05/50] [abbrv] bigtop git commit: Beginning of adding SQL. Added DDL.

Posted by rv...@apache.org.
Beginning of adding SQL.  Added DDL.


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

Branch: refs/heads/BIGTOP-2666
Commit: 3de72710350b4925a29d4ca3f53d441e5608ac0e
Parents: abe9cdb
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Oct 7 16:39:25 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:45:55 2017 -0800

----------------------------------------------------------------------
 .../odpi/specs/runtime/hive/JdbcConnector.java  |  68 ++++++
 .../org/odpi/specs/runtime/hive/TestJdbc.java   |  74 ++-----
 .../org/odpi/specs/runtime/hive/TestSql.java    | 210 +++++++++++++++++++
 3 files changed, 294 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/3de72710/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
new file mode 100644
index 0000000..4f15ab4
--- /dev/null
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/JdbcConnector.java
@@ -0,0 +1,68 @@
+/**
+ * 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.odpi.specs.runtime.hive;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Properties;
+
+public class JdbcConnector {
+  private static final Log LOG = LogFactory.getLog(JdbcConnector.class.getName());
+
+  protected static final String URL = "odpi.test.hive.jdbc.url";
+  protected static final String USER = "odpi.test.hive.jdbc.user";
+  protected static final String PASSWD = "odpi.test.hive.jdbc.password";
+  protected static final String LOCATION = "odpi.test.hive.location";
+
+  protected static Connection conn;
+
+  @BeforeClass
+  public static void connectToJdbc() throws SQLException {
+    // Assume they've put the URL for the JDBC driver in an environment variable.
+    String jdbcUrl = getProperty(URL, "the JDBC URL");
+    String jdbcUser = getProperty(USER, "the JDBC user name");
+    String jdbcPasswd = getProperty(PASSWD, "the JDBC password");
+
+    Properties props = new Properties();
+    props.put("user", jdbcUser);
+    if (!jdbcPasswd.equals("")) props.put("password", jdbcPasswd);
+    conn = DriverManager.getConnection(jdbcUrl, props);
+  }
+
+  @AfterClass
+  public static void closeJdbc() throws SQLException {
+    if (conn != null) conn.close();
+  }
+
+  protected static String getProperty(String property, String description) {
+    String val = System.getProperty(property);
+    if (val == null) {
+      throw new RuntimeException("You must set the property " + property + " with " +
+          description);
+    }
+    LOG.debug(description + " is " + val);
+    return val;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/3de72710/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestJdbc.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestJdbc.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestJdbc.java
index 54d8475..154fd9c 100644
--- a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestJdbc.java
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestJdbc.java
@@ -1,23 +1,3 @@
-package org.odpi.specs.runtime.hive;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.SQLWarning;
-import java.sql.Statement;
-import java.sql.Types;
-import java.util.Properties;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -35,46 +15,24 @@ import java.util.Properties;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-public class TestJdbc {
-  public static final Log LOG = LogFactory.getLog(TestJdbc.class.getName());
-
-  private static final String URL = "odpiHiveTestJdbcUrl";
-  private static final String USER = "odpiHiveTestJdbcUser";
-  private static final String PASSWD = "odpiHiveTestJdbcPassword";
-
-  protected static Connection conn;
-
-  @BeforeClass
-  public static void connectToJdbc() throws SQLException {
-    // Assume they've put the URL for the JDBC driver in an environment variable.
-    String jdbcUrl = System.getProperty(URL);
-    String jdbcUser = System.getProperty(USER);
-    String jdbcPasswd = System.getProperty(PASSWD);
-
-    LOG.info("URL is " + jdbcUrl);
-    LOG.info("User is " + jdbcUser);
-    LOG.info("Passwd is " + jdbcPasswd);
-    LOG.info("Passwd is null " + (jdbcPasswd == null));
-
-    if (jdbcUrl == null || jdbcUser == null ) {
-      String msg = "You must set the URL, user, and password for the JDBC connection before\n" +
-          "running these tests.  Each is set as a Java property:\n" +
-          URL + " = the URL " +
-          USER + " = the user " +
-          PASSWD + " = the password ";
-      throw new RuntimeException(msg);
-    }
+package org.odpi.specs.runtime.hive;
 
-    Properties props = new Properties();
-    props.put("user", jdbcUser);
-    if (jdbcPasswd != null && jdbcPasswd != "") props.put("password", jdbcPasswd);
-    conn = DriverManager.getConnection(jdbcUrl, props);
-  }
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.Test;
 
-  @AfterClass
-  public static void closeJdbc() throws SQLException {
-    if (conn != null) conn.close();
-  }
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.Statement;
+import java.sql.Types;
+
+public class TestJdbc extends JdbcConnector {
+  private static final Log LOG = LogFactory.getLog(TestJdbc.class.getName());
 
   /**
    * Test simple non-statement related class.  setSchema is tested elsewhere because there's work

http://git-wip-us.apache.org/repos/asf/bigtop/blob/3de72710/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
new file mode 100644
index 0000000..3965f07
--- /dev/null
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestSql.java
@@ -0,0 +1,210 @@
+/**
+ * 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.odpi.specs.runtime.hive;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.Test;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+
+public class TestSql extends JdbcConnector {
+  private static final Log LOG = LogFactory.getLog(TestSql.class.getName());
+
+  @Test
+  public void db() throws SQLException {
+    final String db1 = "odpi_sql_db1";
+    final String db2 = "odpi_sql_db2";
+    try (Statement stmt = conn.createStatement()) {
+      stmt.execute("drop database if exists " + db1 + " cascade");
+
+      // Simple create database
+      stmt.execute("create database " + db1);
+      stmt.execute("drop database " + db1);
+
+      stmt.execute("drop schema if exists " + db2 + " cascade");
+
+      String location = getProperty(LOCATION, "a writable directory in HDFS");
+
+      // All the bells and whistles
+      stmt.execute("create schema if not exists " + db2 + " comment 'a db' location '" + location +
+          "' with dbproperties ('a' = 'b')");
+
+      stmt.execute("alter database " + db2 + " set dbproperties ('c' = 'd')");
+
+      stmt.execute("drop database " + db2 + " restrict");
+    }
+  }
+
+  @Test
+  public void table() throws SQLException {
+    final String table1 = "odpi_sql_table1";
+    final String table2 = "odpi_sql_table2";
+    final String table3 = "odpi_sql_table3";
+    final String table4 = "odpi_sql_table4";
+    final String table5 = "odpi_sql_table5";
+
+    try (Statement stmt = conn.createStatement()) {
+      stmt.execute("drop table if exists " + table1);
+      stmt.execute("drop table if exists " + table2);
+      stmt.execute("drop table if exists " + table3);
+      stmt.execute("drop table if exists " + table4);
+      stmt.execute("drop table if exists " + table5);
+
+      String location = getProperty(LOCATION, "a writable directory in HDFS");
+      stmt.execute("create external table " + table1 + "(a int, b varchar(32)) location '" +
+          location + "'");
+
+      // With a little bit of everything, except partitions, we'll do those below
+      stmt.execute("create table if not exists " + table2 +
+          "(c1 tinyint," +
+          " c2 smallint," +
+          " c3 int comment 'a column comment'," +
+          " c4 bigint," +
+          " c5 float," +
+          " c6 double," +
+          " c7 decimal," +
+          " c8 decimal(12)," +
+          " c9 decimal(8,2)," +
+          " c10 timestamp," +
+          " c11 date," +
+          " c12 string," +
+          " c13 varchar(120)," +
+          " c14 char(10)," +
+          " c15 boolean," +
+          " c16 binary," +
+          " c17 array<string>," +
+          " c18 map <string, string>," +
+          " c19 struct<s1:int, s2:bigint>," +
+          " c20 uniontype<int, string>) " +
+          "comment 'table comment'" +
+          "clustered by (c1) sorted by (c2) into 10 buckets " +
+          "stored as orc " +
+          "tblproperties ('a' = 'b')");
+
+      // NOTES: Not testing SKEWED BY, ROW FORMAT, STORED BY (storage handler
+
+      stmt.execute("create temporary table " + table3 + " like " + table2);
+
+      stmt.execute("insert into " + table1 + " values (3, 'abc'), (4, 'def')");
+
+      stmt.execute("create table " + table4 + " as select a, b from " + table1);
+
+      stmt.execute("truncate table " + table4);
+
+      stmt.execute("alter table " + table4 + " rename to " + table5);
+      stmt.execute("alter table " + table2 + " set tblproperties ('c' = 'd')");
+
+      // NOTE: Not testing alter of clustered or sorted by, because that's suicidal
+      // NOTE: Not testing alter of skewed or serde properties since we didn't test it for create
+      // above.
+
+      stmt.execute("drop table " + table1 + " purge");
+      stmt.execute("drop table " + table2);
+      stmt.execute("drop table " + table3);
+      stmt.execute("drop table " + table5);
+    }
+  }
+
+  @Test
+  public void partitionedTable() throws SQLException {
+    final String table1 = "odpi_sql_ptable1";
+    try (Statement stmt = conn.createStatement()) {
+      stmt.execute("drop table if exists " + table1);
+
+      stmt.execute("create table " + table1 +
+          "(c1 int," +
+          " c2 varchar(32))" +
+          "partitioned by (p1 string comment 'a partition column')" +
+          "stored as orc");
+
+      stmt.execute("alter table " + table1 + " add partition (p1 = 'a')");
+      stmt.execute("insert into " + table1 + " partition (p1 = 'a') values (1, 'abc')");
+      stmt.execute("insert into " + table1 + " partition (p1 = 'a') values (2, 'def')");
+      stmt.execute("insert into " + table1 + " partition (p1 = 'a') values (3, 'ghi')");
+      stmt.execute("alter table " + table1 + " partition (p1 = 'a') concatenate");
+      stmt.execute("alter table " + table1 + " touch partition (p1 = 'a')");
+
+      stmt.execute("alter table " + table1 + " add columns (c3 float)");
+      stmt.execute("alter table " + table1 + " drop partition (p1 = 'a')");
+
+      // NOTE: Not testing rename partition, exchange partition, msck repair, archive/unarchive,
+      // set location, enable/disable no_drop/offline, compact (because not everyone may have
+      // ACID on), change column
+
+      stmt.execute("drop table " + table1);
+
+    }
+  }
+
+  @Test
+  public void view() throws SQLException {
+    final String table1 = "odpi_sql_vtable1";
+    final String view1 = "odpi_sql_view1";
+    final String view2 = "odpi_sql_view2";
+    try (Statement stmt = conn.createStatement()) {
+      stmt.execute("drop table if exists " + table1);
+      stmt.execute("drop view if exists " + view1);
+      stmt.execute("drop view if exists " + view2);
+      stmt.execute("create table " + table1 + "(a int, b varchar(32))");
+      stmt.execute("create view " + view1 + " as select a from " + table1);
+
+      stmt.execute("create view if not exists " + view2 +
+          " comment 'a view comment' " +
+          "tblproperties ('a' = 'b') " +
+          "as select b from " + table1);
+
+      stmt.execute("alter view " + view1 + " as select a, b from " + table1);
+      stmt.execute("alter view " + view2 + " set tblproperties('c' = 'd')");
+
+      stmt.execute("drop view " + view1);
+      stmt.execute("drop view " + view2);
+    }
+  }
+
+  // NOTE: Not testing indices because they are currently useless in Hive
+  // NOTE: Not testing macros because as far as I know no one uses them
+
+  @Test
+  public void function() throws SQLException {
+    final String func1 = "odpi_sql_func1";
+    final String func2 = "odpi_sql_func2";
+    try (Statement stmt = conn.createStatement()) {
+      stmt.execute("create temporary function " + func1 +
+          " as 'org.apache.hadoop.hive.ql.udf.UDFToInteger'");
+      stmt.execute("drop temporary function " + func1);
+
+      stmt.execute("drop function if exists " + func2);
+
+      stmt.execute("create function " + func2 +
+          " as 'org.apache.hadoop.hive.ql.udf.UDFToInteger'");
+      stmt.execute("drop function " + func2);
+    }
+  }
+
+  // NOTE: Not testing grant/revoke/roles as different vendors use different security solutions
+  // and hence different things will work here.
+
+
+}
+
+
+
+
+


[22/50] [abbrv] bigtop git commit: ODPI-182. Adding Hive CLI tests.

Posted by rv...@apache.org.
ODPI-182. Adding Hive CLI tests.


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

Branch: refs/heads/BIGTOP-2666
Commit: e04f24fdf3d8af12623b347d33a247eb552efd26
Parents: 4522c95
Author: Raj Desai <rd...@us.ibm.com>
Authored: Mon Oct 24 17:00:31 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:45:59 2017 -0800

----------------------------------------------------------------------
 .../org/odpi/specs/runtime/hive/HiveHelper.java | 101 +++++++++
 .../org/odpi/specs/runtime/hive/TestCLI.java    | 213 +++++++++++++++++++
 2 files changed, 314 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/e04f24fd/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
new file mode 100644
index 0000000..2ac9cc8
--- /dev/null
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/HiveHelper.java
@@ -0,0 +1,101 @@
+/**
+ * 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.odpi.specs.runtime.hive;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.exec.CommandLine;
+import org.apache.commons.exec.DefaultExecuteResultHandler;
+import org.apache.commons.exec.DefaultExecutor;
+import org.apache.commons.exec.ExecuteException;
+import org.apache.commons.exec.ExecuteWatchdog;
+import org.apache.commons.exec.Executor;
+import org.apache.commons.exec.PumpStreamHandler;
+import org.apache.commons.exec.environment.EnvironmentUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class HiveHelper {
+	
+	private static final Log LOG = LogFactory.getLog(HiveHelper.class.getName());
+	
+	
+	public static Map<String, String> execCommand(CommandLine commandline) {
+		
+		System.out.println("Executing command:");
+		System.out.println(commandline.toString());
+		Map<String, String> env = null;
+		Map<String, String> entry = new HashMap<String, String>();
+		try {
+			env = EnvironmentUtils.getProcEnvironment();
+		} catch (IOException e1) {
+			// TODO Auto-generated catch block
+			e1.printStackTrace();
+		}
+
+		DefaultExecuteResultHandler resultHandler = new DefaultExecuteResultHandler();
+		ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+		PumpStreamHandler streamHandler = new PumpStreamHandler(outputStream);
+		ExecuteWatchdog watchdog = new ExecuteWatchdog(60*10000);
+		Executor executor = new DefaultExecutor();
+		executor.setExitValue(1);
+		executor.setWatchdog(watchdog);
+		executor.setStreamHandler(streamHandler);
+		try {
+			executor.execute(commandline, env, resultHandler);
+		} catch (ExecuteException e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		} catch (IOException e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		}
+		
+		try {
+			resultHandler.waitFor();
+			/*System.out.println("Command output: "+outputStream.toString());*/
+			entry.put("exitValue", String.valueOf(resultHandler.getExitValue()));
+			entry.put("outputStream", outputStream.toString());
+			return entry;
+		} catch (InterruptedException e) {
+			// TODO Auto-generated catch block
+			/*System.out.println("Command output: "+outputStream.toString());*/
+			LOG.debug("exitValue: "+ String.valueOf(resultHandler.getExitValue()));
+			LOG.debug("outputStream: "+ outputStream.toString());
+			entry.put("exitValue", String.valueOf(resultHandler.getExitValue()));
+			entry.put("outputStream", outputStream.toString());
+			e.printStackTrace();		
+			return entry;
+		}
+	}
+	
+	protected static String getProperty(String property, String description) {
+		String val = System.getProperty(property);
+		if (val == null) {
+			throw new RuntimeException("You must set the property " + property + " with " +
+				description);
+		}
+		LOG.debug(description + " is " + val);
+		return val;
+	 }
+	
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e04f24fd/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestCLI.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestCLI.java b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestCLI.java
new file mode 100644
index 0000000..18ee81d
--- /dev/null
+++ b/bigtop-tests/spec-tests/runtime/src/test/java/org/odpi/specs/runtime/hive/TestCLI.java
@@ -0,0 +1,213 @@
+/**
+ * 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.odpi.specs.runtime.hive;
+
+import java.io.FileNotFoundException;
+import java.io.PrintWriter;
+import java.util.Map;
+
+import org.apache.commons.exec.CommandLine;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.AfterClass;
+import org.junit.Assert;
+
+public class TestCLI {
+	
+	static Map<String, String> results;
+	
+	@BeforeClass
+	public static void setup(){
+		
+		results = HiveHelper.execCommand(new CommandLine("which").addArgument("hive"));
+		Assert.assertEquals("Hive is not in the current path.", 0, Integer.parseInt(results.get("exitValue")));
+
+	}
+	
+	@Test
+	public void help(){		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-H"));
+		//LOG.info(results.get("exitValue"));
+		Assert.assertEquals("Error in executing 'hive -H'", 2, Integer.parseInt(results.get("exitValue")));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--help"));
+		Assert.assertEquals("Error in executing 'hive --help'", 0, Integer.parseInt(results.get("exitValue")));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-U"));
+		Assert.assertEquals("Unrecognized option should exit 1.", 1, Integer.parseInt(results.get("exitValue")));
+	}
+	 
+	@Test
+	public void sqlFromCmdLine(){
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES"));
+		Assert.assertEquals("SHOW DATABASES command failed to execute.", 0, Integer.parseInt(results.get("exitValue")));
+		if(!results.get("outputStream").contains("odpi_runtime_hive")){
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			Assert.assertEquals("Could not create database odpi_runtime_hive.", 0, Integer.parseInt(results.get("exitValue")));
+		}else{
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+			Assert.assertEquals("Could not create database odpi_runtime_hive.", 0, Integer.parseInt(results.get("exitValue")));
+		}
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+	}
+	
+	@Test
+	public void sqlFromFiles() throws FileNotFoundException{
+		try(PrintWriter out = new PrintWriter("hive-f1.sql")){ out.println("SHOW DATABASES;"); }
+		try(PrintWriter out = new PrintWriter("hive-f2.sql")){ out.println("CREATE DATABASE odpi_runtime_hive;"); }
+		try(PrintWriter out = new PrintWriter("hive-f3.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); out.println("CREATE DATABASE odpi_runtime_hive;"); }
+		try(PrintWriter out = new PrintWriter("hive-f4.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); }
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f1.sql"));
+		Assert.assertEquals("SHOW DATABASES command failed to execute.", 0, Integer.parseInt(results.get("exitValue")));
+		if(!results.get("outputStream").contains("odpi_runtime_hive")){
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f2.sql"));
+			Assert.assertEquals("Could not create database odpi_runtime_hive.", 0, Integer.parseInt(results.get("exitValue")));
+		}else{
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f3.sql"));
+			Assert.assertEquals("Could not create database odpi_runtime_hive.", 0, Integer.parseInt(results.get("exitValue")));
+		}
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-f").addArgument("hive-f4.sql"));
+	}
+	
+	@Test
+	public void silent() {
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("-S"));
+		Assert.assertEquals("-S option did not work.", new Boolean(false), results.get("outputStream").contains("Time taken:"));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--silent"));
+		Assert.assertEquals("--silent option did not work.", new Boolean(false), results.get("outputStream").contains("Time taken:"));
+	}
+	
+	@Test
+	public void verbose(){
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("-v"));
+		Assert.assertEquals("-v option did not work.", new Boolean(true), results.get("outputStream").contains("SHOW DATABASES"));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES").addArgument("--verbose"));
+		Assert.assertEquals("--verbose option did not work.", new Boolean(true), results.get("outputStream").contains("SHOW DATABASES"));		
+	}
+	
+	@Test
+	public void initialization() throws FileNotFoundException{
+		try(PrintWriter out = new PrintWriter("hive-init1.sql")){ out.println("CREATE DATABASE odpi_runtime_hive;"); }
+		try(PrintWriter out = new PrintWriter("hive-init2.sql")){ out.println("DROP DATABASE odpi_runtime_hive;"); out.println("CREATE DATABASE odpi_runtime_hive;"); }
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES"));
+		Assert.assertEquals("SHOW DATABASES command failed to execute.", 0, Integer.parseInt(results.get("exitValue")));
+		if(!results.get("outputStream").contains("odpi_runtime_hive")){
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-i").addArgument("hive-init1.sql").addArgument("-e").addArgument("SHOW DATABASES"));
+			Assert.assertEquals("Could not create database odpi_runtime_hive using the init -i option.", 0, Integer.parseInt(results.get("exitValue")));
+			Assert.assertEquals("Could not create database odpi_runtime_hive using the init -i option.", true, results.get("outputStream").contains("odpi_runtime_hive"));
+		}else{
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-i").addArgument("hive-init2.sql").addArgument("-e").addArgument("SHOW DATABASES"));
+			Assert.assertEquals("Could not create database odpi_runtime_hive.", 0, Integer.parseInt(results.get("exitValue")));
+			Assert.assertEquals("Could not create database odpi_runtime_hive using the init -i option.", true, results.get("outputStream").contains("odpi_runtime_hive"));
+		}
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+	}
+	
+	@Test
+	public void database(){
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES"));
+		if(!results.get("outputStream").contains("odpi_runtime_hive")){
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+		}else{
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+		}
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive_1234").addArgument("-e").addArgument("CREATE TABLE odpi ( MYID INT );"));
+		Assert.assertEquals("Non-existent database returned with wrong exit code: "+Integer.parseInt(results.get("exitValue")), 88, Integer.parseInt(results.get("exitValue")));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive").addArgument("-e").addArgument("CREATE TABLE odpi ( MYID INT );"));
+		Assert.assertEquals("Failed to create table using --database argument.", 0, Integer.parseInt(results.get("exitValue")));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive").addArgument("-e").addArgument("DESCRIBE odpi"));
+		Assert.assertEquals("Failed to get expected column after creating odpi table using --database argument.", true, results.get("outputStream").contains("myid"));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--database").addArgument("odpi_runtime_hive").addArgument("-e").addArgument("DROP TABLE odpi"));
+		Assert.assertEquals("Failed to create table using --database argument.", 0, Integer.parseInt(results.get("exitValue")));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+	}
+	
+	@Test
+	public void hiveConf(){
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("--hiveconf").addArgument("hive.root.logger=INFO,console").addArgument("-e").addArgument("SHOW DATABASES"));
+		Assert.assertEquals("The --hiveconf option did not work in setting hive.root.logger=INFO,console.", true, results.get("outputStream").contains("INFO parse.ParseDriver: Parsing command: SHOW DATABASES"));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-hiveconf").addArgument("hive.root.logger=INFO,console").addArgument("-e").addArgument("SHOW DATABASES"));
+		Assert.assertEquals("The -hiveconf variant option did not work in setting hive.root.logger=INFO,console.", true, results.get("outputStream").contains("INFO parse.ParseDriver: Parsing command: SHOW DATABASES"));
+	}
+	
+	@Test
+	public void variableSubsitution() throws FileNotFoundException{
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES"));
+		if(!results.get("outputStream").contains("odpi_runtime_hive")){
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+		}else{
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+		}
+		try(PrintWriter out = new PrintWriter("hive-define.sql")){ out.println("show ${A};"); out.println("quit;"); }
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive -d A=DATABASES < hive-define.sql", false));		
+		Assert.assertEquals("The hive -d A=DATABASES option did not work.", 0, Integer.parseInt(results.get("exitValue")));
+		Assert.assertEquals("The hive -d A=DATABASES option did not work.", true, results.get("outputStream").contains("odpi_runtime_hive"));
+		
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive --define A=DATABASES < hive-define.sql", false));		
+		Assert.assertEquals("The hive --define A=DATABASES option did not work.", 0, Integer.parseInt(results.get("exitValue")));
+		Assert.assertEquals("The hive --define A=DATABASES option did not work.", true, results.get("outputStream").contains("odpi_runtime_hive"));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+	}
+	
+	@Test
+	public void hiveVar() throws FileNotFoundException{
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("SHOW DATABASES"));
+		if(!results.get("outputStream").contains("odpi_runtime_hive")){
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+		}else{
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+			results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("CREATE DATABASE odpi_runtime_hive"));
+		}
+		try(PrintWriter out = new PrintWriter("hive-var.sql")){ out.println("show ${A};"); out.println("quit;"); }
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive --hivevar A=DATABASES < hive-var.sql", false));		
+		Assert.assertEquals("The hive --hivevar A=DATABASES option did not work.", 0, Integer.parseInt(results.get("exitValue")));
+		Assert.assertEquals("The hive --hivevar A=DATABASES option did not work.", true, results.get("outputStream").contains("odpi_runtime_hive"));
+		
+		try(PrintWriter out = new PrintWriter("hiveconf-var.sql")){ out.println("show ${hiveconf:A};"); out.println("quit;"); }
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("hive --hiveconf A=DATABASES < hiveconf-var.sql", false));		
+		Assert.assertEquals("The hive --hiveconf A=DATABASES option did not work.", 0, Integer.parseInt(results.get("exitValue")));
+		Assert.assertEquals("The hive --hiveconf A=DATABASES option did not work.", true, results.get("outputStream").contains("odpi_runtime_hive"));
+		
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+	}
+	
+	@AfterClass
+	public static void cleanup(){
+		results = HiveHelper.execCommand(new CommandLine("hive").addArgument("-e").addArgument("DROP DATABASE odpi_runtime_hive"));
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf hive-f*.sql", false));
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf hive-init*.sql", false));
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf hive-define.sql", false));
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf hive-var.sql", false));
+		results = HiveHelper.execCommand(new CommandLine("/bin/sh").addArgument("-c").addArgument("rm -rf hiveconf-var.sql", false));
+	}
+	 
+}


[47/50] [abbrv] bigtop git commit: Changing file permissions

Posted by rv...@apache.org.
Changing file permissions


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

Branch: refs/heads/BIGTOP-2666
Commit: cfc6685dbd1db2957f7672669f78db3463649be0
Parents: bbc4b47
Author: Raj Desai <rd...@us.ibm.com>
Authored: Thu Nov 10 23:44:06 2016 +0000
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:25 2017 -0800

----------------------------------------------------------------------
 run_itest.sh | 0
 1 file changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/cfc6685d/run_itest.sh
----------------------------------------------------------------------
diff --git a/run_itest.sh b/run_itest.sh
old mode 100644
new mode 100755


[49/50] [abbrv] bigtop git commit: Added key value assignments to arguments.

Posted by rv...@apache.org.
Added key value assignments to arguments.


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

Branch: refs/heads/BIGTOP-2666
Commit: ebe22bc49f7609d8a23a9f6d2985ada323d366ad
Parents: 5966b07
Author: Raj Desai <rd...@us.ibm.com>
Authored: Thu Jan 19 15:08:45 2017 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:26 2017 -0800

----------------------------------------------------------------------
 .../org/odpi/specs/runtime/hive/HCatalogMR.java | 21 +++++++++++++++-----
 1 file changed, 16 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/ebe22bc4/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java b/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
index 7cb9bbe..4110d5d 100644
--- a/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
+++ b/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
@@ -46,14 +46,25 @@ public class HCatalogMR extends Configured implements Tool {
 
   @Override
   public int run(String[] args) throws Exception {
+    String inputTable = null;
+    String outputTable = null;
+    String inputSchemaStr = null;
+    String outputSchemaStr = null;
+    for(int i = 0; i < args.length; i++){
+        if(args[i].equalsIgnoreCase("-it")){
+            inputTable = args[i+1];
+        }else if(args[i].equalsIgnoreCase("-ot")){
+            outputTable = args[i+1];
+        }else if(args[i].equalsIgnoreCase("-is")){
+            inputSchemaStr = args[i+1];
+        }else if(args[i].equalsIgnoreCase("-os")){
+            outputSchemaStr = args[i+1];
+        }
+    }
+    
     Configuration conf = getConf();
     args = new GenericOptionsParser(conf, args).getRemainingArgs();
 
-    String inputTable = args[0];
-    String outputTable = args[1];
-    String inputSchemaStr = args[2];
-    String outputSchemaStr = args[3];
-
     conf.set(INPUT_SCHEMA, inputSchemaStr);
     conf.set(OUTPUT_SCHEMA, outputSchemaStr);
 


[12/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.mysql.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.mysql.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.mysql.sql
new file mode 100755
index 0000000..b0415b1
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.mysql.sql
@@ -0,0 +1,777 @@
+-- MySQL dump 10.13  Distrib 5.5.25, for osx10.6 (i386)
+--
+-- Host: localhost    Database: test
+-- ------------------------------------------------------
+-- Server version	5.5.25
+
+/*!40101 SET @OLD_CHARACTER_SET_CLIENT=@@CHARACTER_SET_CLIENT */;
+/*!40101 SET @OLD_CHARACTER_SET_RESULTS=@@CHARACTER_SET_RESULTS */;
+/*!40101 SET @OLD_COLLATION_CONNECTION=@@COLLATION_CONNECTION */;
+/*!40101 SET NAMES utf8 */;
+/*!40103 SET @OLD_TIME_ZONE=@@TIME_ZONE */;
+/*!40103 SET TIME_ZONE='+00:00' */;
+/*!40014 SET @OLD_UNIQUE_CHECKS=@@UNIQUE_CHECKS, UNIQUE_CHECKS=0 */;
+/*!40014 SET @OLD_FOREIGN_KEY_CHECKS=@@FOREIGN_KEY_CHECKS, FOREIGN_KEY_CHECKS=0 */;
+/*!40101 SET @OLD_SQL_MODE=@@SQL_MODE, SQL_MODE='NO_AUTO_VALUE_ON_ZERO' */;
+/*!40111 SET @OLD_SQL_NOTES=@@SQL_NOTES, SQL_NOTES=0 */;
+
+--
+-- Table structure for table `BUCKETING_COLS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `BUCKETING_COLS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `BUCKET_COL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
+  KEY `BUCKETING_COLS_N49` (`SD_ID`),
+  CONSTRAINT `BUCKETING_COLS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `CDS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `CDS` (
+  `CD_ID` bigint(20) NOT NULL,
+  PRIMARY KEY (`CD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `COLUMNS_V2`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `COLUMNS_V2` (
+  `CD_ID` bigint(20) NOT NULL,
+  `COMMENT` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TYPE_NAME` varchar(4000) DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`CD_ID`,`COLUMN_NAME`),
+  KEY `COLUMNS_V2_N49` (`CD_ID`),
+  CONSTRAINT `COLUMNS_V2_FK1` FOREIGN KEY (`CD_ID`) REFERENCES `CDS` (`CD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `DATABASE_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` (
+  `DB_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(180) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`DB_ID`,`PARAM_KEY`),
+  KEY `DATABASE_PARAMS_N49` (`DB_ID`),
+  CONSTRAINT `DATABASE_PARAMS_FK1` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `DBS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `DBS` (
+  `DB_ID` bigint(20) NOT NULL,
+  `DESC` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `DB_LOCATION_URI` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`DB_ID`),
+  UNIQUE KEY `UNIQUE_DATABASE` (`NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `DB_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `DB_PRIVS` (
+  `DB_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `DB_ID` bigint(20) DEFAULT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `DB_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`DB_GRANT_ID`),
+  UNIQUE KEY `DBPRIVILEGEINDEX` (`DB_ID`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`DB_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `DB_PRIVS_N49` (`DB_ID`),
+  CONSTRAINT `DB_PRIVS_FK1` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `GLOBAL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `GLOBAL_PRIVS` (
+  `USER_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `USER_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`USER_GRANT_ID`),
+  UNIQUE KEY `GLOBALPRIVILEGEINDEX` (`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`USER_PRIV`,`GRANTOR`,`GRANTOR_TYPE`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `IDXS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `IDXS` (
+  `INDEX_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `DEFERRED_REBUILD` bit(1) NOT NULL,
+  `INDEX_HANDLER_CLASS` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INDEX_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INDEX_TBL_ID` bigint(20) DEFAULT NULL,
+  `LAST_ACCESS_TIME` int(11) NOT NULL,
+  `ORIG_TBL_ID` bigint(20) DEFAULT NULL,
+  `SD_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`INDEX_ID`),
+  UNIQUE KEY `UNIQUEINDEX` (`INDEX_NAME`,`ORIG_TBL_ID`),
+  KEY `IDXS_N51` (`SD_ID`),
+  KEY `IDXS_N50` (`INDEX_TBL_ID`),
+  KEY `IDXS_N49` (`ORIG_TBL_ID`),
+  CONSTRAINT `IDXS_FK1` FOREIGN KEY (`ORIG_TBL_ID`) REFERENCES `TBLS` (`TBL_ID`),
+  CONSTRAINT `IDXS_FK2` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`),
+  CONSTRAINT `IDXS_FK3` FOREIGN KEY (`INDEX_TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `INDEX_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `INDEX_PARAMS` (
+  `INDEX_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`),
+  KEY `INDEX_PARAMS_N49` (`INDEX_ID`),
+  CONSTRAINT `INDEX_PARAMS_FK1` FOREIGN KEY (`INDEX_ID`) REFERENCES `IDXS` (`INDEX_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `NUCLEUS_TABLES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `NUCLEUS_TABLES` (
+  `CLASS_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TABLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TYPE` varchar(4) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `OWNER` varchar(2) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `VERSION` varchar(20) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `INTERFACE_NAME` varchar(255) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`CLASS_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITIONS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITIONS` (
+  `PART_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `LAST_ACCESS_TIME` int(11) NOT NULL,
+  `PART_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SD_ID` bigint(20) DEFAULT NULL,
+  `TBL_ID` bigint(20) DEFAULT NULL,
+  `LINK_TARGET_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`PART_ID`),
+  UNIQUE KEY `UNIQUEPARTITION` (`PART_NAME`,`TBL_ID`),
+  KEY `PARTITIONS_N49` (`TBL_ID`),
+  KEY `PARTITIONS_N50` (`SD_ID`),
+  KEY `PARTITIONS_N51` (`LINK_TARGET_ID`),
+  CONSTRAINT `PARTITIONS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`),
+  CONSTRAINT `PARTITIONS_FK2` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`),
+  CONSTRAINT `PARTITIONS_FK3` FOREIGN KEY (`LINK_TARGET_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_EVENTS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_EVENTS` (
+  `PART_NAME_ID` bigint(20) NOT NULL,
+  `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `EVENT_TIME` bigint(20) NOT NULL,
+  `EVENT_TYPE` int(11) NOT NULL,
+  `PARTITION_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_NAME_ID`),
+  KEY `PARTITIONEVENTINDEX` (`PARTITION_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_KEYS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_KEYS` (
+  `TBL_ID` bigint(20) NOT NULL,
+  `PKEY_COMMENT` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PKEY_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PKEY_TYPE` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`TBL_ID`,`PKEY_NAME`),
+  KEY `PARTITION_KEYS_N49` (`TBL_ID`),
+  CONSTRAINT `PARTITION_KEYS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_KEY_VALS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_KEY_VALS` (
+  `PART_ID` bigint(20) NOT NULL,
+  `PART_KEY_VAL` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`PART_ID`,`INTEGER_IDX`),
+  KEY `PARTITION_KEY_VALS_N49` (`PART_ID`),
+  CONSTRAINT `PARTITION_KEY_VALS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PARTITION_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PARTITION_PARAMS` (
+  `PART_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_ID`,`PARAM_KEY`),
+  KEY `PARTITION_PARAMS_N49` (`PART_ID`),
+  CONSTRAINT `PARTITION_PARAMS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PART_COL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PART_COL_PRIVS` (
+  `PART_COLUMN_GRANT_ID` bigint(20) NOT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_ID` bigint(20) DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_COL_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_COLUMN_GRANT_ID`),
+  KEY `PART_COL_PRIVS_N49` (`PART_ID`),
+  KEY `PARTITIONCOLUMNPRIVILEGEINDEX` (`PART_ID`,`COLUMN_NAME`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`PART_COL_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  CONSTRAINT `PART_COL_PRIVS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `PART_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `PART_PRIVS` (
+  `PART_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_ID` bigint(20) DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PART_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`PART_GRANT_ID`),
+  KEY `PARTPRIVILEGEINDEX` (`PART_ID`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`PART_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `PART_PRIVS_N49` (`PART_ID`),
+  CONSTRAINT `PART_PRIVS_FK1` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `ROLES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `ROLES` (
+  `ROLE_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `OWNER_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `ROLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`ROLE_ID`),
+  UNIQUE KEY `ROLEENTITYINDEX` (`ROLE_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `ROLE_MAP`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `ROLE_MAP` (
+  `ROLE_GRANT_ID` bigint(20) NOT NULL,
+  `ADD_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `ROLE_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`ROLE_GRANT_ID`),
+  UNIQUE KEY `USERROLEMAPINDEX` (`PRINCIPAL_NAME`,`ROLE_ID`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `ROLE_MAP_N49` (`ROLE_ID`),
+  CONSTRAINT `ROLE_MAP_FK1` FOREIGN KEY (`ROLE_ID`) REFERENCES `ROLES` (`ROLE_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SDS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SDS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `CD_ID` bigint(20) DEFAULT NULL,
+  `INPUT_FORMAT` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `IS_COMPRESSED` bit(1) NOT NULL,
+  `IS_STOREDASSUBDIRECTORIES` bit(1) NOT NULL,
+  `LOCATION` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `NUM_BUCKETS` int(11) NOT NULL,
+  `OUTPUT_FORMAT` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SERDE_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`SD_ID`),
+  KEY `SDS_N49` (`SERDE_ID`),
+  KEY `SDS_N50` (`CD_ID`),
+  CONSTRAINT `SDS_FK1` FOREIGN KEY (`SERDE_ID`) REFERENCES `SERDES` (`SERDE_ID`),
+  CONSTRAINT `SDS_FK2` FOREIGN KEY (`CD_ID`) REFERENCES `CDS` (`CD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SD_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SD_PARAMS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SD_ID`,`PARAM_KEY`),
+  KEY `SD_PARAMS_N49` (`SD_ID`),
+  CONSTRAINT `SD_PARAMS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SEQUENCE_TABLE`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SEQUENCE_TABLE` (
+  `SEQUENCE_NAME` varchar(255) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `NEXT_VAL` bigint(20) NOT NULL,
+  PRIMARY KEY (`SEQUENCE_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SERDES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SERDES` (
+  `SERDE_ID` bigint(20) NOT NULL,
+  `NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `SLIB` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SERDE_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SERDE_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SERDE_PARAMS` (
+  `SERDE_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`),
+  KEY `SERDE_PARAMS_N49` (`SERDE_ID`),
+  CONSTRAINT `SERDE_PARAMS_FK1` FOREIGN KEY (`SERDE_ID`) REFERENCES `SERDES` (`SERDE_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_COL_NAMES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_COL_NAMES` (
+  `SD_ID` bigint(20) NOT NULL,
+  `SKEWED_COL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
+  KEY `SKEWED_COL_NAMES_N49` (`SD_ID`),
+  CONSTRAINT `SKEWED_COL_NAMES_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_COL_VALUE_LOC_MAP`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_COL_VALUE_LOC_MAP` (
+  `SD_ID` bigint(20) NOT NULL,
+  `STRING_LIST_ID_KID` bigint(20) NOT NULL,
+  `LOCATION` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`),
+  KEY `SKEWED_COL_VALUE_LOC_MAP_N49` (`STRING_LIST_ID_KID`),
+  KEY `SKEWED_COL_VALUE_LOC_MAP_N50` (`SD_ID`),
+  CONSTRAINT `SKEWED_COL_VALUE_LOC_MAP_FK2` FOREIGN KEY (`STRING_LIST_ID_KID`) REFERENCES `SKEWED_STRING_LIST` (`STRING_LIST_ID`),
+  CONSTRAINT `SKEWED_COL_VALUE_LOC_MAP_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_STRING_LIST`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST` (
+  `STRING_LIST_ID` bigint(20) NOT NULL,
+  PRIMARY KEY (`STRING_LIST_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_STRING_LIST_VALUES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST_VALUES` (
+  `STRING_LIST_ID` bigint(20) NOT NULL,
+  `STRING_LIST_VALUE` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`),
+  KEY `SKEWED_STRING_LIST_VALUES_N49` (`STRING_LIST_ID`),
+  CONSTRAINT `SKEWED_STRING_LIST_VALUES_FK1` FOREIGN KEY (`STRING_LIST_ID`) REFERENCES `SKEWED_STRING_LIST` (`STRING_LIST_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SKEWED_VALUES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SKEWED_VALUES` (
+  `SD_ID_OID` bigint(20) NOT NULL,
+  `STRING_LIST_ID_EID` bigint(20) NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`),
+  KEY `SKEWED_VALUES_N50` (`SD_ID_OID`),
+  KEY `SKEWED_VALUES_N49` (`STRING_LIST_ID_EID`),
+  CONSTRAINT `SKEWED_VALUES_FK2` FOREIGN KEY (`STRING_LIST_ID_EID`) REFERENCES `SKEWED_STRING_LIST` (`STRING_LIST_ID`),
+  CONSTRAINT `SKEWED_VALUES_FK1` FOREIGN KEY (`SD_ID_OID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `SORT_COLS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `SORT_COLS` (
+  `SD_ID` bigint(20) NOT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `ORDER` int(11) NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
+  KEY `SORT_COLS_N49` (`SD_ID`),
+  CONSTRAINT `SORT_COLS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TABLE_PARAMS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` (
+  `TBL_ID` bigint(20) NOT NULL,
+  `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`TBL_ID`,`PARAM_KEY`),
+  KEY `TABLE_PARAMS_N49` (`TBL_ID`),
+  CONSTRAINT `TABLE_PARAMS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TBLS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TBLS` (
+  `TBL_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `DB_ID` bigint(20) DEFAULT NULL,
+  `LAST_ACCESS_TIME` int(11) NOT NULL,
+  `OWNER` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `RETENTION` int(11) NOT NULL,
+  `SD_ID` bigint(20) DEFAULT NULL,
+  `TBL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `VIEW_EXPANDED_TEXT` mediumtext,
+  `VIEW_ORIGINAL_TEXT` mediumtext,
+  `LINK_TARGET_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`TBL_ID`),
+  UNIQUE KEY `UNIQUETABLE` (`TBL_NAME`,`DB_ID`),
+  KEY `TBLS_N50` (`SD_ID`),
+  KEY `TBLS_N49` (`DB_ID`),
+  KEY `TBLS_N51` (`LINK_TARGET_ID`),
+  CONSTRAINT `TBLS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`),
+  CONSTRAINT `TBLS_FK2` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` (`DB_ID`),
+  CONSTRAINT `TBLS_FK3` FOREIGN KEY (`LINK_TARGET_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TBL_COL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` (
+  `TBL_COLUMN_GRANT_ID` bigint(20) NOT NULL,
+  `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_COL_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`TBL_COLUMN_GRANT_ID`),
+  KEY `TABLECOLUMNPRIVILEGEINDEX` (`TBL_ID`,`COLUMN_NAME`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`TBL_COL_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  KEY `TBL_COL_PRIVS_N49` (`TBL_ID`),
+  CONSTRAINT `TBL_COL_PRIVS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TBL_PRIVS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TBL_PRIVS` (
+  `TBL_GRANT_ID` bigint(20) NOT NULL,
+  `CREATE_TIME` int(11) NOT NULL,
+  `GRANT_OPTION` smallint(6) NOT NULL,
+  `GRANTOR` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `GRANTOR_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `PRINCIPAL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_PRIV` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TBL_ID` bigint(20) DEFAULT NULL,
+  PRIMARY KEY (`TBL_GRANT_ID`),
+  KEY `TBL_PRIVS_N49` (`TBL_ID`),
+  KEY `TABLEPRIVILEGEINDEX` (`TBL_ID`,`PRINCIPAL_NAME`,`PRINCIPAL_TYPE`,`TBL_PRIV`,`GRANTOR`,`GRANTOR_TYPE`),
+  CONSTRAINT `TBL_PRIVS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TAB_COL_STATS`
+--
+CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` (
+ `CS_ID` bigint(20) NOT NULL,
+ `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `TABLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `TBL_ID` bigint(20) NOT NULL,
+ `LONG_LOW_VALUE` bigint(20),
+ `LONG_HIGH_VALUE` bigint(20),
+ `DOUBLE_HIGH_VALUE` double(53,4),
+ `DOUBLE_LOW_VALUE` double(53,4),
+ `BIG_DECIMAL_LOW_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `BIG_DECIMAL_HIGH_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `NUM_NULLS` bigint(20) NOT NULL,
+ `NUM_DISTINCTS` bigint(20),
+ `AVG_COL_LEN` double(53,4),
+ `MAX_COL_LEN` bigint(20),
+ `NUM_TRUES` bigint(20),
+ `NUM_FALSES` bigint(20),
+ `LAST_ANALYZED` bigint(20) NOT NULL,
+  PRIMARY KEY (`CS_ID`),
+  CONSTRAINT `TAB_COL_STATS_FK` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for table `PART_COL_STATS`
+--
+CREATE TABLE IF NOT EXISTS `PART_COL_STATS` (
+ `CS_ID` bigint(20) NOT NULL,
+ `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `TABLE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `PARTITION_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `COLUMN_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+ `PART_ID` bigint(20) NOT NULL,
+ `LONG_LOW_VALUE` bigint(20),
+ `LONG_HIGH_VALUE` bigint(20),
+ `DOUBLE_HIGH_VALUE` double(53,4),
+ `DOUBLE_LOW_VALUE` double(53,4),
+ `BIG_DECIMAL_LOW_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `BIG_DECIMAL_HIGH_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin,
+ `NUM_NULLS` bigint(20) NOT NULL,
+ `NUM_DISTINCTS` bigint(20),
+ `AVG_COL_LEN` double(53,4),
+ `MAX_COL_LEN` bigint(20),
+ `NUM_TRUES` bigint(20),
+ `NUM_FALSES` bigint(20),
+ `LAST_ANALYZED` bigint(20) NOT NULL,
+  PRIMARY KEY (`CS_ID`),
+  CONSTRAINT `PART_COL_STATS_FK` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for table `TYPES`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TYPES` (
+  `TYPES_ID` bigint(20) NOT NULL,
+  `TYPE_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TYPE1` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `TYPE2` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  PRIMARY KEY (`TYPES_ID`),
+  UNIQUE KEY `UNIQUE_TYPE` (`TYPE_NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
+-- Table structure for table `TYPE_FIELDS`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `TYPE_FIELDS` (
+  `TYPE_NAME` bigint(20) NOT NULL,
+  `COMMENT` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `FIELD_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `FIELD_TYPE` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `INTEGER_IDX` int(11) NOT NULL,
+  PRIMARY KEY (`TYPE_NAME`,`FIELD_NAME`),
+  KEY `TYPE_FIELDS_N49` (`TYPE_NAME`),
+  CONSTRAINT `TYPE_FIELDS_FK1` FOREIGN KEY (`TYPE_NAME`) REFERENCES `TYPES` (`TYPES_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+-- Table `MASTER_KEYS` for classes [org.apache.hadoop.hive.metastore.model.MMasterKey]
+CREATE TABLE IF NOT EXISTS `MASTER_KEYS` 
+(
+    `KEY_ID` INTEGER NOT NULL AUTO_INCREMENT,
+    `MASTER_KEY` VARCHAR(767) BINARY NULL,
+    PRIMARY KEY (`KEY_ID`)
+) ENGINE=INNODB DEFAULT CHARSET=latin1;
+
+-- Table `DELEGATION_TOKENS` for classes [org.apache.hadoop.hive.metastore.model.MDelegationToken]
+CREATE TABLE IF NOT EXISTS `DELEGATION_TOKENS`
+(
+    `TOKEN_IDENT` VARCHAR(767) BINARY NOT NULL,
+    `TOKEN` VARCHAR(767) BINARY NULL,
+    PRIMARY KEY (`TOKEN_IDENT`)
+) ENGINE=INNODB DEFAULT CHARSET=latin1;
+
+--
+-- Table structure for VERSION
+--
+CREATE TABLE IF NOT EXISTS `VERSION` (
+  `VER_ID` BIGINT NOT NULL,
+  `SCHEMA_VERSION` VARCHAR(127) NOT NULL,
+  `VERSION_COMMENT` VARCHAR(255),
+  PRIMARY KEY (`VER_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.12.0', 'Hive release version 0.12.0');
+
+/*!40101 SET character_set_client = @saved_cs_client */;
+/*!40103 SET TIME_ZONE=@OLD_TIME_ZONE */;
+
+/*!40101 SET SQL_MODE=@OLD_SQL_MODE */;
+/*!40014 SET FOREIGN_KEY_CHECKS=@OLD_FOREIGN_KEY_CHECKS */;
+/*!40014 SET UNIQUE_CHECKS=@OLD_UNIQUE_CHECKS */;
+/*!40101 SET CHARACTER_SET_CLIENT=@OLD_CHARACTER_SET_CLIENT */;
+/*!40101 SET CHARACTER_SET_RESULTS=@OLD_CHARACTER_SET_RESULTS */;
+/*!40101 SET COLLATION_CONNECTION=@OLD_COLLATION_CONNECTION */;
+/*!40111 SET SQL_NOTES=@OLD_SQL_NOTES */;
+
+-- Dump completed on 2012-08-23  0:56:31

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.oracle.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.oracle.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.oracle.sql
new file mode 100755
index 0000000..812b897
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.oracle.sql
@@ -0,0 +1,718 @@
+-- Table SEQUENCE_TABLE is an internal table required by DataNucleus.
+-- NOTE: Some versions of SchemaTool do not automatically generate this table.
+-- See http://www.datanucleus.org/servlet/jira/browse/NUCRDBMS-416
+CREATE TABLE SEQUENCE_TABLE
+(
+   SEQUENCE_NAME VARCHAR2(255) NOT NULL,
+   NEXT_VAL NUMBER NOT NULL
+);
+
+ALTER TABLE SEQUENCE_TABLE ADD CONSTRAINT PART_TABLE_PK PRIMARY KEY (SEQUENCE_NAME);
+
+-- Table NUCLEUS_TABLES is an internal table required by DataNucleus.
+-- This table is required if datanucleus.autoStartMechanism=SchemaTable
+-- NOTE: Some versions of SchemaTool do not automatically generate this table.
+-- See http://www.datanucleus.org/servlet/jira/browse/NUCRDBMS-416
+CREATE TABLE NUCLEUS_TABLES
+(
+   CLASS_NAME VARCHAR2(128) NOT NULL,
+   TABLE_NAME VARCHAR2(128) NOT NULL,
+   TYPE VARCHAR2(4) NOT NULL,
+   OWNER VARCHAR2(2) NOT NULL,
+   VERSION VARCHAR2(20) NOT NULL,
+   INTERFACE_NAME VARCHAR2(255) NULL
+);
+
+ALTER TABLE NUCLEUS_TABLES ADD CONSTRAINT NUCLEUS_TABLES_PK PRIMARY KEY (CLASS_NAME);
+
+-- Table PART_COL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege]
+CREATE TABLE PART_COL_PRIVS
+(
+    PART_COLUMN_GRANT_ID NUMBER NOT NULL,
+    "COLUMN_NAME" VARCHAR2(128) NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PART_ID NUMBER NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    PART_COL_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE PART_COL_PRIVS ADD CONSTRAINT PART_COL_PRIVS_PK PRIMARY KEY (PART_COLUMN_GRANT_ID);
+
+-- Table CDS.
+CREATE TABLE CDS
+(
+    CD_ID NUMBER NOT NULL
+);
+
+ALTER TABLE CDS ADD CONSTRAINT CDS_PK PRIMARY KEY (CD_ID);
+
+-- Table COLUMNS_V2 for join relationship
+CREATE TABLE COLUMNS_V2
+(
+    CD_ID NUMBER NOT NULL,
+    "COMMENT" VARCHAR2(256) NULL,
+    "COLUMN_NAME" VARCHAR2(128) NOT NULL,
+    TYPE_NAME VARCHAR2(4000) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE COLUMNS_V2 ADD CONSTRAINT COLUMNS_V2_PK PRIMARY KEY (CD_ID,"COLUMN_NAME");
+
+-- Table PARTITION_KEY_VALS for join relationship
+CREATE TABLE PARTITION_KEY_VALS
+(
+    PART_ID NUMBER NOT NULL,
+    PART_KEY_VAL VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE PARTITION_KEY_VALS ADD CONSTRAINT PARTITION_KEY_VALS_PK PRIMARY KEY (PART_ID,INTEGER_IDX);
+
+-- Table DBS for classes [org.apache.hadoop.hive.metastore.model.MDatabase]
+CREATE TABLE DBS
+(
+    DB_ID NUMBER NOT NULL,
+    "DESC" VARCHAR2(4000) NULL,
+    DB_LOCATION_URI VARCHAR2(4000) NOT NULL,
+    "NAME" VARCHAR2(128) NULL
+);
+
+ALTER TABLE DBS ADD CONSTRAINT DBS_PK PRIMARY KEY (DB_ID);
+
+-- Table PARTITION_PARAMS for join relationship
+CREATE TABLE PARTITION_PARAMS
+(
+    PART_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE PARTITION_PARAMS ADD CONSTRAINT PARTITION_PARAMS_PK PRIMARY KEY (PART_ID,PARAM_KEY);
+
+-- Table SERDES for classes [org.apache.hadoop.hive.metastore.model.MSerDeInfo]
+CREATE TABLE SERDES
+(
+    SERDE_ID NUMBER NOT NULL,
+    "NAME" VARCHAR2(128) NULL,
+    SLIB VARCHAR2(4000) NULL
+);
+
+ALTER TABLE SERDES ADD CONSTRAINT SERDES_PK PRIMARY KEY (SERDE_ID);
+
+-- Table TYPES for classes [org.apache.hadoop.hive.metastore.model.MType]
+CREATE TABLE TYPES
+(
+    TYPES_ID NUMBER NOT NULL,
+    TYPE_NAME VARCHAR2(128) NULL,
+    TYPE1 VARCHAR2(767) NULL,
+    TYPE2 VARCHAR2(767) NULL
+);
+
+ALTER TABLE TYPES ADD CONSTRAINT TYPES_PK PRIMARY KEY (TYPES_ID);
+
+-- Table PARTITION_KEYS for join relationship
+CREATE TABLE PARTITION_KEYS
+(
+    TBL_ID NUMBER NOT NULL,
+    PKEY_COMMENT VARCHAR2(4000) NULL,
+    PKEY_NAME VARCHAR2(128) NOT NULL,
+    PKEY_TYPE VARCHAR2(767) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE PARTITION_KEYS ADD CONSTRAINT PARTITION_KEY_PK PRIMARY KEY (TBL_ID,PKEY_NAME);
+
+-- Table ROLES for classes [org.apache.hadoop.hive.metastore.model.MRole]
+CREATE TABLE ROLES
+(
+    ROLE_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    OWNER_NAME VARCHAR2(128) NULL,
+    ROLE_NAME VARCHAR2(128) NULL
+);
+
+ALTER TABLE ROLES ADD CONSTRAINT ROLES_PK PRIMARY KEY (ROLE_ID);
+
+-- Table PARTITIONS for classes [org.apache.hadoop.hive.metastore.model.MPartition]
+CREATE TABLE PARTITIONS
+(
+    PART_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    LAST_ACCESS_TIME NUMBER (10) NOT NULL,
+    PART_NAME VARCHAR2(767) NULL,
+    SD_ID NUMBER NULL,
+    TBL_ID NUMBER NULL
+);
+
+ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_PK PRIMARY KEY (PART_ID);
+
+-- Table INDEX_PARAMS for join relationship
+CREATE TABLE INDEX_PARAMS
+(
+    INDEX_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE INDEX_PARAMS ADD CONSTRAINT INDEX_PARAMS_PK PRIMARY KEY (INDEX_ID,PARAM_KEY);
+
+-- Table TBL_COL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege]
+CREATE TABLE TBL_COL_PRIVS
+(
+    TBL_COLUMN_GRANT_ID NUMBER NOT NULL,
+    "COLUMN_NAME" VARCHAR2(128) NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    TBL_COL_PRIV VARCHAR2(128) NULL,
+    TBL_ID NUMBER NULL
+);
+
+ALTER TABLE TBL_COL_PRIVS ADD CONSTRAINT TBL_COL_PRIVS_PK PRIMARY KEY (TBL_COLUMN_GRANT_ID);
+
+-- Table IDXS for classes [org.apache.hadoop.hive.metastore.model.MIndex]
+CREATE TABLE IDXS
+(
+    INDEX_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    DEFERRED_REBUILD NUMBER(1) NOT NULL CHECK (DEFERRED_REBUILD IN (1,0)),
+    INDEX_HANDLER_CLASS VARCHAR2(4000) NULL,
+    INDEX_NAME VARCHAR2(128) NULL,
+    INDEX_TBL_ID NUMBER NULL,
+    LAST_ACCESS_TIME NUMBER (10) NOT NULL,
+    ORIG_TBL_ID NUMBER NULL,
+    SD_ID NUMBER NULL
+);
+
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_PK PRIMARY KEY (INDEX_ID);
+
+-- Table BUCKETING_COLS for join relationship
+CREATE TABLE BUCKETING_COLS
+(
+    SD_ID NUMBER NOT NULL,
+    BUCKET_COL_NAME VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE BUCKETING_COLS ADD CONSTRAINT BUCKETING_COLS_PK PRIMARY KEY (SD_ID,INTEGER_IDX);
+
+-- Table TYPE_FIELDS for join relationship
+CREATE TABLE TYPE_FIELDS
+(
+    TYPE_NAME NUMBER NOT NULL,
+    "COMMENT" VARCHAR2(256) NULL,
+    FIELD_NAME VARCHAR2(128) NOT NULL,
+    FIELD_TYPE VARCHAR2(767) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE TYPE_FIELDS ADD CONSTRAINT TYPE_FIELDS_PK PRIMARY KEY (TYPE_NAME,FIELD_NAME);
+
+-- Table SD_PARAMS for join relationship
+CREATE TABLE SD_PARAMS
+(
+    SD_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE SD_PARAMS ADD CONSTRAINT SD_PARAMS_PK PRIMARY KEY (SD_ID,PARAM_KEY);
+
+-- Table GLOBAL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MGlobalPrivilege]
+CREATE TABLE GLOBAL_PRIVS
+(
+    USER_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    USER_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE GLOBAL_PRIVS ADD CONSTRAINT GLOBAL_PRIVS_PK PRIMARY KEY (USER_GRANT_ID);
+
+-- Table SDS for classes [org.apache.hadoop.hive.metastore.model.MStorageDescriptor]
+CREATE TABLE SDS
+(
+    SD_ID NUMBER NOT NULL,
+    CD_ID NUMBER NULL,
+    INPUT_FORMAT VARCHAR2(4000) NULL,
+    IS_COMPRESSED NUMBER(1) NOT NULL CHECK (IS_COMPRESSED IN (1,0)),
+    LOCATION VARCHAR2(4000) NULL,
+    NUM_BUCKETS NUMBER (10) NOT NULL,
+    OUTPUT_FORMAT VARCHAR2(4000) NULL,
+    SERDE_ID NUMBER NULL,
+    IS_STOREDASSUBDIRECTORIES NUMBER(1) NOT NULL CHECK (IS_STOREDASSUBDIRECTORIES IN (1,0))
+);
+
+ALTER TABLE SDS ADD CONSTRAINT SDS_PK PRIMARY KEY (SD_ID);
+
+-- Table TABLE_PARAMS for join relationship
+CREATE TABLE TABLE_PARAMS
+(
+    TBL_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE TABLE_PARAMS ADD CONSTRAINT TABLE_PARAMS_PK PRIMARY KEY (TBL_ID,PARAM_KEY);
+
+-- Table SORT_COLS for join relationship
+CREATE TABLE SORT_COLS
+(
+    SD_ID NUMBER NOT NULL,
+    "COLUMN_NAME" VARCHAR2(128) NULL,
+    "ORDER" NUMBER (10) NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SORT_COLS ADD CONSTRAINT SORT_COLS_PK PRIMARY KEY (SD_ID,INTEGER_IDX);
+
+-- Table TBL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MTablePrivilege]
+CREATE TABLE TBL_PRIVS
+(
+    TBL_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    TBL_PRIV VARCHAR2(128) NULL,
+    TBL_ID NUMBER NULL
+);
+
+ALTER TABLE TBL_PRIVS ADD CONSTRAINT TBL_PRIVS_PK PRIMARY KEY (TBL_GRANT_ID);
+
+-- Table DATABASE_PARAMS for join relationship
+CREATE TABLE DATABASE_PARAMS
+(
+    DB_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(180) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE DATABASE_PARAMS ADD CONSTRAINT DATABASE_PARAMS_PK PRIMARY KEY (DB_ID,PARAM_KEY);
+
+-- Table ROLE_MAP for classes [org.apache.hadoop.hive.metastore.model.MRoleMap]
+CREATE TABLE ROLE_MAP
+(
+    ROLE_GRANT_ID NUMBER NOT NULL,
+    ADD_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    ROLE_ID NUMBER NULL
+);
+
+ALTER TABLE ROLE_MAP ADD CONSTRAINT ROLE_MAP_PK PRIMARY KEY (ROLE_GRANT_ID);
+
+-- Table SERDE_PARAMS for join relationship
+CREATE TABLE SERDE_PARAMS
+(
+    SERDE_ID NUMBER NOT NULL,
+    PARAM_KEY VARCHAR2(256) NOT NULL,
+    PARAM_VALUE VARCHAR2(4000) NULL
+);
+
+ALTER TABLE SERDE_PARAMS ADD CONSTRAINT SERDE_PARAMS_PK PRIMARY KEY (SERDE_ID,PARAM_KEY);
+
+-- Table PART_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MPartitionPrivilege]
+CREATE TABLE PART_PRIVS
+(
+    PART_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PART_ID NUMBER NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    PART_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE PART_PRIVS ADD CONSTRAINT PART_PRIVS_PK PRIMARY KEY (PART_GRANT_ID);
+
+-- Table DB_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MDBPrivilege]
+CREATE TABLE DB_PRIVS
+(
+    DB_GRANT_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    DB_ID NUMBER NULL,
+    GRANT_OPTION NUMBER (5) NOT NULL,
+    GRANTOR VARCHAR2(128) NULL,
+    GRANTOR_TYPE VARCHAR2(128) NULL,
+    PRINCIPAL_NAME VARCHAR2(128) NULL,
+    PRINCIPAL_TYPE VARCHAR2(128) NULL,
+    DB_PRIV VARCHAR2(128) NULL
+);
+
+ALTER TABLE DB_PRIVS ADD CONSTRAINT DB_PRIVS_PK PRIMARY KEY (DB_GRANT_ID);
+
+-- Table TBLS for classes [org.apache.hadoop.hive.metastore.model.MTable]
+CREATE TABLE TBLS
+(
+    TBL_ID NUMBER NOT NULL,
+    CREATE_TIME NUMBER (10) NOT NULL,
+    DB_ID NUMBER NULL,
+    LAST_ACCESS_TIME NUMBER (10) NOT NULL,
+    OWNER VARCHAR2(767) NULL,
+    RETENTION NUMBER (10) NOT NULL,
+    SD_ID NUMBER NULL,
+    TBL_NAME VARCHAR2(128) NULL,
+    TBL_TYPE VARCHAR2(128) NULL,
+    VIEW_EXPANDED_TEXT CLOB NULL,
+    VIEW_ORIGINAL_TEXT CLOB NULL
+);
+
+ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);
+
+-- Table PARTITION_EVENTS for classes [org.apache.hadoop.hive.metastore.model.MPartitionEvent]
+CREATE TABLE PARTITION_EVENTS
+(
+    PART_NAME_ID NUMBER NOT NULL,
+    DB_NAME VARCHAR2(128) NULL,
+    EVENT_TIME NUMBER NOT NULL,
+    EVENT_TYPE NUMBER (10) NOT NULL,
+    PARTITION_NAME VARCHAR2(767) NULL,
+    TBL_NAME VARCHAR2(128) NULL
+);
+
+ALTER TABLE PARTITION_EVENTS ADD CONSTRAINT PARTITION_EVENTS_PK PRIMARY KEY (PART_NAME_ID);
+
+-- Table SKEWED_STRING_LIST for classes [org.apache.hadoop.hive.metastore.model.MStringList]
+CREATE TABLE SKEWED_STRING_LIST
+(
+    STRING_LIST_ID NUMBER NOT NULL
+);
+
+ALTER TABLE SKEWED_STRING_LIST ADD CONSTRAINT SKEWED_STRING_LIST_PK PRIMARY KEY (STRING_LIST_ID);
+
+CREATE TABLE SKEWED_STRING_LIST_VALUES
+(
+    STRING_LIST_ID NUMBER NOT NULL,
+    "STRING_LIST_VALUE" VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SKEWED_STRING_LIST_VALUES ADD CONSTRAINT SKEWED_STRING_LIST_VALUES_PK PRIMARY KEY (STRING_LIST_ID,INTEGER_IDX);
+
+ALTER TABLE SKEWED_STRING_LIST_VALUES ADD CONSTRAINT SKEWED_STRING_LIST_VALUES_FK1 FOREIGN KEY (STRING_LIST_ID) REFERENCES SKEWED_STRING_LIST (STRING_LIST_ID) INITIALLY DEFERRED ;
+
+CREATE TABLE SKEWED_COL_NAMES
+(
+    SD_ID NUMBER NOT NULL,
+    "SKEWED_COL_NAME" VARCHAR2(256) NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SKEWED_COL_NAMES ADD CONSTRAINT SKEWED_COL_NAMES_PK PRIMARY KEY (SD_ID,INTEGER_IDX);
+
+ALTER TABLE SKEWED_COL_NAMES ADD CONSTRAINT SKEWED_COL_NAMES_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE TABLE SKEWED_COL_VALUE_LOC_MAP
+(
+    SD_ID NUMBER NOT NULL,
+    STRING_LIST_ID_KID NUMBER NOT NULL,
+    "LOCATION" VARCHAR2(4000) NULL
+);
+
+CREATE TABLE MASTER_KEYS
+(
+    KEY_ID NUMBER (10) NOT NULL,
+    MASTER_KEY VARCHAR2(767) NULL
+);
+
+CREATE TABLE DELEGATION_TOKENS
+(
+    TOKEN_IDENT VARCHAR2(767) NOT NULL,
+    TOKEN VARCHAR2(767) NULL
+);
+
+ALTER TABLE SKEWED_COL_VALUE_LOC_MAP ADD CONSTRAINT SKEWED_COL_VALUE_LOC_MAP_PK PRIMARY KEY (SD_ID,STRING_LIST_ID_KID);
+
+ALTER TABLE SKEWED_COL_VALUE_LOC_MAP ADD CONSTRAINT SKEWED_COL_VALUE_LOC_MAP_FK1 FOREIGN KEY (STRING_LIST_ID_KID) REFERENCES SKEWED_STRING_LIST (STRING_LIST_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE SKEWED_COL_VALUE_LOC_MAP ADD CONSTRAINT SKEWED_COL_VALUE_LOC_MAP_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE TABLE SKEWED_VALUES
+(
+    SD_ID_OID NUMBER NOT NULL,
+    STRING_LIST_ID_EID NUMBER NOT NULL,
+    INTEGER_IDX NUMBER(10) NOT NULL
+);
+
+ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_PK PRIMARY KEY (SD_ID_OID,INTEGER_IDX);
+
+ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_FK1 FOREIGN KEY (STRING_LIST_ID_EID) REFERENCES SKEWED_STRING_LIST (STRING_LIST_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_FK2 FOREIGN KEY (SD_ID_OID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+-- column statistics
+
+CREATE TABLE TAB_COL_STATS (
+ CS_ID NUMBER NOT NULL,
+ DB_NAME VARCHAR2(128) NOT NULL,
+ TABLE_NAME VARCHAR2(128) NOT NULL, 
+ COLUMN_NAME VARCHAR2(128) NOT NULL,
+ COLUMN_TYPE VARCHAR2(128) NOT NULL,
+ TBL_ID NUMBER NOT NULL,
+ LONG_LOW_VALUE NUMBER,
+ LONG_HIGH_VALUE NUMBER,
+ DOUBLE_LOW_VALUE NUMBER,
+ DOUBLE_HIGH_VALUE NUMBER,
+ BIG_DECIMAL_LOW_VALUE VARCHAR2(4000),
+ BIG_DECIMAL_HIGH_VALUE VARCHAR2(4000),
+ NUM_NULLS NUMBER NOT NULL,
+ NUM_DISTINCTS NUMBER,
+ AVG_COL_LEN NUMBER,
+ MAX_COL_LEN NUMBER,
+ NUM_TRUES NUMBER,
+ NUM_FALSES NUMBER,
+ LAST_ANALYZED NUMBER NOT NULL
+);
+
+CREATE TABLE VERSION (
+  VER_ID NUMBER NOT NULL,
+  SCHEMA_VERSION VARCHAR(127) NOT NULL,
+  VERSION_COMMENT VARCHAR(255)
+);
+ALTER TABLE VERSION ADD CONSTRAINT VERSION_PK PRIMARY KEY (VER_ID);
+
+ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_PKEY PRIMARY KEY (CS_ID);
+
+ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_FK FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TAB_COL_STATS_N49 ON TAB_COL_STATS(TBL_ID);
+
+CREATE TABLE PART_COL_STATS (
+ CS_ID NUMBER NOT NULL,
+ DB_NAME VARCHAR2(128) NOT NULL,
+ TABLE_NAME VARCHAR2(128) NOT NULL,
+ PARTITION_NAME VARCHAR2(767) NOT NULL,
+ COLUMN_NAME VARCHAR2(128) NOT NULL,
+ COLUMN_TYPE VARCHAR2(128) NOT NULL,
+ PART_ID NUMBER NOT NULL,
+ LONG_LOW_VALUE NUMBER,
+ LONG_HIGH_VALUE NUMBER,
+ DOUBLE_LOW_VALUE NUMBER,
+ DOUBLE_HIGH_VALUE NUMBER,
+ BIG_DECIMAL_LOW_VALUE VARCHAR2(4000),
+ BIG_DECIMAL_HIGH_VALUE VARCHAR2(4000),
+ NUM_NULLS NUMBER NOT NULL,
+ NUM_DISTINCTS NUMBER,
+ AVG_COL_LEN NUMBER,
+ MAX_COL_LEN NUMBER,
+ NUM_TRUES NUMBER,
+ NUM_FALSES NUMBER,
+ LAST_ANALYZED NUMBER NOT NULL
+);
+
+ALTER TABLE PART_COL_STATS ADD CONSTRAINT PART_COL_STATS_PKEY PRIMARY KEY (CS_ID);
+
+ALTER TABLE PART_COL_STATS ADD CONSTRAINT PART_COL_STATS_FK FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED;
+
+CREATE INDEX PART_COL_STATS_N49 ON PART_COL_STATS (PART_ID);
+
+-- Constraints for table PART_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege]
+ALTER TABLE PART_COL_PRIVS ADD CONSTRAINT PART_COL_PRIVS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PART_COL_PRIVS_N49 ON PART_COL_PRIVS (PART_ID);
+
+CREATE INDEX PARTITIONCOLUMNPRIVILEGEINDEX ON PART_COL_PRIVS (PART_ID,"COLUMN_NAME",PRINCIPAL_NAME,PRINCIPAL_TYPE,PART_COL_PRIV,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table COLUMNS_V2
+ALTER TABLE COLUMNS_V2 ADD CONSTRAINT COLUMNS_V2_FK1 FOREIGN KEY (CD_ID) REFERENCES CDS (CD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX COLUMNS_V2_N49 ON COLUMNS_V2 (CD_ID);
+
+
+-- Constraints for table PARTITION_KEY_VALS
+ALTER TABLE PARTITION_KEY_VALS ADD CONSTRAINT PARTITION_KEY_VALS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITION_KEY_VALS_N49 ON PARTITION_KEY_VALS (PART_ID);
+
+
+-- Constraints for table DBS for class(es) [org.apache.hadoop.hive.metastore.model.MDatabase]
+CREATE UNIQUE INDEX UNIQUE_DATABASE ON DBS ("NAME");
+
+
+-- Constraints for table PARTITION_PARAMS
+ALTER TABLE PARTITION_PARAMS ADD CONSTRAINT PARTITION_PARAMS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITION_PARAMS_N49 ON PARTITION_PARAMS (PART_ID);
+
+
+-- Constraints for table SERDES for class(es) [org.apache.hadoop.hive.metastore.model.MSerDeInfo]
+
+-- Constraints for table TYPES for class(es) [org.apache.hadoop.hive.metastore.model.MType]
+CREATE UNIQUE INDEX UNIQUE_TYPE ON TYPES (TYPE_NAME);
+
+
+-- Constraints for table PARTITION_KEYS
+ALTER TABLE PARTITION_KEYS ADD CONSTRAINT PARTITION_KEYS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITION_KEYS_N49 ON PARTITION_KEYS (TBL_ID);
+
+
+-- Constraints for table ROLES for class(es) [org.apache.hadoop.hive.metastore.model.MRole]
+CREATE UNIQUE INDEX ROLEENTITYINDEX ON ROLES (ROLE_NAME);
+
+
+-- Constraints for table PARTITIONS for class(es) [org.apache.hadoop.hive.metastore.model.MPartition]
+ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTITIONS_N49 ON PARTITIONS (SD_ID);
+
+CREATE INDEX PARTITIONS_N50 ON PARTITIONS (TBL_ID);
+
+CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (PART_NAME,TBL_ID);
+
+
+-- Constraints for table INDEX_PARAMS
+ALTER TABLE INDEX_PARAMS ADD CONSTRAINT INDEX_PARAMS_FK1 FOREIGN KEY (INDEX_ID) REFERENCES IDXS (INDEX_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX INDEX_PARAMS_N49 ON INDEX_PARAMS (INDEX_ID);
+
+
+-- Constraints for table TBL_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege]
+ALTER TABLE TBL_COL_PRIVS ADD CONSTRAINT TBL_COL_PRIVS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TABLECOLUMNPRIVILEGEINDEX ON TBL_COL_PRIVS (TBL_ID,"COLUMN_NAME",PRINCIPAL_NAME,PRINCIPAL_TYPE,TBL_COL_PRIV,GRANTOR,GRANTOR_TYPE);
+
+CREATE INDEX TBL_COL_PRIVS_N49 ON TBL_COL_PRIVS (TBL_ID);
+
+
+-- Constraints for table IDXS for class(es) [org.apache.hadoop.hive.metastore.model.MIndex]
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK1 FOREIGN KEY (ORIG_TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK3 FOREIGN KEY (INDEX_TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE UNIQUE INDEX UNIQUEINDEX ON IDXS (INDEX_NAME,ORIG_TBL_ID);
+
+CREATE INDEX IDXS_N50 ON IDXS (INDEX_TBL_ID);
+
+CREATE INDEX IDXS_N51 ON IDXS (SD_ID);
+
+CREATE INDEX IDXS_N49 ON IDXS (ORIG_TBL_ID);
+
+
+-- Constraints for table BUCKETING_COLS
+ALTER TABLE BUCKETING_COLS ADD CONSTRAINT BUCKETING_COLS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX BUCKETING_COLS_N49 ON BUCKETING_COLS (SD_ID);
+
+
+-- Constraints for table TYPE_FIELDS
+ALTER TABLE TYPE_FIELDS ADD CONSTRAINT TYPE_FIELDS_FK1 FOREIGN KEY (TYPE_NAME) REFERENCES TYPES (TYPES_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TYPE_FIELDS_N49 ON TYPE_FIELDS (TYPE_NAME);
+
+
+-- Constraints for table SD_PARAMS
+ALTER TABLE SD_PARAMS ADD CONSTRAINT SD_PARAMS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SD_PARAMS_N49 ON SD_PARAMS (SD_ID);
+
+
+-- Constraints for table GLOBAL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MGlobalPrivilege]
+CREATE UNIQUE INDEX GLOBALPRIVILEGEINDEX ON GLOBAL_PRIVS (PRINCIPAL_NAME,PRINCIPAL_TYPE,USER_PRIV,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table SDS for class(es) [org.apache.hadoop.hive.metastore.model.MStorageDescriptor]
+ALTER TABLE SDS ADD CONSTRAINT SDS_FK1 FOREIGN KEY (SERDE_ID) REFERENCES SERDES (SERDE_ID) INITIALLY DEFERRED ;
+ALTER TABLE SDS ADD CONSTRAINT SDS_FK2 FOREIGN KEY (CD_ID) REFERENCES CDS (CD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SDS_N49 ON SDS (SERDE_ID);
+CREATE INDEX SDS_N50 ON SDS (CD_ID);
+
+
+-- Constraints for table TABLE_PARAMS
+ALTER TABLE TABLE_PARAMS ADD CONSTRAINT TABLE_PARAMS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TABLE_PARAMS_N49 ON TABLE_PARAMS (TBL_ID);
+
+
+-- Constraints for table SORT_COLS
+ALTER TABLE SORT_COLS ADD CONSTRAINT SORT_COLS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SORT_COLS_N49 ON SORT_COLS (SD_ID);
+
+
+-- Constraints for table TBL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MTablePrivilege]
+ALTER TABLE TBL_PRIVS ADD CONSTRAINT TBL_PRIVS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TBL_PRIVS_N49 ON TBL_PRIVS (TBL_ID);
+
+CREATE INDEX TABLEPRIVILEGEINDEX ON TBL_PRIVS (TBL_ID,PRINCIPAL_NAME,PRINCIPAL_TYPE,TBL_PRIV,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table DATABASE_PARAMS
+ALTER TABLE DATABASE_PARAMS ADD CONSTRAINT DATABASE_PARAMS_FK1 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX DATABASE_PARAMS_N49 ON DATABASE_PARAMS (DB_ID);
+
+
+-- Constraints for table ROLE_MAP for class(es) [org.apache.hadoop.hive.metastore.model.MRoleMap]
+ALTER TABLE ROLE_MAP ADD CONSTRAINT ROLE_MAP_FK1 FOREIGN KEY (ROLE_ID) REFERENCES ROLES (ROLE_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX ROLE_MAP_N49 ON ROLE_MAP (ROLE_ID);
+
+CREATE UNIQUE INDEX USERROLEMAPINDEX ON ROLE_MAP (PRINCIPAL_NAME,ROLE_ID,GRANTOR,GRANTOR_TYPE);
+
+
+-- Constraints for table SERDE_PARAMS
+ALTER TABLE SERDE_PARAMS ADD CONSTRAINT SERDE_PARAMS_FK1 FOREIGN KEY (SERDE_ID) REFERENCES SERDES (SERDE_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX SERDE_PARAMS_N49 ON SERDE_PARAMS (SERDE_ID);
+
+
+-- Constraints for table PART_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionPrivilege]
+ALTER TABLE PART_PRIVS ADD CONSTRAINT PART_PRIVS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX PARTPRIVILEGEINDEX ON PART_PRIVS (PART_ID,PRINCIPAL_NAME,PRINCIPAL_TYPE,PART_PRIV,GRANTOR,GRANTOR_TYPE);
+
+CREATE INDEX PART_PRIVS_N49 ON PART_PRIVS (PART_ID);
+
+
+-- Constraints for table DB_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MDBPrivilege]
+ALTER TABLE DB_PRIVS ADD CONSTRAINT DB_PRIVS_FK1 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED ;
+
+CREATE UNIQUE INDEX DBPRIVILEGEINDEX ON DB_PRIVS (DB_ID,PRINCIPAL_NAME,PRINCIPAL_TYPE,DB_PRIV,GRANTOR,GRANTOR_TYPE);
+
+CREATE INDEX DB_PRIVS_N49 ON DB_PRIVS (DB_ID);
+
+
+-- Constraints for table TBLS for class(es) [org.apache.hadoop.hive.metastore.model.MTable]
+ALTER TABLE TBLS ADD CONSTRAINT TBLS_FK2 FOREIGN KEY (DB_ID) REFERENCES DBS (DB_ID) INITIALLY DEFERRED ;
+
+ALTER TABLE TBLS ADD CONSTRAINT TBLS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
+
+CREATE INDEX TBLS_N49 ON TBLS (DB_ID);
+
+CREATE UNIQUE INDEX UNIQUETABLE ON TBLS (TBL_NAME,DB_ID);
+
+CREATE INDEX TBLS_N50 ON TBLS (SD_ID);
+
+
+-- Constraints for table PARTITION_EVENTS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionEvent]
+CREATE INDEX PARTITIONEVENTINDEX ON PARTITION_EVENTS (PARTITION_NAME);
+
+INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.12.0', 'Hive release version 0.12.0');
+


[41/50] [abbrv] bigtop git commit: Sort of working prototype. Lots of hard coded pieces at the moment.

Posted by rv...@apache.org.
Sort of working prototype.  Lots of hard coded pieces at the moment.


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

Branch: refs/heads/BIGTOP-2666
Commit: 246939aee4796bbc622e6527034ed9751b92f25e
Parents: 27ba88b
Author: Alan Gates <ga...@hortonworks.com>
Authored: Mon Nov 7 15:52:37 2016 -0800
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:51:24 2017 -0800

----------------------------------------------------------------------
 .../src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/246939ae/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java b/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
index 4a733d6..6fcfe37 100644
--- a/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
+++ b/bigtop-tests/spec-tests/runtime/src/main/java/org/odpi/specs/runtime/hive/HCatalogMR.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 import org.apache.hive.hcatalog.data.DefaultHCatRecord;
 import org.apache.hive.hcatalog.data.HCatRecord;
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
@@ -74,6 +75,7 @@ public class HCatalogMR extends Configured implements Tool {
     job.addCacheArchive(new URI("hdfs:/user/gates/hive-hcatalog-core-1.2.1.jar"));
     job.addCacheArchive(new URI("hdfs:/user/gates/hive-metastore-1.2.1.jar"));
     job.addCacheArchive(new URI("hdfs:/user/gates/hive-exec-1.2.1.jar"));
+    job.addCacheArchive(new URI("hdfs:/user/gates/libfb303-0.9.2.jar"));
 
     return job.waitForCompletion(true) ? 0 : 1;
 
@@ -121,4 +123,9 @@ public class HCatalogMR extends Configured implements Tool {
       context.write(null, output);
     }
   }
+
+  public static void main(String[] args) throws Exception {
+    int exitCode = ToolRunner.run(new HCatalogMR(), args);
+    System.exit(exitCode);
+  }
  }


[02/50] [abbrv] bigtop git commit: Modified vagrant config to include hive and point to trunk artifacts.

Posted by rv...@apache.org.
Modified vagrant config to include hive and point to trunk artifacts.


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

Branch: refs/heads/BIGTOP-2666
Commit: ae817611c4909aa8d0e1ddde38324c8c057e9163
Parents: 6f6bd47
Author: Raj Desai <rd...@us.ibm.com>
Authored: Tue Sep 20 13:56:44 2016 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Fri Feb 24 11:45:54 2017 -0800

----------------------------------------------------------------------
 provisioner/docker/config_centos6.yaml | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/ae817611/provisioner/docker/config_centos6.yaml
----------------------------------------------------------------------
diff --git a/provisioner/docker/config_centos6.yaml b/provisioner/docker/config_centos6.yaml
index 368a1a5..1c143db 100644
--- a/provisioner/docker/config_centos6.yaml
+++ b/provisioner/docker/config_centos6.yaml
@@ -21,9 +21,12 @@ boot2docker:
         memory_size: "4096"
         number_cpus: "1"
 
-repo: "http://build.odpi.org:8080/job/ODPi-v1.0-collect/lastSuccessfulBuild/artifact/1.0/centos-6"
+repo: "http://build.odpi.org:8080/job/ODPi-trunk-collect/lastSuccessfulBuild/artifact/trunk/centos-6"
 distro: centos
-components: [hadoop, yarn]
+components: [hadoop, yarn, hive]
+namenode_ui_port: "50070"
+yarn_ui_port: "8088"
+hbase_ui_port: "60010"
 enable_local_repo: false
-smoke_test_components: [mapreduce,hcfs,hdfs,yarn]
+smoke_test_components: [mapreduce,hcfs,hdfs,yarn,hive]
 jdk: "java-1.7.0-openjdk-devel.x86_64"


[08/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/setup_ranger_hive.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/setup_ranger_hive.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/setup_ranger_hive.py
new file mode 100755
index 0000000..81a4e3e
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/setup_ranger_hive.py
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+from resource_management.core.logger import Logger
+
+def setup_ranger_hive(upgrade_type = None):
+  import params
+
+  if params.has_ranger_admin:
+
+    stack_version = None
+
+    if upgrade_type is not None:
+      stack_version = params.version
+
+    if params.retryAble:
+      Logger.info("Hive: Setup ranger: command retry enables thus retrying if ranger admin is down !")
+    else:
+      Logger.info("Hive: Setup ranger: command retry not enabled thus skipping if ranger admin is down !")
+
+    if params.xml_configurations_supported and params.enable_ranger_hive and params.xa_audit_hdfs_is_enabled:
+      params.HdfsResource("/ranger/audit",
+                         type="directory",
+                         action="create_on_execute",
+                         owner=params.hdfs_user,
+                         group=params.hdfs_user,
+                         mode=0755,
+                         recursive_chmod=True
+      )
+      params.HdfsResource("/ranger/audit/hiveServer2",
+                         type="directory",
+                         action="create_on_execute",
+                         owner=params.hive_user,
+                         group=params.hive_user,
+                         mode=0700,
+                         recursive_chmod=True
+      )
+      params.HdfsResource(None, action="execute")
+
+    if params.xml_configurations_supported:
+      api_version=None
+      if params.stack_supports_ranger_kerberos:
+        api_version='v2'
+      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
+      setup_ranger_plugin('hive-server2', 'hive', params.ranger_previous_jdbc_jar,
+                          params.ranger_downloaded_custom_connector, params.ranger_driver_curl_source,
+                          params.ranger_driver_curl_target, params.java64_home,
+                          params.repo_name, params.hive_ranger_plugin_repo,
+                          params.ranger_env, params.ranger_plugin_properties,
+                          params.policy_user, params.policymgr_mgr_url,
+                          params.enable_ranger_hive, conf_dict=params.hive_server_conf_dir,
+                          component_user=params.hive_user, component_group=params.user_group, cache_service_list=['hiveServer2'],
+                          plugin_audit_properties=params.config['configurations']['ranger-hive-audit'], plugin_audit_attributes=params.config['configuration_attributes']['ranger-hive-audit'],
+                          plugin_security_properties=params.config['configurations']['ranger-hive-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-hive-security'],
+                          plugin_policymgr_ssl_properties=params.config['configurations']['ranger-hive-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-hive-policymgr-ssl'],
+                          component_list=['hive-client', 'hive-metastore', 'hive-server2'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
+                          credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password,
+                          ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
+                          stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble, api_version=api_version,
+                          is_security_enabled = params.security_enabled,
+                          is_stack_supports_ranger_kerberos = params.stack_supports_ranger_kerberos,
+                          component_user_principal=params.hive_principal if params.security_enabled else None,
+                          component_user_keytab=params.hive_server2_keytab if params.security_enabled else None)
+    else:
+      from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
+      setup_ranger_plugin('hive-server2', 'hive', params.ranger_previous_jdbc_jar,
+                        params.ranger_downloaded_custom_connector, params.ranger_driver_curl_source,
+                        params.ranger_driver_curl_target, params.java64_home,
+                        params.repo_name, params.hive_ranger_plugin_repo,
+                        params.ranger_env, params.ranger_plugin_properties,
+                        params.policy_user, params.policymgr_mgr_url,
+                        params.enable_ranger_hive, conf_dict=params.hive_server_conf_dir,
+                        component_user=params.hive_user, component_group=params.user_group, cache_service_list=['hiveServer2'],
+                        plugin_audit_properties=params.config['configurations']['ranger-hive-audit'], plugin_audit_attributes=params.config['configuration_attributes']['ranger-hive-audit'],
+                        plugin_security_properties=params.config['configurations']['ranger-hive-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-hive-security'],
+                        plugin_policymgr_ssl_properties=params.config['configurations']['ranger-hive-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-hive-policymgr-ssl'],
+                        component_list=['hive-client', 'hive-metastore', 'hive-server2'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
+                        credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password,
+                        ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
+                        stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble)
+  else:
+    Logger.info('Ranger admin not installed')

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/setup_ranger_hive_interactive.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/setup_ranger_hive_interactive.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/setup_ranger_hive_interactive.py
new file mode 100755
index 0000000..0b5d5db
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/setup_ranger_hive_interactive.py
@@ -0,0 +1,78 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+from resource_management.core.logger import Logger
+
+def setup_ranger_hive_interactive(upgrade_type = None):
+  import params
+
+  if params.has_ranger_admin:
+
+    stack_version = None
+
+    if upgrade_type is not None:
+      stack_version = params.version
+
+    if params.retryAble:
+      Logger.info("Hive2: Setup ranger: command retry enabled thus retrying if ranger admin is down !")
+    else:
+      Logger.info("Hive2: Setup ranger: command retry not enabled thus skipping if ranger admin is down !")
+
+    if params.xml_configurations_supported and params.enable_ranger_hive and params.xa_audit_hdfs_is_enabled:
+      params.HdfsResource("/ranger/audit",
+                         type="directory",
+                         action="create_on_execute",
+                         owner=params.hdfs_user,
+                         group=params.hdfs_user,
+                         mode=0755,
+                         recursive_chmod=True
+      )
+      params.HdfsResource("/ranger/audit/hive2",
+                         type="directory",
+                         action="create_on_execute",
+                         owner=params.hive_user,
+                         group=params.hive_user,
+                         mode=0700,
+                         recursive_chmod=True
+      )
+      params.HdfsResource(None, action="execute")
+
+      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
+      setup_ranger_plugin('hive-server2-hive2', 'hive', params.ranger_previous_jdbc_jar,
+                          params.ranger_downloaded_custom_connector, params.ranger_driver_curl_source,
+                          params.ranger_driver_curl_target, params.java64_home,
+                          params.repo_name, params.hive_ranger_plugin_repo,
+                          params.ranger_env, params.ranger_plugin_properties,
+                          params.policy_user, params.policymgr_mgr_url,
+                          params.enable_ranger_hive, conf_dict=params.hive_server_interactive_conf_dir,
+                          component_user=params.hive_user, component_group=params.user_group, cache_service_list=['hive-server2-hive2'],
+                          plugin_audit_properties=params.config['configurations']['ranger-hive-audit'], plugin_audit_attributes=params.config['configuration_attributes']['ranger-hive-audit'],
+                          plugin_security_properties=params.config['configurations']['ranger-hive-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-hive-security'],
+                          plugin_policymgr_ssl_properties=params.config['configurations']['ranger-hive-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-hive-policymgr-ssl'],
+                          component_list=['hive-client', 'hive-metastore', 'hive-server2','hive-server2-hive2'], audit_db_is_enabled=False,
+                          credential_file=params.credential_file, xa_audit_db_password=None,
+                          ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
+                          stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble, api_version='v2',
+                          is_security_enabled = params.security_enabled,
+                          is_stack_supports_ranger_kerberos = params.stack_supports_ranger_kerberos,
+                          component_user_principal=params.hive_principal if params.security_enabled else None,
+                          component_user_keytab=params.hive_server2_keytab if params.security_enabled else None)
+
+  else:
+    Logger.info('Ranger admin not installed')

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
new file mode 100755
index 0000000..b7cb148
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/status_params.py
@@ -0,0 +1,118 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+from ambari_commons import OSCheck
+
+from resource_management.libraries.functions import conf_select
+from resource_management.libraries.functions import stack_select
+from resource_management.libraries.functions import format
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.functions.version import format_stack_version
+from resource_management.libraries.functions.default import default
+from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.script.script import Script
+
+
+# a map of the Ambari role to the component name
+# for use with <stack-root>/current/<component>
+SERVER_ROLE_DIRECTORY_MAP = {
+  'HIVE_METASTORE' : 'hive-metastore',
+  'HIVE_SERVER' : 'hive-server2',
+  'WEBHCAT_SERVER' : 'hive-webhcat',
+  'HIVE_CLIENT' : 'hive-client',
+  'HCAT' : 'hive-client',
+  'HIVE_SERVER_INTERACTIVE' : 'hive-server2-hive2'
+}
+
+
+# Either HIVE_METASTORE, HIVE_SERVER, WEBHCAT_SERVER, HIVE_CLIENT, HCAT, HIVE_SERVER_INTERACTIVE
+role = default("/role", None)
+component_directory = Script.get_component_from_role(SERVER_ROLE_DIRECTORY_MAP, "HIVE_CLIENT")
+component_directory_interactive = Script.get_component_from_role(SERVER_ROLE_DIRECTORY_MAP, "HIVE_SERVER_INTERACTIVE")
+
+config = Script.get_config()
+
+stack_root = Script.get_stack_root()
+stack_version_unformatted = config['hostLevelParams']['stack_version']
+stack_version_formatted_major = format_stack_version(stack_version_unformatted)
+
+if OSCheck.is_windows_family():
+  hive_metastore_win_service_name = "metastore"
+  hive_client_win_service_name = "hwi"
+  hive_server_win_service_name = "hiveserver2"
+  webhcat_server_win_service_name = "templeton"
+else:
+  hive_pid_dir = config['configurations']['hive-env']['hive_pid_dir']
+  hive_pid = 'hive-server.pid'
+  hive_interactive_pid = 'hive-interactive.pid'
+  hive_metastore_pid = 'hive.pid'
+
+  hcat_pid_dir = config['configurations']['hive-env']['hcat_pid_dir'] #hcat_pid_dir
+  webhcat_pid_file = format('{hcat_pid_dir}/webhcat.pid')
+
+  process_name = 'mysqld'
+  if OSCheck.is_suse_family() or OSCheck.is_ubuntu_family():
+    daemon_name = 'mysql'
+  else:
+    daemon_name = 'mysqld'
+
+  # Security related/required params
+  hostname = config['hostname']
+  security_enabled = config['configurations']['cluster-env']['security_enabled']
+  kinit_path_local = get_kinit_path(default('/configurations/kerberos-env/executable_search_paths', None))
+  tmp_dir = Script.get_tmp_dir()
+  hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
+  hive_user = config['configurations']['hive-env']['hive_user']
+  webhcat_user = config['configurations']['hive-env']['webhcat_user']
+
+  # default configuration directories
+  hadoop_conf_dir = conf_select.get_hadoop_conf_dir()
+  hadoop_bin_dir = stack_select.get_hadoop_dir("bin")
+  hive_etc_dir_prefix = "/etc/hive"
+  hive_interactive_etc_dir_prefix = "/etc/hive2"
+
+  hive_server_conf_dir = "/etc/hive/conf.server"
+  hive_server_interactive_conf_dir = "/etc/hive2/conf.server"
+
+  webhcat_conf_dir = format("{stack_root}/current/hive-webhcat/conf")
+  hive_home_dir = format("{stack_root}/current/{component_directory}")
+  hive_conf_dir = format("{stack_root}/current/{component_directory}/conf")
+  hive_client_conf_dir = format("{stack_root}/current/{component_directory}/conf")
+
+  if check_stack_feature(StackFeature.CONFIG_VERSIONING, stack_version_formatted_major):
+    hive_server_conf_dir = format("{stack_root}/current/{component_directory}/conf/conf.server")
+    hive_conf_dir = hive_server_conf_dir
+
+  if check_stack_feature(StackFeature.HIVE_WEBHCAT_SPECIFIC_CONFIGS, stack_version_formatted_major):
+    # this is NOT a typo. Configs for hcatalog/webhcat point to a
+    # specific directory which is NOT called 'conf'
+    webhcat_conf_dir = format("{stack_root}/current/hive-webhcat/etc/webhcat")
+
+  # if stack version supports hive serve interactive
+  if check_stack_feature(StackFeature.HIVE_SERVER_INTERACTIVE, stack_version_formatted_major):
+    hive_server_interactive_conf_dir = format("{stack_root}/current/{component_directory_interactive}/conf/conf.server")
+
+  hive_config_dir = hive_client_conf_dir
+
+  if 'role' in config and config['role'] in ["HIVE_SERVER", "HIVE_METASTORE", "HIVE_SERVER_INTERACTIVE"]:
+    hive_config_dir = hive_server_conf_dir
+    
+stack_name = default("/hostLevelParams/stack_name", None)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat.py
new file mode 100755
index 0000000..fe3f34a
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat.py
@@ -0,0 +1,145 @@
+"""
+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.
+
+Ambari Agent
+
+"""
+import sys
+import os.path
+from resource_management import *
+from resource_management.core.resources.system import Execute
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from resource_management.libraries.functions.setup_atlas_hook import has_atlas_in_cluster, setup_atlas_hook
+from ambari_commons import OSConst
+from ambari_commons.constants import SERVICE
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def webhcat():
+  import params
+  XmlConfig("webhcat-site.xml",
+            conf_dir=params.hcat_config_dir,
+            configurations=params.config['configurations']['webhcat-site']
+  )
+  # Manually overriding service logon user & password set by the installation package
+  ServiceConfig(params.webhcat_server_win_service_name,
+                action="change_user",
+                username = params.hcat_user,
+                password = Script.get_password(params.hcat_user))
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def webhcat():
+  import params
+
+  Directory(params.templeton_pid_dir,
+            owner=params.webhcat_user,
+            mode=0755,
+            group=params.user_group,
+            create_parents = True)
+
+  Directory(params.templeton_log_dir,
+            owner=params.webhcat_user,
+            mode=0755,
+            group=params.user_group,
+            create_parents = True)
+
+  Directory(params.config_dir,
+            create_parents = True,
+            owner=params.webhcat_user,
+            group=params.user_group,
+            cd_access="a")
+
+  if params.security_enabled:
+    kinit_if_needed = format("{kinit_path_local} -kt {hdfs_user_keytab} {hdfs_principal_name};")
+  else:
+    kinit_if_needed = ""
+
+  if kinit_if_needed:
+    Execute(kinit_if_needed,
+            user=params.webhcat_user,
+            path='/bin'
+    )
+
+  # Replace _HOST with hostname in relevant principal-related properties
+  webhcat_site = params.config['configurations']['webhcat-site'].copy()
+  for prop_name in ['templeton.hive.properties', 'templeton.kerberos.principal']:
+    if prop_name in webhcat_site:
+      webhcat_site[prop_name] = webhcat_site[prop_name].replace("_HOST", params.hostname)
+
+  XmlConfig("webhcat-site.xml",
+            conf_dir=params.config_dir,
+            configurations=webhcat_site,
+            configuration_attributes=params.config['configuration_attributes']['webhcat-site'],
+            owner=params.webhcat_user,
+            group=params.user_group,
+            )
+
+  # if we're in an upgrade of a secure cluster, make sure hive-site and yarn-site are created
+  if params.stack_version_formatted_major  and check_stack_feature(StackFeature.CONFIG_VERSIONING, params.stack_version_formatted_major) and \
+       params.version and params.stack_root:
+    XmlConfig("hive-site.xml",
+      conf_dir = format("{stack_root}/{version}/hive/conf"),
+      configurations = params.config['configurations']['hive-site'],
+      configuration_attributes = params.config['configuration_attributes']['hive-site'],
+      owner = params.hive_user,
+      group = params.user_group,
+      )
+
+    XmlConfig("yarn-site.xml",
+      conf_dir = format("{stack_root}/{version}/hadoop/conf"),
+      configurations = params.config['configurations']['yarn-site'],
+      configuration_attributes = params.config['configuration_attributes']['yarn-site'],
+      owner = params.yarn_user,
+      group = params.user_group,    
+  )
+  
+
+  File(format("{config_dir}/webhcat-env.sh"),
+       owner=params.webhcat_user,
+       group=params.user_group,
+       content=InlineTemplate(params.webhcat_env_sh_template)
+  )
+  
+  Directory(params.webhcat_conf_dir,
+       cd_access='a',
+       create_parents = True
+  )
+
+  log4j_webhcat_filename = 'webhcat-log4j.properties'
+  if (params.log4j_webhcat_props != None):
+    File(format("{config_dir}/{log4j_webhcat_filename}"),
+         mode=0644,
+         group=params.user_group,
+         owner=params.webhcat_user,
+         content=params.log4j_webhcat_props
+    )
+  elif (os.path.exists("{config_dir}/{log4j_webhcat_filename}.template")):
+    File(format("{config_dir}/{log4j_webhcat_filename}"),
+         mode=0644,
+         group=params.user_group,
+         owner=params.webhcat_user,
+         content=StaticFile(format("{config_dir}/{log4j_webhcat_filename}.template"))
+    )
+
+  # Generate atlas-application.properties.xml file
+  if has_atlas_in_cluster():
+    # WebHCat uses a different config dir than the rest of the daemons in Hive.
+    atlas_hook_filepath = os.path.join(params.config_dir, params.atlas_hook_filename)
+    setup_atlas_hook(SERVICE.HIVE, params.hive_atlas_application_properties, atlas_hook_filepath, params.hive_user, params.user_group)

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_server.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_server.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_server.py
new file mode 100755
index 0000000..34687c4
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_server.py
@@ -0,0 +1,164 @@
+"""
+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.
+
+Ambari Agent
+
+"""
+from resource_management import *
+from resource_management.libraries.functions import conf_select
+from resource_management.libraries.functions import stack_select
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.functions.security_commons import build_expectations, \
+  cached_kinit_executor, get_params_from_filesystem, validate_security_config_properties, \
+  FILE_TYPE_XML
+from webhcat import webhcat
+from webhcat_service import webhcat_service
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
+
+
+class WebHCatServer(Script):
+  def install(self, env):
+    import params
+    self.install_packages(env)
+
+  def start(self, env, upgrade_type=None):
+    import params
+    env.set_params(params)
+    self.configure(env) # FOR SECURITY
+    webhcat_service(action='start', upgrade_type=upgrade_type)
+
+  def stop(self, env, upgrade_type=None):
+    import params
+    env.set_params(params)
+    webhcat_service(action='stop')
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    webhcat()
+
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class WebHCatServerWindows(WebHCatServer):
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    check_windows_service_status(status_params.webhcat_server_win_service_name)
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class WebHCatServerDefault(WebHCatServer):
+  def get_component_name(self):
+    return "hive-webhcat"
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    check_process_status(status_params.webhcat_pid_file)
+
+  def pre_upgrade_restart(self, env, upgrade_type=None):
+    Logger.info("Executing WebHCat Stack Upgrade pre-restart")
+    import params
+    env.set_params(params)
+
+    if params.version and check_stack_feature(StackFeature.ROLLING_UPGRADE, params.version): 
+      # webhcat has no conf, but uses hadoop home, so verify that regular hadoop conf is set
+      conf_select.select(params.stack_name, "hive-hcatalog", params.version)
+      conf_select.select(params.stack_name, "hadoop", params.version)
+      stack_select.select("hive-webhcat", params.version)
+
+  def security_status(self, env):
+    import status_params
+    env.set_params(status_params)
+
+    if status_params.security_enabled:
+      expectations ={}
+      expectations.update(
+        build_expectations(
+          'webhcat-site',
+          {
+            "templeton.kerberos.secret": "secret"
+          },
+          [
+            "templeton.kerberos.keytab",
+            "templeton.kerberos.principal"
+          ],
+          [
+            "templeton.kerberos.keytab"
+          ]
+        )
+      )
+      expectations.update(
+        build_expectations(
+          'hive-site',
+          {
+            "hive.server2.authentication": "KERBEROS",
+            "hive.metastore.sasl.enabled": "true",
+            "hive.security.authorization.enabled": "true"
+          },
+          None,
+          None
+        )
+      )
+
+      security_params = {}
+      security_params.update(get_params_from_filesystem(status_params.hive_conf_dir,
+                                                        {'hive-site.xml': FILE_TYPE_XML}))
+      security_params.update(get_params_from_filesystem(status_params.webhcat_conf_dir,
+                                                        {'webhcat-site.xml': FILE_TYPE_XML}))
+      result_issues = validate_security_config_properties(security_params, expectations)
+      if not result_issues: # If all validations passed successfully
+        try:
+          # Double check the dict before calling execute
+          if 'webhcat-site' not in security_params \
+            or 'templeton.kerberos.keytab' not in security_params['webhcat-site'] \
+            or 'templeton.kerberos.principal' not in security_params['webhcat-site']:
+            self.put_structured_out({"securityState": "UNSECURED"})
+            self.put_structured_out({"securityIssuesFound": "Keytab file or principal are not set property."})
+            return
+
+          cached_kinit_executor(status_params.kinit_path_local,
+                                status_params.webhcat_user,
+                                security_params['webhcat-site']['templeton.kerberos.keytab'],
+                                security_params['webhcat-site']['templeton.kerberos.principal'],
+                                status_params.hostname,
+                                status_params.tmp_dir)
+          self.put_structured_out({"securityState": "SECURED_KERBEROS"})
+        except Exception as e:
+          self.put_structured_out({"securityState": "ERROR"})
+          self.put_structured_out({"securityStateErrorInfo": str(e)})
+      else:
+        issues = []
+        for cf in result_issues:
+          issues.append("Configuration file %s did not pass the validation. Reason: %s" % (cf, result_issues[cf]))
+        self.put_structured_out({"securityIssuesFound": ". ".join(issues)})
+        self.put_structured_out({"securityState": "UNSECURED"})
+    else:
+      self.put_structured_out({"securityState": "UNSECURED"})
+
+  def get_log_folder(self):
+    import params
+    return params.hcat_log_dir
+  
+  def get_user(self):
+    import params
+    return params.webhcat_user
+
+if __name__ == "__main__":
+  WebHCatServer().execute()

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_service.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_service.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_service.py
new file mode 100755
index 0000000..c24db4c
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_service.py
@@ -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.
+
+Ambari Agent
+
+"""
+from resource_management import *
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+from resource_management.core.shell import as_user
+from resource_management.core.logger import Logger
+import traceback
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def webhcat_service(action='start', rolling_restart=False):
+  import params
+  if action == 'start' or action == 'stop':
+    Service(params.webhcat_server_win_service_name, action=action)
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def webhcat_service(action='start', upgrade_type=None):
+  import params
+
+  environ = {
+    'HADOOP_HOME': params.hadoop_home
+  }
+
+  cmd = format('{webhcat_bin_dir}/webhcat_server.sh')
+
+  if action == 'start':
+    if upgrade_type is not None and params.version and params.stack_root:
+      environ['HADOOP_HOME'] = format("{stack_root}/{version}/hadoop")
+
+    daemon_cmd = format('cd {hcat_pid_dir} ; {cmd} start')
+    no_op_test = as_user(format('ls {webhcat_pid_file} >/dev/null 2>&1 && ps -p `cat {webhcat_pid_file}` >/dev/null 2>&1'), user=params.webhcat_user)
+    try:
+      Execute(daemon_cmd,
+              user=params.webhcat_user,
+              not_if=no_op_test,
+              environment = environ)
+    except:
+      show_logs(params.hcat_log_dir, params.webhcat_user)
+      raise
+  elif action == 'stop':
+    try:
+      graceful_stop(cmd, environ)
+    except Fail:
+      show_logs(params.hcat_log_dir, params.webhcat_user)
+      Logger.info(traceback.format_exc())
+
+    pid_expression = "`" + as_user(format("cat {webhcat_pid_file}"), user=params.webhcat_user) + "`"
+    process_id_exists_command = format("ls {webhcat_pid_file} >/dev/null 2>&1 && ps -p {pid_expression} >/dev/null 2>&1")
+    daemon_hard_kill_cmd = format("{sudo} kill -9 {pid_expression}")
+    wait_time = 10
+    Execute(daemon_hard_kill_cmd,
+            not_if = format("! ({process_id_exists_command}) || ( sleep {wait_time} && ! ({process_id_exists_command}) )"),
+            ignore_failures = True
+    )
+
+    try:
+      # check if stopped the process, else fail the task
+      Execute(format("! ({process_id_exists_command})"),
+              tries=20,
+              try_sleep=3,
+      )
+    except:
+      show_logs(params.hcat_log_dir, params.webhcat_user)
+      raise
+
+    File(params.webhcat_pid_file,
+         action="delete",
+    )
+
+def graceful_stop(cmd, environ):
+  import params
+  daemon_cmd = format('{cmd} stop')
+
+  Execute(daemon_cmd,
+          user = params.webhcat_user,
+          environment = environ)

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_service_check.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_service_check.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_service_check.py
new file mode 100755
index 0000000..8e80d48
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/webhcat_service_check.py
@@ -0,0 +1,128 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+import urllib2
+
+from resource_management import *
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+import time
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def webhcat_service_check():
+  Logger.info("Webhcat smoke test - service status")
+
+  import params
+  # AMBARI-11633 [WinTP2] Webhcat service check fails
+  # Hive doesn't pass the environment variables correctly to child processes, which fails the smoke test.
+  # Reducing the amount of URLs checked to the minimum required.
+  #smoke_cmd = os.path.join(params.stack_root,"Run-SmokeTests.cmd")
+  #service = "WEBHCAT"
+  #Execute(format("cmd /C {smoke_cmd} {service}"), user=params.hcat_user, logoutput=True)
+
+  url_tests = [
+    "status",
+    #These are the failing ones:
+    #"ddl/database?user.name=hadoop",
+    #"ddl/database/default/table?user.name=hadoop"
+  ]
+
+
+  import socket
+
+  url_host = socket.getfqdn()
+  url_port = params.config["configurations"]["webhcat-site"]["templeton.port"]
+
+  for url_test in url_tests:
+    url_request = "http://{0}:{1}/templeton/v1/{2}".format(url_host, url_port, url_test)
+    url_response = None
+
+    try:
+      # execute the query for the JSON that includes WebHCat status
+      url_response = urllib2.urlopen(url_request, timeout=30)
+
+      status = url_response.getcode()
+      response = url_response.read()
+
+      if status != 200:
+        Logger.warning("Webhcat service check status: {0}".format(status))
+      Logger.info("Webhcat service check response: {0}".format(response))
+    except urllib2.HTTPError as he:
+      raise Fail("Webhcat check {0} failed: {1}".format(url_request, he.msg))
+    finally:
+      if url_response is not None:
+        try:
+          url_response.close()
+        except:
+          pass
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def webhcat_service_check():
+  import params
+  File(format("{tmp_dir}/templetonSmoke.sh"),
+       content= StaticFile('templetonSmoke.sh'),
+       mode=0755
+  )
+
+  if params.security_enabled:
+    smokeuser_keytab=params.smoke_user_keytab
+    smoke_user_principal=params.smokeuser_principal
+  else:
+    smokeuser_keytab= "no_keytab"
+    smoke_user_principal="no_principal"
+    
+  unique_name = format("{smokeuser}.{timestamp}", timestamp = time.time())
+  templeton_test_script = format("idtest.{unique_name}.pig")
+  templeton_test_input = format("/tmp/idtest.{unique_name}.in")
+  templeton_test_output = format("/tmp/idtest.{unique_name}.out")
+
+  File(format("{tmp_dir}/{templeton_test_script}"),
+       content = Template("templeton_smoke.pig.j2", templeton_test_input=templeton_test_input, templeton_test_output=templeton_test_output),
+       owner=params.hdfs_user
+  )
+  
+  params.HdfsResource(format("/tmp/{templeton_test_script}"),
+                      action = "create_on_execute",
+                      type = "file",
+                      source = format("{tmp_dir}/{templeton_test_script}"),
+                      owner = params.smokeuser
+  )
+  
+  params.HdfsResource(templeton_test_input,
+                      action = "create_on_execute",
+                      type = "file",
+                      source = "/etc/passwd",
+                      owner = params.smokeuser
+  )
+  
+  params.HdfsResource(None, action = "execute")
+
+  cmd = format("{tmp_dir}/templetonSmoke.sh {webhcat_server_host[0]} {smokeuser} {templeton_port} {templeton_test_script} {smokeuser_keytab}"
+               " {security_param} {kinit_path_local} {smoke_user_principal}"
+               " {tmp_dir}")
+
+  Execute(cmd,
+          tries=3,
+          try_sleep=5,
+          path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin',
+          logoutput=True)
+
+
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-hivemetastore.properties.j2
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-hivemetastore.properties.j2 b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-hivemetastore.properties.j2
new file mode 100755
index 0000000..e4d88bc
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-hivemetastore.properties.j2
@@ -0,0 +1,54 @@
+{#
+# 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.
+#}
+
+# 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.
+
+# syntax: [prefix].[source|sink|jmx].[instance].[options]
+# See package.html for org.apache.hadoop.metrics2 for details
+
+{% if has_metric_collector %}
+
+  *.period={{metrics_collection_period}}
+  *.sink.timeline.plugin.urls=file:///usr/lib/ambari-metrics-hadoop-sink/ambari-metrics-hadoop-sink.jar
+  *.sink.timeline.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+  *.sink.timeline.period={{metrics_collection_period}}
+  *.sink.timeline.sendInterval={{metrics_report_interval}}000
+  *.sink.timeline.slave.host.name = {{hostname}}
+
+  # HTTPS properties
+  *.sink.timeline.truststore.path = {{metric_truststore_path}}
+  *.sink.timeline.truststore.type = {{metric_truststore_type}}
+  *.sink.timeline.truststore.password = {{metric_truststore_password}}
+
+  hivemetastore.sink.timeline.collector={{metric_collector_protocol}}://{{metric_collector_host}}:{{metric_collector_port}}
+
+
+{% endif %}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-hiveserver2.properties.j2
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-hiveserver2.properties.j2 b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-hiveserver2.properties.j2
new file mode 100755
index 0000000..b5c4891
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-hiveserver2.properties.j2
@@ -0,0 +1,54 @@
+{#
+# 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.
+#}
+
+# 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.
+
+# syntax: [prefix].[source|sink|jmx].[instance].[options]
+# See package.html for org.apache.hadoop.metrics2 for details
+
+{% if has_metric_collector %}
+
+  *.period={{metrics_collection_period}}
+  *.sink.timeline.plugin.urls=file:///usr/lib/ambari-metrics-hadoop-sink/ambari-metrics-hadoop-sink.jar
+  *.sink.timeline.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+  *.sink.timeline.period={{metrics_collection_period}}
+  *.sink.timeline.sendInterval={{metrics_report_interval}}000
+  *.sink.timeline.slave.host.name = {{hostname}}
+
+  # HTTPS properties
+  *.sink.timeline.truststore.path = {{metric_truststore_path}}
+  *.sink.timeline.truststore.type = {{metric_truststore_type}}
+  *.sink.timeline.truststore.password = {{metric_truststore_password}}
+
+  hiveserver2.sink.timeline.collector={{metric_collector_protocol}}://{{metric_collector_host}}:{{metric_collector_port}}
+
+
+{% endif %}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-llapdaemon.j2
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-llapdaemon.j2 b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-llapdaemon.j2
new file mode 100755
index 0000000..1d75ccf
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-llapdaemon.j2
@@ -0,0 +1,52 @@
+{#
+# 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.
+#}
+
+# 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.
+
+# syntax: [prefix].[source|sink|jmx].[instance].[options]
+# See package.html for org.apache.hadoop.metrics2 for details
+
+{% if has_metric_collector %}
+
+  *.period={{metrics_collection_period}}
+  *.sink.timeline.plugin.urls=file:///usr/lib/ambari-metrics-hadoop-sink/ambari-metrics-hadoop-sink.jar
+  *.sink.timeline.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+  *.sink.timeline.period={{metrics_collection_period}}
+  *.sink.timeline.sendInterval={{metrics_report_interval}}000
+
+  # HTTPS properties
+  *.sink.timeline.truststore.path = {{metric_truststore_path}}
+  *.sink.timeline.truststore.type = {{metric_truststore_type}}
+  *.sink.timeline.truststore.password = {{metric_truststore_password}}
+
+  llapdaemon.sink.timeline.collector={{metric_collector_protocol}}://{{metric_collector_host}}:{{metric_collector_port}}
+
+{% endif %}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-llaptaskscheduler.j2
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-llaptaskscheduler.j2 b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-llaptaskscheduler.j2
new file mode 100755
index 0000000..5ab787c
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hadoop-metrics2-llaptaskscheduler.j2
@@ -0,0 +1,52 @@
+{#
+# 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.
+#}
+
+# 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.
+
+# syntax: [prefix].[source|sink|jmx].[instance].[options]
+# See package.html for org.apache.hadoop.metrics2 for details
+
+{% if has_metric_collector %}
+
+  *.period={{metrics_collection_period}}
+  *.sink.timeline.plugin.urls=file:///usr/lib/ambari-metrics-hadoop-sink/ambari-metrics-hadoop-sink.jar
+  *.sink.timeline.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+  *.sink.timeline.period={{metrics_collection_period}}
+  *.sink.timeline.sendInterval={{metrics_report_interval}}000
+
+  # HTTPS properties
+  *.sink.timeline.truststore.path = {{metric_truststore_path}}
+  *.sink.timeline.truststore.type = {{metric_truststore_type}}
+  *.sink.timeline.truststore.password = {{metric_truststore_password}}
+
+  llaptaskscheduler.sink.timeline.collector={{metric_collector_protocol}}://{{metric_collector_host}}:{{metric_collector_port}}
+
+{% endif %}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hive.conf.j2
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hive.conf.j2 b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hive.conf.j2
new file mode 100755
index 0000000..5af53d0
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/hive.conf.j2
@@ -0,0 +1,35 @@
+{#
+# 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.
+#}
+
+# 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.
+
+{{hive_user}}   - nofile {{hive_user_nofile_limit}}
+{{hive_user}}   - nproc  {{hive_user_nproc_limit}}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/startHiveserver2.sh.j2
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/startHiveserver2.sh.j2 b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/startHiveserver2.sh.j2
new file mode 100755
index 0000000..70b418c
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/startHiveserver2.sh.j2
@@ -0,0 +1,24 @@
+#
+#
+# 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.
+#
+#
+
+HIVE_SERVER2_OPTS=" -hiveconf hive.log.file=hiveserver2.log -hiveconf hive.log.dir=$5"
+HIVE_CONF_DIR=$4 {{hive_bin}}/hiveserver2 -hiveconf hive.metastore.uris=" " ${HIVE_SERVER2_OPTS} > $1 2> $2 &
+echo $!|cat>$3

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/startHiveserver2Interactive.sh.j2
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/startHiveserver2Interactive.sh.j2 b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/startHiveserver2Interactive.sh.j2
new file mode 100755
index 0000000..6062a7e
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/startHiveserver2Interactive.sh.j2
@@ -0,0 +1,24 @@
+#
+#
+# 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.
+#
+#
+
+HIVE_SERVER2_INTERACTIVE_OPTS=" -hiveconf hive.log.file=hiveserver2Interactive.log -hiveconf hive.log.dir=$5"
+HIVE_INTERACTIVE_CONF_DIR=$4 {{hive_interactive_bin}}/hiveserver2 -hiveconf hive.metastore.uris=" " ${HIVE_SERVER2_INTERACTIVE_OPTS} > $1 2> $2 &
+echo $!|cat>$3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/templeton_smoke.pig.j2
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/templeton_smoke.pig.j2 b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/templeton_smoke.pig.j2
new file mode 100755
index 0000000..3153e81
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/templates/templeton_smoke.pig.j2
@@ -0,0 +1,24 @@
+#
+#
+# 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.
+#
+#
+
+A = load '{{templeton_test_input}}' using PigStorage(':');
+B = foreach A generate \$0 as id; 
+store B into '{{templeton_test_output}}';
\ No newline at end of file


[11/50] [abbrv] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.postgres.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.postgres.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.postgres.sql
new file mode 100755
index 0000000..bc6486b
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/etc/hive-schema-0.12.0.postgres.sql
@@ -0,0 +1,1406 @@
+--
+-- PostgreSQL database dump
+--
+
+SET statement_timeout = 0;
+SET client_encoding = 'UTF8';
+SET standard_conforming_strings = off;
+SET check_function_bodies = false;
+SET client_min_messages = warning;
+SET escape_string_warning = off;
+
+SET search_path = public, pg_catalog;
+
+SET default_tablespace = '';
+
+SET default_with_oids = false;
+
+--
+-- Name: BUCKETING_COLS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "BUCKETING_COLS" (
+    "SD_ID" bigint NOT NULL,
+    "BUCKET_COL_NAME" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: CDS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "CDS" (
+    "CD_ID" bigint NOT NULL
+);
+
+
+--
+-- Name: COLUMNS_OLD; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "COLUMNS_OLD" (
+    "SD_ID" bigint NOT NULL,
+    "COMMENT" character varying(256) DEFAULT NULL::character varying,
+    "COLUMN_NAME" character varying(128) NOT NULL,
+    "TYPE_NAME" character varying(4000) NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: COLUMNS_V2; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "COLUMNS_V2" (
+    "CD_ID" bigint NOT NULL,
+    "COMMENT" character varying(4000),
+    "COLUMN_NAME" character varying(128) NOT NULL,
+    "TYPE_NAME" character varying(4000),
+    "INTEGER_IDX" integer NOT NULL
+);
+
+
+--
+-- Name: DATABASE_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "DATABASE_PARAMS" (
+    "DB_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(180) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: DBS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "DBS" (
+    "DB_ID" bigint NOT NULL,
+    "DESC" character varying(4000) DEFAULT NULL::character varying,
+    "DB_LOCATION_URI" character varying(4000) NOT NULL,
+    "NAME" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: DB_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "DB_PRIVS" (
+    "DB_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "DB_ID" bigint,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "DB_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: GLOBAL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "GLOBAL_PRIVS" (
+    "USER_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "USER_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: IDXS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "IDXS" (
+    "INDEX_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "DEFERRED_REBUILD" boolean NOT NULL,
+    "INDEX_HANDLER_CLASS" character varying(4000) DEFAULT NULL::character varying,
+    "INDEX_NAME" character varying(128) DEFAULT NULL::character varying,
+    "INDEX_TBL_ID" bigint,
+    "LAST_ACCESS_TIME" bigint NOT NULL,
+    "ORIG_TBL_ID" bigint,
+    "SD_ID" bigint
+);
+
+
+--
+-- Name: INDEX_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "INDEX_PARAMS" (
+    "INDEX_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: NUCLEUS_TABLES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "NUCLEUS_TABLES" (
+    "CLASS_NAME" character varying(128) NOT NULL,
+    "TABLE_NAME" character varying(128) NOT NULL,
+    "TYPE" character varying(4) NOT NULL,
+    "OWNER" character varying(2) NOT NULL,
+    "VERSION" character varying(20) NOT NULL,
+    "INTERFACE_NAME" character varying(255) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: PARTITIONS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITIONS" (
+    "PART_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "LAST_ACCESS_TIME" bigint NOT NULL,
+    "PART_NAME" character varying(767) DEFAULT NULL::character varying,
+    "SD_ID" bigint,
+    "TBL_ID" bigint
+);
+
+
+--
+-- Name: PARTITION_EVENTS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_EVENTS" (
+    "PART_NAME_ID" bigint NOT NULL,
+    "DB_NAME" character varying(128),
+    "EVENT_TIME" bigint NOT NULL,
+    "EVENT_TYPE" integer NOT NULL,
+    "PARTITION_NAME" character varying(767),
+    "TBL_NAME" character varying(128)
+);
+
+
+--
+-- Name: PARTITION_KEYS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_KEYS" (
+    "TBL_ID" bigint NOT NULL,
+    "PKEY_COMMENT" character varying(4000) DEFAULT NULL::character varying,
+    "PKEY_NAME" character varying(128) NOT NULL,
+    "PKEY_TYPE" character varying(767) NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: PARTITION_KEY_VALS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_KEY_VALS" (
+    "PART_ID" bigint NOT NULL,
+    "PART_KEY_VAL" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: PARTITION_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PARTITION_PARAMS" (
+    "PART_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: PART_COL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PART_COL_PRIVS" (
+    "PART_COLUMN_GRANT_ID" bigint NOT NULL,
+    "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_ID" bigint,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_COL_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: PART_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PART_PRIVS" (
+    "PART_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_ID" bigint,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PART_PRIV" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: ROLES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "ROLES" (
+    "ROLE_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "OWNER_NAME" character varying(128) DEFAULT NULL::character varying,
+    "ROLE_NAME" character varying(128) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: ROLE_MAP; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "ROLE_MAP" (
+    "ROLE_GRANT_ID" bigint NOT NULL,
+    "ADD_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "ROLE_ID" bigint
+);
+
+
+--
+-- Name: SDS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SDS" (
+    "SD_ID" bigint NOT NULL,
+    "INPUT_FORMAT" character varying(4000) DEFAULT NULL::character varying,
+    "IS_COMPRESSED" boolean NOT NULL,
+    "LOCATION" character varying(4000) DEFAULT NULL::character varying,
+    "NUM_BUCKETS" bigint NOT NULL,
+    "OUTPUT_FORMAT" character varying(4000) DEFAULT NULL::character varying,
+    "SERDE_ID" bigint,
+    "CD_ID" bigint,
+    "IS_STOREDASSUBDIRECTORIES" boolean NOT NULL
+);
+
+
+--
+-- Name: SD_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SD_PARAMS" (
+    "SD_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: SEQUENCE_TABLE; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SEQUENCE_TABLE" (
+    "SEQUENCE_NAME" character varying(255) NOT NULL,
+    "NEXT_VAL" bigint NOT NULL
+);
+
+
+--
+-- Name: SERDES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SERDES" (
+    "SERDE_ID" bigint NOT NULL,
+    "NAME" character varying(128) DEFAULT NULL::character varying,
+    "SLIB" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: SERDE_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SERDE_PARAMS" (
+    "SERDE_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: SORT_COLS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "SORT_COLS" (
+    "SD_ID" bigint NOT NULL,
+    "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+    "ORDER" bigint NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: TABLE_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TABLE_PARAMS" (
+    "TBL_ID" bigint NOT NULL,
+    "PARAM_KEY" character varying(256) NOT NULL,
+    "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: TBLS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TBLS" (
+    "TBL_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "DB_ID" bigint,
+    "LAST_ACCESS_TIME" bigint NOT NULL,
+    "OWNER" character varying(767) DEFAULT NULL::character varying,
+    "RETENTION" bigint NOT NULL,
+    "SD_ID" bigint,
+    "TBL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "TBL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "VIEW_EXPANDED_TEXT" text,
+    "VIEW_ORIGINAL_TEXT" text
+);
+
+
+--
+-- Name: TBL_COL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TBL_COL_PRIVS" (
+    "TBL_COLUMN_GRANT_ID" bigint NOT NULL,
+    "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "TBL_COL_PRIV" character varying(128) DEFAULT NULL::character varying,
+    "TBL_ID" bigint
+);
+
+
+--
+-- Name: TBL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TBL_PRIVS" (
+    "TBL_GRANT_ID" bigint NOT NULL,
+    "CREATE_TIME" bigint NOT NULL,
+    "GRANT_OPTION" smallint NOT NULL,
+    "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+    "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+    "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+    "TBL_PRIV" character varying(128) DEFAULT NULL::character varying,
+    "TBL_ID" bigint
+);
+
+
+--
+-- Name: TYPES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TYPES" (
+    "TYPES_ID" bigint NOT NULL,
+    "TYPE_NAME" character varying(128) DEFAULT NULL::character varying,
+    "TYPE1" character varying(767) DEFAULT NULL::character varying,
+    "TYPE2" character varying(767) DEFAULT NULL::character varying
+);
+
+
+--
+-- Name: TYPE_FIELDS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "TYPE_FIELDS" (
+    "TYPE_NAME" bigint NOT NULL,
+    "COMMENT" character varying(256) DEFAULT NULL::character varying,
+    "FIELD_NAME" character varying(128) NOT NULL,
+    "FIELD_TYPE" character varying(767) NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_STRING_LIST" (
+    "STRING_LIST_ID" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_STRING_LIST_VALUES" (
+    "STRING_LIST_ID" bigint NOT NULL,
+    "STRING_LIST_VALUE" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_COL_NAMES" (
+    "SD_ID" bigint NOT NULL,
+    "SKEWED_COL_NAME" character varying(256) DEFAULT NULL::character varying,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+CREATE TABLE "SKEWED_COL_VALUE_LOC_MAP" (
+    "SD_ID" bigint NOT NULL,
+    "STRING_LIST_ID_KID" bigint NOT NULL,
+    "LOCATION" character varying(4000) DEFAULT NULL::character varying
+);
+
+CREATE TABLE "SKEWED_VALUES" (
+    "SD_ID_OID" bigint NOT NULL,
+    "STRING_LIST_ID_EID" bigint NOT NULL,
+    "INTEGER_IDX" bigint NOT NULL
+);
+
+
+--
+-- Name: TAB_COL_STATS Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE  "MASTER_KEYS"
+(
+    "KEY_ID" SERIAL,
+    "MASTER_KEY" varchar(767) NULL,
+    PRIMARY KEY ("KEY_ID")
+);
+
+CREATE TABLE  "DELEGATION_TOKENS"
+(
+    "TOKEN_IDENT" varchar(767) NOT NULL,
+    "TOKEN" varchar(767) NULL,
+    PRIMARY KEY ("TOKEN_IDENT")
+);
+
+CREATE TABLE "TAB_COL_STATS" (
+ "CS_ID" bigint NOT NULL,
+ "DB_NAME" character varying(128) DEFAULT NULL::character varying,
+ "TABLE_NAME" character varying(128) DEFAULT NULL::character varying,
+ "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+ "COLUMN_TYPE" character varying(128) DEFAULT NULL::character varying,
+ "TBL_ID" bigint NOT NULL,
+ "LONG_LOW_VALUE" bigint,
+ "LONG_HIGH_VALUE" bigint,
+ "DOUBLE_LOW_VALUE" double precision,
+ "DOUBLE_HIGH_VALUE" double precision,
+ "BIG_DECIMAL_LOW_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "BIG_DECIMAL_HIGH_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "NUM_NULLS" bigint NOT NULL,
+ "NUM_DISTINCTS" bigint,
+ "AVG_COL_LEN" double precision,
+ "MAX_COL_LEN" bigint,
+ "NUM_TRUES" bigint,
+ "NUM_FALSES" bigint,
+ "LAST_ANALYZED" bigint NOT NULL
+);
+
+--
+-- Table structure for VERSION
+--
+CREATE TABLE "VERSION" (
+  "VER_ID" bigint,
+  "SCHEMA_VERSION" character varying(127) NOT NULL,
+  "VERSION_COMMENT" character varying(255) NOT NULL
+);
+
+--
+-- Name: PART_COL_STATS Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "PART_COL_STATS" (
+ "CS_ID" bigint NOT NULL,
+ "DB_NAME" character varying(128) DEFAULT NULL::character varying,
+ "TABLE_NAME" character varying(128) DEFAULT NULL::character varying,
+ "PARTITION_NAME" character varying(767) DEFAULT NULL::character varying,
+ "COLUMN_NAME" character varying(128) DEFAULT NULL::character varying,
+ "COLUMN_TYPE" character varying(128) DEFAULT NULL::character varying,
+ "PART_ID" bigint NOT NULL,
+ "LONG_LOW_VALUE" bigint,
+ "LONG_HIGH_VALUE" bigint,
+ "DOUBLE_LOW_VALUE" double precision,
+ "DOUBLE_HIGH_VALUE" double precision,
+ "BIG_DECIMAL_LOW_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "BIG_DECIMAL_HIGH_VALUE" character varying(4000) DEFAULT NULL::character varying,
+ "NUM_NULLS" bigint NOT NULL,
+ "NUM_DISTINCTS" bigint,
+ "AVG_COL_LEN" double precision,
+ "MAX_COL_LEN" bigint,
+ "NUM_TRUES" bigint,
+ "NUM_FALSES" bigint,
+ "LAST_ANALYZED" bigint NOT NULL
+);
+
+--
+-- Name: BUCKETING_COLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "BUCKETING_COLS"
+    ADD CONSTRAINT "BUCKETING_COLS_pkey" PRIMARY KEY ("SD_ID", "INTEGER_IDX");
+
+
+--
+-- Name: CDS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "CDS"
+    ADD CONSTRAINT "CDS_pkey" PRIMARY KEY ("CD_ID");
+
+
+--
+-- Name: COLUMNS_V2_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "COLUMNS_V2"
+    ADD CONSTRAINT "COLUMNS_V2_pkey" PRIMARY KEY ("CD_ID", "COLUMN_NAME");
+
+
+--
+-- Name: COLUMNS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "COLUMNS_OLD"
+    ADD CONSTRAINT "COLUMNS_pkey" PRIMARY KEY ("SD_ID", "COLUMN_NAME");
+
+
+--
+-- Name: DATABASE_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DATABASE_PARAMS"
+    ADD CONSTRAINT "DATABASE_PARAMS_pkey" PRIMARY KEY ("DB_ID", "PARAM_KEY");
+
+
+--
+-- Name: DBPRIVILEGEINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DB_PRIVS"
+    ADD CONSTRAINT "DBPRIVILEGEINDEX" UNIQUE ("DB_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DB_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: DBS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DBS"
+    ADD CONSTRAINT "DBS_pkey" PRIMARY KEY ("DB_ID");
+
+
+--
+-- Name: DB_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DB_PRIVS"
+    ADD CONSTRAINT "DB_PRIVS_pkey" PRIMARY KEY ("DB_GRANT_ID");
+
+
+--
+-- Name: GLOBALPRIVILEGEINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "GLOBAL_PRIVS"
+    ADD CONSTRAINT "GLOBALPRIVILEGEINDEX" UNIQUE ("PRINCIPAL_NAME", "PRINCIPAL_TYPE", "USER_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: GLOBAL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "GLOBAL_PRIVS"
+    ADD CONSTRAINT "GLOBAL_PRIVS_pkey" PRIMARY KEY ("USER_GRANT_ID");
+
+
+--
+-- Name: IDXS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_pkey" PRIMARY KEY ("INDEX_ID");
+
+
+--
+-- Name: INDEX_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "INDEX_PARAMS"
+    ADD CONSTRAINT "INDEX_PARAMS_pkey" PRIMARY KEY ("INDEX_ID", "PARAM_KEY");
+
+
+--
+-- Name: NUCLEUS_TABLES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "NUCLEUS_TABLES"
+    ADD CONSTRAINT "NUCLEUS_TABLES_pkey" PRIMARY KEY ("CLASS_NAME");
+
+
+--
+-- Name: PARTITIONS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "PARTITIONS_pkey" PRIMARY KEY ("PART_ID");
+
+
+--
+-- Name: PARTITION_EVENTS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_EVENTS"
+    ADD CONSTRAINT "PARTITION_EVENTS_pkey" PRIMARY KEY ("PART_NAME_ID");
+
+
+--
+-- Name: PARTITION_KEYS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_KEYS"
+    ADD CONSTRAINT "PARTITION_KEYS_pkey" PRIMARY KEY ("TBL_ID", "PKEY_NAME");
+
+
+--
+-- Name: PARTITION_KEY_VALS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_KEY_VALS"
+    ADD CONSTRAINT "PARTITION_KEY_VALS_pkey" PRIMARY KEY ("PART_ID", "INTEGER_IDX");
+
+
+--
+-- Name: PARTITION_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITION_PARAMS"
+    ADD CONSTRAINT "PARTITION_PARAMS_pkey" PRIMARY KEY ("PART_ID", "PARAM_KEY");
+
+
+--
+-- Name: PART_COL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PART_COL_PRIVS"
+    ADD CONSTRAINT "PART_COL_PRIVS_pkey" PRIMARY KEY ("PART_COLUMN_GRANT_ID");
+
+
+--
+-- Name: PART_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PART_PRIVS"
+    ADD CONSTRAINT "PART_PRIVS_pkey" PRIMARY KEY ("PART_GRANT_ID");
+
+
+--
+-- Name: ROLEENTITYINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLES"
+    ADD CONSTRAINT "ROLEENTITYINDEX" UNIQUE ("ROLE_NAME");
+
+
+--
+-- Name: ROLES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLES"
+    ADD CONSTRAINT "ROLES_pkey" PRIMARY KEY ("ROLE_ID");
+
+
+--
+-- Name: ROLE_MAP_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLE_MAP"
+    ADD CONSTRAINT "ROLE_MAP_pkey" PRIMARY KEY ("ROLE_GRANT_ID");
+
+
+--
+-- Name: SDS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SDS"
+    ADD CONSTRAINT "SDS_pkey" PRIMARY KEY ("SD_ID");
+
+
+--
+-- Name: SD_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SD_PARAMS"
+    ADD CONSTRAINT "SD_PARAMS_pkey" PRIMARY KEY ("SD_ID", "PARAM_KEY");
+
+
+--
+-- Name: SEQUENCE_TABLE_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SEQUENCE_TABLE"
+    ADD CONSTRAINT "SEQUENCE_TABLE_pkey" PRIMARY KEY ("SEQUENCE_NAME");
+
+
+--
+-- Name: SERDES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SERDES"
+    ADD CONSTRAINT "SERDES_pkey" PRIMARY KEY ("SERDE_ID");
+
+
+--
+-- Name: SERDE_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SERDE_PARAMS"
+    ADD CONSTRAINT "SERDE_PARAMS_pkey" PRIMARY KEY ("SERDE_ID", "PARAM_KEY");
+
+
+--
+-- Name: SORT_COLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "SORT_COLS"
+    ADD CONSTRAINT "SORT_COLS_pkey" PRIMARY KEY ("SD_ID", "INTEGER_IDX");
+
+
+--
+-- Name: TABLE_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TABLE_PARAMS"
+    ADD CONSTRAINT "TABLE_PARAMS_pkey" PRIMARY KEY ("TBL_ID", "PARAM_KEY");
+
+
+--
+-- Name: TBLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "TBLS_pkey" PRIMARY KEY ("TBL_ID");
+
+
+--
+-- Name: TBL_COL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBL_COL_PRIVS"
+    ADD CONSTRAINT "TBL_COL_PRIVS_pkey" PRIMARY KEY ("TBL_COLUMN_GRANT_ID");
+
+
+--
+-- Name: TBL_PRIVS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBL_PRIVS"
+    ADD CONSTRAINT "TBL_PRIVS_pkey" PRIMARY KEY ("TBL_GRANT_ID");
+
+
+--
+-- Name: TYPES_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TYPES"
+    ADD CONSTRAINT "TYPES_pkey" PRIMARY KEY ("TYPES_ID");
+
+
+--
+-- Name: TYPE_FIELDS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TYPE_FIELDS"
+    ADD CONSTRAINT "TYPE_FIELDS_pkey" PRIMARY KEY ("TYPE_NAME", "FIELD_NAME");
+
+ALTER TABLE ONLY "SKEWED_STRING_LIST"
+    ADD CONSTRAINT "SKEWED_STRING_LIST_pkey" PRIMARY KEY ("STRING_LIST_ID");
+
+ALTER TABLE ONLY "SKEWED_STRING_LIST_VALUES"
+    ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_pkey" PRIMARY KEY ("STRING_LIST_ID", "INTEGER_IDX");
+
+
+ALTER TABLE ONLY "SKEWED_COL_NAMES"
+    ADD CONSTRAINT "SKEWED_COL_NAMES_pkey" PRIMARY KEY ("SD_ID", "INTEGER_IDX");
+
+ALTER TABLE ONLY "SKEWED_COL_VALUE_LOC_MAP"
+    ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_pkey" PRIMARY KEY ("SD_ID", "STRING_LIST_ID_KID");
+
+ALTER TABLE ONLY "SKEWED_VALUES"
+    ADD CONSTRAINT "SKEWED_VALUES_pkey" PRIMARY KEY ("SD_ID_OID", "INTEGER_IDX");
+
+--
+-- Name: TAB_COL_STATS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+ALTER TABLE ONLY "TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_pkey" PRIMARY KEY("CS_ID");
+
+--
+-- Name: PART_COL_STATS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+ALTER TABLE ONLY "PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_pkey" PRIMARY KEY("CS_ID");
+
+--
+-- Name: UNIQUEINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "UNIQUEINDEX" UNIQUE ("INDEX_NAME", "ORIG_TBL_ID");
+
+
+--
+-- Name: UNIQUEPARTITION; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "UNIQUEPARTITION" UNIQUE ("PART_NAME", "TBL_ID");
+
+
+--
+-- Name: UNIQUETABLE; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "UNIQUETABLE" UNIQUE ("TBL_NAME", "DB_ID");
+
+
+--
+-- Name: UNIQUE_DATABASE; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "DBS"
+    ADD CONSTRAINT "UNIQUE_DATABASE" UNIQUE ("NAME");
+
+
+--
+-- Name: UNIQUE_TYPE; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "TYPES"
+    ADD CONSTRAINT "UNIQUE_TYPE" UNIQUE ("TYPE_NAME");
+
+
+--
+-- Name: USERROLEMAPINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+ALTER TABLE ONLY "ROLE_MAP"
+    ADD CONSTRAINT "USERROLEMAPINDEX" UNIQUE ("PRINCIPAL_NAME", "ROLE_ID", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: BUCKETING_COLS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "BUCKETING_COLS_N49" ON "BUCKETING_COLS" USING btree ("SD_ID");
+
+
+--
+-- Name: COLUMNS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "COLUMNS_N49" ON "COLUMNS_OLD" USING btree ("SD_ID");
+
+
+--
+-- Name: DATABASE_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "DATABASE_PARAMS_N49" ON "DATABASE_PARAMS" USING btree ("DB_ID");
+
+
+--
+-- Name: DB_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "DB_PRIVS_N49" ON "DB_PRIVS" USING btree ("DB_ID");
+
+
+--
+-- Name: IDXS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "IDXS_N49" ON "IDXS" USING btree ("ORIG_TBL_ID");
+
+
+--
+-- Name: IDXS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "IDXS_N50" ON "IDXS" USING btree ("INDEX_TBL_ID");
+
+
+--
+-- Name: IDXS_N51; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "IDXS_N51" ON "IDXS" USING btree ("SD_ID");
+
+
+--
+-- Name: INDEX_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "INDEX_PARAMS_N49" ON "INDEX_PARAMS" USING btree ("INDEX_ID");
+
+
+--
+-- Name: PARTITIONCOLUMNPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONCOLUMNPRIVILEGEINDEX" ON "PART_COL_PRIVS" USING btree ("PART_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_COL_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: PARTITIONEVENTINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONEVENTINDEX" ON "PARTITION_EVENTS" USING btree ("PARTITION_NAME");
+
+
+--
+-- Name: PARTITIONS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONS_N49" ON "PARTITIONS" USING btree ("TBL_ID");
+
+
+--
+-- Name: PARTITIONS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITIONS_N50" ON "PARTITIONS" USING btree ("SD_ID");
+
+
+--
+-- Name: PARTITION_KEYS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITION_KEYS_N49" ON "PARTITION_KEYS" USING btree ("TBL_ID");
+
+
+--
+-- Name: PARTITION_KEY_VALS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITION_KEY_VALS_N49" ON "PARTITION_KEY_VALS" USING btree ("PART_ID");
+
+
+--
+-- Name: PARTITION_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTITION_PARAMS_N49" ON "PARTITION_PARAMS" USING btree ("PART_ID");
+
+
+--
+-- Name: PARTPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PARTPRIVILEGEINDEX" ON "PART_PRIVS" USING btree ("PART_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: PART_COL_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PART_COL_PRIVS_N49" ON "PART_COL_PRIVS" USING btree ("PART_ID");
+
+
+--
+-- Name: PART_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PART_PRIVS_N49" ON "PART_PRIVS" USING btree ("PART_ID");
+
+
+--
+-- Name: ROLE_MAP_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "ROLE_MAP_N49" ON "ROLE_MAP" USING btree ("ROLE_ID");
+
+
+--
+-- Name: SDS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SDS_N49" ON "SDS" USING btree ("SERDE_ID");
+
+
+--
+-- Name: SD_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SD_PARAMS_N49" ON "SD_PARAMS" USING btree ("SD_ID");
+
+
+--
+-- Name: SERDE_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SERDE_PARAMS_N49" ON "SERDE_PARAMS" USING btree ("SERDE_ID");
+
+
+--
+-- Name: SORT_COLS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "SORT_COLS_N49" ON "SORT_COLS" USING btree ("SD_ID");
+
+
+--
+-- Name: TABLECOLUMNPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TABLECOLUMNPRIVILEGEINDEX" ON "TBL_COL_PRIVS" USING btree ("TBL_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_COL_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: TABLEPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TABLEPRIVILEGEINDEX" ON "TBL_PRIVS" USING btree ("TBL_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_PRIV", "GRANTOR", "GRANTOR_TYPE");
+
+
+--
+-- Name: TABLE_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TABLE_PARAMS_N49" ON "TABLE_PARAMS" USING btree ("TBL_ID");
+
+
+--
+-- Name: TBLS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBLS_N49" ON "TBLS" USING btree ("DB_ID");
+
+
+--
+-- Name: TBLS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBLS_N50" ON "TBLS" USING btree ("SD_ID");
+
+
+--
+-- Name: TBL_COL_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBL_COL_PRIVS_N49" ON "TBL_COL_PRIVS" USING btree ("TBL_ID");
+
+
+--
+-- Name: TBL_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TBL_PRIVS_N49" ON "TBL_PRIVS" USING btree ("TBL_ID");
+
+
+--
+-- Name: TYPE_FIELDS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TYPE_FIELDS_N49" ON "TYPE_FIELDS" USING btree ("TYPE_NAME");
+
+--
+-- Name: TAB_COL_STATS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TAB_COL_STATS_N49" ON "TAB_COL_STATS" USING btree ("TBL_ID");
+
+--
+-- Name: PART_COL_STATS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "PART_COL_STATS_N49" ON "PART_COL_STATS" USING btree ("PART_ID");
+
+
+ALTER TABLE ONLY "SKEWED_STRING_LIST_VALUES"
+    ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_fkey" FOREIGN KEY ("STRING_LIST_ID") REFERENCES "SKEWED_STRING_LIST"("STRING_LIST_ID") DEFERRABLE;
+
+
+ALTER TABLE ONLY "SKEWED_COL_NAMES"
+    ADD CONSTRAINT "SKEWED_COL_NAMES_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+ALTER TABLE ONLY "SKEWED_COL_VALUE_LOC_MAP"
+    ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_fkey1" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+ALTER TABLE ONLY "SKEWED_COL_VALUE_LOC_MAP"
+    ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_fkey2" FOREIGN KEY ("STRING_LIST_ID_KID") REFERENCES "SKEWED_STRING_LIST"("STRING_LIST_ID") DEFERRABLE;
+
+ALTER TABLE ONLY "SKEWED_VALUES"
+    ADD CONSTRAINT "SKEWED_VALUES_fkey1" FOREIGN KEY ("STRING_LIST_ID_EID") REFERENCES "SKEWED_STRING_LIST"("STRING_LIST_ID") DEFERRABLE;
+
+ALTER TABLE ONLY "SKEWED_VALUES"
+    ADD CONSTRAINT "SKEWED_VALUES_fkey2" FOREIGN KEY ("SD_ID_OID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: BUCKETING_COLS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "BUCKETING_COLS"
+    ADD CONSTRAINT "BUCKETING_COLS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: COLUMNS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "COLUMNS_OLD"
+    ADD CONSTRAINT "COLUMNS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: COLUMNS_V2_CD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "COLUMNS_V2"
+    ADD CONSTRAINT "COLUMNS_V2_CD_ID_fkey" FOREIGN KEY ("CD_ID") REFERENCES "CDS"("CD_ID") DEFERRABLE;
+
+
+--
+-- Name: DATABASE_PARAMS_DB_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "DATABASE_PARAMS"
+    ADD CONSTRAINT "DATABASE_PARAMS_DB_ID_fkey" FOREIGN KEY ("DB_ID") REFERENCES "DBS"("DB_ID") DEFERRABLE;
+
+
+--
+-- Name: DB_PRIVS_DB_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "DB_PRIVS"
+    ADD CONSTRAINT "DB_PRIVS_DB_ID_fkey" FOREIGN KEY ("DB_ID") REFERENCES "DBS"("DB_ID") DEFERRABLE;
+
+
+--
+-- Name: IDXS_INDEX_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_INDEX_TBL_ID_fkey" FOREIGN KEY ("INDEX_TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: IDXS_ORIG_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_ORIG_TBL_ID_fkey" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: IDXS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "IDXS"
+    ADD CONSTRAINT "IDXS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: INDEX_PARAMS_INDEX_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "INDEX_PARAMS"
+    ADD CONSTRAINT "INDEX_PARAMS_INDEX_ID_fkey" FOREIGN KEY ("INDEX_ID") REFERENCES "IDXS"("INDEX_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITIONS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "PARTITIONS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITIONS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITIONS"
+    ADD CONSTRAINT "PARTITIONS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITION_KEYS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITION_KEYS"
+    ADD CONSTRAINT "PARTITION_KEYS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITION_KEY_VALS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITION_KEY_VALS"
+    ADD CONSTRAINT "PARTITION_KEY_VALS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: PARTITION_PARAMS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PARTITION_PARAMS"
+    ADD CONSTRAINT "PARTITION_PARAMS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: PART_COL_PRIVS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PART_COL_PRIVS"
+    ADD CONSTRAINT "PART_COL_PRIVS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: PART_PRIVS_PART_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "PART_PRIVS"
+    ADD CONSTRAINT "PART_PRIVS_PART_ID_fkey" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+--
+-- Name: ROLE_MAP_ROLE_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "ROLE_MAP"
+    ADD CONSTRAINT "ROLE_MAP_ROLE_ID_fkey" FOREIGN KEY ("ROLE_ID") REFERENCES "ROLES"("ROLE_ID") DEFERRABLE;
+
+
+--
+-- Name: SDS_CD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SDS"
+    ADD CONSTRAINT "SDS_CD_ID_fkey" FOREIGN KEY ("CD_ID") REFERENCES "CDS"("CD_ID") DEFERRABLE;
+
+
+--
+-- Name: SDS_SERDE_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SDS"
+    ADD CONSTRAINT "SDS_SERDE_ID_fkey" FOREIGN KEY ("SERDE_ID") REFERENCES "SERDES"("SERDE_ID") DEFERRABLE;
+
+
+--
+-- Name: SD_PARAMS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SD_PARAMS"
+    ADD CONSTRAINT "SD_PARAMS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: SERDE_PARAMS_SERDE_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SERDE_PARAMS"
+    ADD CONSTRAINT "SERDE_PARAMS_SERDE_ID_fkey" FOREIGN KEY ("SERDE_ID") REFERENCES "SERDES"("SERDE_ID") DEFERRABLE;
+
+
+--
+-- Name: SORT_COLS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "SORT_COLS"
+    ADD CONSTRAINT "SORT_COLS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: TABLE_PARAMS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TABLE_PARAMS"
+    ADD CONSTRAINT "TABLE_PARAMS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: TBLS_DB_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "TBLS_DB_ID_fkey" FOREIGN KEY ("DB_ID") REFERENCES "DBS"("DB_ID") DEFERRABLE;
+
+
+--
+-- Name: TBLS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBLS"
+    ADD CONSTRAINT "TBLS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE;
+
+
+--
+-- Name: TBL_COL_PRIVS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBL_COL_PRIVS"
+    ADD CONSTRAINT "TBL_COL_PRIVS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: TBL_PRIVS_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TBL_PRIVS"
+    ADD CONSTRAINT "TBL_PRIVS_TBL_ID_fkey" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: TYPE_FIELDS_TYPE_NAME_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+
+ALTER TABLE ONLY "TYPE_FIELDS"
+    ADD CONSTRAINT "TYPE_FIELDS_TYPE_NAME_fkey" FOREIGN KEY ("TYPE_NAME") REFERENCES "TYPES"("TYPES_ID") DEFERRABLE;
+
+--
+-- Name: TAB_COL_STATS_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+ALTER TABLE ONLY "TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_fkey" FOREIGN KEY("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+
+
+--
+-- Name: PART_COL_STATS_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser
+--
+ALTER TABLE ONLY "PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_fkey" FOREIGN KEY("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
+
+
+ALTER TABLE ONLY "VERSION" ADD CONSTRAINT "VERSION_pkey" PRIMARY KEY ("VER_ID");
+
+--
+-- Name: public; Type: ACL; Schema: -; Owner: hiveuser
+--
+
+REVOKE ALL ON SCHEMA public FROM PUBLIC;
+GRANT ALL ON SCHEMA public TO PUBLIC;
+
+
+INSERT INTO "VERSION" ("VER_ID", "SCHEMA_VERSION", "VERSION_COMMENT") VALUES (1, '0.12.0', 'Hive release version 0.12.0');
+--
+-- PostgreSQL database dump complete
+--
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/addMysqlUser.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/addMysqlUser.sh b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/addMysqlUser.sh
new file mode 100755
index 0000000..862e9b2
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/addMysqlUser.sh
@@ -0,0 +1,39 @@
+#!/usr/bin/env bash
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#
+
+mysqldservice=$1
+mysqldbuser=$2
+mysqldbpasswd=$3
+userhost=$4
+
+# The restart (not start) is required to pick up mysql configuration changes made by sed
+# during install, in case mysql is already started. The changes are required by Hive later on.
+/var/lib/ambari-agent/ambari-sudo.sh service $mysqldservice restart
+
+# MySQL 5.7 installed in non-interactive way uses a socket authentication plugin.
+# "mysql -u root" should be executed from root user
+echo "Adding user $mysqldbuser@% and removing users with empty name"
+/var/lib/ambari-agent/ambari-sudo.sh mysql -u root -e "CREATE USER '$mysqldbuser'@'%' IDENTIFIED BY '$mysqldbpasswd';"
+/var/lib/ambari-agent/ambari-sudo.sh mysql -u root -e "GRANT ALL PRIVILEGES ON *.* TO '$mysqldbuser'@'%';"
+/var/lib/ambari-agent/ambari-sudo.sh mysql -u root -e "DELETE FROM mysql.user WHERE user='';"
+/var/lib/ambari-agent/ambari-sudo.sh mysql -u root -e "flush privileges;"
+/var/lib/ambari-agent/ambari-sudo.sh service $mysqldservice stop

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hcatSmoke.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hcatSmoke.sh b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hcatSmoke.sh
new file mode 100755
index 0000000..39e63a6
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hcatSmoke.sh
@@ -0,0 +1,41 @@
+#!/usr/bin/env bash
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#
+export tablename=$1
+
+export purge_cmd=""
+if [ "$3" == "true" ]; then
+	export purge_cmd="purge"
+fi
+
+case "$2" in
+
+prepare)
+  hcat -e "show tables"
+  hcat -e "drop table IF EXISTS ${tablename} ${purge_cmd}"
+  hcat -e "create table ${tablename} ( id INT, name string ) stored as rcfile ;"
+;;
+
+cleanup)
+  hcat -e "drop table IF EXISTS ${tablename} ${purge_cmd}"
+;;
+
+esac

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveSmoke.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveSmoke.sh b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveSmoke.sh
new file mode 100755
index 0000000..f9f2020
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveSmoke.sh
@@ -0,0 +1,24 @@
+#!/usr/bin/env bash
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#
+export tablename=$1
+echo "CREATE EXTERNAL TABLE IF NOT EXISTS ${tablename} ( foo INT, bar STRING );" | hive
+echo "DESCRIBE ${tablename};" | hive

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveTezSetup.cmd
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveTezSetup.cmd b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveTezSetup.cmd
new file mode 100755
index 0000000..10d6a1c
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveTezSetup.cmd
@@ -0,0 +1,58 @@
+@echo off
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem     http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+
+if not defined HADOOP_HOME (
+  set EXITCODE=5
+  goto :errorexit
+)
+if not defined HIVE_HOME (
+  set EXITCODE=6
+  goto :errorexit
+)
+if not defined TEZ_HOME (
+  set EXITCODE=7
+  goto :errorexit
+)
+
+set EXITCODE=0
+
+if not exist %HIVE_HOME%\conf\hive-tez-configured (
+  %HADOOP_HOME%\bin\hadoop.cmd fs -mkdir /apps/tez
+  set EXITCODE=%ERRORLEVEL%
+  if %EXITCODE% neq 0 goto :errorexit
+
+  %HADOOP_HOME%\bin\hadoop.cmd fs -chmod -R 755 /apps/tez
+  set EXITCODE=%ERRORLEVEL%
+  if %EXITCODE% neq 0 goto :errorexit
+
+  %HADOOP_HOME%\bin\hadoop.cmd fs -chown -R hadoop:users /apps/tez
+  set EXITCODE=%ERRORLEVEL%
+  if %EXITCODE% neq 0 goto :errorexit
+
+  %HADOOP_HOME%\bin\hadoop.cmd fs -put %TEZ_HOME%\* /apps/tez
+  set EXITCODE=%ERRORLEVEL%
+  if %EXITCODE% neq 0 goto :errorexit
+
+  %HADOOP_HOME%\bin\hadoop.cmd fs -rm -r -skipTrash /apps/tez/conf
+  set EXITCODE=%ERRORLEVEL%
+  if %EXITCODE% neq 0 goto :errorexit
+
+  echo done > %HIVE_HOME%\conf\hive-tez-configured
+)
+goto :eof
+
+:errorexit
+exit /B %EXITCODE%

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveserver2.sql
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveserver2.sql b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveserver2.sql
new file mode 100755
index 0000000..99a3865
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveserver2.sql
@@ -0,0 +1,23 @@
+#
+#
+# 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.
+#
+#
+
+CREATE EXTERNAL TABLE IF NOT EXISTS hiveserver2smoke20408 ( foo INT, bar STRING );
+DESCRIBE hiveserver2smoke20408;

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveserver2Smoke.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveserver2Smoke.sh b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveserver2Smoke.sh
new file mode 100755
index 0000000..77d7b3e
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/hiveserver2Smoke.sh
@@ -0,0 +1,32 @@
+#!/usr/bin/env bash
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#
+
+smokeout=`/usr/lib/hive/bin/beeline -u $1 -n fakeuser -p fakepwd -d org.apache.hive.jdbc.HiveDriver -e '!run $2' 2>&1| awk '{print}'|grep Error`
+
+if [ "x$smokeout" == "x" ]; then
+  echo "Smoke test of hiveserver2 passed"
+  exit 0
+else
+  echo "Smoke test of hiveserver2 wasnt passed"
+  echo $smokeout
+  exit 1
+fi

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/pigSmoke.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/pigSmoke.sh b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/pigSmoke.sh
new file mode 100755
index 0000000..2e90ac0
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/pigSmoke.sh
@@ -0,0 +1,18 @@
+# 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
+
+A = load 'passwd' using PigStorage(':');
+B = foreach A generate \$0 as id;
+store B into 'pigsmoke.out';

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/removeMysqlUser.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/removeMysqlUser.sh b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/removeMysqlUser.sh
new file mode 100755
index 0000000..7b6d331
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/removeMysqlUser.sh
@@ -0,0 +1,33 @@
+#!/usr/bin/env bash
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#
+
+mysqldservice=$1
+mysqldbuser=$2
+userhost=$3
+myhostname=$(hostname -f)
+sudo_prefix = "/var/lib/ambari-agent/ambari-sudo.sh -H -E"
+
+$sudo_prefix service $mysqldservice start
+echo "Removing user $mysqldbuser@$userhost"
+/var/lib/ambari-agent/ambari-sudo.sh su mysql -s /bin/bash - -c "mysql -u root -e \"DROP USER '$mysqldbuser'@'$userhost';\""
+/var/lib/ambari-agent/ambari-sudo.sh su mysql -s /bin/bash - -c "mysql -u root -e \"flush privileges;\""
+$sudo_prefix service $mysqldservice stop

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/startMetastore.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/startMetastore.sh b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/startMetastore.sh
new file mode 100755
index 0000000..86541f0
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/startMetastore.sh
@@ -0,0 +1,25 @@
+#!/usr/bin/env bash
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#
+HIVE_BIN=${HIVE_BIN:-"hive"}
+
+HIVE_CONF_DIR=$4 $HIVE_BIN --service metastore -hiveconf hive.log.file=hivemetastore.log -hiveconf hive.log.dir=$5 > $1 2> $2 &
+echo $!|cat>$3

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/templetonSmoke.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/templetonSmoke.sh b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/templetonSmoke.sh
new file mode 100755
index 0000000..dd61631
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/files/templetonSmoke.sh
@@ -0,0 +1,95 @@
+#!/usr/bin/env bash
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#
+
+export ttonhost=$1
+export smoke_test_user=$2
+export templeton_port=$3
+export ttonTestScript=$4
+export smoke_user_keytab=$5
+export security_enabled=$6
+export kinit_path_local=$7
+export smokeuser_principal=$8
+export tmp_dir=$9
+export ttonurl="http://${ttonhost}:${templeton_port}/templeton/v1"
+
+if [[ $security_enabled == "true" ]]; then
+  kinitcmd="${kinit_path_local}  -kt ${smoke_user_keytab} ${smokeuser_principal}; "
+else
+  kinitcmd=""
+fi
+
+export no_proxy=$ttonhost
+cmd="${kinitcmd}curl --negotiate -u : -s -w 'http_code <%{http_code}>'  $ttonurl/status 2>&1"
+retVal=`/var/lib/ambari-agent/ambari-sudo.sh su ${smoke_test_user} -s /bin/bash - -c "$cmd"`
+httpExitCode=`echo $retVal |sed 's/.*http_code <\([0-9]*\)>.*/\1/'`
+
+# try again for 2.3 username requirement
+if [[ "$httpExitCode" == "500" ]] ; then
+  cmd="${kinitcmd}curl --negotiate -u : -s -w 'http_code <%{http_code}>'  $ttonurl/status?user.name=$smoke_test_user 2>&1"
+  retVal=`/var/lib/ambari-agent/ambari-sudo.sh su ${smoke_test_user} -s /bin/bash - -c "$cmd"`
+  httpExitCode=`echo $retVal |sed 's/.*http_code <\([0-9]*\)>.*/\1/'`
+fi
+
+if [[ "$httpExitCode" -ne "200" ]] ; then
+  echo "Templeton Smoke Test (status cmd): Failed. : $retVal"
+  export TEMPLETON_EXIT_CODE=1
+  exit 1
+fi
+
+#try hcat ddl command
+/var/lib/ambari-agent/ambari-sudo.sh rm -f ${tmp_dir}/show_db.post.txt
+echo "user.name=${smoke_test_user}&exec=show databases;" > ${tmp_dir}/show_db.post.txt
+/var/lib/ambari-agent/ambari-sudo.sh chown ${smoke_test_user} ${tmp_dir}/show_db.post.txt
+cmd="${kinitcmd}curl --negotiate -u : -s -w 'http_code <%{http_code}>' -d  @${tmp_dir}/show_db.post.txt  $ttonurl/ddl 2>&1"
+retVal=`/var/lib/ambari-agent/ambari-sudo.sh su ${smoke_test_user} -s /bin/bash - -c "$cmd"`
+httpExitCode=`echo $retVal |sed 's/.*http_code <\([0-9]*\)>.*/\1/'`
+
+if [[ "$httpExitCode" -ne "200" ]] ; then
+  echo "Templeton Smoke Test (ddl cmd): Failed. : $retVal"
+  export TEMPLETON_EXIT_CODE=1
+  exit  1
+fi
+
+# NOT SURE?? SUHAS
+if [[ $security_enabled == "true" ]]; then
+  echo "Templeton Pig Smoke Tests not run in secure mode"
+  exit 0
+fi
+
+#try pig query
+
+#create, copy post args file
+/var/lib/ambari-agent/ambari-sudo.sh rm -f ${tmp_dir}/pig_post.txt
+echo -n "user.name=${smoke_test_user}&file=/tmp/$ttonTestScript" > ${tmp_dir}/pig_post.txt
+/var/lib/ambari-agent/ambari-sudo.sh chown ${smoke_test_user} ${tmp_dir}/pig_post.txt
+
+#submit pig query
+cmd="curl --negotiate -u : -s -w 'http_code <%{http_code}>' -d  @${tmp_dir}/pig_post.txt  $ttonurl/pig 2>&1"
+retVal=`/var/lib/ambari-agent/ambari-sudo.sh su ${smoke_test_user} -s /bin/bash - -c "$cmd"`
+httpExitCode=`echo $retVal |sed 's/.*http_code <\([0-9]*\)>.*/\1/'`
+if [[ "$httpExitCode" -ne "200" ]] ; then
+  echo "Templeton Smoke Test (pig cmd): Failed. : $retVal"
+  export TEMPLETON_EXIT_CODE=1
+  exit 1
+fi
+
+exit 0

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/__init__.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/__init__.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/__init__.py
new file mode 100755
index 0000000..5561e10
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/__init__.py
@@ -0,0 +1,19 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat.py
new file mode 100755
index 0000000..5e2c709
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat.py
@@ -0,0 +1,81 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+import os
+
+from resource_management import *
+import sys
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+from resource_management.libraries.functions.setup_atlas_hook import has_atlas_in_cluster, setup_atlas_hook
+from ambari_commons.constants import SERVICE
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def hcat():
+  import params
+
+  XmlConfig("hive-site.xml",
+            conf_dir = params.hive_conf_dir,
+            configurations = params.config['configurations']['hive-site'],
+            owner=params.hive_user,
+            configuration_attributes=params.config['configuration_attributes']['hive-site']
+  )
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def hcat():
+  import params
+
+  Directory(params.hive_conf_dir,
+            create_parents = True,
+            owner=params.hcat_user,
+            group=params.user_group,
+  )
+
+
+  Directory(params.hcat_conf_dir,
+            create_parents = True,
+            owner=params.hcat_user,
+            group=params.user_group,
+  )
+
+  Directory(params.hcat_pid_dir,
+            owner=params.webhcat_user,
+            create_parents = True
+  )
+
+  XmlConfig("hive-site.xml",
+            conf_dir=params.hive_client_conf_dir,
+            configurations=params.config['configurations']['hive-site'],
+            configuration_attributes=params.config['configuration_attributes']['hive-site'],
+            owner=params.hive_user,
+            group=params.user_group,
+            mode=0644)
+
+  File(format("{hcat_conf_dir}/hcat-env.sh"),
+       owner=params.hcat_user,
+       group=params.user_group,
+       content=InlineTemplate(params.hcat_env_sh_template)
+  )
+
+  # Generate atlas-application.properties.xml file
+  if has_atlas_in_cluster():
+    atlas_hook_filepath = os.path.join(params.hive_config_dir, params.atlas_hook_filename)
+    setup_atlas_hook(SERVICE.HIVE, params.hive_atlas_application_properties, atlas_hook_filepath, params.hive_user, params.user_group)

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat_client.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat_client.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat_client.py
new file mode 100755
index 0000000..b37698e
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat_client.py
@@ -0,0 +1,85 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+from hcat import hcat
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
+from resource_management.core.logger import Logger
+from resource_management.core.exceptions import ClientComponentHasNoStatus
+from resource_management.libraries.functions import stack_select
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.script.script import Script
+
+
+class HCatClient(Script):
+  def install(self, env):
+    import params
+    self.install_packages(env)
+    self.configure(env)
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    hcat()
+
+  def status(self, env):
+    raise ClientComponentHasNoStatus()
+
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class HCatClientWindows(HCatClient):
+  pass
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class HCatClientDefault(HCatClient):
+  def get_component_name(self):
+    # HCat client doesn't have a first-class entry in <stack-selector-tool>. Since clients always
+    # update after daemons, this ensures that the hcat directories are correct on hosts
+    # which do not include the WebHCat daemon
+    return "hive-webhcat"
+
+
+  def pre_upgrade_restart(self, env, upgrade_type=None):
+    """
+    Execute <stack-selector-tool> before reconfiguring this client to the new stack version.
+
+    :param env:
+    :param upgrade_type:
+    :return:
+    """
+    Logger.info("Executing Hive HCat Client Stack Upgrade pre-restart")
+
+    import params
+    env.set_params(params)
+
+    # this function should not execute if the stack version does not support rolling upgrade
+    if not (params.version and check_stack_feature(StackFeature.ROLLING_UPGRADE, params.version)):
+      return
+
+    # HCat client doesn't have a first-class entry in <stack-selector-tool>. Since clients always
+    # update after daemons, this ensures that the hcat directories are correct on hosts
+    # which do not include the WebHCat daemon
+    stack_select.select("hive-webhcat", params.version)
+
+
+if __name__ == "__main__":
+  HCatClient().execute()

http://git-wip-us.apache.org/repos/asf/bigtop/blob/4522c959/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat_service_check.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat_service_check.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat_service_check.py
new file mode 100755
index 0000000..07b4095
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hcat_service_check.py
@@ -0,0 +1,86 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import os
+from resource_management import *
+from resource_management.libraries.functions import get_unique_id_and_date
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def hcat_service_check():
+  import params
+  smoke_cmd = os.path.join(params.stack_root, "Run-SmokeTests.cmd")
+  service = "HCatalog"
+  Execute(format("cmd /C {smoke_cmd} {service}"), user=params.hcat_user, logoutput=True)
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def hcat_service_check():
+    import params
+    unique = get_unique_id_and_date()
+    output_file = format("{hive_apps_whs_dir}/hcatsmoke{unique}")
+    test_cmd = format("fs -test -e {output_file}")
+
+    if params.security_enabled:
+      kinit_cmd = format(
+        "{kinit_path_local} -kt {smoke_user_keytab} {smokeuser_principal}; ")
+    else:
+      kinit_cmd = ""
+
+    File(format("{tmp_dir}/hcatSmoke.sh"),
+         content=StaticFile("hcatSmoke.sh"),
+         mode=0755
+    )
+
+    prepare_cmd = format("{kinit_cmd}env JAVA_HOME={java64_home} {tmp_dir}/hcatSmoke.sh hcatsmoke{unique} prepare {purge_tables}")
+
+    exec_path = params.execute_path
+    if params.version and params.stack_root:
+      upgrade_hive_bin = format("{stack_root}/{version}/hive/bin")
+      exec_path =  os.environ['PATH'] + os.pathsep + params.hadoop_bin_dir + os.pathsep + upgrade_hive_bin
+
+    Execute(prepare_cmd,
+            tries=3,
+            user=params.smokeuser,
+            try_sleep=5,
+            path=['/usr/sbin', '/usr/local/bin', '/bin', '/usr/bin', exec_path],
+            logoutput=True)
+
+    if params.security_enabled:
+      Execute (format("{kinit_path_local} -kt {hdfs_user_keytab} {hdfs_principal_name}"),
+               user = params.hdfs_user,
+      )
+
+    ExecuteHadoop(test_cmd,
+                  user=params.hdfs_user,
+                  logoutput=True,
+                  conf_dir=params.hadoop_conf_dir,
+                  bin_dir=params.execute_path
+    )
+
+    cleanup_cmd = format("{kinit_cmd} {tmp_dir}/hcatSmoke.sh hcatsmoke{unique} cleanup {purge_tables}")
+
+    Execute(cleanup_cmd,
+            tries=3,
+            user=params.smokeuser,
+            try_sleep=5,
+            path=['/usr/sbin', '/usr/local/bin', '/bin', '/usr/bin', exec_path],
+            logoutput=True)