You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sp...@apache.org on 2016/05/06 20:42:32 UTC

[01/50] [abbrv] hive git commit: HIVE-4924: JDBC: Support query timeout for jdbc (Vaibhav Gumashta reviewed by Thejas Nair) [Forced Update!]

Repository: hive
Updated Branches:
  refs/heads/java8 2a455f4ae -> 2695a6356 (forced update)


HIVE-4924: JDBC: Support query timeout for jdbc (Vaibhav Gumashta reviewed by Thejas Nair)


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

Branch: refs/heads/java8
Commit: b6218275b00b64aed7efaf470784cc0441464f67
Parents: 0a5bc94
Author: Vaibhav Gumashta <vg...@hortonworks.com>
Authored: Tue May 3 12:49:22 2016 -0700
Committer: Vaibhav Gumashta <vg...@hortonworks.com>
Committed: Tue May 3 12:49:22 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |  43 +++++++-
 .../cli/session/TestHiveSessionImpl.java        |   2 +-
 .../org/apache/hive/jdbc/HiveStatement.java     |  20 ++--
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  27 +++--
 .../hadoop/hive/ql/history/HiveHistoryImpl.java |   8 +-
 .../hadoop/hive/ql/session/OperationLog.java    |   8 +-
 service-rpc/if/TCLIService.thrift               |   6 +
 .../gen/thrift/gen-cpp/TCLIService_types.cpp    |  30 ++++-
 .../src/gen/thrift/gen-cpp/TCLIService_types.h  |  15 ++-
 .../rpc/thrift/TExecuteStatementReq.java        | 109 ++++++++++++++++++-
 .../service/rpc/thrift/TOperationState.java     |   5 +-
 service-rpc/src/gen/thrift/gen-php/Types.php    |  25 +++++
 .../src/gen/thrift/gen-py/TCLIService/ttypes.py |  18 ++-
 .../gen/thrift/gen-rb/t_c_l_i_service_types.rb  |   9 +-
 .../org/apache/hive/service/cli/CLIService.java |  46 ++++++--
 .../service/cli/EmbeddedCLIServiceClient.java   |  19 ++--
 .../apache/hive/service/cli/ICLIService.java    |  16 +--
 .../apache/hive/service/cli/OperationState.java |   7 +-
 .../operation/ExecuteStatementOperation.java    |   9 +-
 .../cli/operation/HiveCommandOperation.java     |   5 +
 .../cli/operation/MetadataOperation.java        |   7 +-
 .../hive/service/cli/operation/Operation.java   |  17 +--
 .../service/cli/operation/OperationManager.java |  27 +++--
 .../service/cli/operation/SQLOperation.java     | 106 +++++++++++++-----
 .../hive/service/cli/session/HiveSession.java   |  28 ++++-
 .../service/cli/session/HiveSessionImpl.java    |  38 ++++---
 .../thrift/RetryingThriftCLIServiceClient.java  |  22 +++-
 .../service/cli/thrift/ThriftCLIService.java    |  16 +--
 .../cli/thrift/ThriftCLIServiceClient.java      |  32 +++---
 .../cli/thrift/ThriftCLIServiceTest.java        |   6 +-
 .../thrift/ThriftCliServiceTestWithCookie.java  |   3 +-
 31 files changed, 557 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
index 73bc620..7243648 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
@@ -55,6 +55,7 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
+import java.sql.SQLTimeoutException;
 import java.sql.SQLWarning;
 import java.sql.Statement;
 import java.sql.Timestamp;
@@ -2384,7 +2385,7 @@ public void testParseUrlHttpMode() throws SQLException, JdbcUriParseException,
         try {
           System.out.println("Executing query: ");
           stmt.executeQuery("select sleepUDF(t1.under_col) as u0, t1.under_col as u1, " +
-              "t2.under_col as u2 from " + tableName +  "t1 join " + tableName +
+              "t2.under_col as u2 from " + tableName +  " t1 join " + tableName +
               " t2 on t1.under_col = t2.under_col");
           fail("Expecting SQLException");
         } catch (SQLException e) {
@@ -2399,7 +2400,7 @@ public void testParseUrlHttpMode() throws SQLException, JdbcUriParseException,
       @Override
       public void run() {
         try {
-          Thread.sleep(1000);
+          Thread.sleep(10000);
           System.out.println("Cancelling query: ");
           stmt.cancel();
         } catch (Exception e) {
@@ -2414,6 +2415,44 @@ public void testParseUrlHttpMode() throws SQLException, JdbcUriParseException,
     stmt.close();
   }
 
+  @Test
+  public void testQueryTimeout() throws Exception {
+    String udfName = SleepUDF.class.getName();
+    Statement stmt1 = con.createStatement();
+    stmt1.execute("create temporary function sleepUDF as '" + udfName + "'");
+    stmt1.close();
+    Statement stmt = con.createStatement();
+    // Test a query where timeout kicks in
+    // Set query timeout to 15 seconds
+    stmt.setQueryTimeout(15);
+    System.err.println("Executing query: ");
+    try {
+      // Sleep UDF sleeps for 100ms for each select call
+      // The test table has 500 rows, so that should be sufficient time
+      stmt.executeQuery("select sleepUDF(t1.under_col) as u0, t1.under_col as u1, "
+          + "t2.under_col as u2 from " + tableName + " t1 join " + tableName
+          + " t2 on t1.under_col = t2.under_col");
+      fail("Expecting SQLTimeoutException");
+    } catch (SQLTimeoutException e) {
+      assertNotNull(e);
+      System.err.println(e.toString());
+    } catch (SQLException e) {
+      fail("Expecting SQLTimeoutException, but got SQLException: " + e);
+      e.printStackTrace();
+    }
+
+    // Test a query where timeout does not kick in. Set it to 25s
+    stmt.setQueryTimeout(25);
+    try {
+      stmt.executeQuery("show tables");
+    } catch (SQLException e) {
+      fail("Unexpected SQLException: " + e);
+      e.printStackTrace();
+    }
+
+    stmt.close();
+  }
+
   /**
    * Test the non-null value of the Yarn ATS GUID.
    * We spawn 2 threads - one running the query and

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java
index 4d763d2..c9e6a13 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java
@@ -70,7 +70,7 @@ public class TestHiveSessionImpl {
     Map<String, String> confOverlay = new HashMap<String, String>();
     String hql = "drop table if exists table_not_exists";
     Mockito.when(operationManager.newExecuteStatementOperation(same(session), eq(hql),
-        (Map<String, String>)Mockito.any(), eq(true))).thenReturn(operation);
+        (Map<String, String>)Mockito.any(), eq(true), eq(0))).thenReturn(operation);
 
     try {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
index 3cc6b74..38ccc78 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
@@ -43,6 +43,7 @@ import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLTimeoutException;
 import java.sql.SQLWarning;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -111,6 +112,8 @@ public class HiveStatement implements java.sql.Statement {
    */
   private boolean isExecuteStatementFailed = false;
 
+  private int queryTimeout = 0;
+
   public HiveStatement(HiveConnection connection, TCLIService.Iface client,
       TSessionHandle sessHandle) {
     this(connection, client, sessHandle, false, DEFAULT_FETCH_SIZE);
@@ -300,7 +303,7 @@ public class HiveStatement implements java.sql.Statement {
      */
     execReq.setRunAsync(true);
     execReq.setConfOverlay(sessConf);
-
+    execReq.setQueryTimeout(queryTimeout);
     try {
       TExecuteStatementResp execResp = client.ExecuteStatement(execReq);
       Utils.verifySuccessWithInfo(execResp.getStatus());
@@ -323,8 +326,8 @@ public class HiveStatement implements java.sql.Statement {
     while (!isOperationComplete) {
       try {
         /**
-         * For an async SQLOperation, GetOperationStatus will use the long polling approach
-         * It will essentially return after the HIVE_SERVER2_LONG_POLLING_TIMEOUT (a server config) expires
+         * For an async SQLOperation, GetOperationStatus will use the long polling approach It will
+         * essentially return after the HIVE_SERVER2_LONG_POLLING_TIMEOUT (a server config) expires
          */
         statusResp = client.GetOperationStatus(statusReq);
         Utils.verifySuccessWithInfo(statusResp.getStatus());
@@ -338,10 +341,12 @@ public class HiveStatement implements java.sql.Statement {
           case CANCELED_STATE:
             // 01000 -> warning
             throw new SQLException("Query was cancelled", "01000");
+          case TIMEDOUT_STATE:
+            throw new SQLTimeoutException("Query timed out after " + queryTimeout + " seconds");
           case ERROR_STATE:
             // Get the error details from the underlying exception
-            throw new SQLException(statusResp.getErrorMessage(),
-                statusResp.getSqlState(), statusResp.getErrorCode());
+            throw new SQLException(statusResp.getErrorMessage(), statusResp.getSqlState(),
+                statusResp.getErrorCode());
           case UKNOWN_STATE:
             throw new SQLException("Unknown query", "HY000");
           case INITIALIZED_STATE:
@@ -787,10 +792,7 @@ public class HiveStatement implements java.sql.Statement {
 
   @Override
   public void setQueryTimeout(int seconds) throws SQLException {
-    // 0 is supported which means "no limit"
-    if (seconds != 0) {
-      throw new SQLException("Query timeout seconds must be 0");
-    }
+    this.queryTimeout = seconds;
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 32d2cb2..6a610cb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -1725,20 +1725,31 @@ public class Driver implements CommandProcessor {
       }
       LOG.info("Completed executing command(queryId=" + queryId + "); Time taken: " + duration + " seconds");
     }
-    plan.setDone();
 
-    if (SessionState.get() != null) {
-      try {
-        SessionState.get().getHiveHistory().logPlanProgress(plan);
-      } catch (Exception e) {
-        // ignore
-      }
+    releasePlan(plan);
+
+    if (console != null) {
+      console.printInfo("OK");
     }
-    console.printInfo("OK");
 
     return (0);
   }
 
+  private synchronized void releasePlan(QueryPlan plan) {
+    // Plan maybe null if Driver.close is called in another thread for the same Driver object
+    if (plan != null) {
+      plan.setDone();
+      if (SessionState.get() != null) {
+        try {
+          SessionState.get().getHiveHistory().logPlanProgress(plan);
+        } catch (Exception e) {
+          // Log and ignore
+          LOG.warn("Could not log query plan progress", e);
+        }
+      }
+    }
+  }
+
   private void setQueryDisplays(List<Task<? extends Serializable>> tasks) {
     if (tasks != null) {
       for (Task<? extends Serializable> task : tasks) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistoryImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistoryImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistoryImpl.java
index 0234fd9..6582cdd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistoryImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistoryImpl.java
@@ -315,9 +315,11 @@ public class HiveHistoryImpl implements HiveHistory{
 
   @Override
   public void logPlanProgress(QueryPlan plan) throws IOException {
-    Map<String,String> ctrmap = ctrMapFactory.get();
-    ctrmap.put("plan", plan.toString());
-    log(RecordTypes.Counters, ctrmap);
+    if (plan != null) {
+      Map<String,String> ctrmap = ctrMapFactory.get();
+      ctrmap.put("plan", plan.toString());
+      log(RecordTypes.Counters, ctrmap);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/ql/src/java/org/apache/hadoop/hive/ql/session/OperationLog.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/OperationLog.java b/ql/src/java/org/apache/hadoop/hive/ql/session/OperationLog.java
index 6d0f14a..18216f2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/OperationLog.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/OperationLog.java
@@ -166,7 +166,7 @@ public class OperationLog {
       return readResults(maxRows);
     }
 
-    void remove() {
+    synchronized void remove() {
       try {
         if (in != null) {
           in.close();
@@ -174,8 +174,10 @@ public class OperationLog {
         if (out != null) {
           out.close();
         }
-        FileUtils.forceDelete(file);
-        isRemoved = true;
+        if (!isRemoved) {
+          FileUtils.forceDelete(file);
+          isRemoved = true;
+        }
       } catch (Exception e) {
         LOG.error("Failed to remove corresponding log file of operation: " + operationName, e);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service-rpc/if/TCLIService.thrift
----------------------------------------------------------------------
diff --git a/service-rpc/if/TCLIService.thrift b/service-rpc/if/TCLIService.thrift
index 92bcf77..9879b1b 100644
--- a/service-rpc/if/TCLIService.thrift
+++ b/service-rpc/if/TCLIService.thrift
@@ -458,6 +458,9 @@ enum TOperationState {
 
   // The operation is in an pending state
   PENDING_STATE,
+
+  // The operation is in an timedout state
+  TIMEDOUT_STATE,
 }
 
 // A string identifier. This is interpreted literally.
@@ -697,6 +700,9 @@ struct TExecuteStatementReq {
 
   // Execute asynchronously when runAsync is true
   4: optional bool runAsync = false
+
+  // The number of seconds after which the query will timeout on the server
+  5: optional i64 queryTimeout = 0
 }
 
 struct TExecuteStatementResp {

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp b/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp
index 66f5e8c..5229230 100644
--- a/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp
+++ b/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp
@@ -109,7 +109,8 @@ int _kTOperationStateValues[] = {
   TOperationState::CLOSED_STATE,
   TOperationState::ERROR_STATE,
   TOperationState::UKNOWN_STATE,
-  TOperationState::PENDING_STATE
+  TOperationState::PENDING_STATE,
+  TOperationState::TIMEDOUT_STATE
 };
 const char* _kTOperationStateNames[] = {
   "INITIALIZED_STATE",
@@ -119,9 +120,10 @@ const char* _kTOperationStateNames[] = {
   "CLOSED_STATE",
   "ERROR_STATE",
   "UKNOWN_STATE",
-  "PENDING_STATE"
+  "PENDING_STATE",
+  "TIMEDOUT_STATE"
 };
-const std::map<int, const char*> _TOperationState_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(8, _kTOperationStateValues, _kTOperationStateNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+const std::map<int, const char*> _TOperationState_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(9, _kTOperationStateValues, _kTOperationStateNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
 
 int _kTOperationTypeValues[] = {
   TOperationType::EXECUTE_STATEMENT,
@@ -5575,6 +5577,11 @@ void TExecuteStatementReq::__set_runAsync(const bool val) {
 __isset.runAsync = true;
 }
 
+void TExecuteStatementReq::__set_queryTimeout(const int64_t val) {
+  this->queryTimeout = val;
+__isset.queryTimeout = true;
+}
+
 uint32_t TExecuteStatementReq::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -5645,6 +5652,14 @@ uint32_t TExecuteStatementReq::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->queryTimeout);
+          this->__isset.queryTimeout = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -5693,6 +5708,11 @@ uint32_t TExecuteStatementReq::write(::apache::thrift::protocol::TProtocol* opro
     xfer += oprot->writeBool(this->runAsync);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.queryTimeout) {
+    xfer += oprot->writeFieldBegin("queryTimeout", ::apache::thrift::protocol::T_I64, 5);
+    xfer += oprot->writeI64(this->queryTimeout);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -5704,6 +5724,7 @@ void swap(TExecuteStatementReq &a, TExecuteStatementReq &b) {
   swap(a.statement, b.statement);
   swap(a.confOverlay, b.confOverlay);
   swap(a.runAsync, b.runAsync);
+  swap(a.queryTimeout, b.queryTimeout);
   swap(a.__isset, b.__isset);
 }
 
@@ -5712,6 +5733,7 @@ TExecuteStatementReq::TExecuteStatementReq(const TExecuteStatementReq& other222)
   statement = other222.statement;
   confOverlay = other222.confOverlay;
   runAsync = other222.runAsync;
+  queryTimeout = other222.queryTimeout;
   __isset = other222.__isset;
 }
 TExecuteStatementReq& TExecuteStatementReq::operator=(const TExecuteStatementReq& other223) {
@@ -5719,6 +5741,7 @@ TExecuteStatementReq& TExecuteStatementReq::operator=(const TExecuteStatementReq
   statement = other223.statement;
   confOverlay = other223.confOverlay;
   runAsync = other223.runAsync;
+  queryTimeout = other223.queryTimeout;
   __isset = other223.__isset;
   return *this;
 }
@@ -5729,6 +5752,7 @@ void TExecuteStatementReq::printTo(std::ostream& out) const {
   out << ", " << "statement=" << to_string(statement);
   out << ", " << "confOverlay="; (__isset.confOverlay ? (out << to_string(confOverlay)) : (out << "<null>"));
   out << ", " << "runAsync="; (__isset.runAsync ? (out << to_string(runAsync)) : (out << "<null>"));
+  out << ", " << "queryTimeout="; (__isset.queryTimeout ? (out << to_string(queryTimeout)) : (out << "<null>"));
   out << ")";
 }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h b/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h
index 9f937ca..838bf17 100644
--- a/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h
+++ b/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h
@@ -84,7 +84,8 @@ struct TOperationState {
     CLOSED_STATE = 4,
     ERROR_STATE = 5,
     UKNOWN_STATE = 6,
-    PENDING_STATE = 7
+    PENDING_STATE = 7,
+    TIMEDOUT_STATE = 8
   };
 };
 
@@ -2501,9 +2502,10 @@ inline std::ostream& operator<<(std::ostream& out, const TGetInfoResp& obj)
 }
 
 typedef struct _TExecuteStatementReq__isset {
-  _TExecuteStatementReq__isset() : confOverlay(false), runAsync(true) {}
+  _TExecuteStatementReq__isset() : confOverlay(false), runAsync(true), queryTimeout(true) {}
   bool confOverlay :1;
   bool runAsync :1;
+  bool queryTimeout :1;
 } _TExecuteStatementReq__isset;
 
 class TExecuteStatementReq {
@@ -2511,7 +2513,7 @@ class TExecuteStatementReq {
 
   TExecuteStatementReq(const TExecuteStatementReq&);
   TExecuteStatementReq& operator=(const TExecuteStatementReq&);
-  TExecuteStatementReq() : statement(), runAsync(false) {
+  TExecuteStatementReq() : statement(), runAsync(false), queryTimeout(0LL) {
   }
 
   virtual ~TExecuteStatementReq() throw();
@@ -2519,6 +2521,7 @@ class TExecuteStatementReq {
   std::string statement;
   std::map<std::string, std::string>  confOverlay;
   bool runAsync;
+  int64_t queryTimeout;
 
   _TExecuteStatementReq__isset __isset;
 
@@ -2530,6 +2533,8 @@ class TExecuteStatementReq {
 
   void __set_runAsync(const bool val);
 
+  void __set_queryTimeout(const int64_t val);
+
   bool operator == (const TExecuteStatementReq & rhs) const
   {
     if (!(sessionHandle == rhs.sessionHandle))
@@ -2544,6 +2549,10 @@ class TExecuteStatementReq {
       return false;
     else if (__isset.runAsync && !(runAsync == rhs.runAsync))
       return false;
+    if (__isset.queryTimeout != rhs.__isset.queryTimeout)
+      return false;
+    else if (__isset.queryTimeout && !(queryTimeout == rhs.queryTimeout))
+      return false;
     return true;
   }
   bool operator != (const TExecuteStatementReq &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java b/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java
index 2eb4d09..1f73cec 100644
--- a/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java
+++ b/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java
@@ -42,6 +42,7 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
   private static final org.apache.thrift.protocol.TField STATEMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("statement", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField CONF_OVERLAY_FIELD_DESC = new org.apache.thrift.protocol.TField("confOverlay", org.apache.thrift.protocol.TType.MAP, (short)3);
   private static final org.apache.thrift.protocol.TField RUN_ASYNC_FIELD_DESC = new org.apache.thrift.protocol.TField("runAsync", org.apache.thrift.protocol.TType.BOOL, (short)4);
+  private static final org.apache.thrift.protocol.TField QUERY_TIMEOUT_FIELD_DESC = new org.apache.thrift.protocol.TField("queryTimeout", org.apache.thrift.protocol.TType.I64, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -53,13 +54,15 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
   private String statement; // required
   private Map<String,String> confOverlay; // optional
   private boolean runAsync; // optional
+  private long queryTimeout; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     SESSION_HANDLE((short)1, "sessionHandle"),
     STATEMENT((short)2, "statement"),
     CONF_OVERLAY((short)3, "confOverlay"),
-    RUN_ASYNC((short)4, "runAsync");
+    RUN_ASYNC((short)4, "runAsync"),
+    QUERY_TIMEOUT((short)5, "queryTimeout");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -82,6 +85,8 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
           return CONF_OVERLAY;
         case 4: // RUN_ASYNC
           return RUN_ASYNC;
+        case 5: // QUERY_TIMEOUT
+          return QUERY_TIMEOUT;
         default:
           return null;
       }
@@ -123,8 +128,9 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
 
   // isset id assignments
   private static final int __RUNASYNC_ISSET_ID = 0;
+  private static final int __QUERYTIMEOUT_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.CONF_OVERLAY,_Fields.RUN_ASYNC};
+  private static final _Fields optionals[] = {_Fields.CONF_OVERLAY,_Fields.RUN_ASYNC,_Fields.QUERY_TIMEOUT};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -138,6 +144,8 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     tmpMap.put(_Fields.RUN_ASYNC, new org.apache.thrift.meta_data.FieldMetaData("runAsync", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.QUERY_TIMEOUT, new org.apache.thrift.meta_data.FieldMetaData("queryTimeout", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TExecuteStatementReq.class, metaDataMap);
   }
@@ -145,6 +153,8 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
   public TExecuteStatementReq() {
     this.runAsync = false;
 
+    this.queryTimeout = 0L;
+
   }
 
   public TExecuteStatementReq(
@@ -172,6 +182,7 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
       this.confOverlay = __this__confOverlay;
     }
     this.runAsync = other.runAsync;
+    this.queryTimeout = other.queryTimeout;
   }
 
   public TExecuteStatementReq deepCopy() {
@@ -185,6 +196,8 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
     this.confOverlay = null;
     this.runAsync = false;
 
+    this.queryTimeout = 0L;
+
   }
 
   public TSessionHandle getSessionHandle() {
@@ -289,6 +302,28 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RUNASYNC_ISSET_ID, value);
   }
 
+  public long getQueryTimeout() {
+    return this.queryTimeout;
+  }
+
+  public void setQueryTimeout(long queryTimeout) {
+    this.queryTimeout = queryTimeout;
+    setQueryTimeoutIsSet(true);
+  }
+
+  public void unsetQueryTimeout() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __QUERYTIMEOUT_ISSET_ID);
+  }
+
+  /** Returns true if field queryTimeout is set (has been assigned a value) and false otherwise */
+  public boolean isSetQueryTimeout() {
+    return EncodingUtils.testBit(__isset_bitfield, __QUERYTIMEOUT_ISSET_ID);
+  }
+
+  public void setQueryTimeoutIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __QUERYTIMEOUT_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case SESSION_HANDLE:
@@ -323,6 +358,14 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
       }
       break;
 
+    case QUERY_TIMEOUT:
+      if (value == null) {
+        unsetQueryTimeout();
+      } else {
+        setQueryTimeout((Long)value);
+      }
+      break;
+
     }
   }
 
@@ -340,6 +383,9 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
     case RUN_ASYNC:
       return isRunAsync();
 
+    case QUERY_TIMEOUT:
+      return getQueryTimeout();
+
     }
     throw new IllegalStateException();
   }
@@ -359,6 +405,8 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
       return isSetConfOverlay();
     case RUN_ASYNC:
       return isSetRunAsync();
+    case QUERY_TIMEOUT:
+      return isSetQueryTimeout();
     }
     throw new IllegalStateException();
   }
@@ -412,6 +460,15 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
         return false;
     }
 
+    boolean this_present_queryTimeout = true && this.isSetQueryTimeout();
+    boolean that_present_queryTimeout = true && that.isSetQueryTimeout();
+    if (this_present_queryTimeout || that_present_queryTimeout) {
+      if (!(this_present_queryTimeout && that_present_queryTimeout))
+        return false;
+      if (this.queryTimeout != that.queryTimeout)
+        return false;
+    }
+
     return true;
   }
 
@@ -439,6 +496,11 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
     if (present_runAsync)
       list.add(runAsync);
 
+    boolean present_queryTimeout = true && (isSetQueryTimeout());
+    list.add(present_queryTimeout);
+    if (present_queryTimeout)
+      list.add(queryTimeout);
+
     return list.hashCode();
   }
 
@@ -490,6 +552,16 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetQueryTimeout()).compareTo(other.isSetQueryTimeout());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetQueryTimeout()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queryTimeout, other.queryTimeout);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -541,6 +613,12 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
       sb.append(this.runAsync);
       first = false;
     }
+    if (isSetQueryTimeout()) {
+      if (!first) sb.append(", ");
+      sb.append("queryTimeout:");
+      sb.append(this.queryTimeout);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -642,6 +720,14 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 5: // QUERY_TIMEOUT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.queryTimeout = iprot.readI64();
+              struct.setQueryTimeoutIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -685,6 +771,11 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
         oprot.writeBool(struct.runAsync);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetQueryTimeout()) {
+        oprot.writeFieldBegin(QUERY_TIMEOUT_FIELD_DESC);
+        oprot.writeI64(struct.queryTimeout);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -711,7 +802,10 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
       if (struct.isSetRunAsync()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetQueryTimeout()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetConfOverlay()) {
         {
           oprot.writeI32(struct.confOverlay.size());
@@ -725,6 +819,9 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
       if (struct.isSetRunAsync()) {
         oprot.writeBool(struct.runAsync);
       }
+      if (struct.isSetQueryTimeout()) {
+        oprot.writeI64(struct.queryTimeout);
+      }
     }
 
     @Override
@@ -735,7 +832,7 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
       struct.setSessionHandleIsSet(true);
       struct.statement = iprot.readString();
       struct.setStatementIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
           org.apache.thrift.protocol.TMap _map168 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
@@ -755,6 +852,10 @@ public class TExecuteStatementReq implements org.apache.thrift.TBase<TExecuteSta
         struct.runAsync = iprot.readBool();
         struct.setRunAsyncIsSet(true);
       }
+      if (incoming.get(2)) {
+        struct.queryTimeout = iprot.readI64();
+        struct.setQueryTimeoutIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TOperationState.java
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TOperationState.java b/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TOperationState.java
index 3fa49b0..4390b4b 100644
--- a/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TOperationState.java
+++ b/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TOperationState.java
@@ -19,7 +19,8 @@ public enum TOperationState implements org.apache.thrift.TEnum {
   CLOSED_STATE(4),
   ERROR_STATE(5),
   UKNOWN_STATE(6),
-  PENDING_STATE(7);
+  PENDING_STATE(7),
+  TIMEDOUT_STATE(8);
 
   private final int value;
 
@@ -56,6 +57,8 @@ public enum TOperationState implements org.apache.thrift.TEnum {
         return UKNOWN_STATE;
       case 7:
         return PENDING_STATE;
+      case 8:
+        return TIMEDOUT_STATE;
       default:
         return null;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service-rpc/src/gen/thrift/gen-php/Types.php
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-php/Types.php b/service-rpc/src/gen/thrift/gen-php/Types.php
index 7f1f99f..9ed7403 100644
--- a/service-rpc/src/gen/thrift/gen-php/Types.php
+++ b/service-rpc/src/gen/thrift/gen-php/Types.php
@@ -109,6 +109,7 @@ final class TOperationState {
   const ERROR_STATE = 5;
   const UKNOWN_STATE = 6;
   const PENDING_STATE = 7;
+  const TIMEDOUT_STATE = 8;
   static public $__names = array(
     0 => 'INITIALIZED_STATE',
     1 => 'RUNNING_STATE',
@@ -118,6 +119,7 @@ final class TOperationState {
     5 => 'ERROR_STATE',
     6 => 'UKNOWN_STATE',
     7 => 'PENDING_STATE',
+    8 => 'TIMEDOUT_STATE',
   );
 }
 
@@ -5446,6 +5448,10 @@ class TExecuteStatementReq {
    * @var bool
    */
   public $runAsync = false;
+  /**
+   * @var int
+   */
+  public $queryTimeout = 0;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -5475,6 +5481,10 @@ class TExecuteStatementReq {
           'var' => 'runAsync',
           'type' => TType::BOOL,
           ),
+        5 => array(
+          'var' => 'queryTimeout',
+          'type' => TType::I64,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -5490,6 +5500,9 @@ class TExecuteStatementReq {
       if (isset($vals['runAsync'])) {
         $this->runAsync = $vals['runAsync'];
       }
+      if (isset($vals['queryTimeout'])) {
+        $this->queryTimeout = $vals['queryTimeout'];
+      }
     }
   }
 
@@ -5554,6 +5567,13 @@ class TExecuteStatementReq {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 5:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->queryTimeout);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -5603,6 +5623,11 @@ class TExecuteStatementReq {
       $xfer += $output->writeBool($this->runAsync);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->queryTimeout !== null) {
+      $xfer += $output->writeFieldBegin('queryTimeout', TType::I64, 5);
+      $xfer += $output->writeI64($this->queryTimeout);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py b/service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py
index 3bb20b8..44e5462 100644
--- a/service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py
+++ b/service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py
@@ -154,6 +154,7 @@ class TOperationState:
   ERROR_STATE = 5
   UKNOWN_STATE = 6
   PENDING_STATE = 7
+  TIMEDOUT_STATE = 8
 
   _VALUES_TO_NAMES = {
     0: "INITIALIZED_STATE",
@@ -164,6 +165,7 @@ class TOperationState:
     5: "ERROR_STATE",
     6: "UKNOWN_STATE",
     7: "PENDING_STATE",
+    8: "TIMEDOUT_STATE",
   }
 
   _NAMES_TO_VALUES = {
@@ -175,6 +177,7 @@ class TOperationState:
     "ERROR_STATE": 5,
     "UKNOWN_STATE": 6,
     "PENDING_STATE": 7,
+    "TIMEDOUT_STATE": 8,
   }
 
 class TOperationType:
@@ -4162,6 +4165,7 @@ class TExecuteStatementReq:
    - statement
    - confOverlay
    - runAsync
+   - queryTimeout
   """
 
   thrift_spec = (
@@ -4170,13 +4174,15 @@ class TExecuteStatementReq:
     (2, TType.STRING, 'statement', None, None, ), # 2
     (3, TType.MAP, 'confOverlay', (TType.STRING,None,TType.STRING,None), None, ), # 3
     (4, TType.BOOL, 'runAsync', None, False, ), # 4
+    (5, TType.I64, 'queryTimeout', None, 0, ), # 5
   )
 
-  def __init__(self, sessionHandle=None, statement=None, confOverlay=None, runAsync=thrift_spec[4][4],):
+  def __init__(self, sessionHandle=None, statement=None, confOverlay=None, runAsync=thrift_spec[4][4], queryTimeout=thrift_spec[5][4],):
     self.sessionHandle = sessionHandle
     self.statement = statement
     self.confOverlay = confOverlay
     self.runAsync = runAsync
+    self.queryTimeout = queryTimeout
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -4214,6 +4220,11 @@ class TExecuteStatementReq:
           self.runAsync = iprot.readBool()
         else:
           iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I64:
+          self.queryTimeout = iprot.readI64()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -4244,6 +4255,10 @@ class TExecuteStatementReq:
       oprot.writeFieldBegin('runAsync', TType.BOOL, 4)
       oprot.writeBool(self.runAsync)
       oprot.writeFieldEnd()
+    if self.queryTimeout is not None:
+      oprot.writeFieldBegin('queryTimeout', TType.I64, 5)
+      oprot.writeI64(self.queryTimeout)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -4261,6 +4276,7 @@ class TExecuteStatementReq:
     value = (value * 31) ^ hash(self.statement)
     value = (value * 31) ^ hash(self.confOverlay)
     value = (value * 31) ^ hash(self.runAsync)
+    value = (value * 31) ^ hash(self.queryTimeout)
     return value
 
   def __repr__(self):

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb b/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb
index 7208bae..b39ec1e 100644
--- a/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb
+++ b/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb
@@ -65,8 +65,9 @@ module TOperationState
   ERROR_STATE = 5
   UKNOWN_STATE = 6
   PENDING_STATE = 7
-  VALUE_MAP = {0 => "INITIALIZED_STATE", 1 => "RUNNING_STATE", 2 => "FINISHED_STATE", 3 => "CANCELED_STATE", 4 => "CLOSED_STATE", 5 => "ERROR_STATE", 6 => "UKNOWN_STATE", 7 => "PENDING_STATE"}
-  VALID_VALUES = Set.new([INITIALIZED_STATE, RUNNING_STATE, FINISHED_STATE, CANCELED_STATE, CLOSED_STATE, ERROR_STATE, UKNOWN_STATE, PENDING_STATE]).freeze
+  TIMEDOUT_STATE = 8
+  VALUE_MAP = {0 => "INITIALIZED_STATE", 1 => "RUNNING_STATE", 2 => "FINISHED_STATE", 3 => "CANCELED_STATE", 4 => "CLOSED_STATE", 5 => "ERROR_STATE", 6 => "UKNOWN_STATE", 7 => "PENDING_STATE", 8 => "TIMEDOUT_STATE"}
+  VALID_VALUES = Set.new([INITIALIZED_STATE, RUNNING_STATE, FINISHED_STATE, CANCELED_STATE, CLOSED_STATE, ERROR_STATE, UKNOWN_STATE, PENDING_STATE, TIMEDOUT_STATE]).freeze
 end
 
 module TOperationType
@@ -1135,12 +1136,14 @@ class TExecuteStatementReq
   STATEMENT = 2
   CONFOVERLAY = 3
   RUNASYNC = 4
+  QUERYTIMEOUT = 5
 
   FIELDS = {
     SESSIONHANDLE => {:type => ::Thrift::Types::STRUCT, :name => 'sessionHandle', :class => ::TSessionHandle},
     STATEMENT => {:type => ::Thrift::Types::STRING, :name => 'statement'},
     CONFOVERLAY => {:type => ::Thrift::Types::MAP, :name => 'confOverlay', :key => {:type => ::Thrift::Types::STRING}, :value => {:type => ::Thrift::Types::STRING}, :optional => true},
-    RUNASYNC => {:type => ::Thrift::Types::BOOL, :name => 'runAsync', :default => false, :optional => true}
+    RUNASYNC => {:type => ::Thrift::Types::BOOL, :name => 'runAsync', :default => false, :optional => true},
+    QUERYTIMEOUT => {:type => ::Thrift::Types::I64, :name => 'queryTimeout', :default => 0, :optional => true}
   }
 
   def struct_fields; FIELDS; end

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/CLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/CLIService.java b/service/src/java/org/apache/hive/service/cli/CLIService.java
index 4a83e38..ed52b4a 100644
--- a/service/src/java/org/apache/hive/service/cli/CLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/CLIService.java
@@ -248,33 +248,55 @@ public class CLIService extends CompositeService implements ICLIService {
     return infoValue;
   }
 
-  /* (non-Javadoc)
-   * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle,
-   *  java.lang.String, java.util.Map)
+  /**
+   * Execute statement on the server. This is a blocking call.
    */
   @Override
   public OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
-      Map<String, String> confOverlay)
-          throws HiveSQLException {
-    OperationHandle opHandle = sessionManager.getSession(sessionHandle)
-        .executeStatement(statement, confOverlay);
+      Map<String, String> confOverlay) throws HiveSQLException {
+    OperationHandle opHandle =
+        sessionManager.getSession(sessionHandle).executeStatement(statement, confOverlay);
     LOG.debug(sessionHandle + ": executeStatement()");
     return opHandle;
   }
 
-  /* (non-Javadoc)
-   * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle,
-   *  java.lang.String, java.util.Map)
+  /**
+   * Execute statement on the server with a timeout. This is a blocking call.
+   */
+  @Override
+  public OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay, long queryTimeout) throws HiveSQLException {
+    OperationHandle opHandle =
+        sessionManager.getSession(sessionHandle).executeStatement(statement, confOverlay,
+            queryTimeout);
+    LOG.debug(sessionHandle + ": executeStatement()");
+    return opHandle;
+  }
+
+  /**
+   * Execute statement asynchronously on the server. This is a non-blocking call
    */
   @Override
   public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement,
       Map<String, String> confOverlay) throws HiveSQLException {
-    OperationHandle opHandle = sessionManager.getSession(sessionHandle)
-        .executeStatementAsync(statement, confOverlay);
+    OperationHandle opHandle =
+        sessionManager.getSession(sessionHandle).executeStatementAsync(statement, confOverlay);
     LOG.debug(sessionHandle + ": executeStatementAsync()");
     return opHandle;
   }
 
+  /**
+   * Execute statement asynchronously on the server with a timeout. This is a non-blocking call
+   */
+  @Override
+  public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay, long queryTimeout) throws HiveSQLException {
+    OperationHandle opHandle =
+        sessionManager.getSession(sessionHandle).executeStatementAsync(statement, confOverlay,
+            queryTimeout);
+    LOG.debug(sessionHandle + ": executeStatementAsync()");
+    return opHandle;
+  }
 
   /* (non-Javadoc)
    * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle)

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java b/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java
index 79e0024..86e9bb1 100644
--- a/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java
+++ b/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java
@@ -67,26 +67,29 @@ public class EmbeddedCLIServiceClient extends CLIServiceClient {
     return cliService.getInfo(sessionHandle, getInfoType);
   }
 
-  /* (non-Javadoc)
-   * @see org.apache.hive.service.cli.CLIServiceClient#executeStatement(org.apache.hive.service.cli.SessionHandle,
-   *  java.lang.String, java.util.Map)
-   */
   @Override
   public OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
       Map<String, String> confOverlay) throws HiveSQLException {
     return cliService.executeStatement(sessionHandle, statement, confOverlay);
   }
 
-  /* (non-Javadoc)
-   * @see org.apache.hive.service.cli.CLIServiceClient#executeStatementAsync(org.apache.hive.service.cli.SessionHandle,
-   *  java.lang.String, java.util.Map)
-   */
+  @Override
+  public OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay, long queryTimeout) throws HiveSQLException {
+    return cliService.executeStatement(sessionHandle, statement, confOverlay, queryTimeout);
+  }
+
   @Override
   public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement,
       Map<String, String> confOverlay) throws HiveSQLException {
     return cliService.executeStatementAsync(sessionHandle, statement, confOverlay);
   }
 
+  @Override
+  public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay, long queryTimeout) throws HiveSQLException {
+    return cliService.executeStatementAsync(sessionHandle, statement, confOverlay, queryTimeout);
+  }
 
   /* (non-Javadoc)
    * @see org.apache.hive.service.cli.CLIServiceClient#getTypeInfo(org.apache.hive.service.cli.SessionHandle)

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/ICLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/ICLIService.java b/service/src/java/org/apache/hive/service/cli/ICLIService.java
index e4aef96..fef772d 100644
--- a/service/src/java/org/apache/hive/service/cli/ICLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/ICLIService.java
@@ -39,12 +39,16 @@ public interface ICLIService {
       throws HiveSQLException;
 
   OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
-      Map<String, String> confOverlay)
-          throws HiveSQLException;
+      Map<String, String> confOverlay) throws HiveSQLException;
 
-  OperationHandle executeStatementAsync(SessionHandle sessionHandle,
-      String statement, Map<String, String> confOverlay)
-          throws HiveSQLException;
+  OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay, long queryTimeout) throws HiveSQLException;
+
+  OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay) throws HiveSQLException;
+
+  OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay, long queryTimeout) throws HiveSQLException;
 
   OperationHandle getTypeInfo(SessionHandle sessionHandle)
       throws HiveSQLException;
@@ -105,6 +109,4 @@ public interface ICLIService {
     String primaryCatalog, String primarySchema, String primaryTable,
     String foreignCatalog, String foreignSchema, String foreignTable)
     throws HiveSQLException;
-
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/OperationState.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/OperationState.java b/service/src/java/org/apache/hive/service/cli/OperationState.java
index 6a67a1d..ae1ff5e 100644
--- a/service/src/java/org/apache/hive/service/cli/OperationState.java
+++ b/service/src/java/org/apache/hive/service/cli/OperationState.java
@@ -32,7 +32,8 @@ public enum OperationState {
   CLOSED(TOperationState.CLOSED_STATE, true),
   ERROR(TOperationState.ERROR_STATE, true),
   UNKNOWN(TOperationState.UKNOWN_STATE, false),
-  PENDING(TOperationState.PENDING_STATE, false);
+  PENDING(TOperationState.PENDING_STATE, false),
+  TIMEDOUT(TOperationState.TIMEDOUT_STATE, true);
 
   private final TOperationState tOperationState;
   private final boolean terminal;
@@ -57,6 +58,7 @@ public enum OperationState {
       case RUNNING:
       case CANCELED:
       case CLOSED:
+      case TIMEDOUT:
         return;
       }
       break;
@@ -67,6 +69,7 @@ public enum OperationState {
       case CANCELED:
       case ERROR:
       case CLOSED:
+      case TIMEDOUT:
         return;
       }
       break;
@@ -76,11 +79,13 @@ public enum OperationState {
       case CANCELED:
       case ERROR:
       case CLOSED:
+      case TIMEDOUT:
         return;
       }
       break;
     case FINISHED:
     case CANCELED:
+    case TIMEDOUT:
     case ERROR:
       if (OperationState.CLOSED.equals(newState)) {
         return;

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
index b3d9b52..ff46ed8 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
@@ -39,9 +39,9 @@ public abstract class ExecuteStatementOperation extends Operation {
     return statement;
   }
 
-  public static ExecuteStatementOperation newExecuteStatementOperation(
-      HiveSession parentSession, String statement, Map<String, String> confOverlay, boolean runAsync)
-          throws HiveSQLException {
+  public static ExecuteStatementOperation newExecuteStatementOperation(HiveSession parentSession,
+      String statement, Map<String, String> confOverlay, boolean runAsync, long queryTimeout)
+      throws HiveSQLException {
     String[] tokens = statement.trim().split("\\s+");
     CommandProcessor processor = null;
     try {
@@ -50,7 +50,8 @@ public abstract class ExecuteStatementOperation extends Operation {
       throw new HiveSQLException(e.getMessage(), e.getSQLState(), e);
     }
     if (processor == null) {
-      return new SQLOperation(parentSession, statement, confOverlay, runAsync);
+      // runAsync, queryTimeout makes sense only for a SQLOperation
+      return new SQLOperation(parentSession, statement, confOverlay, runAsync, queryTimeout);
     }
     return new HiveCommandOperation(parentSession, statement, processor, confOverlay);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
index f18dc67..8f08c2e 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
@@ -229,4 +229,9 @@ public class HiveCommandOperation extends ExecuteStatementOperation {
       resultReader = null;
     }
   }
+
+  @Override
+  public void cancel(OperationState stateAfterCancel) throws HiveSQLException {
+    throw new UnsupportedOperationException("HiveCommandOperation.cancel()");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
index 77228fa..fd6e428 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
@@ -109,7 +109,7 @@ public abstract class MetadataOperation extends Operation {
     pattern = replaceAll(pattern, "^_", ".");
     return pattern;
   }
-  
+
   private String replaceAll(String input, final String pattern, final String replace) {
     while (true) {
       String replaced = input.replaceAll(pattern, replace);
@@ -145,4 +145,9 @@ public abstract class MetadataOperation extends Operation {
     }
   }
 
+  @Override
+  public void cancel(OperationState stateAfterCancel) throws HiveSQLException {
+    throw new UnsupportedOperationException("MetadataOperation.cancel()");
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/operation/Operation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/Operation.java b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
index 63b1a48..0932884 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/Operation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
@@ -327,22 +327,23 @@ public abstract class Operation {
     }
   }
 
-  protected void cleanupOperationLog() {
+  protected synchronized void cleanupOperationLog() {
     if (isOperationLogEnabled) {
+      if (opHandle == null) {
+        LOG.warn("Operation seems to be in invalid state, opHandle is null");
+        return;
+      }
       if (operationLog == null) {
-        LOG.error("Operation [ " + opHandle.getHandleIdentifier() + " ] "
-          + "logging is enabled, but its OperationLog object cannot be found.");
+        LOG.warn("Operation [ " + opHandle.getHandleIdentifier() + " ] " + "logging is enabled, "
+            + "but its OperationLog object cannot be found. "
+            + "Perhaps the operation has already terminated.");
       } else {
         operationLog.close();
       }
     }
   }
 
-  // TODO: make this abstract and implement in subclasses.
-  public void cancel() throws HiveSQLException {
-    setState(OperationState.CANCELED);
-    throw new UnsupportedOperationException("SQLOperation.cancel()");
-  }
+  public abstract void cancel(OperationState stateAfterCancel) throws HiveSQLException;
 
   public abstract void close() throws HiveSQLException;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java b/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
index 52e4b4d..2f18231 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
@@ -90,12 +90,10 @@ public class OperationManager extends AbstractService {
   @Override
   public synchronized void start() {
     super.start();
-    // TODO
   }
 
   @Override
   public synchronized void stop() {
-    // TODO
     super.stop();
   }
 
@@ -111,10 +109,11 @@ public class OperationManager extends AbstractService {
   }
 
   public ExecuteStatementOperation newExecuteStatementOperation(HiveSession parentSession,
-      String statement, Map<String, String> confOverlay, boolean runAsync)
-          throws HiveSQLException {
-    ExecuteStatementOperation executeStatementOperation = ExecuteStatementOperation
-        .newExecuteStatementOperation(parentSession, statement, confOverlay, runAsync);
+      String statement, Map<String, String> confOverlay, boolean runAsync, long queryTimeout)
+      throws HiveSQLException {
+    ExecuteStatementOperation executeStatementOperation =
+        ExecuteStatementOperation.newExecuteStatementOperation(parentSession, statement,
+            confOverlay, runAsync, queryTimeout);
     addOperation(executeStatementOperation);
     return executeStatementOperation;
   }
@@ -250,20 +249,20 @@ public class OperationManager extends AbstractService {
     return getOperation(opHandle).getStatus();
   }
 
+  /**
+   * Cancel the running operation unless it is already in a terminal state
+   * @param opHandle
+   * @throws HiveSQLException
+   */
   public void cancelOperation(OperationHandle opHandle) throws HiveSQLException {
     Operation operation = getOperation(opHandle);
     OperationState opState = operation.getStatus().getState();
-    if (opState == OperationState.CANCELED ||
-        opState == OperationState.CLOSED ||
-        opState == OperationState.FINISHED ||
-        opState == OperationState.ERROR ||
-        opState == OperationState.UNKNOWN) {
+    if (opState.isTerminal()) {
       // Cancel should be a no-op in either cases
       LOG.debug(opHandle + ": Operation is already aborted in state - " + opState);
-    }
-    else {
+    } else {
       LOG.debug(opHandle + ": Attempting to cancel from state - " + opState);
-      operation.cancel();
+      operation.cancel(OperationState.CANCELED);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 01dd48c..67e0e52 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -18,12 +18,24 @@
 
 package org.apache.hive.service.cli.operation;
 
-import java.io.*;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.Serializable;
+import java.io.UnsupportedEncodingException;
 import java.security.PrivilegedExceptionAction;
 import java.sql.SQLException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 import java.util.concurrent.Future;
 import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.codec.binary.Base64;
@@ -84,9 +96,10 @@ public class SQLOperation extends ExecuteStatementOperation {
   private SerDe serde = null;
   private boolean fetchStarted = false;
   private volatile MetricsScope currentSQLStateScope;
-
-  //Display for WebUI.
+  // Display for WebUI.
   private SQLOperationDisplay sqlOpDisplay;
+  private long queryTimeout;
+  private ScheduledExecutorService timeoutExecutor;
 
   /**
    * A map to track query count running by each user
@@ -94,10 +107,11 @@ public class SQLOperation extends ExecuteStatementOperation {
   private static Map<String, AtomicInteger> userQueries = new HashMap<String, AtomicInteger>();
   private static final String ACTIVE_SQL_USER = MetricsConstant.SQL_OPERATION_PREFIX + "active_user";
 
-  public SQLOperation(HiveSession parentSession, String statement, Map<String,
-      String> confOverlay, boolean runInBackground) {
+  public SQLOperation(HiveSession parentSession, String statement, Map<String, String> confOverlay,
+      boolean runInBackground, long queryTimeout) {
     // TODO: call setRemoteUser in ExecuteStatementOperation or higher.
     super(parentSession, statement, confOverlay, runInBackground);
+    this.queryTimeout = queryTimeout;
     setupSessionIO(parentSession.getSessionState());
     try {
       sqlOpDisplay = new SQLOperationDisplay(this);
@@ -121,7 +135,7 @@ public class SQLOperation extends ExecuteStatementOperation {
       }
     }
 
-  /***
+  /**
    * Compile the query and extract metadata
    * @param sqlOperationConf
    * @throws HiveSQLException
@@ -130,6 +144,29 @@ public class SQLOperation extends ExecuteStatementOperation {
     setState(OperationState.RUNNING);
     try {
       driver = new Driver(queryState, getParentSession().getUserName());
+
+      // Start the timer thread for cancelling the query when query timeout is reached
+      // queryTimeout == 0 means no timeout
+      if (queryTimeout > 0) {
+        timeoutExecutor = new ScheduledThreadPoolExecutor(1);
+        Runnable timeoutTask = new Runnable() {
+          @Override
+          public void run() {
+            try {
+              LOG.info("Query timed out after: " + queryTimeout
+                  + " seconds. Cancelling the execution now.");
+              SQLOperation.this.cancel(OperationState.TIMEDOUT);
+            } catch (HiveSQLException e) {
+              LOG.error("Error cancelling the query after timeout: " + queryTimeout + " seconds", e);
+            } finally {
+              // Stop
+              timeoutExecutor.shutdown();
+            }
+          }
+        };
+        timeoutExecutor.schedule(timeoutTask, queryTimeout, TimeUnit.SECONDS);
+      }
+
       sqlOpDisplay.setQueryDisplay(driver.getQueryDisplay());
 
       // set the operation handle information in Driver, so that thrift API users
@@ -184,6 +221,13 @@ public class SQLOperation extends ExecuteStatementOperation {
 
   private void runQuery() throws HiveSQLException {
     try {
+      OperationState opState = getStatus().getState();
+      // Operation may have been cancelled by another thread
+      if (opState.isTerminal()) {
+        LOG.info("Not running the query. Operation is already in terminal state: " + opState
+            + ", perhaps cancelled due to query timeout or by another thread.");
+        return;
+      }
       // In Hive server mode, we are not able to retry in the FetchTask
       // case, when calling fetch queries since execute() has returned.
       // For now, we disable the test attempts.
@@ -193,14 +237,16 @@ public class SQLOperation extends ExecuteStatementOperation {
         throw toSQLException("Error while processing statement", response);
       }
     } catch (HiveSQLException e) {
-      // If the operation was cancelled by another thread,
-      // Driver#run will return a non-zero response code.
-      // We will simply return if the operation state is CANCELED,
-      // otherwise throw an exception
-      if (getStatus().getState() == OperationState.CANCELED) {
+      /**
+       * If the operation was cancelled by another thread, or the execution timed out, Driver#run
+       * may return a non-zero response code. We will simply return if the operation state is
+       * CANCELED, TIMEDOUT or CLOSED, otherwise throw an exception
+       */
+      if ((getStatus().getState() == OperationState.CANCELED)
+          || (getStatus().getState() == OperationState.TIMEDOUT)
+          || (getStatus().getState() == OperationState.CLOSED)) {
         return;
-      }
-      else {
+      } else {
         setState(OperationState.ERROR);
         throw e;
       }
@@ -312,8 +358,22 @@ public class SQLOperation extends ExecuteStatementOperation {
     }
   }
 
-  private void cleanup(OperationState state) throws HiveSQLException {
+  private synchronized void cleanup(OperationState state) throws HiveSQLException {
     setState(state);
+    if (driver != null) {
+      driver.close();
+      driver.destroy();
+    }
+    driver = null;
+
+    SessionState ss = SessionState.get();
+    if (ss == null) {
+      LOG.warn("Operation seems to be in invalid state, SessionState is null");
+    } else {
+      ss.deleteTmpOutputFile();
+      ss.deleteTmpErrOutputFile();
+    }
+
     if (shouldRunAsync()) {
       Future<?> backgroundHandle = getBackgroundHandle();
       if (backgroundHandle != null) {
@@ -321,20 +381,16 @@ public class SQLOperation extends ExecuteStatementOperation {
       }
     }
 
-    if (driver != null) {
-      driver.close();
-      driver.destroy();
+    // Shutdown the timeout thread if any, while closing this operation
+    if ((timeoutExecutor != null) && (state != OperationState.TIMEDOUT) && (state.isTerminal())) {
+      timeoutExecutor.shutdownNow();
     }
-    driver = null;
-
-    SessionState ss = SessionState.get();
-    ss.deleteTmpOutputFile();
-    ss.deleteTmpErrOutputFile();
   }
 
   @Override
-  public void cancel() throws HiveSQLException {
-    cleanup(OperationState.CANCELED);
+  public void cancel(OperationState stateAfterCancel) throws HiveSQLException {
+    cleanup(stateAfterCancel);
+    cleanupOperationLog();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/session/HiveSession.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSession.java b/service/src/java/org/apache/hive/service/cli/session/HiveSession.java
index 9ea643b..78ff388 100644
--- a/service/src/java/org/apache/hive/service/cli/session/HiveSession.java
+++ b/service/src/java/org/apache/hive/service/cli/session/HiveSession.java
@@ -56,18 +56,38 @@ public interface HiveSession extends HiveSessionBase {
    * @return
    * @throws HiveSQLException
    */
-  OperationHandle executeStatement(String statement,
-      Map<String, String> confOverlay) throws HiveSQLException;
+  OperationHandle executeStatement(String statement, Map<String, String> confOverlay) throws HiveSQLException;
 
   /**
    * execute operation handler
    * @param statement
    * @param confOverlay
+   * @param queryTimeout
    * @return
    * @throws HiveSQLException
    */
-  OperationHandle executeStatementAsync(String statement,
-      Map<String, String> confOverlay) throws HiveSQLException;
+  OperationHandle executeStatement(String statement, Map<String, String> confOverlay,
+      long queryTimeout) throws HiveSQLException;
+
+  /**
+   * execute operation handler
+   * @param statement
+   * @param confOverlay
+   * @return
+   * @throws HiveSQLException
+   */
+  OperationHandle executeStatementAsync(String statement, Map<String, String> confOverlay) throws HiveSQLException;
+
+  /**
+   * execute operation handler
+   * @param statement
+   * @param confOverlay
+   * @param queryTimeout
+   * @return
+   * @throws HiveSQLException
+   */
+  OperationHandle executeStatementAsync(String statement, Map<String, String> confOverlay,
+      long queryTimeout) throws HiveSQLException;
 
   /**
    * getTypeInfo operation handler

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
index 0cfec7a..a0015eb 100644
--- a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
+++ b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
@@ -204,7 +204,7 @@ public class HiveSessionImpl implements HiveSession {
       OperationHandle opHandle = null;
       try {
         //execute in sync mode
-        opHandle = executeStatementInternal(cmd_trimed, null, false);
+        opHandle = executeStatementInternal(cmd_trimed, null, false, 0);
       } catch (HiveSQLException e) {
         LOG.warn("Failed to execute command in global .hiverc file.", e);
         return -1;
@@ -426,33 +426,43 @@ public class HiveSessionImpl implements HiveSession {
   }
 
   @Override
-  public OperationHandle executeStatement(String statement, Map<String, String> confOverlay)
-      throws HiveSQLException {
-    return executeStatementInternal(statement, confOverlay, false);
+  public OperationHandle executeStatement(String statement, Map<String, String> confOverlay) throws HiveSQLException {
+    return executeStatementInternal(statement, confOverlay, false, 0);
   }
 
   @Override
-  public OperationHandle executeStatementAsync(String statement, Map<String, String> confOverlay)
-      throws HiveSQLException {
-    return executeStatementInternal(statement, confOverlay, true);
+  public OperationHandle executeStatement(String statement, Map<String, String> confOverlay,
+      long queryTimeout) throws HiveSQLException {
+    return executeStatementInternal(statement, confOverlay, false, queryTimeout);
   }
 
-  private OperationHandle executeStatementInternal(String statement, Map<String, String> confOverlay,
-      boolean runAsync)
-          throws HiveSQLException {
+  @Override
+  public OperationHandle executeStatementAsync(String statement, Map<String, String> confOverlay) throws HiveSQLException {
+    return executeStatementInternal(statement, confOverlay, true, 0);
+  }
+
+  @Override
+  public OperationHandle executeStatementAsync(String statement, Map<String, String> confOverlay,
+      long queryTimeout) throws HiveSQLException {
+    return executeStatementInternal(statement, confOverlay, true, queryTimeout);
+  }
+
+  private OperationHandle executeStatementInternal(String statement,
+      Map<String, String> confOverlay, boolean runAsync, long queryTimeout) throws HiveSQLException {
     acquire(true);
 
     OperationManager operationManager = getOperationManager();
-    ExecuteStatementOperation operation = operationManager
-        .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync);
+    ExecuteStatementOperation operation =
+        operationManager.newExecuteStatementOperation(getSession(), statement, confOverlay,
+            runAsync, queryTimeout);
     OperationHandle opHandle = operation.getHandle();
     try {
       operation.run();
       addOpHandle(opHandle);
       return opHandle;
     } catch (HiveSQLException e) {
-      // Refering to SQLOperation.java,there is no chance that a HiveSQLException throws and the asyn
-      // background operation submits to thread pool successfully at the same time. So, Cleanup
+      // Refering to SQLOperation.java, there is no chance that a HiveSQLException throws and the
+      // async background operation submits to thread pool successfully at the same time. So, Cleanup
       // opHandle directly when got HiveSQLException
       operationManager.closeOperation(opHandle);
       throw e;

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java b/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java
index b2e0e9e..933750b 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java
@@ -126,20 +126,30 @@ public class RetryingThriftCLIServiceClient implements InvocationHandler {
     }
 
     @Override
-    public OperationHandle executeStatement(SessionHandle sessionHandle,
-                                            String statement,
-                                            Map<String, String> confOverlay) throws HiveSQLException {
+    public OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
+        Map<String, String> confOverlay) throws HiveSQLException {
       return cliService.executeStatement(sessionHandle, statement, confOverlay);
     }
 
     @Override
-    public OperationHandle executeStatementAsync(SessionHandle sessionHandle,
-                                                 String statement,
-                                                 Map<String, String> confOverlay) throws HiveSQLException {
+    public OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
+        Map<String, String> confOverlay, long queryTimeout) throws HiveSQLException {
+      return cliService.executeStatement(sessionHandle, statement, confOverlay, queryTimeout);
+    }
+
+    @Override
+    public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement,
+        Map<String, String> confOverlay) throws HiveSQLException {
       return cliService.executeStatementAsync(sessionHandle, statement, confOverlay);
     }
 
     @Override
+    public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement,
+        Map<String, String> confOverlay, long queryTimeout) throws HiveSQLException {
+      return cliService.executeStatementAsync(sessionHandle, statement, confOverlay, queryTimeout);
+    }
+
+    @Override
     public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException {
       return cliService.getTypeInfo(sessionHandle);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
index 6ede1d7..5464e58 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
@@ -493,15 +493,17 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       String statement = req.getStatement();
       Map<String, String> confOverlay = req.getConfOverlay();
       Boolean runAsync = req.isRunAsync();
-      OperationHandle operationHandle = runAsync ?
-          cliService.executeStatementAsync(sessionHandle, statement, confOverlay)
-          : cliService.executeStatement(sessionHandle, statement, confOverlay);
-          resp.setOperationHandle(operationHandle.toTOperationHandle());
-          resp.setStatus(OK_STATUS);
+      long queryTimeout = req.getQueryTimeout();
+      OperationHandle operationHandle =
+          runAsync ? cliService.executeStatementAsync(sessionHandle, statement, confOverlay,
+              queryTimeout) : cliService.executeStatement(sessionHandle, statement, confOverlay,
+              queryTimeout);
+      resp.setOperationHandle(operationHandle.toTOperationHandle());
+      resp.setStatus(OK_STATUS);
     } catch (Exception e) {
       // Note: it's rather important that this (and other methods) catch Exception, not Throwable;
-      //       in combination with HiveSessionProxy.invoke code, perhaps unintentionally, it used
-      //       to also catch all errors; and now it allows OOMs only to propagate.
+      // in combination with HiveSessionProxy.invoke code, perhaps unintentionally, it used
+      // to also catch all errors; and now it allows OOMs only to propagate.
       LOG.warn("Error executing statement: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java
index 098aea6..82ac42d 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java
@@ -166,34 +166,38 @@ public class ThriftCLIServiceClient extends CLIServiceClient {
     }
   }
 
-  /* (non-Javadoc)
-   * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map)
-   */
   @Override
   public OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
-      Map<String, String> confOverlay)
-          throws HiveSQLException {
-    return executeStatementInternal(sessionHandle, statement, confOverlay, false);
+      Map<String, String> confOverlay) throws HiveSQLException {
+    return executeStatementInternal(sessionHandle, statement, confOverlay, false, 0);
+  }
+
+  @Override
+  public OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay, long queryTimeout) throws HiveSQLException {
+    return executeStatementInternal(sessionHandle, statement, confOverlay, false, queryTimeout);
   }
 
-  /* (non-Javadoc)
-   * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map)
-   */
   @Override
   public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement,
-      Map<String, String> confOverlay)
-          throws HiveSQLException {
-    return executeStatementInternal(sessionHandle, statement, confOverlay, true);
+      Map<String, String> confOverlay) throws HiveSQLException {
+    return executeStatementInternal(sessionHandle, statement, confOverlay, true, 0);
+  }
+
+  @Override
+  public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay, long queryTimeout) throws HiveSQLException {
+    return executeStatementInternal(sessionHandle, statement, confOverlay, true, queryTimeout);
   }
 
   private OperationHandle executeStatementInternal(SessionHandle sessionHandle, String statement,
-      Map<String, String> confOverlay, boolean isAsync)
-          throws HiveSQLException {
+      Map<String, String> confOverlay, boolean isAsync, long queryTimeout) throws HiveSQLException {
     try {
       TExecuteStatementReq req =
           new TExecuteStatementReq(sessionHandle.toTSessionHandle(), statement);
       req.setConfOverlay(confOverlay);
       req.setRunAsync(isAsync);
+      req.setQueryTimeout(queryTimeout);
       TExecuteStatementResp resp = cliService.ExecuteStatement(req);
       checkStatus(resp.getStatus());
       TProtocolVersion protocol = sessionHandle.getProtocolVersion();

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java b/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
index 1740079..abb1ecf 100644
--- a/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
+++ b/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
@@ -178,8 +178,7 @@ public abstract class ThriftCLIServiceTest {
 
     // Execute another query
     queryString = "SELECT ID+1 FROM TEST_EXEC_THRIFT";
-    OperationHandle opHandle = client.executeStatement(sessHandle,
-        queryString, opConf);
+    OperationHandle opHandle = client.executeStatement(sessHandle, queryString, opConf);
     assertNotNull(opHandle);
 
     OperationStatus opStatus = client.getOperationStatus(opHandle);
@@ -229,8 +228,7 @@ public abstract class ThriftCLIServiceTest {
     // Execute another query
     queryString = "SELECT ID+1 FROM TEST_EXEC_ASYNC_THRIFT";
     System.out.println("Will attempt to execute: " + queryString);
-    opHandle = client.executeStatementAsync(sessHandle,
-        queryString, opConf);
+    opHandle = client.executeStatementAsync(sessHandle, queryString, opConf);
     assertNotNull(opHandle);
 
     // Poll on the operation status till the query is completed

http://git-wip-us.apache.org/repos/asf/hive/blob/b6218275/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java b/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
index a1ef1fc..ab20c4c 100644
--- a/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
+++ b/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
@@ -200,8 +200,7 @@ public class ThriftCliServiceTestWithCookie {
 
     // Execute another query
     queryString = "SELECT ID+1 FROM TEST_EXEC_THRIFT";
-    OperationHandle opHandle = client.executeStatement(sessHandle,
-        queryString, opConf);
+    OperationHandle opHandle = client.executeStatement(sessHandle, queryString, opConf);
     assertNotNull(opHandle);
 
     OperationStatus opStatus = client.getOperationStatus(opHandle);


[14/50] [abbrv] hive git commit: HIVE-13638: CBO rule to pull up constants through Sort/Limit (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
HIVE-13638: CBO rule to pull up constants through Sort/Limit (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/java8
Commit: b04dc95f4fa7dda9d4806c45dbe52aed4b9f1a18
Parents: 2d33d09
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Sat Apr 30 11:49:47 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed May 4 18:57:30 2016 +0100

----------------------------------------------------------------------
 .../rules/HiveReduceExpressionsRule.java        | 125 ++++
 .../rules/HiveSortLimitPullUpConstantsRule.java | 157 +++++
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   3 +
 .../test/queries/clientpositive/cbo_input26.q   |  54 ++
 .../results/clientpositive/cbo_input26.q.out    | 596 +++++++++++++++++++
 .../clientpositive/load_dyn_part14.q.out        |   6 +-
 .../clientpositive/spark/load_dyn_part14.q.out  |   6 +-
 .../clientpositive/spark/union_remove_25.q.out  |  60 +-
 .../clientpositive/union_remove_25.q.out        |  20 +-
 9 files changed, 985 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b04dc95f/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
index 9006f45..2fe9b75 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
@@ -396,6 +396,131 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
     assert constExps.size() == addCasts.size();
   }
 
+  /** Creates a map containing each (e, constant) pair that occurs within
+   * a predicate list.
+   *
+   * @param clazz Class of expression that is considered constant
+   * @param rexBuilder Rex builder
+   * @param predicates Predicate list
+   * @param <C> what to consider a constant: {@link RexLiteral} to use a narrow
+   *           definition of constant, or {@link RexNode} to use
+   *           {@link RexUtil#isConstant(RexNode)}
+   * @return Map from values to constants
+   */
+  public static <C extends RexNode> ImmutableMap<RexNode, C> predicateConstants(
+          Class<C> clazz, RexBuilder rexBuilder, RelOptPredicateList predicates) {
+    // We cannot use an ImmutableMap.Builder here. If there are multiple entries
+    // with the same key (e.g. "WHERE deptno = 1 AND deptno = 2"), it doesn't
+    // matter which we take, so the latter will replace the former.
+    // The basic idea is to find all the pairs of RexNode = RexLiteral
+    // (1) If 'predicates' contain a non-EQUALS, we bail out.
+    // (2) It is OK if a RexNode is equal to the same RexLiteral several times,
+    // (e.g. "WHERE deptno = 1 AND deptno = 1")
+    // (3) It will return false if there are inconsistent constraints (e.g.
+    // "WHERE deptno = 1 AND deptno = 2")
+    final Map<RexNode, C> map = new HashMap<>();
+    final Set<RexNode> excludeSet = new HashSet<>();
+    for (RexNode predicate : predicates.pulledUpPredicates) {
+      gatherConstraints(clazz, predicate, map, excludeSet, rexBuilder);
+    }
+    final ImmutableMap.Builder<RexNode, C> builder =
+        ImmutableMap.builder();
+    for (Map.Entry<RexNode, C> entry : map.entrySet()) {
+      RexNode rexNode = entry.getKey();
+      if (!overlap(rexNode, excludeSet)) {
+        builder.put(rexNode, entry.getValue());
+      }
+    }
+    return builder.build();
+  }
+
+  private static <C extends RexNode> void gatherConstraints(Class<C> clazz,
+      RexNode predicate, Map<RexNode, C> map, Set<RexNode> excludeSet,
+      RexBuilder rexBuilder) {
+    if (predicate.getKind() != SqlKind.EQUALS) {
+      decompose(excludeSet, predicate);
+      return;
+    }
+    final List<RexNode> operands = ((RexCall) predicate).getOperands();
+    if (operands.size() != 2) {
+      decompose(excludeSet, predicate);
+      return;
+    }
+    // if it reaches here, we have rexNode equals rexNode
+    final RexNode left = operands.get(0);
+    final RexNode right = operands.get(1);
+    // note that literals are immutable too and they can only be compared through
+    // values.
+    gatherConstraint(clazz, left, right, map, excludeSet, rexBuilder);
+    gatherConstraint(clazz, right, left, map, excludeSet, rexBuilder);
+  }
+
+  /** Returns whether a value of {@code type2} can be assigned to a variable
+   * of {@code type1}.
+   *
+   * <p>For example:
+   * <ul>
+   *   <li>{@code canAssignFrom(BIGINT, TINYINT)} returns {@code true}</li>
+   *   <li>{@code canAssignFrom(TINYINT, BIGINT)} returns {@code false}</li>
+   *   <li>{@code canAssignFrom(BIGINT, VARCHAR)} returns {@code false}</li>
+   * </ul>
+   */
+  private static boolean canAssignFrom(RelDataType type1, RelDataType type2) {
+    final SqlTypeName name1 = type1.getSqlTypeName();
+    final SqlTypeName name2 = type2.getSqlTypeName();
+    if (name1.getFamily() == name2.getFamily()) {
+      switch (name1.getFamily()) {
+      case NUMERIC:
+        return name1.compareTo(name2) >= 0;
+      default:
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static <C extends RexNode> void gatherConstraint(Class<C> clazz,
+      RexNode left, RexNode right, Map<RexNode, C> map, Set<RexNode> excludeSet,
+      RexBuilder rexBuilder) {
+    if (!clazz.isInstance(right)) {
+      return;
+    }
+    if (!RexUtil.isConstant(right)) {
+      return;
+    }
+    C constant = clazz.cast(right);
+    if (excludeSet.contains(left)) {
+      return;
+    }
+    final C existedValue = map.get(left);
+    if (existedValue == null) {
+      switch (left.getKind()) {
+      case CAST:
+        // Convert "CAST(c) = literal" to "c = literal", as long as it is a
+        // widening cast.
+        final RexNode operand = ((RexCall) left).getOperands().get(0);
+        if (canAssignFrom(left.getType(), operand.getType())) {
+          final RexNode castRight =
+              rexBuilder.makeCast(operand.getType(), constant);
+          if (castRight instanceof RexLiteral) {
+            left = operand;
+            constant = clazz.cast(castRight);
+          }
+        }
+      }
+      map.put(left, constant);
+    } else {
+      if (existedValue instanceof RexLiteral
+          && constant instanceof RexLiteral
+          && !((RexLiteral) existedValue).getValue()
+              .equals(((RexLiteral) constant).getValue())) {
+        // we found conflicting values, e.g. left = 10 and left = 20
+        map.remove(left);
+        excludeSet.add(left);
+      }
+    }
+  }
+  
   protected static ImmutableMap<RexNode, RexLiteral> predicateConstants(
       RelOptPredicateList predicates) {
     // We cannot use an ImmutableMap.Builder here. If there are multiple entries

http://git-wip-us.apache.org/repos/asf/hive/blob/b04dc95f/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java
new file mode 100644
index 0000000..d14b0ba
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java
@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Planner rule that pulls up constant keys through a SortLimit operator.
+ * 
+ * This rule is only applied on SortLimit operators that are not the root
+ * of the plan tree. This is done because the interaction of this rule
+ * with the AST conversion may cause some optimizations to not kick in
+ * e.g. SimpleFetchOptimizer. Nevertheless, this will not have any
+ * performance impact in the resulting plans.
+ */
+public class HiveSortLimitPullUpConstantsRule extends RelOptRule {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(HiveSortLimitPullUpConstantsRule.class);
+
+
+  public static final HiveSortLimitPullUpConstantsRule INSTANCE =
+          new HiveSortLimitPullUpConstantsRule(HiveSortLimit.class,
+                  HiveRelFactories.HIVE_BUILDER);
+
+  private HiveSortLimitPullUpConstantsRule(
+      Class<? extends Sort> sortClass,
+      RelBuilderFactory relBuilderFactory) {
+    super(operand(RelNode.class,
+            operand(sortClass, any())),
+            relBuilderFactory, null);
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final RelNode parent = call.rel(0);
+    final Sort sort = call.rel(1);
+
+    final int count = sort.getInput().getRowType().getFieldCount();
+    if (count == 1) {
+      // No room for optimization since we cannot convert to an empty
+      // Project operator.
+      return;
+    }
+
+    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final RelOptPredicateList predicates = mq.getPulledUpPredicates(sort.getInput());
+    if (predicates == null) {
+      return;
+    }
+
+    Map<RexNode, RexNode> constants = HiveReduceExpressionsRule.predicateConstants(
+            RexNode.class, rexBuilder, predicates);
+
+    // None of the expressions are constant. Nothing to do.
+    if (constants.isEmpty()) {
+      return;
+    }
+
+    if (count == constants.size()) {
+      // At least a single item in project is required.
+      final Map<RexNode, RexNode> map = new HashMap<>(constants);
+      map.remove(map.keySet().iterator().next());
+      constants = map;
+    }
+
+    // Create expressions for Project operators before and after the Sort
+    List<RelDataTypeField> fields = sort.getInput().getRowType().getFieldList();
+    List<Pair<RexNode, String>> newChildExprs = new ArrayList<>();
+    List<RexNode> topChildExprs = new ArrayList<>();
+    List<String> topChildExprsFields = new ArrayList<>();
+    for (int i = 0; i < count ; i++) {
+      RexNode expr = rexBuilder.makeInputRef(sort.getInput(), i);
+      RelDataTypeField field = fields.get(i);
+      if (constants.containsKey(expr)) {
+        topChildExprs.add(constants.get(expr));
+        topChildExprsFields.add(field.getName());
+      } else {
+        newChildExprs.add(Pair.<RexNode,String>of(expr, field.getName()));
+        topChildExprs.add(expr);
+        topChildExprsFields.add(field.getName());
+      }
+    }
+
+    // Update field collations
+    final Mappings.TargetMapping mapping =
+            RelOptUtil.permutation(Pair.left(newChildExprs), sort.getInput().getRowType()).inverse();
+    List<RelFieldCollation> fieldCollations = new ArrayList<>();
+    for (RelFieldCollation fc : sort.getCollation().getFieldCollations()) {
+      final int target = mapping.getTargetOpt(fc.getFieldIndex());
+      if (target < 0) {
+        // It is a constant, we can ignore it
+        continue;
+      }
+      fieldCollations.add(fc.copy(target));
+    }
+
+    // Update top Project positions
+    topChildExprs = ImmutableList.copyOf(RexUtil.apply(mapping, topChildExprs));
+
+    // Create new Project-Sort-Project sequence
+    final RelBuilder relBuilder = call.builder();
+    relBuilder.push(sort.getInput());
+    relBuilder.project(Pair.left(newChildExprs), Pair.right(newChildExprs));
+    final ImmutableList<RexNode> sortFields =
+            relBuilder.fields(RelCollations.of(fieldCollations));
+    relBuilder.sortLimit(sort.offset == null ? -1 : RexLiteral.intValue(sort.offset),
+            sort.fetch == null ? -1 : RexLiteral.intValue(sort.fetch), sortFields);
+    relBuilder.project(topChildExprs, topChildExprsFields);
+
+    call.transformTo(parent.copy(parent.getTraitSet(), ImmutableList.of(relBuilder.build())));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/b04dc95f/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 8e00e0b..377573b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -153,6 +153,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinCommuteRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinProjectTransposeRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinPushTransitivePredicatesRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinToMultiJoinRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortLimitPullUpConstantsRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePartitionPruneRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePointLookupOptimizerRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePreFilteringRule;
@@ -1163,6 +1164,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
       rules.add(HiveJoinAddNotNullRule.INSTANCE_SEMIJOIN);
       rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_JOIN);
       rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_SEMIJOIN);
+      rules.add(HiveSortMergeRule.INSTANCE);
+      rules.add(HiveSortLimitPullUpConstantsRule.INSTANCE);
       perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
       basePlan = hepPlan(basePlan, true, mdProvider, executorProvider, HepMatchOrder.BOTTOM_UP,
               rules.toArray(new RelOptRule[rules.size()]));

http://git-wip-us.apache.org/repos/asf/hive/blob/b04dc95f/ql/src/test/queries/clientpositive/cbo_input26.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_input26.q b/ql/src/test/queries/clientpositive/cbo_input26.q
new file mode 100644
index 0000000..40050f9
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/cbo_input26.q
@@ -0,0 +1,54 @@
+set hive.mapred.mode=nonstrict;
+set hive.optimize.constant.propagation=false;
+
+explain
+select * from (
+  select * from (select * from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select * from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq;
+
+select * from (
+  select * from (select * from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select * from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq;
+
+explain
+select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq;
+
+select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq;
+
+explain
+select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.hr,a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq;
+
+select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.hr,a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq;
+
+explain
+select * from (
+  select * from (select a.key, a.ds, a.value from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.ds limit 5)pa
+    union all
+  select * from (select b.key, b.ds, b.value from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq;
+
+select * from (
+  select * from (select a.key, a.ds, a.value from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.ds limit 5)pa
+    union all
+  select * from (select b.key, b.ds, b.value from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq;

http://git-wip-us.apache.org/repos/asf/hive/blob/b04dc95f/ql/src/test/results/clientpositive/cbo_input26.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_input26.q.out b/ql/src/test/results/clientpositive/cbo_input26.q.out
new file mode 100644
index 0000000..5c4c771
--- /dev/null
+++ b/ql/src/test/results/clientpositive/cbo_input26.q.out
@@ -0,0 +1,596 @@
+PREHOOK: query: explain
+select * from (
+  select * from (select * from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select * from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select * from (
+  select * from (select * from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select * from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1, Stage-3
+  Stage-3 is a root stage
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                TopN Hash Memory Usage: 0.1
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 5
+            Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), '11' (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3
+              Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Union
+              Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          TableScan
+            Union
+              Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((ds = '2008-04-08') and (hr = '14')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Limit
+                  Number of rows: 5
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    TopN Hash Memory Usage: 0.1
+                    value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Limit
+            Number of rows: 5
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Select Operator
+              expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), '14' (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select * from (
+  select * from (select * from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select * from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+POSTHOOK: query: select * from (
+  select * from (select * from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select * from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+10	val_10	2008-04-08	11
+100	val_100	2008-04-08	11
+PREHOOK: query: explain
+select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1, Stage-3
+  Stage-3 is a root stage
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                TopN Hash Memory Usage: 0.1
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: string)
+          outputColumnNames: _col0
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 5
+            Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: '2008-04-08' (type: string), _col0 (type: string), '11' (type: string)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Union
+              Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          TableScan
+            Union
+              Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((ds = '2008-04-08') and (hr = '14')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Limit
+                  Number of rows: 5
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    TopN Hash Memory Usage: 0.1
+                    value expressions: _col0 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string)
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Limit
+            Number of rows: 5
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Select Operator
+              expressions: '2008-04-08' (type: string), _col0 (type: string), '14' (type: string)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+POSTHOOK: query: select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+2008-04-08	0	11
+2008-04-08	0	11
+2008-04-08	0	11
+2008-04-08	10	11
+2008-04-08	100	11
+PREHOOK: query: explain
+select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.hr,a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.hr,a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1, Stage-3
+  Stage-3 is a root stage
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                TopN Hash Memory Usage: 0.1
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: string)
+          outputColumnNames: _col0
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 5
+            Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: '2008-04-08' (type: string), _col0 (type: string), '11' (type: string)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Union
+              Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          TableScan
+            Union
+              Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((ds = '2008-04-08') and (hr = '14')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Limit
+                  Number of rows: 5
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    TopN Hash Memory Usage: 0.1
+                    value expressions: _col0 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string)
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Limit
+            Number of rows: 5
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Select Operator
+              expressions: '2008-04-08' (type: string), _col0 (type: string), '14' (type: string)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.hr,a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+POSTHOOK: query: select * from (
+  select * from (select a.ds, a.key, a.hr from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.hr,a.key limit 5)pa
+    union all
+  select * from (select b.ds, b.key, b.hr from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+2008-04-08	0	11
+2008-04-08	0	11
+2008-04-08	0	11
+2008-04-08	10	11
+2008-04-08	100	11
+PREHOOK: query: explain
+select * from (
+  select * from (select a.key, a.ds, a.value from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.ds limit 5)pa
+    union all
+  select * from (select b.key, b.ds, b.value from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select * from (
+  select * from (select a.key, a.ds, a.value from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.ds limit 5)pa
+    union all
+  select * from (select b.key, b.ds, b.value from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1, Stage-3
+  Stage-3 is a root stage
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Limit
+                Number of rows: 5
+                Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                  TopN Hash Memory Usage: 0.1
+                  value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 5
+            Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: _col0 (type: string), '2008-04-08' (type: string), _col1 (type: string)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Union
+              Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          TableScan
+            Union
+              Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((ds = '2008-04-08') and (hr = '14')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Limit
+                  Number of rows: 5
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    TopN Hash Memory Usage: 0.1
+                    value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Limit
+            Number of rows: 5
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Select Operator
+              expressions: _col0 (type: string), '2008-04-08' (type: string), _col1 (type: string)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select * from (
+  select * from (select a.key, a.ds, a.value from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.ds limit 5)pa
+    union all
+  select * from (select b.key, b.ds, b.value from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+POSTHOOK: query: select * from (
+  select * from (select a.key, a.ds, a.value from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.ds limit 5)pa
+    union all
+  select * from (select b.key, b.ds, b.value from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5)pb
+)subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+165	2008-04-08	val_165
+27	2008-04-08	val_27
+311	2008-04-08	val_311
+86	2008-04-08	val_86
+238	2008-04-08	val_238

http://git-wip-us.apache.org/repos/asf/hive/blob/b04dc95f/ql/src/test/results/clientpositive/load_dyn_part14.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/load_dyn_part14.q.out b/ql/src/test/results/clientpositive/load_dyn_part14.q.out
index 53e9df3..57c4287 100644
--- a/ql/src/test/results/clientpositive/load_dyn_part14.q.out
+++ b/ql/src/test/results/clientpositive/load_dyn_part14.q.out
@@ -74,13 +74,13 @@ STAGE PLANS:
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
               Limit
                 Number of rows: 2
-                Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   TopN Hash Memory Usage: 0.1
       Reduce Operator Tree:
         Limit

http://git-wip-us.apache.org/repos/asf/hive/blob/b04dc95f/ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out b/ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out
index 84d99c3..1940561 100644
--- a/ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out
+++ b/ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out
@@ -73,13 +73,13 @@ STAGE PLANS:
                   alias: src
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                     Limit
                       Number of rows: 2
-                      Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         sort order: 
-                        Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                        Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
         Reducer 2 
             Reduce Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/b04dc95f/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_25.q.out b/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
index 253bf8f..190bea5 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
@@ -438,7 +438,7 @@ STAGE PLANS:
                   Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: string), value (type: string), hr (type: string)
-                    outputColumnNames: _col0, _col1, _col3
+                    outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                     Limit
                       Number of rows: 1000
@@ -447,49 +447,57 @@ STAGE PLANS:
                         sort order: 
                         Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
                         TopN Hash Memory Usage: 0.1
-                        value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+                        value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 2 
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col3 (type: string)
-                outputColumnNames: _col0, _col1, _col3
+                expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
                 Limit
                   Number of rows: 1000
                   Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col3 (type: string)
-                    outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
-                    File Output Operator
-                      compressed: false
+                    expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+                    outputColumnNames: _col0, _col1, _col3
+                    Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col3 (type: string)
+                      outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
-                      table:
-                          input format: org.apache.hadoop.mapred.TextInputFormat
-                          output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                          name: default.outputtbl3
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                            name: default.outputtbl3
         Reducer 4 
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col3 (type: string)
-                outputColumnNames: _col0, _col1, _col3
+                expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
                 Limit
                   Number of rows: 1000
                   Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col3 (type: string)
-                    outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
-                    File Output Operator
-                      compressed: false
+                    expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+                    outputColumnNames: _col0, _col1, _col3
+                    Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col3 (type: string)
+                      outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
-                      table:
-                          input format: org.apache.hadoop.mapred.TextInputFormat
-                          output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                          name: default.outputtbl3
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                            name: default.outputtbl3
 
   Stage: Stage-0
     Move Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/b04dc95f/ql/src/test/results/clientpositive/union_remove_25.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_25.q.out b/ql/src/test/results/clientpositive/union_remove_25.q.out
index 54ddf56..3869735 100644
--- a/ql/src/test/results/clientpositive/union_remove_25.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_25.q.out
@@ -461,7 +461,7 @@ STAGE PLANS:
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string), hr (type: string)
-              outputColumnNames: _col0, _col1, _col3
+              outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
               Limit
                 Number of rows: 1000
@@ -470,17 +470,17 @@ STAGE PLANS:
                   sort order: 
                   Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
                   TopN Hash Memory Usage: 0.1
-                  value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+                  value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
       Reduce Operator Tree:
         Select Operator
-          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col3 (type: string)
-          outputColumnNames: _col0, _col1, _col3
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
+          outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
           Limit
             Number of rows: 1000
             Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col3 (type: string)
+              expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col2 (type: string)
               outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
@@ -513,7 +513,7 @@ STAGE PLANS:
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string), hr (type: string)
-              outputColumnNames: _col0, _col1, _col3
+              outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
               Limit
                 Number of rows: 1000
@@ -522,17 +522,17 @@ STAGE PLANS:
                   sort order: 
                   Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
                   TopN Hash Memory Usage: 0.1
-                  value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+                  value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
       Reduce Operator Tree:
         Select Operator
-          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col3 (type: string)
-          outputColumnNames: _col0, _col1, _col3
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
+          outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
           Limit
             Number of rows: 1000
             Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col3 (type: string)
+              expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col2 (type: string)
               outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
               File Output Operator


[03/50] [abbrv] hive git commit: HIVE-13445 : LLAP: token should encode application and cluster ids (Sergey Shelukhin, reviewed by Siddharth Seth)

Posted by sp...@apache.org.
HIVE-13445 : LLAP: token should encode application and cluster ids (Sergey Shelukhin, reviewed by Siddharth Seth)


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

Branch: refs/heads/java8
Commit: 868e5e141856ce75af48d854d9e3eb13372d11f4
Parents: b621827
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue May 3 12:01:32 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue May 3 13:38:03 2016 -0700

----------------------------------------------------------------------
 .../daemon/rpc/LlapDaemonProtocolProtos.java    | 209 +++++++++++++++++--
 .../org/apache/hadoop/hive/llap/DaemonId.java   |  41 ++++
 .../hive/llap/security/LlapTokenIdentifier.java |  39 +++-
 .../hive/llap/security/LlapTokenProvider.java   |   2 +-
 .../src/protobuf/LlapDaemonProtocol.proto       |   1 +
 .../hive/llap/daemon/ContainerRunner.java       |   9 +-
 .../llap/daemon/impl/ContainerRunnerImpl.java   |  47 +++--
 .../hive/llap/daemon/impl/LlapDaemon.java       |  52 ++++-
 .../daemon/impl/LlapProtocolServerImpl.java     |  41 ++--
 .../hive/llap/daemon/impl/LlapTokenChecker.java | 137 ++++++++++++
 .../hadoop/hive/llap/daemon/impl/QueryInfo.java |  17 +-
 .../hive/llap/daemon/impl/QueryTracker.java     |  85 +++++---
 .../hadoop/hive/llap/daemon/impl/Scheduler.java |   2 +
 .../llap/daemon/impl/TaskExecutorService.java   |   9 +
 .../hive/llap/security/LlapSecurityHelper.java  |  15 +-
 .../hive/llap/security/SecretManager.java       |  19 +-
 .../hive/llap/daemon/MiniLlapCluster.java       |   2 +-
 .../daemon/impl/TaskExecutorTestHelpers.java    |   2 +-
 .../impl/TestLlapDaemonProtocolServerImpl.java  |   2 +-
 .../llap/daemon/impl/TestLlapTokenChecker.java  |  96 +++++++++
 .../hive/ql/exec/tez/TezSessionState.java       |   3 +-
 21 files changed, 702 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
----------------------------------------------------------------------
diff --git a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
index 4ab7b32..820f6be 100644
--- a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
+++ b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
@@ -12821,6 +12821,21 @@ public final class LlapDaemonProtocolProtos {
 
   public interface GetTokenRequestProtoOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
+
+    // optional string app_id = 1;
+    /**
+     * <code>optional string app_id = 1;</code>
+     */
+    boolean hasAppId();
+    /**
+     * <code>optional string app_id = 1;</code>
+     */
+    java.lang.String getAppId();
+    /**
+     * <code>optional string app_id = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getAppIdBytes();
   }
   /**
    * Protobuf type {@code GetTokenRequestProto}
@@ -12855,6 +12870,7 @@ public final class LlapDaemonProtocolProtos {
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
+      int mutable_bitField0_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -12872,6 +12888,11 @@ public final class LlapDaemonProtocolProtos {
               }
               break;
             }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              appId_ = input.readBytes();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -12911,7 +12932,52 @@ public final class LlapDaemonProtocolProtos {
       return PARSER;
     }
 
+    private int bitField0_;
+    // optional string app_id = 1;
+    public static final int APP_ID_FIELD_NUMBER = 1;
+    private java.lang.Object appId_;
+    /**
+     * <code>optional string app_id = 1;</code>
+     */
+    public boolean hasAppId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional string app_id = 1;</code>
+     */
+    public java.lang.String getAppId() {
+      java.lang.Object ref = appId_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          appId_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string app_id = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getAppIdBytes() {
+      java.lang.Object ref = appId_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        appId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
     private void initFields() {
+      appId_ = "";
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -12925,6 +12991,9 @@ public final class LlapDaemonProtocolProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getAppIdBytes());
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -12934,6 +13003,10 @@ public final class LlapDaemonProtocolProtos {
       if (size != -1) return size;
 
       size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getAppIdBytes());
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -12957,6 +13030,11 @@ public final class LlapDaemonProtocolProtos {
       org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto) obj;
 
       boolean result = true;
+      result = result && (hasAppId() == other.hasAppId());
+      if (hasAppId()) {
+        result = result && getAppId()
+            .equals(other.getAppId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -12970,6 +13048,10 @@ public final class LlapDaemonProtocolProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasAppId()) {
+        hash = (37 * hash) + APP_ID_FIELD_NUMBER;
+        hash = (53 * hash) + getAppId().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -13079,6 +13161,8 @@ public final class LlapDaemonProtocolProtos {
 
       public Builder clear() {
         super.clear();
+        appId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -13105,6 +13189,13 @@ public final class LlapDaemonProtocolProtos {
 
       public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto buildPartial() {
         org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto result = new org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.appId_ = appId_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -13120,6 +13211,11 @@ public final class LlapDaemonProtocolProtos {
 
       public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto other) {
         if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto.getDefaultInstance()) return this;
+        if (other.hasAppId()) {
+          bitField0_ |= 0x00000001;
+          appId_ = other.appId_;
+          onChanged();
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -13145,6 +13241,81 @@ public final class LlapDaemonProtocolProtos {
         }
         return this;
       }
+      private int bitField0_;
+
+      // optional string app_id = 1;
+      private java.lang.Object appId_ = "";
+      /**
+       * <code>optional string app_id = 1;</code>
+       */
+      public boolean hasAppId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional string app_id = 1;</code>
+       */
+      public java.lang.String getAppId() {
+        java.lang.Object ref = appId_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          appId_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string app_id = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getAppIdBytes() {
+        java.lang.Object ref = appId_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          appId_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string app_id = 1;</code>
+       */
+      public Builder setAppId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        appId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string app_id = 1;</code>
+       */
+      public Builder clearAppId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        appId_ = getDefaultInstance().getAppId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string app_id = 1;</code>
+       */
+      public Builder setAppIdBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        appId_ = value;
+        onChanged();
+        return this;
+      }
 
       // @@protoc_insertion_point(builder_scope:GetTokenRequestProto)
     }
@@ -14414,24 +14585,24 @@ public final class LlapDaemonProtocolProtos {
       "RequestProto\022/\n\020query_identifier\030\001 \001(\0132\025" +
       ".QueryIdentifierProto\022\"\n\032fragment_identi" +
       "fier_string\030\002 \001(\t\" \n\036TerminateFragmentRe" +
-      "sponseProto\"\026\n\024GetTokenRequestProto\"&\n\025G",
-      "etTokenResponseProto\022\r\n\005token\030\001 \001(\014*2\n\020S" +
-      "ourceStateProto\022\017\n\013S_SUCCEEDED\020\001\022\r\n\tS_RU" +
-      "NNING\020\002*E\n\024SubmissionStateProto\022\014\n\010ACCEP" +
-      "TED\020\001\022\014\n\010REJECTED\020\002\022\021\n\rEVICTED_OTHER\020\0032\316" +
-      "\002\n\022LlapDaemonProtocol\022?\n\nsubmitWork\022\027.Su" +
-      "bmitWorkRequestProto\032\030.SubmitWorkRespons" +
-      "eProto\022W\n\022sourceStateUpdated\022\037.SourceSta" +
-      "teUpdatedRequestProto\032 .SourceStateUpdat" +
-      "edResponseProto\022H\n\rqueryComplete\022\032.Query" +
-      "CompleteRequestProto\032\033.QueryCompleteResp",
-      "onseProto\022T\n\021terminateFragment\022\036.Termina" +
-      "teFragmentRequestProto\032\037.TerminateFragme" +
-      "ntResponseProto2]\n\026LlapManagementProtoco" +
-      "l\022C\n\022getDelegationToken\022\025.GetTokenReques" +
-      "tProto\032\026.GetTokenResponseProtoBH\n&org.ap" +
-      "ache.hadoop.hive.llap.daemon.rpcB\030LlapDa" +
-      "emonProtocolProtos\210\001\001\240\001\001"
+      "sponseProto\"&\n\024GetTokenRequestProto\022\016\n\006a",
+      "pp_id\030\001 \001(\t\"&\n\025GetTokenResponseProto\022\r\n\005" +
+      "token\030\001 \001(\014*2\n\020SourceStateProto\022\017\n\013S_SUC" +
+      "CEEDED\020\001\022\r\n\tS_RUNNING\020\002*E\n\024SubmissionSta" +
+      "teProto\022\014\n\010ACCEPTED\020\001\022\014\n\010REJECTED\020\002\022\021\n\rE" +
+      "VICTED_OTHER\020\0032\316\002\n\022LlapDaemonProtocol\022?\n" +
+      "\nsubmitWork\022\027.SubmitWorkRequestProto\032\030.S" +
+      "ubmitWorkResponseProto\022W\n\022sourceStateUpd" +
+      "ated\022\037.SourceStateUpdatedRequestProto\032 ." +
+      "SourceStateUpdatedResponseProto\022H\n\rquery" +
+      "Complete\022\032.QueryCompleteRequestProto\032\033.Q",
+      "ueryCompleteResponseProto\022T\n\021terminateFr" +
+      "agment\022\036.TerminateFragmentRequestProto\032\037" +
+      ".TerminateFragmentResponseProto2]\n\026LlapM" +
+      "anagementProtocol\022C\n\022getDelegationToken\022" +
+      "\025.GetTokenRequestProto\032\026.GetTokenRespons" +
+      "eProtoBH\n&org.apache.hadoop.hive.llap.da" +
+      "emon.rpcB\030LlapDaemonProtocolProtos\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -14533,7 +14704,7 @@ public final class LlapDaemonProtocolProtos {
           internal_static_GetTokenRequestProto_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_GetTokenRequestProto_descriptor,
-              new java.lang.String[] { });
+              new java.lang.String[] { "AppId", });
           internal_static_GetTokenResponseProto_descriptor =
             getDescriptor().getMessageTypes().get(16);
           internal_static_GetTokenResponseProto_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-common/src/java/org/apache/hadoop/hive/llap/DaemonId.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/DaemonId.java b/llap-common/src/java/org/apache/hadoop/hive/llap/DaemonId.java
new file mode 100644
index 0000000..18355e6
--- /dev/null
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/DaemonId.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.llap;
+
+public class DaemonId {
+  private final String userName;
+  private final String clusterName;
+  private final String appId;
+  private final String hostName;
+  private final long startTime;
+
+  public DaemonId(String userName, String clusterName, String hostName, String appId,
+      long startTime) {
+    this.userName = userName;
+    this.clusterName = clusterName;
+    this.appId = appId;
+    this.hostName = hostName;
+    this.startTime = startTime;
+    // TODO: we could also get an unique number per daemon.
+  }
+
+  public String getClusterString() {
+    return userName + "_" + clusterName + "_" + appId;
+  }
+
+  public String getApplicationId() {
+    return appId;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenIdentifier.java b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenIdentifier.java
index 23980d0..e28eddd 100644
--- a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenIdentifier.java
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenIdentifier.java
@@ -22,6 +22,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.Token;
@@ -31,25 +32,32 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti
 public class LlapTokenIdentifier extends AbstractDelegationTokenIdentifier {
   private static final String KIND = "LLAP_TOKEN";
   public static final Text KIND_NAME = new Text(KIND);
+  private String clusterId;
+  private String appId;
 
   public LlapTokenIdentifier() {
     super();
   }
 
-  public LlapTokenIdentifier(Text owner, Text renewer, Text realUser) {
+  public LlapTokenIdentifier(Text owner, Text renewer, Text realUser,
+      String clusterId, String appId) {
     super(owner, renewer, realUser);
+    this.clusterId = clusterId;
+    this.appId = appId == null ? "" : appId;
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
-    // Nothing right now.
+    out.writeUTF(clusterId);
+    out.writeUTF(appId);
   }
 
   @Override
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
-    // Nothing right now.
+    clusterId = in.readUTF();
+    appId = in.readUTF();
   }
 
   @Override
@@ -57,21 +65,34 @@ public class LlapTokenIdentifier extends AbstractDelegationTokenIdentifier {
     return KIND_NAME;
   }
 
+  public String getAppId() {
+    return appId;
+  }
+
+  public String getClusterId() {
+    return clusterId;
+  }
+
   @Override
   public int hashCode() {
-    // Nothing else right now.
-    return super.hashCode();
+    final int prime = 31;
+    int result = prime * super.hashCode() + ((appId == null) ? 0 : appId.hashCode());
+    return prime * result + ((clusterId == null) ? 0 : clusterId.hashCode());
   }
 
   @Override
-  public boolean equals(Object other) {
-    // Nothing else right now.
-    return super.equals(other);
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (!(obj instanceof LlapTokenIdentifier) || !super.equals(obj)) return false;
+    LlapTokenIdentifier other = (LlapTokenIdentifier) obj;
+    return (appId == null ? other.appId == null : appId.equals(other.appId))
+        && (clusterId == null ? other.clusterId == null : clusterId.equals(other.clusterId));
   }
 
   @Override
   public String toString() {
-    return KIND + "; " + super.toString();
+    return KIND + "; " + super.toString() + ", cluster " + clusterId + ", app secret hash "
+        + (StringUtils.isBlank(appId) ? 0 : appId.hashCode());
   }
 
   @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenProvider.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenProvider.java b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenProvider.java
index 2e99a28..edf9b18 100644
--- a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenProvider.java
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenProvider.java
@@ -23,5 +23,5 @@ import java.io.IOException;
 import org.apache.hadoop.security.token.Token;
 
 public interface LlapTokenProvider {
-  Token<LlapTokenIdentifier> getDelegationToken() throws IOException;
+  Token<LlapTokenIdentifier> getDelegationToken(String appId) throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-common/src/protobuf/LlapDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/llap-common/src/protobuf/LlapDaemonProtocol.proto b/llap-common/src/protobuf/LlapDaemonProtocol.proto
index 944c96c..5cdc02e 100644
--- a/llap-common/src/protobuf/LlapDaemonProtocol.proto
+++ b/llap-common/src/protobuf/LlapDaemonProtocol.proto
@@ -130,6 +130,7 @@ message TerminateFragmentResponseProto {
 }
 
 message GetTokenRequestProto {
+  optional string app_id = 1;
 }
 
 message GetTokenResponseProto {

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/ContainerRunner.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/ContainerRunner.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/ContainerRunner.java
index fc29371..c346aed 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/ContainerRunner.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/ContainerRunner.java
@@ -29,9 +29,12 @@ public interface ContainerRunner {
 
   SubmitWorkResponseProto submitWork(SubmitWorkRequestProto request) throws IOException;
 
-  SourceStateUpdatedResponseProto sourceStateUpdated(SourceStateUpdatedRequestProto request);
+  SourceStateUpdatedResponseProto sourceStateUpdated(
+      SourceStateUpdatedRequestProto request) throws IOException;
 
-  QueryCompleteResponseProto queryComplete(QueryCompleteRequestProto request);
+  QueryCompleteResponseProto queryComplete(
+      QueryCompleteRequestProto request) throws IOException;
 
-  TerminateFragmentResponseProto terminateFragment(TerminateFragmentRequestProto request);
+  TerminateFragmentResponseProto terminateFragment(
+      TerminateFragmentRequestProto request)  throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
index 3d45c7a..78b37f7 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
@@ -92,7 +92,7 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
       boolean enablePreemption, String[] localDirsBase, AtomicReference<Integer> localShufflePort,
       AtomicReference<InetSocketAddress> localAddress,
       long totalMemoryAvailableBytes, LlapDaemonExecutorMetrics metrics,
-      AMReporter amReporter, ClassLoader classLoader) {
+      AMReporter amReporter, ClassLoader classLoader, String clusterId) {
     super("ContainerRunnerImpl");
     this.conf = conf;
     Preconditions.checkState(numExecutors > 0,
@@ -101,7 +101,7 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
     this.localShufflePort = localShufflePort;
     this.amReporter = amReporter;
 
-    this.queryTracker = new QueryTracker(conf, localDirsBase);
+    this.queryTracker = new QueryTracker(conf, localDirsBase, clusterId);
     addIfService(queryTracker);
     String waitQueueSchedulerClassName = HiveConf.getVar(
         conf, ConfVars.LLAP_DAEMON_WAIT_QUEUE_COMPARATOR_CLASS_NAME);
@@ -175,7 +175,8 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
           fragmentSpec.getFragmentIdentifierString());
       int dagIdentifier = taskAttemptId.getTaskID().getVertexID().getDAGId().getId();
 
-      QueryIdentifier queryIdentifier = new QueryIdentifier(request.getApplicationIdString(), dagIdentifier);
+      QueryIdentifier queryIdentifier = new QueryIdentifier(
+          request.getApplicationIdString(), dagIdentifier);
 
       Credentials credentials = new Credentials();
       DataInputBuffer dib = new DataInputBuffer();
@@ -193,6 +194,7 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
               fragmentSpec.getAttemptNumber(), request.getUser(), request.getFragmentSpec(),
               jobToken);
 
+
       String[] localDirs = fragmentInfo.getLocalDirs();
       Preconditions.checkNotNull(localDirs);
       if (LOG.isDebugEnabled()) {
@@ -200,7 +202,6 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
       }
       // May need to setup localDir for re-localization, which is usually setup as Environment.PWD.
       // Used for re-localization, to add the user specified configuration (conf_pb_binary_stream)
-
       TaskRunnerCallable callable = new TaskRunnerCallable(request, fragmentInfo, new Configuration(getConfig()),
           new LlapExecutionContext(localAddress.get().getHostName(), queryTracker), env,
           credentials, memoryPerExecutor, amReporter, confParams, metrics, killedTaskHandler,
@@ -248,24 +249,23 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
 
   @Override
   public SourceStateUpdatedResponseProto sourceStateUpdated(
-      SourceStateUpdatedRequestProto request) {
+      SourceStateUpdatedRequestProto request) throws IOException {
     LOG.info("Processing state update: " + stringifySourceStateUpdateRequest(request));
-    queryTracker.registerSourceStateChange(
-        new QueryIdentifier(request.getQueryIdentifier().getAppIdentifier(),
-            request.getQueryIdentifier().getDagIdentifier()), request.getSrcName(),
-        request.getState());
+    QueryIdentifier queryId = new QueryIdentifier(request.getQueryIdentifier().getAppIdentifier(),
+        request.getQueryIdentifier().getDagIdentifier());
+    queryTracker.registerSourceStateChange(queryId, request.getSrcName(), request.getState());
     return SourceStateUpdatedResponseProto.getDefaultInstance();
   }
 
   @Override
-  public QueryCompleteResponseProto queryComplete(QueryCompleteRequestProto request) {
+  public QueryCompleteResponseProto queryComplete(
+      QueryCompleteRequestProto request) throws IOException {
     QueryIdentifier queryIdentifier =
         new QueryIdentifier(request.getQueryIdentifier().getAppIdentifier(),
             request.getQueryIdentifier().getDagIdentifier());
     LOG.info("Processing queryComplete notification for {}", queryIdentifier);
-    List<QueryFragmentInfo> knownFragments =
-        queryTracker
-            .queryComplete(queryIdentifier, request.getDeleteDelay());
+    List<QueryFragmentInfo> knownFragments = queryTracker.queryComplete(
+        queryIdentifier, request.getDeleteDelay(), false);
     LOG.info("DBG: Pending fragment count for completed query {} = {}", queryIdentifier,
         knownFragments.size());
     for (QueryFragmentInfo fragmentInfo : knownFragments) {
@@ -277,9 +277,16 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
   }
 
   @Override
-  public TerminateFragmentResponseProto terminateFragment(TerminateFragmentRequestProto request) {
-    LOG.info("DBG: Received terminateFragment request for {}", request.getFragmentIdentifierString());
-    executorService.killFragment(request.getFragmentIdentifierString());
+  public TerminateFragmentResponseProto terminateFragment(
+      TerminateFragmentRequestProto request) throws IOException {
+    String fragmentId = request.getFragmentIdentifierString();
+    LOG.info("DBG: Received terminateFragment request for {}", fragmentId);
+    // TODO: ideally, QueryTracker should have fragment-to-query mapping.
+    QueryIdentifier queryId = executorService.findQueryByFragment(fragmentId);
+    // checkPermissions returns false if query is not found, throws on failure.
+    if (queryId != null && queryTracker.checkPermissionsForQuery(queryId)) {
+      executorService.killFragment(fragmentId);
+    }
     return TerminateFragmentResponseProto.getDefaultInstance();
   }
 
@@ -355,8 +362,12 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
   @Override
   public void queryFailed(QueryIdentifier queryIdentifier) {
     LOG.info("Processing query failed notification for {}", queryIdentifier);
-    List<QueryFragmentInfo> knownFragments =
-        queryTracker.queryComplete(queryIdentifier, -1);
+    List<QueryFragmentInfo> knownFragments;
+    try {
+      knownFragments = queryTracker.queryComplete(queryIdentifier, -1, true);
+    } catch (IOException e) {
+      throw new RuntimeException(e); // Should never happen here, no permission check.
+    }
     LOG.info("DBG: Pending fragment count for failed query {} = {}", queryIdentifier,
         knownFragments.size());
     for (QueryFragmentInfo fragmentInfo : knownFragments) {

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index 63cb16b..d23a44a 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -26,12 +26,14 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
 
 import javax.management.ObjectName;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.DaemonId;
 import org.apache.hadoop.hive.llap.LlapUtil;
 import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration;
 import org.apache.hadoop.hive.llap.daemon.ContainerRunner;
@@ -57,11 +59,13 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge.UdfWhitelistChecker;
 import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.JvmPauseMonitor;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hive.common.util.ShutdownHookManager;
 import org.apache.logging.log4j.core.config.Configurator;
 import org.slf4j.Logger;
@@ -97,6 +101,13 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
   private final int numExecutors;
   private final long maxJvmMemory;
   private final String[] localDirs;
+  private final DaemonId daemonId;
+
+  private final static Pattern hostsRe = Pattern.compile("[^A-Za-z0-9_-]");
+  private static String generateClusterName(Configuration conf) {
+    String hosts = HiveConf.getTrimmedVar(conf, ConfVars.LLAP_DAEMON_SERVICE_HOSTS);
+    return hostsRe.matcher(hosts.startsWith("@") ? hosts.substring(1) : hosts).replaceAll("_");
+  }
 
   // TODO Not the best way to share the address
   private final AtomicReference<InetSocketAddress> srvAddress = new AtomicReference<>(),
@@ -105,11 +116,10 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
 
   public LlapDaemon(Configuration daemonConf, int numExecutors, long executorMemoryBytes,
       boolean ioEnabled, boolean isDirectCache, long ioMemoryBytes, String[] localDirs, int srvPort,
-      int mngPort, int shufflePort, int webPort) {
+      int mngPort, int shufflePort, int webPort, String appName) {
     super("LlapDaemon");
 
     initializeLogging();
-
     printAsciiArt();
 
     Preconditions.checkArgument(numExecutors > 0);
@@ -129,6 +139,14 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
           "LLAP service hosts startswith '@' but hive.zookeeper.quorum is not set." +
               " hive.zookeeper.quorum must be set.");
     }
+    String hostName = MetricsUtils.getHostName();
+    try {
+      daemonId = new DaemonId(UserGroupInformation.getCurrentUser().getUserName(),
+          generateClusterName(daemonConf), hostName, appName, System.currentTimeMillis());
+    } catch (IOException ex) {
+      throw new RuntimeException(ex);
+    }
+
 
     this.maxJvmMemory = getTotalHeapSize();
     this.llapIoEnabled = ioEnabled;
@@ -193,7 +211,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     LlapMetricsSystem.initialize("LlapDaemon");
     this.pauseMonitor = new JvmPauseMonitor(daemonConf);
     pauseMonitor.start();
-    String displayName = "LlapDaemonExecutorMetrics-" + MetricsUtils.getHostName();
+    String displayName = "LlapDaemonExecutorMetrics-" + hostName;
     String sessionId = MetricsUtils.getUUID();
     daemonConf.set("llap.daemon.metrics.sessionid", sessionId);
     String[] strIntervals = HiveConf.getTrimmedStringsVar(daemonConf,
@@ -223,11 +241,11 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     this.amReporter = new AMReporter(srvAddress, new QueryFailedHandlerProxy(), daemonConf);
 
     this.server = new LlapProtocolServerImpl(
-        numHandlers, this, srvAddress, mngAddress, srvPort, mngPort);
+        numHandlers, this, srvAddress, mngAddress, srvPort, mngPort, daemonId);
 
     this.containerRunner = new ContainerRunnerImpl(daemonConf, numExecutors, waitQueueSize,
         enablePreemption, localDirs, this.shufflePort, srvAddress, executorMemoryBytes, metrics,
-        amReporter, executorClassLoader);
+        amReporter, executorClassLoader, daemonId.getClusterString());
     addIfService(containerRunner);
 
     // Not adding the registry as a service, since we need to control when it is initialized - conf used to pickup properties.
@@ -377,10 +395,18 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       LlapDaemonConfiguration daemonConf = new LlapDaemonConfiguration();
 
       String containerIdStr = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name());
+      String appName = null;
       if (containerIdStr != null && !containerIdStr.isEmpty()) {
         daemonConf.set(ConfVars.LLAP_DAEMON_CONTAINER_ID.varname, containerIdStr);
+        appName = ConverterUtils.toContainerId(containerIdStr)
+            .getApplicationAttemptId().getApplicationId().toString();
       } else {
         daemonConf.unset(ConfVars.LLAP_DAEMON_CONTAINER_ID.varname);
+        // Note, we assume production LLAP always runs under YARN.
+        LOG.error("Cannot find " + ApplicationConstants.Environment.CONTAINER_ID.toString()
+            + "; LLAP tokens may grant access to subsequent instances of the cluster with"
+            + " the same name");
+        appName = null;
       }
 
       int numExecutors = HiveConf.getIntVar(daemonConf, ConfVars.LLAP_DAEMON_NUM_EXECUTORS);
@@ -400,7 +426,8 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       boolean isDirectCache = HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_ALLOCATOR_DIRECT);
       boolean isLlapIo = HiveConf.getBoolVar(daemonConf, HiveConf.ConfVars.LLAP_IO_ENABLED, true);
       llapDaemon = new LlapDaemon(daemonConf, numExecutors, executorMemoryBytes, isLlapIo,
-              isDirectCache, ioMemoryBytes, localDirs, rpcPort, mngPort, shufflePort, webPort);
+          isDirectCache, ioMemoryBytes, localDirs, rpcPort, mngPort, shufflePort, webPort,
+          appName);
 
       LOG.info("Adding shutdown hook for LlapDaemon");
       ShutdownHookManager.addShutdownHook(new CompositeServiceShutdownHook(llapDaemon), 1);
@@ -420,24 +447,27 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
   }
 
   @Override
-  public SubmitWorkResponseProto submitWork(SubmitWorkRequestProto request) throws
-      IOException {
+  public SubmitWorkResponseProto submitWork(
+      SubmitWorkRequestProto request) throws IOException {
     numSubmissions.incrementAndGet();
     return containerRunner.submitWork(request);
   }
 
   @Override
-  public SourceStateUpdatedResponseProto sourceStateUpdated(SourceStateUpdatedRequestProto request) {
+  public SourceStateUpdatedResponseProto sourceStateUpdated(
+      SourceStateUpdatedRequestProto request) throws IOException {
     return containerRunner.sourceStateUpdated(request);
   }
 
   @Override
-  public QueryCompleteResponseProto queryComplete(QueryCompleteRequestProto request) {
+  public QueryCompleteResponseProto queryComplete(
+      QueryCompleteRequestProto request) throws IOException {
     return containerRunner.queryComplete(request);
   }
 
   @Override
-  public TerminateFragmentResponseProto terminateFragment(TerminateFragmentRequestProto request) {
+  public TerminateFragmentResponseProto terminateFragment(
+      TerminateFragmentRequestProto request) throws IOException {
     return containerRunner.terminateFragment(request);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
index dae1a3a..db8bfa6 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.DaemonId;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenResponseProto;
@@ -71,13 +72,11 @@ public class LlapProtocolServerImpl extends AbstractService
   private final AtomicReference<InetSocketAddress> srvAddress, mngAddress;
   private SecretManager zkSecretManager;
   private String restrictedToUser = null;
+  private final DaemonId daemonId;
 
-  public LlapProtocolServerImpl(int numHandlers,
-                                ContainerRunner containerRunner,
-                                AtomicReference<InetSocketAddress> srvAddress,
-                                AtomicReference<InetSocketAddress> mngAddress,
-                                int srvPort,
-                                int mngPort) {
+  public LlapProtocolServerImpl(int numHandlers, ContainerRunner containerRunner,
+      AtomicReference<InetSocketAddress> srvAddress, AtomicReference<InetSocketAddress> mngAddress,
+      int srvPort, int mngPort, DaemonId daemonId) {
     super("LlapDaemonProtocolServerImpl");
     this.numHandlers = numHandlers;
     this.containerRunner = containerRunner;
@@ -85,14 +84,14 @@ public class LlapProtocolServerImpl extends AbstractService
     this.srvPort = srvPort;
     this.mngAddress = mngAddress;
     this.mngPort = mngPort;
+    this.daemonId = daemonId;
     LOG.info("Creating: " + LlapProtocolServerImpl.class.getSimpleName() +
         " with port configured to: " + srvPort);
   }
 
   @Override
   public SubmitWorkResponseProto submitWork(RpcController controller,
-                                            SubmitWorkRequestProto request) throws
-      ServiceException {
+      SubmitWorkRequestProto request) throws ServiceException {
     try {
       return containerRunner.submitWork(request);
     } catch (IOException e) {
@@ -103,20 +102,31 @@ public class LlapProtocolServerImpl extends AbstractService
   @Override
   public SourceStateUpdatedResponseProto sourceStateUpdated(RpcController controller,
       SourceStateUpdatedRequestProto request) throws ServiceException {
-    return containerRunner.sourceStateUpdated(request);
+    try {
+      return containerRunner.sourceStateUpdated(request);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
   }
 
   @Override
   public QueryCompleteResponseProto queryComplete(RpcController controller,
       QueryCompleteRequestProto request) throws ServiceException {
-    return containerRunner.queryComplete(request);
+    try {
+      return containerRunner.queryComplete(request);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
   }
 
   @Override
   public TerminateFragmentResponseProto terminateFragment(
-      RpcController controller,
-      TerminateFragmentRequestProto request) throws ServiceException {
-    return containerRunner.terminateFragment(request);
+      RpcController controller, TerminateFragmentRequestProto request) throws ServiceException {
+    try {
+      return containerRunner.terminateFragment(request);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
   }
 
   @Override
@@ -140,7 +150,7 @@ public class LlapProtocolServerImpl extends AbstractService
     }
     String llapPrincipal = HiveConf.getVar(conf, ConfVars.LLAP_KERBEROS_PRINCIPAL),
         llapKeytab = HiveConf.getVar(conf, ConfVars.LLAP_KERBEROS_KEYTAB_FILE);
-    zkSecretManager = SecretManager.createSecretManager(conf, llapPrincipal, llapKeytab);
+    zkSecretManager = SecretManager.createSecretManager(conf, llapPrincipal, llapKeytab, daemonId);
 
     // Start the protocol server after properly authenticating with daemon keytab.
     UserGroupInformation daemonUgi = null;
@@ -275,7 +285,8 @@ public class LlapProtocolServerImpl extends AbstractService
       realUser = new Text(ugi.getRealUser().getUserName());
     }
     Text renewer = new Text(ugi.getShortUserName());
-    LlapTokenIdentifier llapId = new LlapTokenIdentifier(owner, renewer, realUser);
+    LlapTokenIdentifier llapId = new LlapTokenIdentifier(owner, renewer, realUser,
+        daemonId.getClusterString(), request.hasAppId() ? request.getAppId() : null);
     // TODO: note that the token is not renewable right now and will last for 2 weeks by default.
     Token<LlapTokenIdentifier> token = new Token<LlapTokenIdentifier>(llapId, zkSecretManager);
     if (LOG.isInfoEnabled()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTokenChecker.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTokenChecker.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTokenChecker.java
new file mode 100644
index 0000000..03ee055
--- /dev/null
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTokenChecker.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.llap.daemon.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import java.util.ArrayList;
+
+import java.util.List;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hive.llap.security.LlapTokenIdentifier;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class LlapTokenChecker {
+  private static final Logger LOG = LoggerFactory.getLogger(LlapTokenChecker.class);
+
+  private static final ImmutablePair<String, String> NO_SECURITY = new ImmutablePair<>(null, null);
+  public static Pair<String, String> getTokenInfo(String clusterId) throws IOException {
+    if (!UserGroupInformation.isSecurityEnabled()) return NO_SECURITY;
+    UserGroupInformation current = UserGroupInformation.getCurrentUser();
+    String kerberosName = current.hasKerberosCredentials() ? current.getShortUserName() : null;
+    List<LlapTokenIdentifier> tokens = getLlapTokens(current, clusterId);
+    if ((tokens == null || tokens.isEmpty()) && kerberosName == null) {
+      throw new SecurityException("No tokens or kerberos for " + current);
+    }
+    return getTokenInfoInternal(kerberosName, tokens);
+  }
+
+  private static List<LlapTokenIdentifier> getLlapTokens(
+      UserGroupInformation ugi, String clusterId) {
+    List<LlapTokenIdentifier> tokens = null;
+    for (TokenIdentifier id : ugi.getTokenIdentifiers()) {
+      if (!LlapTokenIdentifier.KIND_NAME.equals(id.getKind())) continue;
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Token {}", id);
+      }
+      LlapTokenIdentifier llapId = (LlapTokenIdentifier)id;
+      if (!clusterId.equals(llapId.getClusterId())) continue;
+      if (tokens == null) {
+        tokens = new ArrayList<>();
+      }
+      tokens.add((LlapTokenIdentifier)id);
+    }
+    return tokens;
+  }
+
+  @VisibleForTesting
+  static Pair<String, String> getTokenInfoInternal(
+      String kerberosName, List<LlapTokenIdentifier> tokens) {
+    assert (tokens != null && !tokens.isEmpty()) || kerberosName != null;
+    if (tokens == null) {
+      return new ImmutablePair<String, String>(kerberosName, null);
+    }
+    String userName = kerberosName, appId = null;
+    for (LlapTokenIdentifier llapId : tokens) {
+      String newUserName = llapId.getRealUser().toString();
+      if (userName != null && !userName.equals(newUserName)) {
+        throw new SecurityException("Ambiguous user name from credentials - " + userName
+            + " and " + newUserName + " from " + llapId
+            + ((kerberosName == null) ? ("; has kerberos credentials for " + kerberosName) : ""));
+      }
+      userName = newUserName;
+      String newAppId = llapId.getAppId();
+      if (!StringUtils.isEmpty(newAppId)) {
+        if (!StringUtils.isEmpty(appId) && !appId.equals(newAppId)) {
+          throw new SecurityException("Ambiguous app ID from credentials - " + appId
+              + " and " + newAppId + " from " + llapId);
+        }
+        appId = newAppId;
+      }
+    }
+    assert userName != null;
+    return new ImmutablePair<String, String>(userName, appId);
+  }
+
+  public static void checkPermissions(
+      String clusterId, String userName, String appId, Object hint) throws IOException {
+    if (!UserGroupInformation.isSecurityEnabled()) return;
+    UserGroupInformation current = UserGroupInformation.getCurrentUser();
+    String kerberosName = current.hasKerberosCredentials() ? current.getShortUserName() : null;
+    List<LlapTokenIdentifier> tokens = getLlapTokens(current, clusterId);
+    checkPermissionsInternal(kerberosName, tokens, userName, appId, hint);
+  }
+
+  @VisibleForTesting
+  static void checkPermissionsInternal(String kerberosName, List<LlapTokenIdentifier> tokens,
+      String userName, String appId, Object hint) {
+    if (kerberosName != null && StringUtils.isEmpty(appId) && kerberosName.equals(userName)) {
+      return;
+    }
+    if (tokens != null) {
+      for (LlapTokenIdentifier llapId : tokens) {
+        String tokenUser = llapId.getRealUser().toString(), tokenAppId = llapId.getAppId();
+        if (checkTokenPermissions(userName, appId, tokenUser, tokenAppId)) return;
+      }
+    }
+    throw new SecurityException("Unauthorized to access "
+        + userName + ", " + appId.hashCode() + " (" + hint + ")");
+  }
+
+  public static void checkPermissions(
+      Pair<String, String> prm, String userName, String appId, Object hint) {
+    if (userName == null) {
+      assert StringUtils.isEmpty(appId);
+      return;
+    }
+    if (!checkTokenPermissions(userName, appId, prm.getLeft(), prm.getRight())) {
+      throw new SecurityException("Unauthorized to access "
+          + userName + ", " + appId.hashCode() + " (" + hint + ")");
+    }
+  }
+
+  private static boolean checkTokenPermissions(
+      String userName, String appId, String tokenUser, String tokenAppId) {
+    return userName.equals(tokenUser)
+        && (StringUtils.isEmpty(appId) || appId.equals(tokenAppId));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
index 64c2b58..8daef9e 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
@@ -55,10 +55,11 @@ public class QueryInfo {
   private final ConcurrentMap<String, SourceStateProto> sourceStateMap;
 
   private final FinishableStateTracker finishableStateTracker = new FinishableStateTracker();
+  private final String tokenUserName, appId;
 
-  public QueryInfo(QueryIdentifier queryIdentifier, String appIdString, String dagName, int dagIdentifier,
-                   String user, ConcurrentMap<String, SourceStateProto> sourceStateMap,
-                   String[] localDirsBase, FileSystem localFs) {
+  public QueryInfo(QueryIdentifier queryIdentifier, String appIdString, String dagName,
+      int dagIdentifier, String user, ConcurrentMap<String, SourceStateProto> sourceStateMap,
+      String[] localDirsBase, FileSystem localFs, String tokenUserName, String tokenAppId) {
     this.queryIdentifier = queryIdentifier;
     this.appIdString = appIdString;
     this.dagName = dagName;
@@ -67,6 +68,8 @@ public class QueryInfo {
     this.user = user;
     this.localDirsBase = localDirsBase;
     this.localFs = localFs;
+    this.tokenUserName = tokenUserName;
+    this.appId = tokenAppId;
   }
 
   public QueryIdentifier getQueryIdentifier() {
@@ -270,4 +273,12 @@ public class QueryInfo {
       this.lastFinishableState = lastFinishableState;
     }
   }
+
+  public String getTokenUserName() {
+    return tokenUserName;
+  }
+
+  public String getTokenAppId() {
+    return appId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
index 14657e6..cb3be2b 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
@@ -18,6 +18,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.token.Token;
 import org.apache.tez.common.CallableWithNdc;
@@ -60,6 +62,7 @@ public class QueryTracker extends AbstractService {
 
   private final String[] localDirsBase;
   private final FileSystem localFs;
+  private final String clusterId;
   private final long defaultDeleteDelaySeconds;
 
   // TODO At the moment there's no way of knowing whether a query is running or not.
@@ -89,9 +92,10 @@ public class QueryTracker extends AbstractService {
   private final ConcurrentHashMap<QueryIdentifier, String> queryIdentifierToHiveQueryId =
       new ConcurrentHashMap<>();
 
-  public QueryTracker(Configuration conf, String[] localDirsBase) {
+  public QueryTracker(Configuration conf, String[] localDirsBase, String clusterId) {
     super("QueryTracker");
     this.localDirsBase = localDirsBase;
+    this.clusterId = clusterId;
     try {
       localFs = FileSystem.getLocal(conf);
     } catch (IOException e) {
@@ -119,35 +123,50 @@ public class QueryTracker extends AbstractService {
    * @param user
    * @throws IOException
    */
-  QueryFragmentInfo registerFragment(QueryIdentifier queryIdentifier, String appIdString, String dagName,
-                                     int dagIdentifier, String vertexName, int fragmentNumber, int attemptNumber, String user,
-                                     FragmentSpecProto fragmentSpec, Token<JobTokenIdentifier> appToken) throws IOException {
+  QueryFragmentInfo registerFragment(QueryIdentifier queryIdentifier, String appIdString,
+      String dagName, int dagIdentifier, String vertexName, int fragmentNumber, int attemptNumber,
+      String user, FragmentSpecProto fragmentSpec, Token<JobTokenIdentifier> appToken)
+          throws IOException {
     ReadWriteLock dagLock = getDagLock(queryIdentifier);
     dagLock.readLock().lock();
     try {
-      if (!completedDagMap.contains(queryIdentifier)) {
-        QueryInfo queryInfo = queryInfoMap.get(queryIdentifier);
-        if (queryInfo == null) {
-          queryInfo = new QueryInfo(queryIdentifier, appIdString, dagName, dagIdentifier, user,
-              getSourceCompletionMap(queryIdentifier), localDirsBase, localFs);
-          queryInfoMap.putIfAbsent(queryIdentifier, queryInfo);
-        }
-
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Registering request for {} with the ShuffleHandler", queryIdentifier);
-        }
-        ShuffleHandler.get()
-            .registerDag(appIdString, dagIdentifier, appToken,
-                user, queryInfo.getLocalDirs());
-
-        return queryInfo.registerFragment(vertexName, fragmentNumber, attemptNumber, fragmentSpec);
-      } else {
+      if (completedDagMap.contains(queryIdentifier)) {
         // Cleanup the dag lock here, since it may have been created after the query completed
         dagSpecificLocks.remove(queryIdentifier);
         throw new RuntimeException(
             "Dag " + dagName + " already complete. Rejecting fragment ["
                 + vertexName + ", " + fragmentNumber + ", " + attemptNumber + "]");
       }
+      // TODO: for now, we get the secure username out of UGI... after signing, we can take it
+      //       out of the request provided that it's signed.
+      Pair<String, String> tokenInfo = LlapTokenChecker.getTokenInfo(clusterId);
+      boolean isExistingQueryInfo = true;
+      QueryInfo queryInfo = queryInfoMap.get(queryIdentifier);
+      if (queryInfo == null) {
+        queryInfo = new QueryInfo(queryIdentifier, appIdString, dagName, dagIdentifier, user,
+            getSourceCompletionMap(queryIdentifier), localDirsBase, localFs,
+            tokenInfo.getLeft(), tokenInfo.getRight());
+        QueryInfo old = queryInfoMap.putIfAbsent(queryIdentifier, queryInfo);
+        if (old != null) {
+          queryInfo = old;
+        } else {
+          isExistingQueryInfo = false;
+        }
+      }
+      if (isExistingQueryInfo) {
+        // We already retrieved the incoming info, check without UGI.
+        LlapTokenChecker.checkPermissions(tokenInfo, queryInfo.getTokenUserName(),
+            queryInfo.getTokenAppId(), queryInfo.getQueryIdentifier());
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Registering request for {} with the ShuffleHandler", queryIdentifier);
+      }
+      ShuffleHandler.get()
+          .registerDag(appIdString, dagIdentifier, appToken,
+              user, queryInfo.getLocalDirs());
+
+      return queryInfo.registerFragment(vertexName, fragmentNumber, attemptNumber, fragmentSpec);
     } finally {
       dagLock.readLock().unlock();
     }
@@ -174,17 +193,20 @@ public class QueryTracker extends AbstractService {
    * @param queryIdentifier
    * @param deleteDelay
    */
-  List<QueryFragmentInfo> queryComplete(QueryIdentifier queryIdentifier, long deleteDelay) {
+  List<QueryFragmentInfo> queryComplete(QueryIdentifier queryIdentifier, long deleteDelay,
+      boolean isInternal) throws IOException {
     if (deleteDelay == -1) {
       deleteDelay = defaultDeleteDelaySeconds;
     }
     ReadWriteLock dagLock = getDagLock(queryIdentifier);
     dagLock.writeLock().lock();
     try {
+      QueryInfo queryInfo = isInternal
+          ? queryInfoMap.get(queryIdentifier) : checkPermissionsAndGetQuery(queryIdentifier);
       rememberCompletedDag(queryIdentifier);
       LOG.info("Processing queryComplete for queryIdentifier={} with deleteDelay={} seconds", queryIdentifier,
           deleteDelay);
-      QueryInfo queryInfo = queryInfoMap.remove(queryIdentifier);
+      queryInfoMap.remove(queryIdentifier);
       if (queryInfo == null) {
         LOG.warn("Ignoring query complete for unknown dag: {}", queryIdentifier);
         return Collections.emptyList();
@@ -229,9 +251,10 @@ public class QueryTracker extends AbstractService {
    * @param sourceName
    * @param sourceState
    */
-  void registerSourceStateChange(QueryIdentifier queryIdentifier, String sourceName, SourceStateProto sourceState) {
+  void registerSourceStateChange(QueryIdentifier queryIdentifier, String sourceName,
+      SourceStateProto sourceState) throws IOException {
     getSourceCompletionMap(queryIdentifier).put(sourceName, sourceState);
-    QueryInfo queryInfo = queryInfoMap.get(queryIdentifier);
+    QueryInfo queryInfo = checkPermissionsAndGetQuery(queryIdentifier);
     if (queryInfo != null) {
       queryInfo.sourceStateUpdated(sourceName);
     } else {
@@ -322,4 +345,16 @@ public class QueryTracker extends AbstractService {
       return null;
     }
   }
+
+  private QueryInfo checkPermissionsAndGetQuery(QueryIdentifier queryId) throws IOException {
+    QueryInfo queryInfo = queryInfoMap.get(queryId);
+    if (queryInfo == null) return null;
+    LlapTokenChecker.checkPermissions(clusterId, queryInfo.getTokenAppId(),
+        queryInfo.getTokenUserName(), queryInfo.getQueryIdentifier());
+    return queryInfo;
+  }
+
+  public boolean checkPermissionsForQuery(QueryIdentifier queryId) throws IOException {
+    return checkPermissionsAndGetQuery(queryId) != null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/Scheduler.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/Scheduler.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/Scheduler.java
index 26c8e55..fd6234a 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/Scheduler.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/Scheduler.java
@@ -44,4 +44,6 @@ public interface Scheduler<T> {
   void killFragment(String fragmentId);
 
   Set<String> getExecutorsStatus();
+
+  QueryIdentifier findQueryByFragment(String fragmentId);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
index f621af2..1933eb1 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
@@ -420,6 +420,15 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
   }
 
   @Override
+  public QueryIdentifier findQueryByFragment(String fragmentId) {
+    synchronized (lock) {
+      TaskWrapper taskWrapper = knownTasks.get(fragmentId);
+      return taskWrapper == null ? null : taskWrapper.getTaskRunnerCallable()
+          .getFragmentInfo().getQueryInfo().getQueryIdentifier();
+    }
+  }
+
+  @Override
   public void killFragment(String fragmentId) {
     synchronized (lock) {
       TaskWrapper taskWrapper = knownTasks.remove(fragmentId);

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSecurityHelper.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSecurityHelper.java b/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSecurityHelper.java
index 76ba225..f958bc4 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSecurityHelper.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSecurityHelper.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 
 import javax.net.SocketFactory;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.llap.impl.LlapManagementProtocolClientImpl;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto;
@@ -81,7 +82,7 @@ public class LlapSecurityHelper implements LlapTokenProvider {
   }
 
   @Override
-  public Token<LlapTokenIdentifier> getDelegationToken() throws IOException {
+  public Token<LlapTokenIdentifier> getDelegationToken(String appId) throws IOException {
     if (!UserGroupInformation.isSecurityEnabled()) return null;
     if (llapUgi == null) {
       llapUgi = UserGroupInformation.getCurrentUser();
@@ -98,7 +99,7 @@ public class LlapSecurityHelper implements LlapTokenProvider {
     boolean hasRefreshed = false;
     while (true) {
       try {
-        tokenBytes = getTokenBytes();
+        tokenBytes = getTokenBytes(appId);
         break;
       } catch (InterruptedException ie) {
         throw new RuntimeException(ie);
@@ -128,7 +129,8 @@ public class LlapSecurityHelper implements LlapTokenProvider {
     return token;
   }
 
-  private ByteString getTokenBytes() throws InterruptedException, IOException {
+  private ByteString getTokenBytes(
+      final String appId) throws InterruptedException, IOException {
     return llapUgi.doAs(new PrivilegedExceptionAction<ByteString>() {
       @Override
       public ByteString run() throws Exception {
@@ -138,8 +140,11 @@ public class LlapSecurityHelper implements LlapTokenProvider {
               clientInstance.getManagementPort(), retryPolicy, socketFactory);
         }
         // Client only connects on the first call, so this has to be done in doAs.
-        GetTokenRequestProto req = GetTokenRequestProto.newBuilder().build();
-        return client.getDelegationToken(null, req).getToken();
+        GetTokenRequestProto.Builder req = GetTokenRequestProto.newBuilder();
+        if (!StringUtils.isBlank(appId)) {
+          req.setAppId(appId);
+        }
+        return client.getDelegationToken(null, req.build()).getToken();
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java b/llap-server/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
index 8c7a539..c54e726 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
@@ -23,6 +23,7 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.DaemonId;
 import org.apache.hadoop.hive.llap.security.LlapTokenIdentifier;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -33,6 +34,7 @@ import org.slf4j.LoggerFactory;
 
 public class SecretManager extends ZKDelegationTokenSecretManager<LlapTokenIdentifier> {
   private static final Logger LOG = LoggerFactory.getLogger(SecretManager.class);
+
   public SecretManager(Configuration conf) {
     super(conf);
     checkForZKDTSMBug(conf);
@@ -82,16 +84,8 @@ public class SecretManager extends ZKDelegationTokenSecretManager<LlapTokenIdent
     return id;
   }
 
-  private final static Pattern hostsRe = Pattern.compile("[^A-Za-z0-9_-]");
-  private static String deriveZkPath(Configuration conf) throws IOException {
-    String hosts = HiveConf.getTrimmedVar(conf, ConfVars.LLAP_DAEMON_SERVICE_HOSTS);
-    String clusterName = hosts.startsWith("@") ? hosts.substring(1) : hosts;
-    String userName = UserGroupInformation.getCurrentUser().getShortUserName();
-    return hostsRe.matcher(userName + "_" + clusterName).replaceAll("_") ;
-  }
-
   public static SecretManager createSecretManager(
-      final Configuration conf, String llapPrincipal, String llapKeytab) {
+      final Configuration conf, String llapPrincipal, String llapKeytab, DaemonId daemonId) {
     // Create ZK connection under a separate ugi (if specified) - ZK works in mysterious ways.
     UserGroupInformation zkUgi = null;
     String principal = HiveConf.getVar(conf, ConfVars.LLAP_ZKSM_KERBEROS_PRINCIPAL, llapPrincipal);
@@ -110,12 +104,7 @@ public class SecretManager extends ZKDelegationTokenSecretManager<LlapTokenIdent
     zkConf.setLong(DelegationTokenManager.RENEW_INTERVAL, tokenLifetime);
     zkConf.set(SecretManager.ZK_DTSM_ZK_KERBEROS_PRINCIPAL, principal);
     zkConf.set(SecretManager.ZK_DTSM_ZK_KERBEROS_KEYTAB, keyTab);
-    String zkPath;
-    try {
-      zkPath = deriveZkPath(conf);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
+    String zkPath = daemonId.getClusterString();
     LOG.info("Using {} as ZK secret manager path", zkPath);
     zkConf.set(SecretManager.ZK_DTSM_ZNODE_WORKING_PATH, "zkdtsm_" + zkPath);
     setZkConfIfNotSet(zkConf, SecretManager.ZK_DTSM_ZK_AUTH_TYPE, "sasl");

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
index 610f266..dde5be0 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
@@ -193,7 +193,7 @@ public class MiniLlapCluster extends AbstractService {
     LOG.info("Initializing {} llap instances for MiniLlapCluster with name={}", numInstances, clusterNameTrimmed);
     for (int i = 0 ;i < numInstances ; i++) {
       llapDaemons[i] = new LlapDaemon(conf, numExecutorsPerService, execBytesPerService, llapIoEnabled,
-          ioIsDirect, ioBytesPerService, localDirs, rpcPort, mngPort, shufflePort, webPort);
+          ioIsDirect, ioBytesPerService, localDirs, rpcPort, mngPort, shufflePort, webPort, clusterNameTrimmed);
       llapDaemons[i].init(new Configuration(conf));
     }
     LOG.info("Initialized {} llap instances for MiniLlapCluster with name={}", numInstances, clusterNameTrimmed);

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
index 24f4442..c6ba14e 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
@@ -78,7 +78,7 @@ public class TaskExecutorTestHelpers {
     QueryInfo queryInfo =
         new QueryInfo(queryIdentifier, "fake_app_id_string", "fake_dag_name", 1, "fakeUser",
             new ConcurrentHashMap<String, LlapDaemonProtocolProtos.SourceStateProto>(),
-            new String[0], null);
+            new String[0], null, null, null);
     return queryInfo;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
index a65bf5c..fd37a06 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
@@ -46,7 +46,7 @@ public class TestLlapDaemonProtocolServerImpl {
     LlapProtocolServerImpl server =
         new LlapProtocolServerImpl(numHandlers, containerRunnerMock,
            new AtomicReference<InetSocketAddress>(), new AtomicReference<InetSocketAddress>(),
-           rpcPort, rpcPort + 1);
+           rpcPort, rpcPort + 1, null);
     when(containerRunnerMock.submitWork(any(SubmitWorkRequestProto.class))).thenReturn(
         SubmitWorkResponseProto
             .newBuilder()

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapTokenChecker.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapTokenChecker.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapTokenChecker.java
new file mode 100644
index 0000000..aaaa762
--- /dev/null
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapTokenChecker.java
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.llap.daemon.impl;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.io.Text;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hive.llap.security.LlapTokenIdentifier;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+public class TestLlapTokenChecker {
+
+  @Test
+  public void testGetToken() {
+    check(LlapTokenChecker.getTokenInfoInternal("u", null), "u", null);
+    check(LlapTokenChecker.getTokenInfoInternal(null, createTokens("u", null)), "u", null);
+    check(LlapTokenChecker.getTokenInfoInternal(null, createTokens("u", "a")), "u", "a");
+    check(LlapTokenChecker.getTokenInfoInternal("u", createTokens("u", "a")), "u", "a");
+    check(LlapTokenChecker.getTokenInfoInternal("u", createTokens("u", "a", "u", null)),
+        "u", "a");
+    // Note - some of these scenarios could be handled, but they are not supported right now.
+    // The reason is that we bind a query to app/user using the signed token information, and
+    // we don't want to bother figuring out which one to use in case of ambiguity w/o a use case.
+    checkGetThrows("u", createTokens("u2", "a")); // Ambiguous user.
+    checkGetThrows("u2", createTokens("u2", "a", "u3", "a")); // Ambiguous user.
+    checkGetThrows(null, createTokens("u2", "a", "u3", "a")); // Ambiguous user.
+    checkGetThrows(null, createTokens("u2", "a", "u2", "a1")); // Ambiguous app.
+  }
+
+  @Test
+  public void testCheckPermissions() {
+    LlapTokenChecker.checkPermissionsInternal("u", null, "u", null, null);
+    LlapTokenChecker.checkPermissionsInternal(null, createTokens("u", null) , "u", null, null);
+    LlapTokenChecker.checkPermissionsInternal("u", createTokens("u", "a") , "u", "a", null);
+    // No access.
+    checkPrmThrows("u2", null, "u", "a");
+    checkPrmThrows("u", null, "u", "a"); // Note - Kerberos user w/o appId doesn't have access.
+    checkPrmThrows(null, createTokens("u2", "a"), "u", "a");
+    checkPrmThrows(null, createTokens("u", "a2"), "u", "a");
+    checkPrmThrows(null, createTokens("u", null), "u", "a");
+  }
+
+  private List<LlapTokenIdentifier> createTokens(String... args) {
+    List<LlapTokenIdentifier> tokens = new ArrayList<>();
+    for (int i = 0; i < args.length; i += 2) {
+      tokens.add(new LlapTokenIdentifier(null, null, new Text(args[i]), "c", args[i + 1]));
+    }
+    return tokens;
+  }
+
+  private void checkGetThrows(String kerberosName, List<LlapTokenIdentifier> tokens) {
+    try {
+      LlapTokenChecker.getTokenInfoInternal(kerberosName, tokens);
+      fail("Didn't throw");
+    } catch (SecurityException ex) {
+      // Expected.
+    }
+  }
+
+  private void checkPrmThrows(
+      String kerberosName, List<LlapTokenIdentifier> tokens, String userName, String appId) {
+    try {
+      LlapTokenChecker.checkPermissionsInternal(kerberosName, tokens, userName, appId, null);
+      fail("Didn't throw");
+    } catch (SecurityException ex) {
+      // Expected.
+    }
+  }
+
+  private void check(Pair<String, String> p, String user, String appId) {
+    assertEquals(user, p.getLeft());
+    assertEquals(appId, p.getRight());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
index 3ea5ef9..fd6465a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
@@ -275,7 +275,8 @@ public class TezSessionState {
     if (llapMode) {
       if (UserGroupInformation.isSecurityEnabled()) {
         LlapTokenProvider tp = LlapProxy.getOrInitTokenProvider(conf);
-        Token<LlapTokenIdentifier> token = tp.getDelegationToken();
+        // For Tez, we don't use appId to distinguish the tokens; security scope is the user.
+        Token<LlapTokenIdentifier> token = tp.getDelegationToken(null);
         if (LOG.isInfoEnabled()) {
           LOG.info("Obtained a LLAP token: " + token);
         }


[43/50] [abbrv] hive git commit: HIVE-13701: LLAP: Use different prefix for llap task scheduler metrics (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Posted by sp...@apache.org.
HIVE-13701: LLAP: Use different prefix for llap task scheduler metrics (Prasanth Jayachandran reviewed by Sergey Shelukhin)


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

Branch: refs/heads/java8
Commit: 0cc40456586aa5f3c54a34ceaf65eaef9a3d311b
Parents: 3517a99
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Thu May 5 21:43:48 2016 -0500
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Thu May 5 21:43:48 2016 -0500

----------------------------------------------------------------------
 ...doop-metrics2-llapdaemon.properties.template | 50 ++++++++++++++++++++
 ...trics2-llaptaskscheduler.properties.template | 50 ++++++++++++++++++++
 .../hadoop-metrics2.properties.template         | 50 --------------------
 .../tezplugins/LlapTaskSchedulerService.java    |  2 +-
 .../metrics/LlapTaskSchedulerMetrics.java       |  6 +--
 5 files changed, 104 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0cc40456/llap-server/src/main/resources/hadoop-metrics2-llapdaemon.properties.template
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/hadoop-metrics2-llapdaemon.properties.template b/llap-server/src/main/resources/hadoop-metrics2-llapdaemon.properties.template
new file mode 100644
index 0000000..994acaa
--- /dev/null
+++ b/llap-server/src/main/resources/hadoop-metrics2-llapdaemon.properties.template
@@ -0,0 +1,50 @@
+#
+# 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].[instance].[options]
+# See javadoc of package-info.java for org.apache.hadoop.metrics2 for details
+
+#*.sink.file.class=org.apache.hadoop.metrics2.sink.FileSink
+# default sampling period, in seconds
+#*.sink.file.period=10
+
+# *.sink.timeline.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+# *.sink.timeline.period=60
+
+# llapdeamon metrics for all contexts (jvm,queue,executors,cache) will go to this file
+# llapdaemon.sink.file.filename=llapdaemon-metrics.out
+
+# to configure separate files per context define following for each context
+# llapdaemon.sink.file_jvm.class=org.apache.hadoop.metrics2.sink.FileSink
+# llapdaemon.sink.file_jvm.context=jvm
+# llapdaemon.sink.file_jvm.filename=llapdaemon-jvm-metrics.out

http://git-wip-us.apache.org/repos/asf/hive/blob/0cc40456/llap-server/src/main/resources/hadoop-metrics2-llaptaskscheduler.properties.template
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/hadoop-metrics2-llaptaskscheduler.properties.template b/llap-server/src/main/resources/hadoop-metrics2-llaptaskscheduler.properties.template
new file mode 100644
index 0000000..5cf71a7
--- /dev/null
+++ b/llap-server/src/main/resources/hadoop-metrics2-llaptaskscheduler.properties.template
@@ -0,0 +1,50 @@
+#
+# 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].[instance].[options]
+# See javadoc of package-info.java for org.apache.hadoop.metrics2 for details
+
+#*.sink.file.class=org.apache.hadoop.metrics2.sink.FileSink
+# default sampling period, in seconds
+#*.sink.file.period=10
+
+# *.sink.timeline.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+# *.sink.timeline.period=60
+
+# llapdeamon metrics for all contexts (jvm,queue,executors,cache) will go to this file
+# llaptaskscheduler.sink.file.filename=llaptaskscheduler-metrics.out
+
+# to configure separate files per context define following for each context
+# llaptaskscheduler.sink.file_jvm.class=org.apache.hadoop.metrics2.sink.FileSink
+# llaptaskscheduler.sink.file_jvm.context=jvm
+# llaptaskscheduler.sink.file_jvm.filename=llaptaskscheduler-jvm-metrics.out

http://git-wip-us.apache.org/repos/asf/hive/blob/0cc40456/llap-server/src/main/resources/hadoop-metrics2.properties.template
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/hadoop-metrics2.properties.template b/llap-server/src/main/resources/hadoop-metrics2.properties.template
deleted file mode 100644
index 994acaa..0000000
--- a/llap-server/src/main/resources/hadoop-metrics2.properties.template
+++ /dev/null
@@ -1,50 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#   http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#}
-
-# 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].[instance].[options]
-# See javadoc of package-info.java for org.apache.hadoop.metrics2 for details
-
-#*.sink.file.class=org.apache.hadoop.metrics2.sink.FileSink
-# default sampling period, in seconds
-#*.sink.file.period=10
-
-# *.sink.timeline.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
-# *.sink.timeline.period=60
-
-# llapdeamon metrics for all contexts (jvm,queue,executors,cache) will go to this file
-# llapdaemon.sink.file.filename=llapdaemon-metrics.out
-
-# to configure separate files per context define following for each context
-# llapdaemon.sink.file_jvm.class=org.apache.hadoop.metrics2.sink.FileSink
-# llapdaemon.sink.file_jvm.context=jvm
-# llapdaemon.sink.file_jvm.filename=llapdaemon-jvm-metrics.out

http://git-wip-us.apache.org/repos/asf/hive/blob/0cc40456/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
index da1e17f..733049d 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
@@ -267,7 +267,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
 
     if (initMetrics) {
       // Initialize the metrics system
-      LlapMetricsSystem.initialize("LlapDaemon");
+      LlapMetricsSystem.initialize("LlapTaskScheduler");
       this.pauseMonitor = new JvmPauseMonitor(conf);
       pauseMonitor.start();
       String displayName = "LlapTaskSchedulerMetrics-" + MetricsUtils.getHostName();

http://git-wip-us.apache.org/repos/asf/hive/blob/0cc40456/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerMetrics.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerMetrics.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerMetrics.java
index b3230e2..04fd815 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerMetrics.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/metrics/LlapTaskSchedulerMetrics.java
@@ -46,9 +46,9 @@ import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 
 /**
- * Metrics about the llap daemon task scheduler.
+ * Metrics about the llap task scheduler.
  */
-@Metrics(about = "LlapDaemon Task Scheduler Metrics", context = "scheduler")
+@Metrics(about = "Llap Task Scheduler Metrics", context = "scheduler")
 public class LlapTaskSchedulerMetrics implements MetricsSource {
 
   private final String name;
@@ -99,7 +99,7 @@ public class LlapTaskSchedulerMetrics implements MetricsSource {
   public void getMetrics(MetricsCollector collector, boolean b) {
     MetricsRecordBuilder rb = collector.addRecord(SchedulerMetrics)
         .setContext("scheduler")
-        .tag(ProcessName, MetricsUtils.METRICS_PROCESS_NAME)
+        .tag(ProcessName, "DAGAppMaster")
         .tag(SessionId, sessionId);
     getTaskSchedulerStats(rb);
   }


[24/50] [abbrv] hive git commit: HIVE-13592 : metastore calls map is not thread safe (Sergey Shelukhin, reviewed by Aihua Xu)

Posted by sp...@apache.org.
HIVE-13592 : metastore calls map is not thread safe (Sergey Shelukhin, reviewed by Aihua Xu)


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

Branch: refs/heads/java8
Commit: f68b5dbb59a9e837209e64aefe5aa994476c0bdc
Parents: e68783c
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed May 4 17:05:20 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed May 4 17:05:39 2016 -0700

----------------------------------------------------------------------
 .../hive/metastore/RetryingMetaStoreClient.java    | 17 +++++++++--------
 .../org/apache/hadoop/hive/ql/metadata/Hive.java   |  3 ++-
 2 files changed, 11 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f68b5dbb/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
index f672adf..3c125e0 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
@@ -25,6 +25,7 @@ import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 
 import org.slf4j.Logger;
@@ -55,14 +56,14 @@ public class RetryingMetaStoreClient implements InvocationHandler {
   private final IMetaStoreClient base;
   private final int retryLimit;
   private final long retryDelaySeconds;
-  private final Map<String, Long> metaCallTimeMap;
+  private final ConcurrentHashMap<String, Long> metaCallTimeMap;
   private final long connectionLifeTimeInMillis;
   private long lastConnectionTime;
   private boolean localMetaStore;
 
 
   protected RetryingMetaStoreClient(HiveConf hiveConf, Class<?>[] constructorArgTypes,
-      Object[] constructorArgs, Map<String, Long> metaCallTimeMap,
+      Object[] constructorArgs, ConcurrentHashMap<String, Long> metaCallTimeMap,
       Class<? extends IMetaStoreClient> msClientClass) throws MetaException {
 
     this.retryLimit = hiveConf.getIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES);
@@ -94,7 +95,7 @@ public class RetryingMetaStoreClient implements InvocationHandler {
   }
 
   public static IMetaStoreClient getProxy(HiveConf hiveConf, HiveMetaHookLoader hookLoader,
-      Map<String, Long> metaCallTimeMap, String mscClassName, boolean allowEmbedded)
+      ConcurrentHashMap<String, Long> metaCallTimeMap, String mscClassName, boolean allowEmbedded)
           throws MetaException {
 
     return getProxy(hiveConf,
@@ -119,7 +120,7 @@ public class RetryingMetaStoreClient implements InvocationHandler {
    * Please use getProxy(HiveConf hiveConf, HiveMetaHookLoader hookLoader) for external purpose.
    */
   public static IMetaStoreClient getProxy(HiveConf hiveConf, Class<?>[] constructorArgTypes,
-      Object[] constructorArgs, Map<String, Long> metaCallTimeMap,
+      Object[] constructorArgs, ConcurrentHashMap<String, Long> metaCallTimeMap,
       String mscClassName) throws MetaException {
 
     @SuppressWarnings("unchecked")
@@ -202,11 +203,11 @@ public class RetryingMetaStoreClient implements InvocationHandler {
 
   private void addMethodTime(Method method, long timeTaken) {
     String methodStr = getMethodString(method);
-    Long curTime = metaCallTimeMap.get(methodStr);
-    if (curTime != null) {
-      timeTaken += curTime;
+    while (true) {
+      Long curTime = metaCallTimeMap.get(methodStr), newTime = timeTaken;
+      if (curTime != null && metaCallTimeMap.replace(methodStr, curTime, newTime + curTime)) break;
+      if (curTime == null && (null == metaCallTimeMap.putIfAbsent(methodStr, newTime))) break;
     }
-    metaCallTimeMap.put(methodStr, timeTaken);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/f68b5dbb/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index 6862f70..f4a9772 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -48,6 +48,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.ConcurrentHashMap;
 
 import com.google.common.collect.ImmutableMap;
 
@@ -162,7 +163,7 @@ public class Hive {
   private UserGroupInformation owner;
 
   // metastore calls timing information
-  private final Map<String, Long> metaCallTimeMap = new HashMap<String, Long>();
+  private final ConcurrentHashMap<String, Long> metaCallTimeMap = new ConcurrentHashMap<>();
 
   private static ThreadLocal<Hive> hiveDB = new ThreadLocal<Hive>() {
     @Override


[10/50] [abbrv] hive git commit: HIVE-13646 make hive.optimize.sort.dynamic.partition compatible with ACID tables (Eugene Koifman, reviewed by Wei Zheng)

Posted by sp...@apache.org.
HIVE-13646 make hive.optimize.sort.dynamic.partition compatible with ACID tables (Eugene Koifman, reviewed by Wei Zheng)


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

Branch: refs/heads/java8
Commit: 8729966296a041b7ea952ba67f148d2c48c27749
Parents: 70fe310
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Tue May 3 17:11:47 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Tue May 3 17:11:47 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   1 -
 .../dynpart_sort_optimization_acid.q.out        | 120 +++++++++++++++----
 2 files changed, 100 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/87299662/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 06db7f9..2983d38 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -7030,7 +7030,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     conf.setBoolVar(ConfVars.HIVEOPTREDUCEDEDUPLICATION, true);
     conf.setIntVar(ConfVars.HIVEOPTREDUCEDEDUPLICATIONMINREDUCER, 1);
     conf.set(AcidUtils.CONF_ACID_KEY, "true");
-    conf.setBoolVar(ConfVars.HIVEOPTSORTDYNAMICPARTITION, false);
 
     if (table.getNumBuckets() < 1) {
       throw new SemanticException(ErrorMsg.ACID_OP_ON_NONACID_TABLE, table.getTableName());

http://git-wip-us.apache.org/repos/asf/hive/blob/87299662/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
index eca29df..62399e3 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
@@ -380,8 +380,9 @@ POSTHOOK: query: explain update acid set value = 'bar' where key = 'foo' and ds
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-  Stage-2 depends on stages: Stage-0
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+  Stage-3 depends on stages: Stage-0
 
 STAGE PLANS:
   Stage: Stage-1
@@ -397,12 +398,31 @@ STAGE PLANS:
                 Reduce Output Operator
                   key expressions: _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
                   sort order: +
-                  Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                   value expressions: _col3 (type: string)
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), 'foo' (type: string), 'bar' (type: string), VALUE._col2 (type: string)
-          outputColumnNames: _col0, _col1, _col2, _col3
+          expressions: KEY.reducesinkkey0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col2 (type: string)
+          outputColumnNames: _col0, _col3
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col3 (type: string), '_bucket_number' (type: string), _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
+              sort order: +++
+              Map-reduce partition columns: _col3 (type: string)
+              value expressions: 'foo' (type: string), 'bar' (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY.'_bucket_number' (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, '_bucket_number'
           File Output Operator
             compressed: false
             table:
@@ -423,7 +443,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
               name: default.acid
 
-  Stage: Stage-2
+  Stage: Stage-3
     Stats-Aggr Operator
 
 PREHOOK: query: update acid set value = 'bar' where key = 'foo' and ds in ('2008-04-08')
@@ -875,8 +895,9 @@ POSTHOOK: query: explain update acid set value = 'bar' where key = 'foo' and ds=
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-  Stage-2 depends on stages: Stage-0
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+  Stage-3 depends on stages: Stage-0
 
 STAGE PLANS:
   Stage: Stage-1
@@ -892,12 +913,31 @@ STAGE PLANS:
                 Reduce Output Operator
                   key expressions: _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
                   sort order: +
-                  Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                   value expressions: _col4 (type: int)
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), VALUE._col3 (type: int)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4
+          expressions: KEY.reducesinkkey0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col3 (type: int)
+          outputColumnNames: _col0, _col4
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: '2008-04-08' (type: string), _col4 (type: int), '_bucket_number' (type: string), _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
+              sort order: ++++
+              Map-reduce partition columns: '2008-04-08' (type: string), _col4 (type: int)
+              value expressions: 'foo' (type: string), 'bar' (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int), KEY.'_bucket_number' (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, '_bucket_number'
           File Output Operator
             compressed: false
             table:
@@ -919,7 +959,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
               name: default.acid
 
-  Stage: Stage-2
+  Stage: Stage-3
     Stats-Aggr Operator
 
 PREHOOK: query: update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11
@@ -1053,8 +1093,9 @@ POSTHOOK: query: explain update acid set value = 'bar' where key = 'foo' and ds=
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-  Stage-2 depends on stages: Stage-0
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+  Stage-3 depends on stages: Stage-0
 
 STAGE PLANS:
   Stage: Stage-1
@@ -1070,7 +1111,6 @@ STAGE PLANS:
                 Reduce Output Operator
                   key expressions: _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
                   sort order: +
-                  Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                   value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: int)
       Reduce Operator Tree:
         Select Operator
@@ -1079,6 +1119,26 @@ STAGE PLANS:
           File Output Operator
             compressed: false
             table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col3 (type: string), _col4 (type: int), '_bucket_number' (type: string), _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
+              sort order: ++++
+              Map-reduce partition columns: _col3 (type: string), _col4 (type: int)
+              value expressions: _col1 (type: string), _col2 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int), KEY.'_bucket_number' (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, '_bucket_number'
+          File Output Operator
+            compressed: false
+            table:
                 input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
                 serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
@@ -1097,7 +1157,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
               name: default.acid
 
-  Stage: Stage-2
+  Stage: Stage-3
     Stats-Aggr Operator
 
 PREHOOK: query: update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11
@@ -1127,8 +1187,9 @@ POSTHOOK: query: explain update acid set value = 'bar' where key = 'foo' and ds=
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-  Stage-2 depends on stages: Stage-0
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+  Stage-3 depends on stages: Stage-0
 
 STAGE PLANS:
   Stage: Stage-1
@@ -1144,7 +1205,6 @@ STAGE PLANS:
                 Reduce Output Operator
                   key expressions: _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
                   sort order: +
-                  Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                   value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: int)
       Reduce Operator Tree:
         Select Operator
@@ -1153,6 +1213,26 @@ STAGE PLANS:
           File Output Operator
             compressed: false
             table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col3 (type: string), _col4 (type: int), '_bucket_number' (type: string), _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
+              sort order: ++++
+              Map-reduce partition columns: _col3 (type: string), _col4 (type: int)
+              value expressions: _col1 (type: string), _col2 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int), KEY.'_bucket_number' (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, '_bucket_number'
+          File Output Operator
+            compressed: false
+            table:
                 input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
                 serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
@@ -1171,7 +1251,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
               name: default.acid
 
-  Stage: Stage-2
+  Stage: Stage-3
     Stats-Aggr Operator
 
 PREHOOK: query: update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11


[04/50] [abbrv] hive git commit: HIVE-13213 make DbLockManger work for non-acid resources (Eugene Koifman, reviewed by Alan Gates)

Posted by sp...@apache.org.
HIVE-13213 make DbLockManger work for non-acid resources (Eugene Koifman, reviewed by Alan Gates)


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

Branch: refs/heads/java8
Commit: b70efa447d9ae5883315d88e84ad1262d371213d
Parents: 47bf055
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Tue May 3 13:38:42 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Tue May 3 13:53:02 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/txn/TxnHandler.java   |  5 ++
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    | 12 +++
 .../apache/hadoop/hive/ql/TestTxnCommands2.java | 22 ++++++
 .../hive/ql/lockmgr/TestDbTxnManager2.java      | 81 ++++++++++++++++++++
 4 files changed, 120 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b70efa44/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index c32b0b0..c0fa97a 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -87,6 +87,11 @@ import java.util.regex.Pattern;
  * If we ever decide to run remote Derby server, according to
  * https://db.apache.org/derby/docs/10.0/manuals/develop/develop78.html all transactions will be
  * seriazlied, so that would also work though has not been tested.
+ *
+ * General design note:
+ * It's imperative that any operation on a txn (e.g. commit), ensure (atomically) that this txn is
+ * still valid and active.  In the code this is usually achieved at the same time the txn record
+ * is locked for some operation.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving

http://git-wip-us.apache.org/repos/asf/hive/blob/b70efa44/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index e8ebe55..3aec8eb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.lockmgr;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hive.common.util.ShutdownHookManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -213,6 +214,17 @@ public class DbTxnManager extends HiveTxnManagerImpl {
           break;
 
         case INSERT:
+          t = output.getTable();
+          if(t == null) {
+            throw new IllegalStateException("No table info for " + output);
+          }
+          if(AcidUtils.isAcidTable(t)) {
+            compBuilder.setShared();
+          }
+          else {
+            compBuilder.setExclusive();
+          }
+          break;
         case DDL_SHARED:
           compBuilder.setShared();
           break;

http://git-wip-us.apache.org/repos/asf/hive/blob/b70efa44/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
index 04c1d17..1030987 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
@@ -433,6 +433,28 @@ public class TestTxnCommands2 {
   }
 
   /**
+   * Test update that hits multiple partitions (i.e. requries dynamic partition insert to process)
+   * @throws Exception
+   */
+  @Test
+  public void updateDeletePartitioned() throws Exception {
+    int[][] tableData = {{1,2},{3,4},{5,6}};
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p=1) (a,b) " + makeValuesClause(tableData));
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p=2) (a,b) " + makeValuesClause(tableData));
+    TxnStore txnHandler = TxnUtils.getTxnStore(hiveConf);
+    txnHandler.compact(new CompactionRequest("default", Table.ACIDTBLPART.name(), CompactionType.MAJOR));
+    runWorker(hiveConf);
+    runCleaner(hiveConf);
+    runStatementOnDriver("update " + Table.ACIDTBLPART + " set b = b + 1 where a = 3");
+    txnHandler.compact(new CompactionRequest("default", Table.ACIDTBLPART.toString(), CompactionType.MAJOR));
+    runWorker(hiveConf);
+    runCleaner(hiveConf);
+    List<String> rs = runStatementOnDriver("select p,a,b from " + Table.ACIDTBLPART + " order by p, a, b");
+    int[][] expectedData = {{1,1,2},{1,3,5},{1,5,6},{2,1,2},{2,3,5},{2,5,6}};
+    Assert.assertEquals("Update " + Table.ACIDTBLPART + " didn't match:", stringifyValues(expectedData), rs);
+  }
+
+  /**
    * https://issues.apache.org/jira/browse/HIVE-10151
    */
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/b70efa44/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index 6e2cf30..e94af55 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -536,6 +536,87 @@ public class TestDbTxnManager2 {
     Assert.assertEquals(0, count);
   }
 
+  /**
+   * collection of queries where we ensure that we get the locks that are expected
+   * @throws Exception
+   */
+  @Test
+  public void checkExpectedLocks() throws Exception {
+    CommandProcessorResponse cpr = null;
+    cpr = driver.run("create table acidPart(a int, b int) partitioned by (p string) clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    cpr = driver.run("create table nonAcidPart(a int, b int) partitioned by (p string) stored as orc");
+    checkCmdOnDriver(cpr);
+
+    cpr = driver.compileAndRespond("insert into nonAcidPart partition(p) values(1,2,3)");
+    checkCmdOnDriver(cpr);
+    LockState lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+    List<ShowLocksResponseElement> locks = getLocks();
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "values__tmp__table__1", null, locks.get(0));
+    checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "nonAcidPart", null, locks.get(1));
+    List<HiveLock> relLocks = new ArrayList<HiveLock>(2);
+    relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+    relLocks.add(new DbLockManager.DbHiveLock(locks.get(1).getLockid()));
+    txnMgr.getLockManager().releaseLocks(relLocks);
+
+    cpr = driver.compileAndRespond("insert into nonAcidPart partition(p=1) values(5,6)");
+    checkCmdOnDriver(cpr);
+    lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+    locks = getLocks();
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "values__tmp__table__2", null, locks.get(0));
+    checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "nonAcidPart", "p=1", locks.get(1));
+    relLocks = new ArrayList<HiveLock>(2);
+    relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+    relLocks.add(new DbLockManager.DbHiveLock(locks.get(1).getLockid()));
+    txnMgr.getLockManager().releaseLocks(relLocks);
+
+    cpr = driver.compileAndRespond("insert into acidPart partition(p) values(1,2,3)");
+    checkCmdOnDriver(cpr);
+    lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+    locks = getLocks();
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "values__tmp__table__3", null, locks.get(0));
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "acidPart", null, locks.get(1));
+    relLocks = new ArrayList<HiveLock>(2);
+    relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+    relLocks.add(new DbLockManager.DbHiveLock(locks.get(1).getLockid()));
+    txnMgr.getLockManager().releaseLocks(relLocks);
+
+    cpr = driver.compileAndRespond("insert into acidPart partition(p=1) values(5,6)");
+    checkCmdOnDriver(cpr);
+    lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+    locks = getLocks();
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "values__tmp__table__4", null, locks.get(0));
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "acidPart", "p=1", locks.get(1));
+    relLocks = new ArrayList<HiveLock>(2);
+    relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+    relLocks.add(new DbLockManager.DbHiveLock(locks.get(1).getLockid()));
+    txnMgr.getLockManager().releaseLocks(relLocks);
+
+    cpr = driver.compileAndRespond("update acidPart set b = 17 where a = 1");
+    checkCmdOnDriver(cpr);
+    lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+    locks = getLocks();
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "acidPart", null, locks.get(0));
+    relLocks = new ArrayList<HiveLock>(2);
+    relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+    txnMgr.getLockManager().releaseLocks(relLocks);
+
+    cpr = driver.compileAndRespond("update acidPart set b = 17 where p = 1");
+    checkCmdOnDriver(cpr);
+    lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+    locks = getLocks();
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "acidPart", null, locks.get(0));//https://issues.apache.org/jira/browse/HIVE-13212
+    relLocks = new ArrayList<HiveLock>(2);
+    relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+    txnMgr.getLockManager().releaseLocks(relLocks);
+  }
+
   private void checkLock(LockType type, LockState state, String db, String table, String partition, ShowLocksResponseElement l) {
     Assert.assertEquals(l.toString(),l.getType(), type);
     Assert.assertEquals(l.toString(),l.getState(), state);


[26/50] [abbrv] hive git commit: HIVE-12837 : Better memory estimation/allocation for hybrid grace hash join during hash table loading (Wei Zheng, reviewed by Vikram Dixit K)

Posted by sp...@apache.org.
HIVE-12837 : Better memory estimation/allocation for hybrid grace hash join during hash table loading (Wei Zheng, reviewed by Vikram Dixit K)


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

Branch: refs/heads/java8
Commit: cbebb4d78064a9098e4145a0f7532f08885c9b27
Parents: a88050b
Author: Wei Zheng <we...@apache.org>
Authored: Wed May 4 23:09:08 2016 -0700
Committer: Wei Zheng <we...@apache.org>
Committed: Wed May 4 23:09:08 2016 -0700

----------------------------------------------------------------------
 .../persistence/HybridHashTableContainer.java   | 60 +++++++++++++++-----
 .../ql/exec/persistence/KeyValueContainer.java  |  4 ++
 2 files changed, 51 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cbebb4d7/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
index f5da5a4..5552dfb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
@@ -90,6 +90,7 @@ public class HybridHashTableContainer
   private boolean lastPartitionInMem;           // only one (last one) partition is left in memory
   private final int memoryCheckFrequency;       // how often (# of rows apart) to check if memory is full
   private final HybridHashTableConf nwayConf;         // configuration for n-way join
+  private int writeBufferSize;                  // write buffer size for BytesBytesMultiHashMap
 
   /** The OI used to deserialize values. We never deserialize keys. */
   private LazyBinaryStructObjectInspector internalValueOi;
@@ -294,7 +295,6 @@ public class HybridHashTableContainer
     this.spillLocalDirs = spillLocalDirs;
 
     this.nwayConf = nwayConf;
-    int writeBufferSize;
     int numPartitions;
     if (nwayConf == null) { // binary join
       numPartitions = calcNumPartitions(memoryThreshold, estimatedTableSize, minNumParts, minWbSize);
@@ -327,7 +327,9 @@ public class HybridHashTableContainer
         writeBufferSize : Integer.highestOneBit(writeBufferSize);
 
     // Cap WriteBufferSize to avoid large preallocations
-    writeBufferSize = writeBufferSize < minWbSize ? minWbSize : Math.min(maxWbSize, writeBufferSize);
+    // We also want to limit the size of writeBuffer, because we normally have 16 partitions, that
+    // makes spilling prediction (isMemoryFull) to be too defensive which results in unnecessary spilling
+    writeBufferSize = writeBufferSize < minWbSize ? minWbSize : Math.min(maxWbSize / numPartitions, writeBufferSize);
 
     this.bloom1 = new BloomFilter(newKeyCount);
 
@@ -417,6 +419,11 @@ public class HybridHashTableContainer
     for (HashPartition hp : hashPartitions) {
       if (hp.hashMap != null) {
         memUsed += hp.hashMap.memorySize();
+      } else {
+        // also include the still-in-memory sidefile, before it has been truely spilled
+        if (hp.sidefileKVContainer != null) {
+          memUsed += hp.sidefileKVContainer.numRowsInReadBuffer() * tableRowSize;
+        }
       }
     }
     return memoryUsed = memUsed;
@@ -454,6 +461,8 @@ public class HybridHashTableContainer
   private MapJoinKey internalPutRow(KeyValueHelper keyValueHelper,
           Writable currentKey, Writable currentValue) throws SerDeException, IOException {
 
+    boolean putToSidefile = false; // by default we put row into partition in memory
+
     // Next, put row into corresponding hash partition
     int keyHash = keyValueHelper.getHashFromKey();
     int partitionId = keyHash & (hashPartitions.length - 1);
@@ -461,15 +470,13 @@ public class HybridHashTableContainer
 
     bloom1.addLong(keyHash);
 
-    if (isOnDisk(partitionId) || isHashMapSpilledOnCreation(partitionId)) {
-      KeyValueContainer kvContainer = hashPartition.getSidefileKVContainer();
-      kvContainer.add((HiveKey) currentKey, (BytesWritable) currentValue);
-    } else {
-      hashPartition.hashMap.put(keyValueHelper, keyHash); // Pass along hashcode to avoid recalculation
-      totalInMemRowCount++;
-
-      if ((totalInMemRowCount & (this.memoryCheckFrequency - 1)) == 0 &&  // check periodically
-          !lastPartitionInMem) { // If this is the only partition in memory, proceed without check
+    if (isOnDisk(partitionId) || isHashMapSpilledOnCreation(partitionId)) { // destination on disk
+      putToSidefile = true;
+    } else {  // destination in memory
+      if (!lastPartitionInMem &&        // If this is the only partition in memory, proceed without check
+          (hashPartition.size() == 0 || // Destination partition being empty indicates a write buffer
+                                        // will be allocated, thus need to check if memory is full
+           (totalInMemRowCount & (this.memoryCheckFrequency - 1)) == 0)) {  // check periodically
         if (isMemoryFull()) {
           if ((numPartitionsSpilled == hashPartitions.length - 1) ) {
             LOG.warn("This LAST partition in memory won't be spilled!");
@@ -479,9 +486,16 @@ public class HybridHashTableContainer
               int biggest = biggestPartition();
               spillPartition(biggest);
               this.setSpill(true);
+              if (partitionId == biggest) { // destination hash partition has just be spilled
+                putToSidefile = true;
+              }
             } else {                // n-way join
               LOG.info("N-way spilling: spill tail partition from previously loaded small tables");
+              int biggest = nwayConf.getNextSpillPartition();
               memoryThreshold += nwayConf.spill();
+              if (biggest != 0 && partitionId == biggest) { // destination hash partition has just be spilled
+                putToSidefile = true;
+              }
               LOG.info("Memory threshold has been increased to: " + memoryThreshold);
             }
             numPartitionsSpilled++;
@@ -490,6 +504,15 @@ public class HybridHashTableContainer
       }
     }
 
+    // Now we know where to put row
+    if (putToSidefile) {
+      KeyValueContainer kvContainer = hashPartition.getSidefileKVContainer();
+      kvContainer.add((HiveKey) currentKey, (BytesWritable) currentValue);
+    } else {
+      hashPartition.hashMap.put(keyValueHelper, keyHash); // Pass along hashcode to avoid recalculation
+      totalInMemRowCount++;
+    }
+
     return null; // there's no key to return
   }
 
@@ -513,11 +536,21 @@ public class HybridHashTableContainer
   }
 
   /**
-   * Check if the memory threshold is reached
+   * Check if the memory threshold is about to be reached.
+   * Since all the write buffer will be lazily allocated in BytesBytesMultiHashMap, we need to
+   * consider those as well.
    * @return true if memory is full, false if not
    */
   private boolean isMemoryFull() {
-    return refreshMemoryUsed() >= memoryThreshold;
+    int numPartitionsInMem = 0;
+
+    for (HashPartition hp : hashPartitions) {
+      if (!hp.isHashMapOnDisk()) {
+        numPartitionsInMem++;
+      }
+    }
+
+    return refreshMemoryUsed() + writeBufferSize * numPartitionsInMem >= memoryThreshold;
   }
 
   /**
@@ -561,6 +594,7 @@ public class HybridHashTableContainer
         new com.esotericsoftware.kryo.io.Output(outputStream);
     Kryo kryo = SerializationUtilities.borrowKryo();
     try {
+      LOG.info("Trying to spill hash partition " + partitionId + " ...");
       kryo.writeObject(output, partition.hashMap);  // use Kryo to serialize hashmap
       output.close();
       outputStream.close();

http://git-wip-us.apache.org/repos/asf/hive/blob/cbebb4d7/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/KeyValueContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/KeyValueContainer.java
index e2b22d3..72faf8b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/KeyValueContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/KeyValueContainer.java
@@ -215,6 +215,10 @@ public class KeyValueContainer {
     return row;
   }
 
+  public int numRowsInReadBuffer() {
+    return rowsInReadBuffer;
+  }
+
   public int size() {
     return rowsInReadBuffer + rowsOnDisk;
   }


[38/50] [abbrv] hive git commit: HIVE-13637: Fold CASE into NVL when CBO optimized the plan (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
HIVE-13637: Fold CASE into NVL when CBO optimized the plan (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/java8
Commit: da82819bc112589e0d96874947c942e834681ed2
Parents: 10d0549
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed May 4 01:27:30 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu May 5 22:13:10 2016 +0100

----------------------------------------------------------------------
 .../calcite/translator/JoinTypeCheckCtx.java    |  2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  | 17 ++++++++-----
 .../hadoop/hive/ql/parse/TypeCheckCtx.java      | 19 +++++++++-----
 .../hive/ql/parse/TypeCheckProcFactory.java     | 26 ++++++++++++++++++++
 .../queries/clientpositive/constantPropWhen.q   |  2 ++
 5 files changed, 53 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/da82819b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java
index dccd1d9..f166bb6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java
@@ -53,7 +53,7 @@ public class JoinTypeCheckCtx extends TypeCheckCtx {
 
   public JoinTypeCheckCtx(RowResolver leftRR, RowResolver rightRR, JoinType hiveJoinType)
       throws SemanticException {
-    super(RowResolver.getCombinedRR(leftRR, rightRR), true, false, false, false, false, false, false,
+    super(RowResolver.getCombinedRR(leftRR, rightRR), true, false, false, false, false, false, false, false,
         false, false);
     this.inputRRLst = ImmutableList.of(leftRR, rightRR);
     this.outerJoin = (hiveJoinType == JoinType.LEFTOUTER) || (hiveJoinType == JoinType.RIGHTOUTER)

http://git-wip-us.apache.org/repos/asf/hive/blob/da82819b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 2983d38..f79a525 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -3143,8 +3143,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     OpParseContext inputCtx = opParseCtx.get(input);
     RowResolver inputRR = inputCtx.getRowResolver();
     Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
-        new FilterDesc(genExprNodeDesc(condn, inputRR, useCaching), false), new RowSchema(
-            inputRR.getColumnInfos()), input), inputRR);
+        new FilterDesc(genExprNodeDesc(condn, inputRR, useCaching, isCBOExecuted()), false),
+        new RowSchema(inputRR.getColumnInfos()), input), inputRR);
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Created Filter Plan for " + qb.getId() + " row schema: "
@@ -4146,7 +4146,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
              expr, col_list, null, inputRR, starRR, pos, out_rwsch, qb.getAliases(), false);
       } else {
         // Case when this is an expression
-        TypeCheckCtx tcCtx = new TypeCheckCtx(inputRR);
+        TypeCheckCtx tcCtx = new TypeCheckCtx(inputRR, true, isCBOExecuted());
         // We allow stateful functions in the SELECT list (but nowhere else)
         tcCtx.setAllowStatefulFunctions(true);
         tcCtx.setAllowDistinctFunctions(false);
@@ -7777,7 +7777,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       List<ASTNode> expressions = joinTree.getExpressions().get(i);
       joinKeys[i] = new ExprNodeDesc[expressions.size()];
       for (int j = 0; j < joinKeys[i].length; j++) {
-        joinKeys[i][j] = genExprNodeDesc(expressions.get(j), inputRR);
+        joinKeys[i][j] = genExprNodeDesc(expressions.get(j), inputRR, true, isCBOExecuted());
       }
     }
     // Type checking and implicit type conversion for join keys
@@ -10999,12 +10999,17 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       throws SemanticException {
     // Since the user didn't supply a customized type-checking context,
     // use default settings.
-    return genExprNodeDesc(expr, input, true);
+    return genExprNodeDesc(expr, input, true, false);
   }
 
   public ExprNodeDesc genExprNodeDesc(ASTNode expr, RowResolver input, boolean useCaching)
       throws SemanticException {
-    TypeCheckCtx tcCtx = new TypeCheckCtx(input, useCaching);
+    return genExprNodeDesc(expr, input, useCaching, false);
+  }
+
+  public ExprNodeDesc genExprNodeDesc(ASTNode expr, RowResolver input, boolean useCaching,
+      boolean foldExpr) throws SemanticException {
+    TypeCheckCtx tcCtx = new TypeCheckCtx(input, useCaching, foldExpr);
     return genExprNodeDesc(expr, input, tcCtx);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/da82819b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java
index de1c043..02896ff 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java
@@ -37,6 +37,8 @@ public class TypeCheckCtx implements NodeProcessorCtx {
 
   private final boolean useCaching;
 
+  private final boolean foldExpr;
+
   /**
    * Receives translations which will need to be applied during unparse.
    */
@@ -79,20 +81,21 @@ public class TypeCheckCtx implements NodeProcessorCtx {
    *          The input row resolver of the previous operator.
    */
   public TypeCheckCtx(RowResolver inputRR) {
-    this(inputRR, true);
+    this(inputRR, true, false);
   }
 
-  public TypeCheckCtx(RowResolver inputRR, boolean useCaching) {
-    this(inputRR, useCaching, false, true, true, true, true, true, true, true);
+  public TypeCheckCtx(RowResolver inputRR, boolean useCaching, boolean foldExpr) {
+    this(inputRR, useCaching, foldExpr, false, true, true, true, true, true, true, true);
   }
 
-  public TypeCheckCtx(RowResolver inputRR, boolean useCaching, boolean allowStatefulFunctions,
-      boolean allowDistinctFunctions, boolean allowGBExprElimination, boolean allowAllColRef,
-      boolean allowFunctionStar, boolean allowWindowing,
+  public TypeCheckCtx(RowResolver inputRR, boolean useCaching, boolean foldExpr,
+      boolean allowStatefulFunctions, boolean allowDistinctFunctions, boolean allowGBExprElimination,
+      boolean allowAllColRef, boolean allowFunctionStar, boolean allowWindowing,
       boolean allowIndexExpr, boolean allowSubQueryExpr) {
     setInputRR(inputRR);
     error = null;
     this.useCaching = useCaching;
+    this.foldExpr = foldExpr;
     this.allowStatefulFunctions = allowStatefulFunctions;
     this.allowDistinctFunctions = allowDistinctFunctions;
     this.allowGBExprElimination = allowGBExprElimination;
@@ -209,4 +212,8 @@ public class TypeCheckCtx implements NodeProcessorCtx {
   public boolean isUseCaching() {
     return useCaching;
   }
+
+  public boolean isFoldExpr() {
+    return foldExpr;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/da82819b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index da236d5..ceeb9b4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -61,9 +61,12 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.udf.SettableUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseCompare;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFNvl;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNot;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -1055,6 +1058,14 @@ public class TypeCheckProcFactory {
           }
           desc = ExprNodeGenericFuncDesc.newInstance(genericUDF, funcText,
               childrenList);
+        } else if (ctx.isFoldExpr() && canConvertIntoNvl(genericUDF, children)) {
+          // Rewrite CASE into NVL
+          desc = ExprNodeGenericFuncDesc.newInstance(new GenericUDFNvl(),
+                  Lists.newArrayList(children.get(0), new ExprNodeConstantDesc(false)));
+          if (Boolean.FALSE.equals(((ExprNodeConstantDesc) children.get(1)).getValue())) {
+            desc = ExprNodeGenericFuncDesc.newInstance(new GenericUDFOPNot(),
+                    Lists.newArrayList(desc));
+          }
         } else {
           desc = ExprNodeGenericFuncDesc.newInstance(genericUDF, funcText,
               children);
@@ -1072,6 +1083,21 @@ public class TypeCheckProcFactory {
       return desc;
     }
 
+    private boolean canConvertIntoNvl(GenericUDF genericUDF, ArrayList<ExprNodeDesc> children) {
+      if (genericUDF instanceof GenericUDFWhen && children.size() == 3 &&
+              children.get(1) instanceof ExprNodeConstantDesc &&
+              children.get(2) instanceof ExprNodeConstantDesc) {
+        ExprNodeConstantDesc constThen = (ExprNodeConstantDesc) children.get(1);
+        ExprNodeConstantDesc constElse = (ExprNodeConstantDesc) children.get(2);
+        Object thenVal = constThen.getValue();
+        Object elseVal = constElse.getValue();
+        if (thenVal instanceof Boolean && elseVal instanceof Boolean) {
+          return true;
+        }
+      }
+      return false;
+    }
+
     /**
      * Returns true if des is a descendant of ans (ancestor)
      */

http://git-wip-us.apache.org/repos/asf/hive/blob/da82819b/ql/src/test/queries/clientpositive/constantPropWhen.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/constantPropWhen.q b/ql/src/test/queries/clientpositive/constantPropWhen.q
index c1d4885..03bfd54 100644
--- a/ql/src/test/queries/clientpositive/constantPropWhen.q
+++ b/ql/src/test/queries/clientpositive/constantPropWhen.q
@@ -1,4 +1,5 @@
 set hive.mapred.mode=nonstrict;
+set hive.optimize.constant.propagation=false;
 
 drop table test_1; 
 
@@ -24,6 +25,7 @@ SELECT cast(CASE id when id2 THEN TRUE ELSE FALSE END AS BOOLEAN) AS b FROM test
 
 
 set hive.cbo.enable=false;
+set hive.optimize.constant.propagation=true;
 
 explain SELECT cast(CASE WHEN id = id2 THEN FALSE ELSE TRUE END AS BOOLEAN) AS b FROM test_1; 
 


[06/50] [abbrv] hive git commit: HIVE-13442 : LLAP: refactor submit API to be amenable to signing (Sergey Shelukhin, reviewed by Siddharth Seth)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java b/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java
index ec6e439..e43b72b 100644
--- a/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java
@@ -22,9 +22,11 @@ import com.google.protobuf.ByteString;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UserPayloadProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tez.common.TezCommonUtils;
 import org.apache.tez.dag.api.EntityDescriptor;
 import org.apache.tez.dag.api.InputDescriptor;
@@ -33,7 +35,10 @@ import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.runtime.api.impl.GroupInputSpec;
 import org.apache.tez.runtime.api.impl.InputSpec;
 import org.apache.tez.runtime.api.impl.OutputSpec;
@@ -41,55 +46,88 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class Converters {
 
-  public static TaskSpec getTaskSpecfromProto(FragmentSpecProto FragmentSpecProto) {
-    TezTaskAttemptID taskAttemptID =
-        TezTaskAttemptID.fromString(FragmentSpecProto.getFragmentIdentifierString());
+  public static TaskSpec getTaskSpecfromProto(SignableVertexSpec vectorProto,
+      int fragmentNum, int attemptNum, TezTaskAttemptID attemptId) {
+    VertexIdentifier vertexId = vectorProto.getVertexIdentifier();
+    TezTaskAttemptID taskAttemptID = attemptId != null ? attemptId
+        : createTaskAttemptId(vertexId, fragmentNum, attemptNum);
 
     ProcessorDescriptor processorDescriptor = null;
-    if (FragmentSpecProto.hasProcessorDescriptor()) {
+    if (vectorProto.hasProcessorDescriptor()) {
       processorDescriptor = convertProcessorDescriptorFromProto(
-          FragmentSpecProto.getProcessorDescriptor());
+          vectorProto.getProcessorDescriptor());
     }
 
-    List<InputSpec> inputSpecList = new ArrayList<InputSpec>(FragmentSpecProto.getInputSpecsCount());
-    if (FragmentSpecProto.getInputSpecsCount() > 0) {
-      for (IOSpecProto inputSpecProto : FragmentSpecProto.getInputSpecsList()) {
+    List<InputSpec> inputSpecList = new ArrayList<InputSpec>(vectorProto.getInputSpecsCount());
+    if (vectorProto.getInputSpecsCount() > 0) {
+      for (IOSpecProto inputSpecProto : vectorProto.getInputSpecsList()) {
         inputSpecList.add(getInputSpecFromProto(inputSpecProto));
       }
     }
 
     List<OutputSpec> outputSpecList =
-        new ArrayList<OutputSpec>(FragmentSpecProto.getOutputSpecsCount());
-    if (FragmentSpecProto.getOutputSpecsCount() > 0) {
-      for (IOSpecProto outputSpecProto : FragmentSpecProto.getOutputSpecsList()) {
+        new ArrayList<OutputSpec>(vectorProto.getOutputSpecsCount());
+    if (vectorProto.getOutputSpecsCount() > 0) {
+      for (IOSpecProto outputSpecProto : vectorProto.getOutputSpecsList()) {
         outputSpecList.add(getOutputSpecFromProto(outputSpecProto));
       }
     }
 
     List<GroupInputSpec> groupInputSpecs =
-        new ArrayList<GroupInputSpec>(FragmentSpecProto.getGroupedInputSpecsCount());
-    if (FragmentSpecProto.getGroupedInputSpecsCount() > 0) {
-      for (GroupInputSpecProto groupInputSpecProto : FragmentSpecProto.getGroupedInputSpecsList()) {
+        new ArrayList<GroupInputSpec>(vectorProto.getGroupedInputSpecsCount());
+    if (vectorProto.getGroupedInputSpecsCount() > 0) {
+      for (GroupInputSpecProto groupInputSpecProto : vectorProto.getGroupedInputSpecsList()) {
         groupInputSpecs.add(getGroupInputSpecFromProto(groupInputSpecProto));
       }
     }
 
     TaskSpec taskSpec =
-        new TaskSpec(taskAttemptID, FragmentSpecProto.getDagName(), FragmentSpecProto.getVertexName(),
-            FragmentSpecProto.getVertexParallelism(), processorDescriptor, inputSpecList,
+        new TaskSpec(taskAttemptID, vectorProto.getDagName(), vectorProto.getVertexName(),
+            vectorProto.getVertexParallelism(), processorDescriptor, inputSpecList,
             outputSpecList, groupInputSpecs);
     return taskSpec;
   }
 
-  public static FragmentSpecProto convertTaskSpecToProto(TaskSpec taskSpec) {
-    FragmentSpecProto.Builder builder = FragmentSpecProto.newBuilder();
-    builder.setFragmentIdentifierString(taskSpec.getTaskAttemptID().toString());
+  public static TezTaskAttemptID createTaskAttemptId(
+      VertexIdentifier vertexId, int fragmentNum, int attemptNum) {
+    // Come ride the API roller-coaster!
+    return TezTaskAttemptID.getInstance(
+            TezTaskID.getInstance(
+                TezVertexID.getInstance(
+                    TezDAGID.getInstance(
+                        ConverterUtils.toApplicationId(
+                            vertexId.getApplicationIdString()),
+                        vertexId.getDagId()),
+                    vertexId.getVertexId()),
+                fragmentNum),
+            attemptNum);
+  }
+
+  public static VertexIdentifier createVertexIdentifier(
+      TezTaskAttemptID taId, int appAttemptId) {
+    VertexIdentifier.Builder idBuilder = VertexIdentifier.newBuilder();
+    idBuilder.setApplicationIdString(
+        taId.getTaskID().getVertexID().getDAGId().getApplicationId().toString());
+    idBuilder.setAppAttemptNumber(appAttemptId);
+    idBuilder.setDagId(taId.getTaskID().getVertexID().getDAGId().getId());
+    idBuilder.setVertexId(taId.getTaskID().getVertexID().getId());
+    return idBuilder.build();
+  }
+
+  public static SignableVertexSpec convertTaskSpecToProto(TaskSpec taskSpec,
+      int appAttemptId, String tokenIdentifier, Integer signatureKeyId, String user) {
+    TezTaskAttemptID tId = taskSpec.getTaskAttemptID();
+
+    SignableVertexSpec.Builder builder = SignableVertexSpec.newBuilder();
+    builder.setVertexIdentifier(createVertexIdentifier(tId, appAttemptId));
     builder.setDagName(taskSpec.getDAGName());
-    builder.setDagId(taskSpec.getDagIdentifier());
     builder.setVertexName(taskSpec.getVertexName());
     builder.setVertexParallelism(taskSpec.getVertexParallelism());
-    builder.setFragmentNumber(taskSpec.getTaskAttemptID().getTaskID().getId());
-    builder.setAttemptNumber(taskSpec.getTaskAttemptID().getId());
+    builder.setTokenIdentifier(tokenIdentifier);
+    builder.setUser(user);
+    if (signatureKeyId != null) {
+      builder.setSignatureKeyId(signatureKeyId);
+    }
 
     if (taskSpec.getProcessorDescriptor() != null) {
       builder.setProcessorDescriptor(

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-common/src/protobuf/LlapDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/llap-common/src/protobuf/LlapDaemonProtocol.proto b/llap-common/src/protobuf/LlapDaemonProtocol.proto
index 5cdc02e..486ba0a 100644
--- a/llap-common/src/protobuf/LlapDaemonProtocol.proto
+++ b/llap-common/src/protobuf/LlapDaemonProtocol.proto
@@ -46,19 +46,38 @@ message GroupInputSpecProto {
   optional EntityDescriptorProto merged_input_descriptor = 3;
 }
 
+message VertexIdentifier {
+  optional string application_id_string = 1;
+  optional int32 app_attempt_number = 2;
+  optional int32 dag_id = 3;
+  optional int32 vertex_id = 4;
+}
+
+// The part of SubmitWork that can be signed 
+message SignableVertexSpec
+{
+  optional string user = 1;
+  optional int64 signatureKeyId = 2;
+
+  optional VertexIdentifier vertexIdentifier = 3;
+  // Display names cannot be modified by the client for now. If needed, they should be sent to HS2 who will put them here.
+  optional string dag_name = 4;
+  optional string vertex_name = 5;
+
+  // The core vertex stuff 
+  optional string token_identifier = 6;
+  optional EntityDescriptorProto processor_descriptor = 7;
+  repeated IOSpecProto input_specs = 8;
+  repeated IOSpecProto output_specs = 9;
+  repeated GroupInputSpecProto grouped_input_specs = 10;
+
+  optional int32 vertex_parallelism = 11; // An internal field required for Tez.
+}
 
-message FragmentSpecProto {
-  optional string fragment_identifier_string = 1;
-  optional string dag_name = 2;
-  optional int32 dag_id = 11;
-  optional string vertex_name = 3;
-  optional EntityDescriptorProto processor_descriptor = 4;
-  repeated IOSpecProto input_specs = 5;
-  repeated IOSpecProto output_specs = 6;
-  repeated GroupInputSpecProto grouped_input_specs = 7;
-  optional int32 vertex_parallelism = 8;
-  optional int32 fragment_number =9;
-  optional int32 attempt_number = 10;
+// Union
+message VertexOrBinary {
+  optional SignableVertexSpec vertex = 1;
+  optional bytes vertexBinary = 2; // SignableVertexSpec
 }
 
 message FragmentRuntimeInfo {
@@ -81,18 +100,24 @@ message QueryIdentifierProto {
 }
 
 message SubmitWorkRequestProto {
-  optional string container_id_string = 1;
-  optional string am_host = 2;
-  optional int32 am_port = 3;
-  optional string token_identifier = 4;
-  optional bytes credentials_binary = 5;
-  optional string user = 6;
-  optional string application_id_string = 7;
-  optional int32 app_attempt_number = 8;
-  optional FragmentSpecProto fragment_spec = 9;
-  optional FragmentRuntimeInfo fragment_runtime_info = 10;
+  optional VertexOrBinary work_spec = 1;
+  optional bytes work_spec_signature = 2;
+
+  optional int32 fragment_number = 3;
+  optional int32 attempt_number = 4;
+
+  optional string container_id_string = 5;
+  optional string am_host = 6;
+  optional int32 am_port = 7;
+
+  // Credentials are not signed - the client can add e.g. his own HDFS tokens.
+  optional bytes credentials_binary = 8;
+
+  // Not supported/honored for external clients right now.
+  optional FragmentRuntimeInfo fragment_runtime_info = 9;
 }
 
+
 enum SubmissionStateProto {
   ACCEPTED = 1;
   REJECTED = 2;

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java
----------------------------------------------------------------------
diff --git a/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java b/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java
index d4cdac1..349ee14 100644
--- a/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java
+++ b/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java
@@ -23,8 +23,8 @@ import java.util.List;
 import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UserPayloadProto;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.tez.dag.api.InputDescriptor;
@@ -77,28 +77,24 @@ public class TestConverters {
         new TaskSpec(tezTaskAttemptId, "dagName", "vertexName", 10, processorDescriptor,
             inputSpecList, outputSpecList, null);
 
+    SignableVertexSpec vertexProto = Converters.convertTaskSpecToProto(taskSpec, 0, "", null, "");
 
-    FragmentSpecProto fragmentSpecProto = Converters.convertTaskSpecToProto(taskSpec);
-
-
-    assertEquals("dagName", fragmentSpecProto.getDagName());
-    assertEquals("vertexName", fragmentSpecProto.getVertexName());
-    assertEquals(tezTaskAttemptId.toString(), fragmentSpecProto.getFragmentIdentifierString());
-    assertEquals(tezDagId.getId(), fragmentSpecProto.getDagId());
-    assertEquals(tezTaskAttemptId.getId(), fragmentSpecProto.getAttemptNumber());
-    assertEquals(tezTaskId.getId(), fragmentSpecProto.getFragmentNumber());
+    assertEquals("dagName", vertexProto.getDagName());
+    assertEquals("vertexName", vertexProto.getVertexName());
+    assertEquals(appId.toString(), vertexProto.getVertexIdentifier().getApplicationIdString());
+    assertEquals(tezDagId.getId(), vertexProto.getVertexIdentifier().getDagId());
     assertEquals(processorDescriptor.getClassName(),
-        fragmentSpecProto.getProcessorDescriptor().getClassName());
+        vertexProto.getProcessorDescriptor().getClassName());
     assertEquals(processorDescriptor.getUserPayload().getPayload(),
-        fragmentSpecProto.getProcessorDescriptor().getUserPayload().getUserPayload()
+        vertexProto.getProcessorDescriptor().getUserPayload().getUserPayload()
             .asReadOnlyByteBuffer());
-    assertEquals(2, fragmentSpecProto.getInputSpecsCount());
-    assertEquals(2, fragmentSpecProto.getOutputSpecsCount());
+    assertEquals(2, vertexProto.getInputSpecsCount());
+    assertEquals(2, vertexProto.getOutputSpecsCount());
 
-    verifyInputSpecAndProto(inputSpec1, fragmentSpecProto.getInputSpecs(0));
-    verifyInputSpecAndProto(inputSpec2, fragmentSpecProto.getInputSpecs(1));
-    verifyOutputSpecAndProto(outputSpec1, fragmentSpecProto.getOutputSpecs(0));
-    verifyOutputSpecAndProto(outputSpec2, fragmentSpecProto.getOutputSpecs(1));
+    verifyInputSpecAndProto(inputSpec1, vertexProto.getInputSpecs(0));
+    verifyInputSpecAndProto(inputSpec2, vertexProto.getInputSpecs(1));
+    verifyOutputSpecAndProto(outputSpec1, vertexProto.getOutputSpecs(0));
+    verifyOutputSpecAndProto(outputSpec2, vertexProto.getOutputSpecs(1));
 
   }
 
@@ -120,11 +116,10 @@ public class TestConverters {
     TezTaskID tezTaskId = TezTaskID.getInstance(tezVertexId, 500);
     TezTaskAttemptID tezTaskAttemptId = TezTaskAttemptID.getInstance(tezTaskId, 600);
 
-    FragmentSpecProto.Builder builder = FragmentSpecProto.newBuilder();
-    builder.setFragmentIdentifierString(tezTaskAttemptId.toString());
+    SignableVertexSpec.Builder builder = SignableVertexSpec.newBuilder();
+    builder.setVertexIdentifier(Converters.createVertexIdentifier(tezTaskAttemptId, 0));
     builder.setDagName("dagName");
     builder.setVertexName("vertexName");
-    builder.setDagId(tezDagId.getId());
     builder.setProcessorDescriptor(
         EntityDescriptorProto.newBuilder().setClassName("fakeProcessorName").setUserPayload(
             UserPayloadProto.newBuilder().setUserPayload(ByteString.copyFrom(procBb))));
@@ -145,9 +140,9 @@ public class TestConverters {
             EntityDescriptorProto.newBuilder().setClassName("outputClassName").setUserPayload(
                 UserPayloadProto.newBuilder().setUserPayload(ByteString.copyFrom(output1Bb)))));
 
-    FragmentSpecProto fragmentSpecProto = builder.build();
+    SignableVertexSpec vertexProto = builder.build();
 
-    TaskSpec taskSpec = Converters.getTaskSpecfromProto(fragmentSpecProto);
+    TaskSpec taskSpec = Converters.getTaskSpecfromProto(vertexProto, 0, 0, null);
 
     assertEquals("dagName", taskSpec.getDAGName());
     assertEquals("vertexName", taskSpec.getVertexName());
@@ -160,12 +155,10 @@ public class TestConverters {
     assertEquals(2, taskSpec.getInputs().size());
     assertEquals(2, taskSpec.getOutputs().size());
 
-    verifyInputSpecAndProto(taskSpec.getInputs().get(0), fragmentSpecProto.getInputSpecs(0));
-    verifyInputSpecAndProto(taskSpec.getInputs().get(1), fragmentSpecProto.getInputSpecs(1));
-    verifyOutputSpecAndProto(taskSpec.getOutputs().get(0), fragmentSpecProto.getOutputSpecs(0));
-    verifyOutputSpecAndProto(taskSpec.getOutputs().get(1), fragmentSpecProto.getOutputSpecs(1));
-
-
+    verifyInputSpecAndProto(taskSpec.getInputs().get(0), vertexProto.getInputSpecs(0));
+    verifyInputSpecAndProto(taskSpec.getInputs().get(1), vertexProto.getInputSpecs(1));
+    verifyOutputSpecAndProto(taskSpec.getOutputs().get(0), vertexProto.getOutputSpecs(0));
+    verifyOutputSpecAndProto(taskSpec.getOutputs().get(1), vertexProto.getOutputSpecs(1));
   }
 
   private void verifyInputSpecAndProto(InputSpec inputSpec,

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
index 78b37f7..2bfe3ed 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
@@ -33,11 +33,11 @@ import org.apache.hadoop.hive.llap.daemon.HistoryLogger;
 import org.apache.hadoop.hive.llap.daemon.KilledTaskHandler;
 import org.apache.hadoop.hive.llap.daemon.QueryFailedHandler;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteRequestProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteResponseProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedResponseProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmissionStateProto;
@@ -45,7 +45,9 @@ import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWor
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkResponseProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentRequestProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier;
 import org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorMetrics;
+import org.apache.hadoop.hive.llap.tez.Converters;
 import org.apache.hadoop.hive.ql.exec.tez.TezProcessor;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.security.Credentials;
@@ -151,32 +153,35 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
 
   @Override
   public SubmitWorkResponseProto submitWork(SubmitWorkRequestProto request) throws IOException {
-    HistoryLogger.logFragmentStart(request.getApplicationIdString(), request.getContainerIdString(),
-        localAddress.get().getHostName(), request.getFragmentSpec().getDagName(), request.getFragmentSpec().getDagId(),
-        request.getFragmentSpec().getVertexName(), request.getFragmentSpec().getFragmentNumber(),
-        request.getFragmentSpec().getAttemptNumber());
+    // TODO: also support binary. Actually, we should figure out the binary stuff here and 
+    //       stop passing the protobuf around. We should pass around some plain objects/values.
+    SignableVertexSpec vertex = request.getWorkSpec().getVertex();
     if (LOG.isInfoEnabled()) {
-      LOG.info("Queueing container for execution: " + stringifySubmitRequest(request));
+      LOG.info("Queueing container for execution: " + stringifySubmitRequest(request, vertex));
     }
+    VertexIdentifier vId = vertex.getVertexIdentifier();
+    TezTaskAttemptID attemptId = Converters.createTaskAttemptId(
+        vId, request.getFragmentNumber(), request.getAttemptNumber());
+    String fragmentIdString = attemptId.toString();
+    HistoryLogger.logFragmentStart(vId.getApplicationIdString(), request.getContainerIdString(),
+        localAddress.get().getHostName(), vertex.getDagName(), vId.getDagId(),
+        vertex.getVertexName(), request.getFragmentNumber(), request.getAttemptNumber());
     // This is the start of container-annotated logging.
     // TODO Reduce the length of this string. Way too verbose at the moment.
-    String ndcContextString = request.getFragmentSpec().getFragmentIdentifierString();
-    NDC.push(ndcContextString);
+    NDC.push(fragmentIdString);
     Scheduler.SubmissionState submissionState;
     SubmitWorkResponseProto.Builder responseBuilder = SubmitWorkResponseProto.newBuilder();
     try {
       Map<String, String> env = new HashMap<>();
       // TODO What else is required in this environment map.
       env.putAll(localEnv);
-      env.put(ApplicationConstants.Environment.USER.name(), request.getUser());
+      env.put(ApplicationConstants.Environment.USER.name(), vertex.getUser());
 
-      FragmentSpecProto fragmentSpec = request.getFragmentSpec();
-      TezTaskAttemptID taskAttemptId = TezTaskAttemptID.fromString(
-          fragmentSpec.getFragmentIdentifierString());
+      TezTaskAttemptID taskAttemptId = TezTaskAttemptID.fromString(fragmentIdString);
       int dagIdentifier = taskAttemptId.getTaskID().getVertexID().getDAGId().getId();
 
       QueryIdentifier queryIdentifier = new QueryIdentifier(
-          request.getApplicationIdString(), dagIdentifier);
+          vId.getApplicationIdString(), dagIdentifier);
 
       Credentials credentials = new Credentials();
       DataInputBuffer dib = new DataInputBuffer();
@@ -186,14 +191,10 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
 
       Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
 
-      QueryFragmentInfo fragmentInfo = queryTracker
-          .registerFragment(queryIdentifier, request.getApplicationIdString(),
-              fragmentSpec.getDagName(),
-              dagIdentifier,
-              fragmentSpec.getVertexName(), fragmentSpec.getFragmentNumber(),
-              fragmentSpec.getAttemptNumber(), request.getUser(), request.getFragmentSpec(),
-              jobToken);
-
+      QueryFragmentInfo fragmentInfo = queryTracker.registerFragment(
+          queryIdentifier, vId.getApplicationIdString(), vertex.getDagName(), dagIdentifier,
+          vertex.getVertexName(), request.getFragmentNumber(), request.getAttemptNumber(),
+          vertex.getUser(), vertex, jobToken, fragmentIdString);
 
       String[] localDirs = fragmentInfo.getLocalDirs();
       Preconditions.checkNotNull(localDirs);
@@ -202,14 +203,16 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
       }
       // May need to setup localDir for re-localization, which is usually setup as Environment.PWD.
       // Used for re-localization, to add the user specified configuration (conf_pb_binary_stream)
-      TaskRunnerCallable callable = new TaskRunnerCallable(request, fragmentInfo, new Configuration(getConfig()),
+
+      Configuration callableConf = new Configuration(getConfig());
+      TaskRunnerCallable callable = new TaskRunnerCallable(request, fragmentInfo, callableConf,
           new LlapExecutionContext(localAddress.get().getHostName(), queryTracker), env,
           credentials, memoryPerExecutor, amReporter, confParams, metrics, killedTaskHandler,
-          this, tezHadoopShim);
+          this, tezHadoopShim, attemptId);
       submissionState = executorService.schedule(callable);
 
       if (LOG.isInfoEnabled()) {
-        LOG.info("SubmissionState for {} : {} ", ndcContextString, submissionState);
+        LOG.info("SubmissionState for {} : {} ", fragmentIdString, submissionState);
       }
 
       if (submissionState.equals(Scheduler.SubmissionState.REJECTED)) {
@@ -300,24 +303,25 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
     return sb.toString();
   }
 
-  public static String stringifySubmitRequest(SubmitWorkRequestProto request) {
+  public static String stringifySubmitRequest(
+      SubmitWorkRequestProto request, SignableVertexSpec vertex) {
     StringBuilder sb = new StringBuilder();
-    FragmentSpecProto fragmentSpec = request.getFragmentSpec();
     sb.append("am_details=").append(request.getAmHost()).append(":").append(request.getAmPort());
-    sb.append(", taskInfo=").append(fragmentSpec.getFragmentIdentifierString());
-    sb.append(", user=").append(request.getUser());
-    sb.append(", appIdString=").append(request.getApplicationIdString());
-    sb.append(", appAttemptNum=").append(request.getAppAttemptNumber());
+    sb.append(", taskInfo=").append(vertex.getVertexIdentifier()).append(" fragment ")
+      .append(request.getFragmentNumber()).append(" attempt ").append(request.getAttemptNumber());
+    sb.append(", user=").append(vertex.getUser());
+    sb.append(", appIdString=").append(vertex.getVertexIdentifier().getApplicationIdString());
+    sb.append(", appAttemptNum=").append(vertex.getVertexIdentifier().getAppAttemptNumber());
     sb.append(", containerIdString=").append(request.getContainerIdString());
-    sb.append(", dagName=").append(fragmentSpec.getDagName());
-    sb.append(", vertexName=").append(fragmentSpec.getVertexName());
-    sb.append(", processor=").append(fragmentSpec.getProcessorDescriptor().getClassName());
-    sb.append(", numInputs=").append(fragmentSpec.getInputSpecsCount());
-    sb.append(", numOutputs=").append(fragmentSpec.getOutputSpecsCount());
-    sb.append(", numGroupedInputs=").append(fragmentSpec.getGroupedInputSpecsCount());
+    sb.append(", dagName=").append(vertex.getDagName());
+    sb.append(", vertexName=").append(vertex.getVertexName());
+    sb.append(", processor=").append(vertex.getProcessorDescriptor().getClassName());
+    sb.append(", numInputs=").append(vertex.getInputSpecsCount());
+    sb.append(", numOutputs=").append(vertex.getOutputSpecsCount());
+    sb.append(", numGroupedInputs=").append(vertex.getGroupedInputSpecsCount());
     sb.append(", Inputs={");
-    if (fragmentSpec.getInputSpecsCount() > 0) {
-      for (IOSpecProto ioSpec : fragmentSpec.getInputSpecsList()) {
+    if (vertex.getInputSpecsCount() > 0) {
+      for (IOSpecProto ioSpec : vertex.getInputSpecsList()) {
         sb.append("{").append(ioSpec.getConnectedVertexName()).append(",")
             .append(ioSpec.getIoDescriptor().getClassName()).append(",")
             .append(ioSpec.getPhysicalEdgeCount()).append("}");
@@ -325,8 +329,8 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
     }
     sb.append("}");
     sb.append(", Outputs={");
-    if (fragmentSpec.getOutputSpecsCount() > 0) {
-      for (IOSpecProto ioSpec : fragmentSpec.getOutputSpecsList()) {
+    if (vertex.getOutputSpecsCount() > 0) {
+      for (IOSpecProto ioSpec : vertex.getOutputSpecsList()) {
         sb.append("{").append(ioSpec.getConnectedVertexName()).append(",")
             .append(ioSpec.getIoDescriptor().getClassName()).append(",")
             .append(ioSpec.getPhysicalEdgeCount()).append("}");
@@ -334,8 +338,8 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
     }
     sb.append("}");
     sb.append(", GroupedInputs={");
-    if (fragmentSpec.getGroupedInputSpecsCount() > 0) {
-      for (GroupInputSpecProto group : fragmentSpec.getGroupedInputSpecsList()) {
+    if (vertex.getGroupedInputSpecsCount() > 0) {
+      for (GroupInputSpecProto group : vertex.getGroupedInputSpecsList()) {
         sb.append("{").append("groupName=").append(group.getGroupName()).append(", elements=")
             .append(group.getGroupVerticesList()).append("}");
         sb.append(group.getGroupVerticesList());

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryFragmentInfo.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryFragmentInfo.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryFragmentInfo.java
index 480a394..195775e 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryFragmentInfo.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryFragmentInfo.java
@@ -21,8 +21,8 @@ import java.util.List;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hive.llap.daemon.FinishableStateUpdateHandler;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.tezplugins.LlapTezUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,19 +35,20 @@ public class QueryFragmentInfo {
   private final String vertexName;
   private final int fragmentNumber;
   private final int attemptNumber;
-  private final FragmentSpecProto fragmentSpec;
+  private final SignableVertexSpec vertexSpec;
+  private final String fragmentIdString;
 
   public QueryFragmentInfo(QueryInfo queryInfo, String vertexName, int fragmentNumber,
-                           int attemptNumber,
-                           FragmentSpecProto fragmentSpec) {
+      int attemptNumber, SignableVertexSpec vertexSpec, String fragmentIdString) {
     Preconditions.checkNotNull(queryInfo);
     Preconditions.checkNotNull(vertexName);
-    Preconditions.checkNotNull(fragmentSpec);
+    Preconditions.checkNotNull(vertexSpec);
     this.queryInfo = queryInfo;
     this.vertexName = vertexName;
     this.fragmentNumber = fragmentNumber;
     this.attemptNumber = attemptNumber;
-    this.fragmentSpec = fragmentSpec;
+    this.vertexSpec = vertexSpec;
+    this.fragmentIdString = fragmentIdString;
   }
 
   // Only meant for use by the QueryTracker
@@ -55,8 +56,8 @@ public class QueryFragmentInfo {
     return this.queryInfo;
   }
 
-  public FragmentSpecProto getFragmentSpec() {
-    return fragmentSpec;
+  public SignableVertexSpec getVertexSpec() {
+    return vertexSpec;
   }
 
   public String getVertexName() {
@@ -72,7 +73,7 @@ public class QueryFragmentInfo {
   }
 
   public String getFragmentIdentifierString() {
-    return fragmentSpec.getFragmentIdentifierString();
+    return fragmentIdString;
   }
 
   /**
@@ -85,7 +86,7 @@ public class QueryFragmentInfo {
    * @return true if the task can finish, false otherwise
    */
   public boolean canFinish() {
-    List<IOSpecProto> inputSpecList = fragmentSpec.getInputSpecsList();
+    List<IOSpecProto> inputSpecList = vertexSpec.getInputSpecsList();
     boolean canFinish = true;
     if (inputSpecList != null && !inputSpecList.isEmpty()) {
       for (IOSpecProto inputSpec : inputSpecList) {
@@ -126,7 +127,7 @@ public class QueryFragmentInfo {
   public boolean registerForFinishableStateUpdates(FinishableStateUpdateHandler handler,
                                                 boolean lastFinishableState) {
     List<String> sourcesOfInterest = new LinkedList<>();
-    List<IOSpecProto> inputSpecList = fragmentSpec.getInputSpecsList();
+    List<IOSpecProto> inputSpecList = vertexSpec.getInputSpecsList();
     if (inputSpecList != null && !inputSpecList.isEmpty()) {
       for (IOSpecProto inputSpec : inputSpecList) {
         if (LlapTezUtils.isSourceOfInterest(inputSpec.getIoDescriptor().getClassName())) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
index 8daef9e..6914134 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
@@ -35,7 +35,7 @@ import com.google.common.collect.Multimap;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.llap.daemon.FinishableStateUpdateHandler;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto;
 
 public class QueryInfo {
@@ -92,9 +92,10 @@ public class QueryInfo {
     return sourceStateMap;
   }
 
-  public QueryFragmentInfo registerFragment(String vertexName, int fragmentNumber, int attemptNumber, FragmentSpecProto fragmentSpec) {
-    QueryFragmentInfo fragmentInfo = new QueryFragmentInfo(this, vertexName, fragmentNumber, attemptNumber,
-        fragmentSpec);
+  public QueryFragmentInfo registerFragment(String vertexName, int fragmentNumber,
+      int attemptNumber, SignableVertexSpec vertexSpec, String fragmentIdString) {
+    QueryFragmentInfo fragmentInfo = new QueryFragmentInfo(
+        this, vertexName, fragmentNumber, attemptNumber, vertexSpec, fragmentIdString);
     knownFragments.add(fragmentInfo);
     return fragmentInfo;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
index cb3be2b..8abd198 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto;
 import org.apache.hadoop.hive.llap.shufflehandler.ShuffleHandler;
 import org.apache.hadoop.hive.ql.exec.ObjectCacheFactory;
@@ -113,20 +113,11 @@ public class QueryTracker extends AbstractService {
 
   /**
    * Register a new fragment for a specific query
-   * @param queryIdentifier
-   * @param appIdString
-   * @param dagName
-   * @param dagIdentifier
-   * @param vertexName
-   * @param fragmentNumber
-   * @param attemptNumber
-   * @param user
-   * @throws IOException
    */
   QueryFragmentInfo registerFragment(QueryIdentifier queryIdentifier, String appIdString,
       String dagName, int dagIdentifier, String vertexName, int fragmentNumber, int attemptNumber,
-      String user, FragmentSpecProto fragmentSpec, Token<JobTokenIdentifier> appToken)
-          throws IOException {
+      String user, SignableVertexSpec vertex, Token<JobTokenIdentifier> appToken,
+      String fragmentIdString) throws IOException {
     ReadWriteLock dagLock = getDagLock(queryIdentifier);
     dagLock.readLock().lock();
     try {
@@ -166,7 +157,8 @@ public class QueryTracker extends AbstractService {
           .registerDag(appIdString, dagIdentifier, appToken,
               user, queryInfo.getLocalDirs());
 
-      return queryInfo.registerFragment(vertexName, fragmentNumber, attemptNumber, fragmentSpec);
+      return queryInfo.registerFragment(
+          vertexName, fragmentNumber, attemptNumber, vertex, fragmentIdString);
     } finally {
       dagLock.readLock().unlock();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
index 1933eb1..eac0e8f 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
@@ -43,7 +43,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.hadoop.hive.llap.daemon.FinishableStateUpdateHandler;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorMetrics;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.tez.runtime.task.EndReason;
@@ -191,8 +191,8 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
       TaskWrapper task = e.getValue();
       boolean isFirst = true;
       TaskRunnerCallable c = task.getTaskRunnerCallable();
-      if (c != null && c.getRequest() != null && c.getRequest().getFragmentSpec() != null) {
-        FragmentSpecProto fs = c.getRequest().getFragmentSpec();
+      if (c != null && c.getVertexSpec() != null) {
+        SignableVertexSpec fs = c.getVertexSpec();
         value.append(isFirst ? " (" : ", ").append(fs.getDagName())
           .append("/").append(fs.getVertexName());
         isFirst = false;
@@ -781,7 +781,7 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
           ", firstAttemptStartTime=" + taskRunnerCallable.getFragmentRuntimeInfo().getFirstAttemptStartTime() +
           ", dagStartTime=" + taskRunnerCallable.getFragmentRuntimeInfo().getDagStartTime() +
           ", withinDagPriority=" + taskRunnerCallable.getFragmentRuntimeInfo().getWithinDagPriority() +
-          ", vertexParallelism= " + taskRunnerCallable.getFragmentSpec().getVertexParallelism() +
+          ", vertexParallelism= " + taskRunnerCallable.getVertexSpec().getVertexParallelism() +
           ", selfAndUpstreamParallelism= " + taskRunnerCallable.getFragmentRuntimeInfo().getNumSelfAndUpstreamTasks() +
           ", selfAndUpstreamComplete= " + taskRunnerCallable.getFragmentRuntimeInfo().getNumSelfAndUpstreamCompletedTasks() +
           '}';

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
index fcfa940..3093de7 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
@@ -33,8 +33,8 @@ import org.apache.hadoop.hive.llap.daemon.FragmentCompletionHandler;
 import org.apache.hadoop.hive.llap.daemon.HistoryLogger;
 import org.apache.hadoop.hive.llap.daemon.KilledTaskHandler;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
 import org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorMetrics;
 import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
@@ -113,6 +113,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
   private final AtomicBoolean isStarted = new AtomicBoolean(false);
   private final AtomicBoolean isCompleted = new AtomicBoolean(false);
   private final AtomicBoolean killInvoked = new AtomicBoolean(false);
+  private final SignableVertexSpec vertex;
 
   @VisibleForTesting
   public TaskRunnerCallable(SubmitWorkRequestProto request, QueryFragmentInfo fragmentInfo,
@@ -123,7 +124,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
                      ConfParams confParams, LlapDaemonExecutorMetrics metrics,
                      KilledTaskHandler killedTaskHandler,
                      FragmentCompletionHandler fragmentCompleteHandler,
-                     HadoopShim tezHadoopShim) {
+                     HadoopShim tezHadoopShim, TezTaskAttemptID attemptId) {
     this.request = request;
     this.fragmentInfo = fragmentInfo;
     this.conf = conf;
@@ -134,17 +135,20 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
     this.memoryAvailable = memoryAvailable;
     this.confParams = confParams;
     this.jobToken = TokenCache.getSessionToken(credentials);
-    this.taskSpec = Converters.getTaskSpecfromProto(request.getFragmentSpec());
+    // TODO: support binary spec here or above
+    this.vertex = request.getWorkSpec().getVertex();
+    this.taskSpec = Converters.getTaskSpecfromProto(
+        vertex, request.getFragmentNumber(), request.getAttemptNumber(), attemptId);
     this.amReporter = amReporter;
     // Register with the AMReporter when the callable is setup. Unregister once it starts running.
     if (jobToken != null) {
     this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
-        request.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
+        vertex.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
     }
     this.metrics = metrics;
-    this.requestId = request.getFragmentSpec().getFragmentIdentifierString();
+    this.requestId = taskSpec.getTaskAttemptID().toString();
     // TODO Change this to the queryId/Name when that's available.
-    this.queryId = request.getFragmentSpec().getDagName();
+    this.queryId = vertex.getDagName();
     this.killedTaskHandler = killedTaskHandler;
     this.fragmentCompletionHanler = fragmentCompleteHandler;
     this.tezHadoopShim = tezHadoopShim;
@@ -184,16 +188,16 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
 
     // TODO Consolidate this code with TezChild.
     runtimeWatch.start();
-    UserGroupInformation taskUgi = UserGroupInformation.createRemoteUser(request.getUser());
+    UserGroupInformation taskUgi = UserGroupInformation.createRemoteUser(vertex.getUser());
     taskUgi.addCredentials(credentials);
 
     Map<String, ByteBuffer> serviceConsumerMetadata = new HashMap<>();
     serviceConsumerMetadata.put(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID,
         TezCommonUtils.convertJobTokenToBytes(jobToken));
-    Multimap<String, String> startedInputsMap = createStartedInputMap(request.getFragmentSpec());
+    Multimap<String, String> startedInputsMap = createStartedInputMap(vertex);
 
     UserGroupInformation taskOwner =
-        UserGroupInformation.createRemoteUser(request.getTokenIdentifier());
+        UserGroupInformation.createRemoteUser(vertex.getTokenIdentifier());
     final InetSocketAddress address =
         NetUtils.createSocketAddrForHost(request.getAmHost(), request.getAmPort());
     SecurityUtil.setTokenService(jobToken, address);
@@ -228,7 +232,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
         if (shouldRunTask) {
           taskRunner = new TezTaskRunner2(conf, taskUgi, fragmentInfo.getLocalDirs(),
               taskSpec,
-              request.getAppAttemptNumber(),
+              vertex.getVertexIdentifier().getAppAttemptNumber(),
               serviceConsumerMetadata, envMap, startedInputsMap, taskReporter, executor,
               objectRegistry,
               pid,
@@ -313,7 +317,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
    */
   public void reportTaskKilled() {
     killedTaskHandler
-        .taskKilled(request.getAmHost(), request.getAmPort(), request.getUser(), jobToken,
+        .taskKilled(request.getAmHost(), request.getAmPort(), vertex.getUser(), jobToken,
             fragmentInfo.getQueryInfo().getQueryIdentifier(), taskSpec.getTaskAttemptID());
   }
 
@@ -321,15 +325,15 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
     return fragmentInfo.canFinish();
   }
 
-  private Multimap<String, String> createStartedInputMap(FragmentSpecProto fragmentSpec) {
+  private static Multimap<String, String> createStartedInputMap(SignableVertexSpec vertex) {
     Multimap<String, String> startedInputMap = HashMultimap.create();
     // Let the Processor control start for Broadcast inputs.
 
     // TODO For now, this affects non broadcast unsorted cases as well. Make use of the edge
     // property when it's available.
-    for (IOSpecProto inputSpec : fragmentSpec.getInputSpecsList()) {
+    for (IOSpecProto inputSpec : vertex.getInputSpecsList()) {
       if (inputSpec.getIoDescriptor().getClassName().equals(UnorderedKVInput.class.getName())) {
-        startedInputMap.put(fragmentSpec.getVertexName(), inputSpec.getConnectedVertexName());
+        startedInputMap.put(vertex.getVertexName(), inputSpec.getConnectedVertexName());
       }
     }
     return startedInputMap;
@@ -350,7 +354,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
   @Override
   public String toString() {
     return requestId + " {canFinish: " + canFinish() +
-        ", vertexParallelism: " + request.getFragmentSpec().getVertexParallelism() +
+        ", vertexParallelism: " + vertex.getVertexParallelism() +
         ", selfAndUpstreamParallelism: " + request.getFragmentRuntimeInfo().getNumSelfAndUpstreamTasks() +
         ", selfAndUpstreamComplete: " + request.getFragmentRuntimeInfo().getNumSelfAndUpstreamCompletedTasks() +
         ", firstAttemptStartTime: " + getFragmentRuntimeInfo().getFirstAttemptStartTime() +
@@ -454,14 +458,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
       fragmentCompletionHanler.fragmentComplete(fragmentInfo);
 
       taskRunnerCallable.shutdown();
-      HistoryLogger
-          .logFragmentEnd(request.getApplicationIdString(), request.getContainerIdString(),
-              executionContext.getHostName(), request.getFragmentSpec().getDagName(),
-              fragmentInfo.getQueryInfo().getDagIdentifier(),
-              request.getFragmentSpec().getVertexName(),
-              request.getFragmentSpec().getFragmentNumber(),
-              request.getFragmentSpec().getAttemptNumber(), taskRunnerCallable.threadName,
-              taskRunnerCallable.startTime, true);
+      logFragmentEnd(true);
     }
 
     @Override
@@ -471,14 +468,15 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
       fragmentCompletionHanler.fragmentComplete(fragmentInfo);
       // TODO HIVE-10236 Report a fatal error over the umbilical
       taskRunnerCallable.shutdown();
-      HistoryLogger
-          .logFragmentEnd(request.getApplicationIdString(), request.getContainerIdString(),
-              executionContext.getHostName(), request.getFragmentSpec().getDagName(),
-              fragmentInfo.getQueryInfo().getDagIdentifier(),
-              request.getFragmentSpec().getVertexName(),
-              request.getFragmentSpec().getFragmentNumber(),
-              request.getFragmentSpec().getAttemptNumber(), taskRunnerCallable.threadName,
-              taskRunnerCallable.startTime, false);
+      logFragmentEnd(false);
+    }
+
+    protected void logFragmentEnd(boolean success) {
+      HistoryLogger.logFragmentEnd(vertex.getVertexIdentifier().getApplicationIdString(),
+          request.getContainerIdString(), executionContext.getHostName(), vertex.getDagName(),
+          fragmentInfo.getQueryInfo().getDagIdentifier(), vertex.getVertexName(),
+          request.getFragmentNumber(), request.getAttemptNumber(), taskRunnerCallable.threadName,
+          taskRunnerCallable.startTime, success);
     }
   }
 
@@ -498,12 +496,14 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
   public static String getTaskIdentifierString(
       SubmitWorkRequestProto request) {
     StringBuilder sb = new StringBuilder();
-    sb.append("AppId=").append(request.getApplicationIdString())
+    // TODO: also support the binary version
+    SignableVertexSpec vertex = request.getWorkSpec().getVertex();
+    sb.append("AppId=").append(vertex.getVertexIdentifier().getApplicationIdString())
         .append(", containerId=").append(request.getContainerIdString())
-        .append(", Dag=").append(request.getFragmentSpec().getDagName())
-        .append(", Vertex=").append(request.getFragmentSpec().getVertexName())
-        .append(", FragmentNum=").append(request.getFragmentSpec().getFragmentNumber())
-        .append(", Attempt=").append(request.getFragmentSpec().getAttemptNumber());
+        .append(", Dag=").append(vertex.getDagName())
+        .append(", Vertex=").append(vertex.getVertexName())
+        .append(", FragmentNum=").append(request.getFragmentNumber())
+        .append(", Attempt=").append(request.getAttemptNumber());
     return sb.toString();
   }
 
@@ -511,7 +511,8 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
     return request.getFragmentRuntimeInfo();
   }
 
-  public FragmentSpecProto getFragmentSpec() {
-    return request.getFragmentSpec();
+  public SignableVertexSpec getVertexSpec() {
+    // TODO: support for binary spec? presumably we'd parse it somewhere earlier
+    return vertex;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
index c6ba14e..d699f20 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
@@ -26,9 +26,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.llap.daemon.FragmentCompletionHandler;
 import org.apache.hadoop.hive.llap.daemon.KilledTaskHandler;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary;
 import org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorMetrics;
+import org.apache.hadoop.hive.llap.tez.Converters;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.tez.dag.records.TezDAGID;
@@ -51,26 +53,25 @@ public class TaskExecutorTestHelpers {
     SubmitWorkRequestProto
         requestProto = createSubmitWorkRequestProto(fragmentNum, parallelism,
         startTime);
-    QueryFragmentInfo queryFragmentInfo = createQueryFragmentInfo(requestProto.getFragmentSpec());
+    QueryFragmentInfo queryFragmentInfo = createQueryFragmentInfo(
+        requestProto.getWorkSpec().getVertex(), requestProto.getFragmentNumber());
     MockRequest mockRequest = new MockRequest(requestProto, queryFragmentInfo, canFinish, workTime);
     return mockRequest;
   }
 
   public static TaskExecutorService.TaskWrapper createTaskWrapper(
       SubmitWorkRequestProto request, boolean canFinish, int workTime) {
-    QueryFragmentInfo queryFragmentInfo = createQueryFragmentInfo(request.getFragmentSpec());
+    QueryFragmentInfo queryFragmentInfo = createQueryFragmentInfo(
+        request.getWorkSpec().getVertex(), request.getFragmentNumber());
     MockRequest mockRequest = new MockRequest(request, queryFragmentInfo, canFinish, workTime);
     TaskExecutorService.TaskWrapper
         taskWrapper = new TaskExecutorService.TaskWrapper(mockRequest, null);
     return taskWrapper;
   }
 
-  public static QueryFragmentInfo createQueryFragmentInfo(FragmentSpecProto fragmentSpecProto) {
-    QueryInfo queryInfo = createQueryInfo();
-    QueryFragmentInfo fragmentInfo =
-        new QueryFragmentInfo(queryInfo, "fakeVertexName", fragmentSpecProto.getFragmentNumber(), 0,
-            fragmentSpecProto);
-    return fragmentInfo;
+  public static QueryFragmentInfo createQueryFragmentInfo(
+      SignableVertexSpec vertex, int fragmentNum) {
+    return new QueryFragmentInfo(createQueryInfo(), "fakeVertexName", fragmentNum, 0, vertex, "");
   }
 
   public static QueryInfo createQueryInfo() {
@@ -100,20 +101,23 @@ public class TaskExecutorTestHelpers {
     TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tId, fragmentNumber);
     return SubmitWorkRequestProto
         .newBuilder()
-        .setFragmentSpec(
-            FragmentSpecProto
-                .newBuilder()
-                .setAttemptNumber(0)
+        .setAttemptNumber(0)
+        .setFragmentNumber(fragmentNumber)
+        .setWorkSpec(
+            VertexOrBinary.newBuilder().setVertex(
+            SignableVertexSpec.newBuilder()
                 .setDagName("MockDag")
-                .setFragmentNumber(fragmentNumber)
+                .setUser("MockUser")
+                .setTokenIdentifier("MockToken_1")
+                .setVertexIdentifier(Converters.createVertexIdentifier(taId, 0))
                 .setVertexName("MockVertex")
                 .setProcessorDescriptor(
                     LlapDaemonProtocolProtos.EntityDescriptorProto.newBuilder()
                         .setClassName("MockProcessor").build())
-                .setFragmentIdentifierString(taId.toString()).build()).setAmHost("localhost")
-        .setAmPort(12345).setAppAttemptNumber(0).setApplicationIdString("MockApp_1")
-        .setContainerIdString("MockContainer_1").setUser("MockUser")
-        .setTokenIdentifier("MockToken_1")
+                .build()).build())
+        .setAmHost("localhost")
+        .setAmPort(12345)
+        .setContainerIdString("MockContainer_1")
         .setFragmentRuntimeInfo(LlapDaemonProtocolProtos
             .FragmentRuntimeInfo
             .newBuilder()
@@ -146,7 +150,7 @@ public class TaskExecutorTestHelpers {
           new ExecutionContextImpl("localhost"), null, new Credentials(), 0, null, null, mock(
               LlapDaemonExecutorMetrics.class),
           mock(KilledTaskHandler.class), mock(
-              FragmentCompletionHandler.class), new DefaultHadoopShim());
+              FragmentCompletionHandler.class), new DefaultHadoopShim(), null);
       this.workTime = workTime;
       this.canFinish = canFinish;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/comparator/TestFirstInFirstOutComparator.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/comparator/TestFirstInFirstOutComparator.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/comparator/TestFirstInFirstOutComparator.java
index 08ee769..a250882 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/comparator/TestFirstInFirstOutComparator.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/comparator/TestFirstInFirstOutComparator.java
@@ -31,8 +31,11 @@ import org.apache.hadoop.hive.llap.daemon.impl.TaskExecutorService.TaskWrapper;
 import org.apache.hadoop.hive.llap.daemon.impl.TaskRunnerCallable;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary;
+import org.apache.hadoop.hive.llap.tez.Converters;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.tez.dag.records.TezDAGID;
@@ -59,7 +62,7 @@ public class TestFirstInFirstOutComparator {
       super(requestProto, mock(QueryFragmentInfo.class), conf,
           new ExecutionContextImpl("localhost"), null, cred, 0, null, null, null,
           mock(KilledTaskHandler.class), mock(
-          FragmentCompletionHandler.class), new DefaultHadoopShim());
+          FragmentCompletionHandler.class), new DefaultHadoopShim(), null);
       this.workTime = workTime;
       this.canFinish = canFinish;
     }
@@ -102,19 +105,23 @@ public class TestFirstInFirstOutComparator {
     TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tId, fragmentNumber);
     return SubmitWorkRequestProto
         .newBuilder()
-        .setFragmentSpec(
-            FragmentSpecProto
+        .setAttemptNumber(0)
+        .setFragmentNumber(fragmentNumber)
+        .setWorkSpec(
+            VertexOrBinary.newBuilder().setVertex(
+            SignableVertexSpec
                 .newBuilder()
-                .setAttemptNumber(0)
+                .setVertexIdentifier(Converters.createVertexIdentifier(taId, 0))
                 .setDagName("MockDag")
-                .setFragmentNumber(fragmentNumber)
                 .setVertexName("MockVertex")
+                .setUser("MockUser")
+                .setTokenIdentifier("MockToken_1")
                 .setProcessorDescriptor(
                     EntityDescriptorProto.newBuilder().setClassName("MockProcessor").build())
-                .setFragmentIdentifierString(taId.toString()).build()).setAmHost("localhost")
-        .setAmPort(12345).setAppAttemptNumber(0).setApplicationIdString("MockApp_1")
-        .setContainerIdString("MockContainer_1").setUser("MockUser")
-        .setTokenIdentifier("MockToken_1")
+                .build()).build())
+        .setAmHost("localhost")
+        .setAmPort(12345)
+        .setContainerIdString("MockContainer_1")
         .setFragmentRuntimeInfo(LlapDaemonProtocolProtos
             .FragmentRuntimeInfo
             .newBuilder()

http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
index b4b041a..a3f2eb8 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWor
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkResponseProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentRequestProto;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary;
 import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
 import org.apache.hadoop.hive.llap.registry.ServiceInstance;
 import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService;
@@ -89,10 +90,7 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
   private static final Logger LOG = LoggerFactory.getLogger(LlapTaskCommunicator.class);
 
   private static final boolean isInfoEnabled = LOG.isInfoEnabled();
-  private static final boolean isDebugEnabed = LOG.isDebugEnabled();
-
-  private final SubmitWorkRequestProto BASE_SUBMIT_WORK_REQUEST;
-
+  
   private final ConcurrentMap<QueryIdentifierProto, ByteBuffer> credentialMap;
 
   // Tracks containerIds and taskAttemptIds, so can be kept independent of the running DAG.
@@ -105,6 +103,8 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
   private long deleteDelayOnDagComplete;
   private final LlapTaskUmbilicalProtocol umbilical;
   private final Token<LlapTokenIdentifier> token;
+  private final int appAttemptId;
+  private final String user;
 
   // These two structures track the list of known nodes, and the list of nodes which are sending in keep-alive heartbeats.
   // Primarily for debugging purposes a.t.m, since there's some unexplained TASK_TIMEOUTS which are currently being observed.
@@ -113,8 +113,6 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
 
   private final LlapRegistryService serviceRegistry;
 
-
-  private volatile int currentDagId;
   private volatile QueryIdentifierProto currentQueryIdentifierProto;
 
   public LlapTaskCommunicator(
@@ -138,17 +136,10 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
     serviceRegistry = LlapRegistryService.getClient(conf);
 
     umbilical = new LlapTaskUmbilicalProtocolImpl(getUmbilical());
-    SubmitWorkRequestProto.Builder baseBuilder = SubmitWorkRequestProto.newBuilder();
 
     // TODO Avoid reading this from the environment
-    baseBuilder.setUser(System.getenv(ApplicationConstants.Environment.USER.name()));
-    baseBuilder.setApplicationIdString(
-        taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString());
-    baseBuilder
-        .setAppAttemptNumber(taskCommunicatorContext.getApplicationAttemptId().getAttemptId());
-    baseBuilder.setTokenIdentifier(getTokenIdentifier());
-
-    BASE_SUBMIT_WORK_REQUEST = baseBuilder.build();
+    user = System.getenv(ApplicationConstants.Environment.USER.name());
+    appAttemptId = taskCommunicatorContext.getApplicationAttemptId().getAttemptId();
 
     credentialMap = new ConcurrentHashMap<>();
     sourceStateTracker = new SourceStateTracker(getContext(), this);
@@ -316,7 +307,6 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
               t = se.getCause();
             }
             if (t instanceof RemoteException) {
-              RemoteException re = (RemoteException) t;
               // All others from the remote service cause the task to FAIL.
               LOG.info(
                   "Failed to run task: " + taskSpec.getTaskAttemptID() + " on containerId: " +
@@ -591,8 +581,9 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
                                                             TaskSpec taskSpec,
                                                             FragmentRuntimeInfo fragmentRuntimeInfo) throws
       IOException {
-    SubmitWorkRequestProto.Builder builder =
-        SubmitWorkRequestProto.newBuilder(BASE_SUBMIT_WORK_REQUEST);
+    SubmitWorkRequestProto.Builder builder = SubmitWorkRequestProto.newBuilder();
+    builder.setFragmentNumber(taskSpec.getTaskAttemptID().getTaskID().getId());
+    builder.setAttemptNumber(taskSpec.getTaskAttemptID().getId());
     builder.setContainerIdString(containerId.toString());
     builder.setAmHost(getAddress().getHostName());
     builder.setAmPort(getAddress().getPort());
@@ -607,7 +598,9 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
       credentialsBinary = credentialsBinary.duplicate();
     }
     builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
-    builder.setFragmentSpec(Converters.convertTaskSpecToProto(taskSpec));
+    builder.setWorkSpec(VertexOrBinary.newBuilder().setVertex(Converters.convertTaskSpecToProto(
+        taskSpec, appAttemptId, getTokenIdentifier(), null, user)).build());
+    // Don't call builder.setWorkSpecSignature() - Tez doesn't sign fragments
     builder.setFragmentRuntimeInfo(fragmentRuntimeInfo);
     return builder.build();
   }


[50/50] [abbrv] hive git commit: HIVE-13587: Set Hive pom to use Hadoop 2.6.1 (Mohit Sabharwal, reviewd by Sergio Pena)

Posted by sp...@apache.org.
HIVE-13587: Set Hive pom to use Hadoop 2.6.1 (Mohit Sabharwal, reviewd by Sergio Pena)


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

Branch: refs/heads/java8
Commit: 2695a6356ab4902b305e1e9fb6fa5b61d683a73d
Parents: 92a9ae8
Author: Sergio Pena <se...@cloudera.com>
Authored: Mon May 2 13:08:54 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Fri May 6 15:41:24 2016 -0500

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2695a635/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c99b05a..517d340 100644
--- a/pom.xml
+++ b/pom.xml
@@ -132,7 +132,7 @@
     <dropwizard-metrics-hadoop-metrics2-reporter.version>0.1.0</dropwizard-metrics-hadoop-metrics2-reporter.version>
     <guava.version>14.0.1</guava.version>
     <groovy.version>2.4.4</groovy.version>
-    <hadoop.version>2.6.0</hadoop.version>
+    <hadoop.version>2.6.1</hadoop.version>
     <hadoop.bin.path>${basedir}/${hive.path.to.root}/testutils/hadoop</hadoop.bin.path>
     <hbase.version>1.1.1</hbase.version>
     <!-- required for logging test to avoid including hbase which pulls disruptor transitively -->


[18/50] [abbrv] hive git commit: HIVE-13351: Support drop Primary Key/Foreign Key constraints (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 8da883d..36a0f96 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -8999,6 +8999,138 @@ void ForeignKeysResponse::printTo(std::ostream& out) const {
 }
 
 
+DropConstraintRequest::~DropConstraintRequest() throw() {
+}
+
+
+void DropConstraintRequest::__set_dbname(const std::string& val) {
+  this->dbname = val;
+}
+
+void DropConstraintRequest::__set_tablename(const std::string& val) {
+  this->tablename = val;
+}
+
+void DropConstraintRequest::__set_constraintname(const std::string& val) {
+  this->constraintname = val;
+}
+
+uint32_t DropConstraintRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_dbname = false;
+  bool isset_tablename = false;
+  bool isset_constraintname = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->dbname);
+          isset_dbname = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tablename);
+          isset_tablename = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->constraintname);
+          isset_constraintname = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_dbname)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_tablename)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_constraintname)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t DropConstraintRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("DropConstraintRequest");
+
+  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->dbname);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->tablename);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("constraintname", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->constraintname);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(DropConstraintRequest &a, DropConstraintRequest &b) {
+  using ::std::swap;
+  swap(a.dbname, b.dbname);
+  swap(a.tablename, b.tablename);
+  swap(a.constraintname, b.constraintname);
+}
+
+DropConstraintRequest::DropConstraintRequest(const DropConstraintRequest& other377) {
+  dbname = other377.dbname;
+  tablename = other377.tablename;
+  constraintname = other377.constraintname;
+}
+DropConstraintRequest& DropConstraintRequest::operator=(const DropConstraintRequest& other378) {
+  dbname = other378.dbname;
+  tablename = other378.tablename;
+  constraintname = other378.constraintname;
+  return *this;
+}
+void DropConstraintRequest::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "DropConstraintRequest(";
+  out << "dbname=" << to_string(dbname);
+  out << ", " << "tablename=" << to_string(tablename);
+  out << ", " << "constraintname=" << to_string(constraintname);
+  out << ")";
+}
+
+
 PartitionsByExprResult::~PartitionsByExprResult() throw() {
 }
 
@@ -9038,14 +9170,14 @@ uint32_t PartitionsByExprResult::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size377;
-            ::apache::thrift::protocol::TType _etype380;
-            xfer += iprot->readListBegin(_etype380, _size377);
-            this->partitions.resize(_size377);
-            uint32_t _i381;
-            for (_i381 = 0; _i381 < _size377; ++_i381)
+            uint32_t _size379;
+            ::apache::thrift::protocol::TType _etype382;
+            xfer += iprot->readListBegin(_etype382, _size379);
+            this->partitions.resize(_size379);
+            uint32_t _i383;
+            for (_i383 = 0; _i383 < _size379; ++_i383)
             {
-              xfer += this->partitions[_i381].read(iprot);
+              xfer += this->partitions[_i383].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9086,10 +9218,10 @@ uint32_t PartitionsByExprResult::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitions.size()));
-    std::vector<Partition> ::const_iterator _iter382;
-    for (_iter382 = this->partitions.begin(); _iter382 != this->partitions.end(); ++_iter382)
+    std::vector<Partition> ::const_iterator _iter384;
+    for (_iter384 = this->partitions.begin(); _iter384 != this->partitions.end(); ++_iter384)
     {
-      xfer += (*_iter382).write(oprot);
+      xfer += (*_iter384).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9110,13 +9242,13 @@ void swap(PartitionsByExprResult &a, PartitionsByExprResult &b) {
   swap(a.hasUnknownPartitions, b.hasUnknownPartitions);
 }
 
-PartitionsByExprResult::PartitionsByExprResult(const PartitionsByExprResult& other383) {
-  partitions = other383.partitions;
-  hasUnknownPartitions = other383.hasUnknownPartitions;
+PartitionsByExprResult::PartitionsByExprResult(const PartitionsByExprResult& other385) {
+  partitions = other385.partitions;
+  hasUnknownPartitions = other385.hasUnknownPartitions;
 }
-PartitionsByExprResult& PartitionsByExprResult::operator=(const PartitionsByExprResult& other384) {
-  partitions = other384.partitions;
-  hasUnknownPartitions = other384.hasUnknownPartitions;
+PartitionsByExprResult& PartitionsByExprResult::operator=(const PartitionsByExprResult& other386) {
+  partitions = other386.partitions;
+  hasUnknownPartitions = other386.hasUnknownPartitions;
   return *this;
 }
 void PartitionsByExprResult::printTo(std::ostream& out) const {
@@ -9278,21 +9410,21 @@ void swap(PartitionsByExprRequest &a, PartitionsByExprRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest& other385) {
-  dbName = other385.dbName;
-  tblName = other385.tblName;
-  expr = other385.expr;
-  defaultPartitionName = other385.defaultPartitionName;
-  maxParts = other385.maxParts;
-  __isset = other385.__isset;
-}
-PartitionsByExprRequest& PartitionsByExprRequest::operator=(const PartitionsByExprRequest& other386) {
-  dbName = other386.dbName;
-  tblName = other386.tblName;
-  expr = other386.expr;
-  defaultPartitionName = other386.defaultPartitionName;
-  maxParts = other386.maxParts;
-  __isset = other386.__isset;
+PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest& other387) {
+  dbName = other387.dbName;
+  tblName = other387.tblName;
+  expr = other387.expr;
+  defaultPartitionName = other387.defaultPartitionName;
+  maxParts = other387.maxParts;
+  __isset = other387.__isset;
+}
+PartitionsByExprRequest& PartitionsByExprRequest::operator=(const PartitionsByExprRequest& other388) {
+  dbName = other388.dbName;
+  tblName = other388.tblName;
+  expr = other388.expr;
+  defaultPartitionName = other388.defaultPartitionName;
+  maxParts = other388.maxParts;
+  __isset = other388.__isset;
   return *this;
 }
 void PartitionsByExprRequest::printTo(std::ostream& out) const {
@@ -9341,14 +9473,14 @@ uint32_t TableStatsResult::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tableStats.clear();
-            uint32_t _size387;
-            ::apache::thrift::protocol::TType _etype390;
-            xfer += iprot->readListBegin(_etype390, _size387);
-            this->tableStats.resize(_size387);
-            uint32_t _i391;
-            for (_i391 = 0; _i391 < _size387; ++_i391)
+            uint32_t _size389;
+            ::apache::thrift::protocol::TType _etype392;
+            xfer += iprot->readListBegin(_etype392, _size389);
+            this->tableStats.resize(_size389);
+            uint32_t _i393;
+            for (_i393 = 0; _i393 < _size389; ++_i393)
             {
-              xfer += this->tableStats[_i391].read(iprot);
+              xfer += this->tableStats[_i393].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9379,10 +9511,10 @@ uint32_t TableStatsResult::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("tableStats", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->tableStats.size()));
-    std::vector<ColumnStatisticsObj> ::const_iterator _iter392;
-    for (_iter392 = this->tableStats.begin(); _iter392 != this->tableStats.end(); ++_iter392)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter394;
+    for (_iter394 = this->tableStats.begin(); _iter394 != this->tableStats.end(); ++_iter394)
     {
-      xfer += (*_iter392).write(oprot);
+      xfer += (*_iter394).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9398,11 +9530,11 @@ void swap(TableStatsResult &a, TableStatsResult &b) {
   swap(a.tableStats, b.tableStats);
 }
 
-TableStatsResult::TableStatsResult(const TableStatsResult& other393) {
-  tableStats = other393.tableStats;
+TableStatsResult::TableStatsResult(const TableStatsResult& other395) {
+  tableStats = other395.tableStats;
 }
-TableStatsResult& TableStatsResult::operator=(const TableStatsResult& other394) {
-  tableStats = other394.tableStats;
+TableStatsResult& TableStatsResult::operator=(const TableStatsResult& other396) {
+  tableStats = other396.tableStats;
   return *this;
 }
 void TableStatsResult::printTo(std::ostream& out) const {
@@ -9447,26 +9579,26 @@ uint32_t PartitionsStatsResult::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partStats.clear();
-            uint32_t _size395;
-            ::apache::thrift::protocol::TType _ktype396;
-            ::apache::thrift::protocol::TType _vtype397;
-            xfer += iprot->readMapBegin(_ktype396, _vtype397, _size395);
-            uint32_t _i399;
-            for (_i399 = 0; _i399 < _size395; ++_i399)
+            uint32_t _size397;
+            ::apache::thrift::protocol::TType _ktype398;
+            ::apache::thrift::protocol::TType _vtype399;
+            xfer += iprot->readMapBegin(_ktype398, _vtype399, _size397);
+            uint32_t _i401;
+            for (_i401 = 0; _i401 < _size397; ++_i401)
             {
-              std::string _key400;
-              xfer += iprot->readString(_key400);
-              std::vector<ColumnStatisticsObj> & _val401 = this->partStats[_key400];
+              std::string _key402;
+              xfer += iprot->readString(_key402);
+              std::vector<ColumnStatisticsObj> & _val403 = this->partStats[_key402];
               {
-                _val401.clear();
-                uint32_t _size402;
-                ::apache::thrift::protocol::TType _etype405;
-                xfer += iprot->readListBegin(_etype405, _size402);
-                _val401.resize(_size402);
-                uint32_t _i406;
-                for (_i406 = 0; _i406 < _size402; ++_i406)
+                _val403.clear();
+                uint32_t _size404;
+                ::apache::thrift::protocol::TType _etype407;
+                xfer += iprot->readListBegin(_etype407, _size404);
+                _val403.resize(_size404);
+                uint32_t _i408;
+                for (_i408 = 0; _i408 < _size404; ++_i408)
                 {
-                  xfer += _val401[_i406].read(iprot);
+                  xfer += _val403[_i408].read(iprot);
                 }
                 xfer += iprot->readListEnd();
               }
@@ -9500,16 +9632,16 @@ uint32_t PartitionsStatsResult::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("partStats", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->partStats.size()));
-    std::map<std::string, std::vector<ColumnStatisticsObj> > ::const_iterator _iter407;
-    for (_iter407 = this->partStats.begin(); _iter407 != this->partStats.end(); ++_iter407)
+    std::map<std::string, std::vector<ColumnStatisticsObj> > ::const_iterator _iter409;
+    for (_iter409 = this->partStats.begin(); _iter409 != this->partStats.end(); ++_iter409)
     {
-      xfer += oprot->writeString(_iter407->first);
+      xfer += oprot->writeString(_iter409->first);
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter407->second.size()));
-        std::vector<ColumnStatisticsObj> ::const_iterator _iter408;
-        for (_iter408 = _iter407->second.begin(); _iter408 != _iter407->second.end(); ++_iter408)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter409->second.size()));
+        std::vector<ColumnStatisticsObj> ::const_iterator _iter410;
+        for (_iter410 = _iter409->second.begin(); _iter410 != _iter409->second.end(); ++_iter410)
         {
-          xfer += (*_iter408).write(oprot);
+          xfer += (*_iter410).write(oprot);
         }
         xfer += oprot->writeListEnd();
       }
@@ -9528,11 +9660,11 @@ void swap(PartitionsStatsResult &a, PartitionsStatsResult &b) {
   swap(a.partStats, b.partStats);
 }
 
-PartitionsStatsResult::PartitionsStatsResult(const PartitionsStatsResult& other409) {
-  partStats = other409.partStats;
+PartitionsStatsResult::PartitionsStatsResult(const PartitionsStatsResult& other411) {
+  partStats = other411.partStats;
 }
-PartitionsStatsResult& PartitionsStatsResult::operator=(const PartitionsStatsResult& other410) {
-  partStats = other410.partStats;
+PartitionsStatsResult& PartitionsStatsResult::operator=(const PartitionsStatsResult& other412) {
+  partStats = other412.partStats;
   return *this;
 }
 void PartitionsStatsResult::printTo(std::ostream& out) const {
@@ -9603,14 +9735,14 @@ uint32_t TableStatsRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colNames.clear();
-            uint32_t _size411;
-            ::apache::thrift::protocol::TType _etype414;
-            xfer += iprot->readListBegin(_etype414, _size411);
-            this->colNames.resize(_size411);
-            uint32_t _i415;
-            for (_i415 = 0; _i415 < _size411; ++_i415)
+            uint32_t _size413;
+            ::apache::thrift::protocol::TType _etype416;
+            xfer += iprot->readListBegin(_etype416, _size413);
+            this->colNames.resize(_size413);
+            uint32_t _i417;
+            for (_i417 = 0; _i417 < _size413; ++_i417)
             {
-              xfer += iprot->readString(this->colNames[_i415]);
+              xfer += iprot->readString(this->colNames[_i417]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9653,10 +9785,10 @@ uint32_t TableStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("colNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->colNames.size()));
-    std::vector<std::string> ::const_iterator _iter416;
-    for (_iter416 = this->colNames.begin(); _iter416 != this->colNames.end(); ++_iter416)
+    std::vector<std::string> ::const_iterator _iter418;
+    for (_iter418 = this->colNames.begin(); _iter418 != this->colNames.end(); ++_iter418)
     {
-      xfer += oprot->writeString((*_iter416));
+      xfer += oprot->writeString((*_iter418));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9674,15 +9806,15 @@ void swap(TableStatsRequest &a, TableStatsRequest &b) {
   swap(a.colNames, b.colNames);
 }
 
-TableStatsRequest::TableStatsRequest(const TableStatsRequest& other417) {
-  dbName = other417.dbName;
-  tblName = other417.tblName;
-  colNames = other417.colNames;
+TableStatsRequest::TableStatsRequest(const TableStatsRequest& other419) {
+  dbName = other419.dbName;
+  tblName = other419.tblName;
+  colNames = other419.colNames;
 }
-TableStatsRequest& TableStatsRequest::operator=(const TableStatsRequest& other418) {
-  dbName = other418.dbName;
-  tblName = other418.tblName;
-  colNames = other418.colNames;
+TableStatsRequest& TableStatsRequest::operator=(const TableStatsRequest& other420) {
+  dbName = other420.dbName;
+  tblName = other420.tblName;
+  colNames = other420.colNames;
   return *this;
 }
 void TableStatsRequest::printTo(std::ostream& out) const {
@@ -9760,14 +9892,14 @@ uint32_t PartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colNames.clear();
-            uint32_t _size419;
-            ::apache::thrift::protocol::TType _etype422;
-            xfer += iprot->readListBegin(_etype422, _size419);
-            this->colNames.resize(_size419);
-            uint32_t _i423;
-            for (_i423 = 0; _i423 < _size419; ++_i423)
+            uint32_t _size421;
+            ::apache::thrift::protocol::TType _etype424;
+            xfer += iprot->readListBegin(_etype424, _size421);
+            this->colNames.resize(_size421);
+            uint32_t _i425;
+            for (_i425 = 0; _i425 < _size421; ++_i425)
             {
-              xfer += iprot->readString(this->colNames[_i423]);
+              xfer += iprot->readString(this->colNames[_i425]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9780,14 +9912,14 @@ uint32_t PartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size424;
-            ::apache::thrift::protocol::TType _etype427;
-            xfer += iprot->readListBegin(_etype427, _size424);
-            this->partNames.resize(_size424);
-            uint32_t _i428;
-            for (_i428 = 0; _i428 < _size424; ++_i428)
+            uint32_t _size426;
+            ::apache::thrift::protocol::TType _etype429;
+            xfer += iprot->readListBegin(_etype429, _size426);
+            this->partNames.resize(_size426);
+            uint32_t _i430;
+            for (_i430 = 0; _i430 < _size426; ++_i430)
             {
-              xfer += iprot->readString(this->partNames[_i428]);
+              xfer += iprot->readString(this->partNames[_i430]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9832,10 +9964,10 @@ uint32_t PartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("colNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->colNames.size()));
-    std::vector<std::string> ::const_iterator _iter429;
-    for (_iter429 = this->colNames.begin(); _iter429 != this->colNames.end(); ++_iter429)
+    std::vector<std::string> ::const_iterator _iter431;
+    for (_iter431 = this->colNames.begin(); _iter431 != this->colNames.end(); ++_iter431)
     {
-      xfer += oprot->writeString((*_iter429));
+      xfer += oprot->writeString((*_iter431));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9844,10 +9976,10 @@ uint32_t PartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter430;
-    for (_iter430 = this->partNames.begin(); _iter430 != this->partNames.end(); ++_iter430)
+    std::vector<std::string> ::const_iterator _iter432;
+    for (_iter432 = this->partNames.begin(); _iter432 != this->partNames.end(); ++_iter432)
     {
-      xfer += oprot->writeString((*_iter430));
+      xfer += oprot->writeString((*_iter432));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9866,17 +9998,17 @@ void swap(PartitionsStatsRequest &a, PartitionsStatsRequest &b) {
   swap(a.partNames, b.partNames);
 }
 
-PartitionsStatsRequest::PartitionsStatsRequest(const PartitionsStatsRequest& other431) {
-  dbName = other431.dbName;
-  tblName = other431.tblName;
-  colNames = other431.colNames;
-  partNames = other431.partNames;
+PartitionsStatsRequest::PartitionsStatsRequest(const PartitionsStatsRequest& other433) {
+  dbName = other433.dbName;
+  tblName = other433.tblName;
+  colNames = other433.colNames;
+  partNames = other433.partNames;
 }
-PartitionsStatsRequest& PartitionsStatsRequest::operator=(const PartitionsStatsRequest& other432) {
-  dbName = other432.dbName;
-  tblName = other432.tblName;
-  colNames = other432.colNames;
-  partNames = other432.partNames;
+PartitionsStatsRequest& PartitionsStatsRequest::operator=(const PartitionsStatsRequest& other434) {
+  dbName = other434.dbName;
+  tblName = other434.tblName;
+  colNames = other434.colNames;
+  partNames = other434.partNames;
   return *this;
 }
 void PartitionsStatsRequest::printTo(std::ostream& out) const {
@@ -9924,14 +10056,14 @@ uint32_t AddPartitionsResult::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size433;
-            ::apache::thrift::protocol::TType _etype436;
-            xfer += iprot->readListBegin(_etype436, _size433);
-            this->partitions.resize(_size433);
-            uint32_t _i437;
-            for (_i437 = 0; _i437 < _size433; ++_i437)
+            uint32_t _size435;
+            ::apache::thrift::protocol::TType _etype438;
+            xfer += iprot->readListBegin(_etype438, _size435);
+            this->partitions.resize(_size435);
+            uint32_t _i439;
+            for (_i439 = 0; _i439 < _size435; ++_i439)
             {
-              xfer += this->partitions[_i437].read(iprot);
+              xfer += this->partitions[_i439].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9961,10 +10093,10 @@ uint32_t AddPartitionsResult::write(::apache::thrift::protocol::TProtocol* oprot
     xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitions.size()));
-      std::vector<Partition> ::const_iterator _iter438;
-      for (_iter438 = this->partitions.begin(); _iter438 != this->partitions.end(); ++_iter438)
+      std::vector<Partition> ::const_iterator _iter440;
+      for (_iter440 = this->partitions.begin(); _iter440 != this->partitions.end(); ++_iter440)
       {
-        xfer += (*_iter438).write(oprot);
+        xfer += (*_iter440).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -9981,13 +10113,13 @@ void swap(AddPartitionsResult &a, AddPartitionsResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddPartitionsResult::AddPartitionsResult(const AddPartitionsResult& other439) {
-  partitions = other439.partitions;
-  __isset = other439.__isset;
+AddPartitionsResult::AddPartitionsResult(const AddPartitionsResult& other441) {
+  partitions = other441.partitions;
+  __isset = other441.__isset;
 }
-AddPartitionsResult& AddPartitionsResult::operator=(const AddPartitionsResult& other440) {
-  partitions = other440.partitions;
-  __isset = other440.__isset;
+AddPartitionsResult& AddPartitionsResult::operator=(const AddPartitionsResult& other442) {
+  partitions = other442.partitions;
+  __isset = other442.__isset;
   return *this;
 }
 void AddPartitionsResult::printTo(std::ostream& out) const {
@@ -10068,14 +10200,14 @@ uint32_t AddPartitionsRequest::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->parts.clear();
-            uint32_t _size441;
-            ::apache::thrift::protocol::TType _etype444;
-            xfer += iprot->readListBegin(_etype444, _size441);
-            this->parts.resize(_size441);
-            uint32_t _i445;
-            for (_i445 = 0; _i445 < _size441; ++_i445)
+            uint32_t _size443;
+            ::apache::thrift::protocol::TType _etype446;
+            xfer += iprot->readListBegin(_etype446, _size443);
+            this->parts.resize(_size443);
+            uint32_t _i447;
+            for (_i447 = 0; _i447 < _size443; ++_i447)
             {
-              xfer += this->parts[_i445].read(iprot);
+              xfer += this->parts[_i447].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10136,10 +10268,10 @@ uint32_t AddPartitionsRequest::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->parts.size()));
-    std::vector<Partition> ::const_iterator _iter446;
-    for (_iter446 = this->parts.begin(); _iter446 != this->parts.end(); ++_iter446)
+    std::vector<Partition> ::const_iterator _iter448;
+    for (_iter448 = this->parts.begin(); _iter448 != this->parts.end(); ++_iter448)
     {
-      xfer += (*_iter446).write(oprot);
+      xfer += (*_iter448).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10169,21 +10301,21 @@ void swap(AddPartitionsRequest &a, AddPartitionsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddPartitionsRequest::AddPartitionsRequest(const AddPartitionsRequest& other447) {
-  dbName = other447.dbName;
-  tblName = other447.tblName;
-  parts = other447.parts;
-  ifNotExists = other447.ifNotExists;
-  needResult = other447.needResult;
-  __isset = other447.__isset;
-}
-AddPartitionsRequest& AddPartitionsRequest::operator=(const AddPartitionsRequest& other448) {
-  dbName = other448.dbName;
-  tblName = other448.tblName;
-  parts = other448.parts;
-  ifNotExists = other448.ifNotExists;
-  needResult = other448.needResult;
-  __isset = other448.__isset;
+AddPartitionsRequest::AddPartitionsRequest(const AddPartitionsRequest& other449) {
+  dbName = other449.dbName;
+  tblName = other449.tblName;
+  parts = other449.parts;
+  ifNotExists = other449.ifNotExists;
+  needResult = other449.needResult;
+  __isset = other449.__isset;
+}
+AddPartitionsRequest& AddPartitionsRequest::operator=(const AddPartitionsRequest& other450) {
+  dbName = other450.dbName;
+  tblName = other450.tblName;
+  parts = other450.parts;
+  ifNotExists = other450.ifNotExists;
+  needResult = other450.needResult;
+  __isset = other450.__isset;
   return *this;
 }
 void AddPartitionsRequest::printTo(std::ostream& out) const {
@@ -10232,14 +10364,14 @@ uint32_t DropPartitionsResult::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size449;
-            ::apache::thrift::protocol::TType _etype452;
-            xfer += iprot->readListBegin(_etype452, _size449);
-            this->partitions.resize(_size449);
-            uint32_t _i453;
-            for (_i453 = 0; _i453 < _size449; ++_i453)
+            uint32_t _size451;
+            ::apache::thrift::protocol::TType _etype454;
+            xfer += iprot->readListBegin(_etype454, _size451);
+            this->partitions.resize(_size451);
+            uint32_t _i455;
+            for (_i455 = 0; _i455 < _size451; ++_i455)
             {
-              xfer += this->partitions[_i453].read(iprot);
+              xfer += this->partitions[_i455].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10269,10 +10401,10 @@ uint32_t DropPartitionsResult::write(::apache::thrift::protocol::TProtocol* opro
     xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitions.size()));
-      std::vector<Partition> ::const_iterator _iter454;
-      for (_iter454 = this->partitions.begin(); _iter454 != this->partitions.end(); ++_iter454)
+      std::vector<Partition> ::const_iterator _iter456;
+      for (_iter456 = this->partitions.begin(); _iter456 != this->partitions.end(); ++_iter456)
       {
-        xfer += (*_iter454).write(oprot);
+        xfer += (*_iter456).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -10289,13 +10421,13 @@ void swap(DropPartitionsResult &a, DropPartitionsResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-DropPartitionsResult::DropPartitionsResult(const DropPartitionsResult& other455) {
-  partitions = other455.partitions;
-  __isset = other455.__isset;
+DropPartitionsResult::DropPartitionsResult(const DropPartitionsResult& other457) {
+  partitions = other457.partitions;
+  __isset = other457.__isset;
 }
-DropPartitionsResult& DropPartitionsResult::operator=(const DropPartitionsResult& other456) {
-  partitions = other456.partitions;
-  __isset = other456.__isset;
+DropPartitionsResult& DropPartitionsResult::operator=(const DropPartitionsResult& other458) {
+  partitions = other458.partitions;
+  __isset = other458.__isset;
   return *this;
 }
 void DropPartitionsResult::printTo(std::ostream& out) const {
@@ -10397,15 +10529,15 @@ void swap(DropPartitionsExpr &a, DropPartitionsExpr &b) {
   swap(a.__isset, b.__isset);
 }
 
-DropPartitionsExpr::DropPartitionsExpr(const DropPartitionsExpr& other457) {
-  expr = other457.expr;
-  partArchiveLevel = other457.partArchiveLevel;
-  __isset = other457.__isset;
+DropPartitionsExpr::DropPartitionsExpr(const DropPartitionsExpr& other459) {
+  expr = other459.expr;
+  partArchiveLevel = other459.partArchiveLevel;
+  __isset = other459.__isset;
 }
-DropPartitionsExpr& DropPartitionsExpr::operator=(const DropPartitionsExpr& other458) {
-  expr = other458.expr;
-  partArchiveLevel = other458.partArchiveLevel;
-  __isset = other458.__isset;
+DropPartitionsExpr& DropPartitionsExpr::operator=(const DropPartitionsExpr& other460) {
+  expr = other460.expr;
+  partArchiveLevel = other460.partArchiveLevel;
+  __isset = other460.__isset;
   return *this;
 }
 void DropPartitionsExpr::printTo(std::ostream& out) const {
@@ -10454,14 +10586,14 @@ uint32_t RequestPartsSpec::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->names.clear();
-            uint32_t _size459;
-            ::apache::thrift::protocol::TType _etype462;
-            xfer += iprot->readListBegin(_etype462, _size459);
-            this->names.resize(_size459);
-            uint32_t _i463;
-            for (_i463 = 0; _i463 < _size459; ++_i463)
+            uint32_t _size461;
+            ::apache::thrift::protocol::TType _etype464;
+            xfer += iprot->readListBegin(_etype464, _size461);
+            this->names.resize(_size461);
+            uint32_t _i465;
+            for (_i465 = 0; _i465 < _size461; ++_i465)
             {
-              xfer += iprot->readString(this->names[_i463]);
+              xfer += iprot->readString(this->names[_i465]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10474,14 +10606,14 @@ uint32_t RequestPartsSpec::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->exprs.clear();
-            uint32_t _size464;
-            ::apache::thrift::protocol::TType _etype467;
-            xfer += iprot->readListBegin(_etype467, _size464);
-            this->exprs.resize(_size464);
-            uint32_t _i468;
-            for (_i468 = 0; _i468 < _size464; ++_i468)
+            uint32_t _size466;
+            ::apache::thrift::protocol::TType _etype469;
+            xfer += iprot->readListBegin(_etype469, _size466);
+            this->exprs.resize(_size466);
+            uint32_t _i470;
+            for (_i470 = 0; _i470 < _size466; ++_i470)
             {
-              xfer += this->exprs[_i468].read(iprot);
+              xfer += this->exprs[_i470].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10510,10 +10642,10 @@ uint32_t RequestPartsSpec::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->names.size()));
-    std::vector<std::string> ::const_iterator _iter469;
-    for (_iter469 = this->names.begin(); _iter469 != this->names.end(); ++_iter469)
+    std::vector<std::string> ::const_iterator _iter471;
+    for (_iter471 = this->names.begin(); _iter471 != this->names.end(); ++_iter471)
     {
-      xfer += oprot->writeString((*_iter469));
+      xfer += oprot->writeString((*_iter471));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10522,10 +10654,10 @@ uint32_t RequestPartsSpec::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("exprs", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->exprs.size()));
-    std::vector<DropPartitionsExpr> ::const_iterator _iter470;
-    for (_iter470 = this->exprs.begin(); _iter470 != this->exprs.end(); ++_iter470)
+    std::vector<DropPartitionsExpr> ::const_iterator _iter472;
+    for (_iter472 = this->exprs.begin(); _iter472 != this->exprs.end(); ++_iter472)
     {
-      xfer += (*_iter470).write(oprot);
+      xfer += (*_iter472).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10543,15 +10675,15 @@ void swap(RequestPartsSpec &a, RequestPartsSpec &b) {
   swap(a.__isset, b.__isset);
 }
 
-RequestPartsSpec::RequestPartsSpec(const RequestPartsSpec& other471) {
-  names = other471.names;
-  exprs = other471.exprs;
-  __isset = other471.__isset;
+RequestPartsSpec::RequestPartsSpec(const RequestPartsSpec& other473) {
+  names = other473.names;
+  exprs = other473.exprs;
+  __isset = other473.__isset;
 }
-RequestPartsSpec& RequestPartsSpec::operator=(const RequestPartsSpec& other472) {
-  names = other472.names;
-  exprs = other472.exprs;
-  __isset = other472.__isset;
+RequestPartsSpec& RequestPartsSpec::operator=(const RequestPartsSpec& other474) {
+  names = other474.names;
+  exprs = other474.exprs;
+  __isset = other474.__isset;
   return *this;
 }
 void RequestPartsSpec::printTo(std::ostream& out) const {
@@ -10770,27 +10902,27 @@ void swap(DropPartitionsRequest &a, DropPartitionsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-DropPartitionsRequest::DropPartitionsRequest(const DropPartitionsRequest& other473) {
-  dbName = other473.dbName;
-  tblName = other473.tblName;
-  parts = other473.parts;
-  deleteData = other473.deleteData;
-  ifExists = other473.ifExists;
-  ignoreProtection = other473.ignoreProtection;
-  environmentContext = other473.environmentContext;
-  needResult = other473.needResult;
-  __isset = other473.__isset;
-}
-DropPartitionsRequest& DropPartitionsRequest::operator=(const DropPartitionsRequest& other474) {
-  dbName = other474.dbName;
-  tblName = other474.tblName;
-  parts = other474.parts;
-  deleteData = other474.deleteData;
-  ifExists = other474.ifExists;
-  ignoreProtection = other474.ignoreProtection;
-  environmentContext = other474.environmentContext;
-  needResult = other474.needResult;
-  __isset = other474.__isset;
+DropPartitionsRequest::DropPartitionsRequest(const DropPartitionsRequest& other475) {
+  dbName = other475.dbName;
+  tblName = other475.tblName;
+  parts = other475.parts;
+  deleteData = other475.deleteData;
+  ifExists = other475.ifExists;
+  ignoreProtection = other475.ignoreProtection;
+  environmentContext = other475.environmentContext;
+  needResult = other475.needResult;
+  __isset = other475.__isset;
+}
+DropPartitionsRequest& DropPartitionsRequest::operator=(const DropPartitionsRequest& other476) {
+  dbName = other476.dbName;
+  tblName = other476.tblName;
+  parts = other476.parts;
+  deleteData = other476.deleteData;
+  ifExists = other476.ifExists;
+  ignoreProtection = other476.ignoreProtection;
+  environmentContext = other476.environmentContext;
+  needResult = other476.needResult;
+  __isset = other476.__isset;
   return *this;
 }
 void DropPartitionsRequest::printTo(std::ostream& out) const {
@@ -10843,9 +10975,9 @@ uint32_t ResourceUri::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast475;
-          xfer += iprot->readI32(ecast475);
-          this->resourceType = (ResourceType::type)ecast475;
+          int32_t ecast477;
+          xfer += iprot->readI32(ecast477);
+          this->resourceType = (ResourceType::type)ecast477;
           this->__isset.resourceType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -10896,15 +11028,15 @@ void swap(ResourceUri &a, ResourceUri &b) {
   swap(a.__isset, b.__isset);
 }
 
-ResourceUri::ResourceUri(const ResourceUri& other476) {
-  resourceType = other476.resourceType;
-  uri = other476.uri;
-  __isset = other476.__isset;
+ResourceUri::ResourceUri(const ResourceUri& other478) {
+  resourceType = other478.resourceType;
+  uri = other478.uri;
+  __isset = other478.__isset;
 }
-ResourceUri& ResourceUri::operator=(const ResourceUri& other477) {
-  resourceType = other477.resourceType;
-  uri = other477.uri;
-  __isset = other477.__isset;
+ResourceUri& ResourceUri::operator=(const ResourceUri& other479) {
+  resourceType = other479.resourceType;
+  uri = other479.uri;
+  __isset = other479.__isset;
   return *this;
 }
 void ResourceUri::printTo(std::ostream& out) const {
@@ -11007,9 +11139,9 @@ uint32_t Function::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast478;
-          xfer += iprot->readI32(ecast478);
-          this->ownerType = (PrincipalType::type)ecast478;
+          int32_t ecast480;
+          xfer += iprot->readI32(ecast480);
+          this->ownerType = (PrincipalType::type)ecast480;
           this->__isset.ownerType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -11025,9 +11157,9 @@ uint32_t Function::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast479;
-          xfer += iprot->readI32(ecast479);
-          this->functionType = (FunctionType::type)ecast479;
+          int32_t ecast481;
+          xfer += iprot->readI32(ecast481);
+          this->functionType = (FunctionType::type)ecast481;
           this->__isset.functionType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -11037,14 +11169,14 @@ uint32_t Function::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->resourceUris.clear();
-            uint32_t _size480;
-            ::apache::thrift::protocol::TType _etype483;
-            xfer += iprot->readListBegin(_etype483, _size480);
-            this->resourceUris.resize(_size480);
-            uint32_t _i484;
-            for (_i484 = 0; _i484 < _size480; ++_i484)
+            uint32_t _size482;
+            ::apache::thrift::protocol::TType _etype485;
+            xfer += iprot->readListBegin(_etype485, _size482);
+            this->resourceUris.resize(_size482);
+            uint32_t _i486;
+            for (_i486 = 0; _i486 < _size482; ++_i486)
             {
-              xfer += this->resourceUris[_i484].read(iprot);
+              xfer += this->resourceUris[_i486].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11101,10 +11233,10 @@ uint32_t Function::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("resourceUris", ::apache::thrift::protocol::T_LIST, 8);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->resourceUris.size()));
-    std::vector<ResourceUri> ::const_iterator _iter485;
-    for (_iter485 = this->resourceUris.begin(); _iter485 != this->resourceUris.end(); ++_iter485)
+    std::vector<ResourceUri> ::const_iterator _iter487;
+    for (_iter487 = this->resourceUris.begin(); _iter487 != this->resourceUris.end(); ++_iter487)
     {
-      xfer += (*_iter485).write(oprot);
+      xfer += (*_iter487).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11128,27 +11260,27 @@ void swap(Function &a, Function &b) {
   swap(a.__isset, b.__isset);
 }
 
-Function::Function(const Function& other486) {
-  functionName = other486.functionName;
-  dbName = other486.dbName;
-  className = other486.className;
-  ownerName = other486.ownerName;
-  ownerType = other486.ownerType;
-  createTime = other486.createTime;
-  functionType = other486.functionType;
-  resourceUris = other486.resourceUris;
-  __isset = other486.__isset;
-}
-Function& Function::operator=(const Function& other487) {
-  functionName = other487.functionName;
-  dbName = other487.dbName;
-  className = other487.className;
-  ownerName = other487.ownerName;
-  ownerType = other487.ownerType;
-  createTime = other487.createTime;
-  functionType = other487.functionType;
-  resourceUris = other487.resourceUris;
-  __isset = other487.__isset;
+Function::Function(const Function& other488) {
+  functionName = other488.functionName;
+  dbName = other488.dbName;
+  className = other488.className;
+  ownerName = other488.ownerName;
+  ownerType = other488.ownerType;
+  createTime = other488.createTime;
+  functionType = other488.functionType;
+  resourceUris = other488.resourceUris;
+  __isset = other488.__isset;
+}
+Function& Function::operator=(const Function& other489) {
+  functionName = other489.functionName;
+  dbName = other489.dbName;
+  className = other489.className;
+  ownerName = other489.ownerName;
+  ownerType = other489.ownerType;
+  createTime = other489.createTime;
+  functionType = other489.functionType;
+  resourceUris = other489.resourceUris;
+  __isset = other489.__isset;
   return *this;
 }
 void Function::printTo(std::ostream& out) const {
@@ -11236,9 +11368,9 @@ uint32_t TxnInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast488;
-          xfer += iprot->readI32(ecast488);
-          this->state = (TxnState::type)ecast488;
+          int32_t ecast490;
+          xfer += iprot->readI32(ecast490);
+          this->state = (TxnState::type)ecast490;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -11357,25 +11489,25 @@ void swap(TxnInfo &a, TxnInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnInfo::TxnInfo(const TxnInfo& other489) {
-  id = other489.id;
-  state = other489.state;
-  user = other489.user;
-  hostname = other489.hostname;
-  agentInfo = other489.agentInfo;
-  heartbeatCount = other489.heartbeatCount;
-  metaInfo = other489.metaInfo;
-  __isset = other489.__isset;
-}
-TxnInfo& TxnInfo::operator=(const TxnInfo& other490) {
-  id = other490.id;
-  state = other490.state;
-  user = other490.user;
-  hostname = other490.hostname;
-  agentInfo = other490.agentInfo;
-  heartbeatCount = other490.heartbeatCount;
-  metaInfo = other490.metaInfo;
-  __isset = other490.__isset;
+TxnInfo::TxnInfo(const TxnInfo& other491) {
+  id = other491.id;
+  state = other491.state;
+  user = other491.user;
+  hostname = other491.hostname;
+  agentInfo = other491.agentInfo;
+  heartbeatCount = other491.heartbeatCount;
+  metaInfo = other491.metaInfo;
+  __isset = other491.__isset;
+}
+TxnInfo& TxnInfo::operator=(const TxnInfo& other492) {
+  id = other492.id;
+  state = other492.state;
+  user = other492.user;
+  hostname = other492.hostname;
+  agentInfo = other492.agentInfo;
+  heartbeatCount = other492.heartbeatCount;
+  metaInfo = other492.metaInfo;
+  __isset = other492.__isset;
   return *this;
 }
 void TxnInfo::printTo(std::ostream& out) const {
@@ -11439,14 +11571,14 @@ uint32_t GetOpenTxnsInfoResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->open_txns.clear();
-            uint32_t _size491;
-            ::apache::thrift::protocol::TType _etype494;
-            xfer += iprot->readListBegin(_etype494, _size491);
-            this->open_txns.resize(_size491);
-            uint32_t _i495;
-            for (_i495 = 0; _i495 < _size491; ++_i495)
+            uint32_t _size493;
+            ::apache::thrift::protocol::TType _etype496;
+            xfer += iprot->readListBegin(_etype496, _size493);
+            this->open_txns.resize(_size493);
+            uint32_t _i497;
+            for (_i497 = 0; _i497 < _size493; ++_i497)
             {
-              xfer += this->open_txns[_i495].read(iprot);
+              xfer += this->open_txns[_i497].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11483,10 +11615,10 @@ uint32_t GetOpenTxnsInfoResponse::write(::apache::thrift::protocol::TProtocol* o
   xfer += oprot->writeFieldBegin("open_txns", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->open_txns.size()));
-    std::vector<TxnInfo> ::const_iterator _iter496;
-    for (_iter496 = this->open_txns.begin(); _iter496 != this->open_txns.end(); ++_iter496)
+    std::vector<TxnInfo> ::const_iterator _iter498;
+    for (_iter498 = this->open_txns.begin(); _iter498 != this->open_txns.end(); ++_iter498)
     {
-      xfer += (*_iter496).write(oprot);
+      xfer += (*_iter498).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11503,13 +11635,13 @@ void swap(GetOpenTxnsInfoResponse &a, GetOpenTxnsInfoResponse &b) {
   swap(a.open_txns, b.open_txns);
 }
 
-GetOpenTxnsInfoResponse::GetOpenTxnsInfoResponse(const GetOpenTxnsInfoResponse& other497) {
-  txn_high_water_mark = other497.txn_high_water_mark;
-  open_txns = other497.open_txns;
+GetOpenTxnsInfoResponse::GetOpenTxnsInfoResponse(const GetOpenTxnsInfoResponse& other499) {
+  txn_high_water_mark = other499.txn_high_water_mark;
+  open_txns = other499.open_txns;
 }
-GetOpenTxnsInfoResponse& GetOpenTxnsInfoResponse::operator=(const GetOpenTxnsInfoResponse& other498) {
-  txn_high_water_mark = other498.txn_high_water_mark;
-  open_txns = other498.open_txns;
+GetOpenTxnsInfoResponse& GetOpenTxnsInfoResponse::operator=(const GetOpenTxnsInfoResponse& other500) {
+  txn_high_water_mark = other500.txn_high_water_mark;
+  open_txns = other500.open_txns;
   return *this;
 }
 void GetOpenTxnsInfoResponse::printTo(std::ostream& out) const {
@@ -11568,15 +11700,15 @@ uint32_t GetOpenTxnsResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->open_txns.clear();
-            uint32_t _size499;
-            ::apache::thrift::protocol::TType _etype502;
-            xfer += iprot->readSetBegin(_etype502, _size499);
-            uint32_t _i503;
-            for (_i503 = 0; _i503 < _size499; ++_i503)
+            uint32_t _size501;
+            ::apache::thrift::protocol::TType _etype504;
+            xfer += iprot->readSetBegin(_etype504, _size501);
+            uint32_t _i505;
+            for (_i505 = 0; _i505 < _size501; ++_i505)
             {
-              int64_t _elem504;
-              xfer += iprot->readI64(_elem504);
-              this->open_txns.insert(_elem504);
+              int64_t _elem506;
+              xfer += iprot->readI64(_elem506);
+              this->open_txns.insert(_elem506);
             }
             xfer += iprot->readSetEnd();
           }
@@ -11613,10 +11745,10 @@ uint32_t GetOpenTxnsResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("open_txns", ::apache::thrift::protocol::T_SET, 2);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->open_txns.size()));
-    std::set<int64_t> ::const_iterator _iter505;
-    for (_iter505 = this->open_txns.begin(); _iter505 != this->open_txns.end(); ++_iter505)
+    std::set<int64_t> ::const_iterator _iter507;
+    for (_iter507 = this->open_txns.begin(); _iter507 != this->open_txns.end(); ++_iter507)
     {
-      xfer += oprot->writeI64((*_iter505));
+      xfer += oprot->writeI64((*_iter507));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -11633,13 +11765,13 @@ void swap(GetOpenTxnsResponse &a, GetOpenTxnsResponse &b) {
   swap(a.open_txns, b.open_txns);
 }
 
-GetOpenTxnsResponse::GetOpenTxnsResponse(const GetOpenTxnsResponse& other506) {
-  txn_high_water_mark = other506.txn_high_water_mark;
-  open_txns = other506.open_txns;
+GetOpenTxnsResponse::GetOpenTxnsResponse(const GetOpenTxnsResponse& other508) {
+  txn_high_water_mark = other508.txn_high_water_mark;
+  open_txns = other508.open_txns;
 }
-GetOpenTxnsResponse& GetOpenTxnsResponse::operator=(const GetOpenTxnsResponse& other507) {
-  txn_high_water_mark = other507.txn_high_water_mark;
-  open_txns = other507.open_txns;
+GetOpenTxnsResponse& GetOpenTxnsResponse::operator=(const GetOpenTxnsResponse& other509) {
+  txn_high_water_mark = other509.txn_high_water_mark;
+  open_txns = other509.open_txns;
   return *this;
 }
 void GetOpenTxnsResponse::printTo(std::ostream& out) const {
@@ -11782,19 +11914,19 @@ void swap(OpenTxnRequest &a, OpenTxnRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-OpenTxnRequest::OpenTxnRequest(const OpenTxnRequest& other508) {
-  num_txns = other508.num_txns;
-  user = other508.user;
-  hostname = other508.hostname;
-  agentInfo = other508.agentInfo;
-  __isset = other508.__isset;
+OpenTxnRequest::OpenTxnRequest(const OpenTxnRequest& other510) {
+  num_txns = other510.num_txns;
+  user = other510.user;
+  hostname = other510.hostname;
+  agentInfo = other510.agentInfo;
+  __isset = other510.__isset;
 }
-OpenTxnRequest& OpenTxnRequest::operator=(const OpenTxnRequest& other509) {
-  num_txns = other509.num_txns;
-  user = other509.user;
-  hostname = other509.hostname;
-  agentInfo = other509.agentInfo;
-  __isset = other509.__isset;
+OpenTxnRequest& OpenTxnRequest::operator=(const OpenTxnRequest& other511) {
+  num_txns = other511.num_txns;
+  user = other511.user;
+  hostname = other511.hostname;
+  agentInfo = other511.agentInfo;
+  __isset = other511.__isset;
   return *this;
 }
 void OpenTxnRequest::printTo(std::ostream& out) const {
@@ -11842,14 +11974,14 @@ uint32_t OpenTxnsResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->txn_ids.clear();
-            uint32_t _size510;
-            ::apache::thrift::protocol::TType _etype513;
-            xfer += iprot->readListBegin(_etype513, _size510);
-            this->txn_ids.resize(_size510);
-            uint32_t _i514;
-            for (_i514 = 0; _i514 < _size510; ++_i514)
+            uint32_t _size512;
+            ::apache::thrift::protocol::TType _etype515;
+            xfer += iprot->readListBegin(_etype515, _size512);
+            this->txn_ids.resize(_size512);
+            uint32_t _i516;
+            for (_i516 = 0; _i516 < _size512; ++_i516)
             {
-              xfer += iprot->readI64(this->txn_ids[_i514]);
+              xfer += iprot->readI64(this->txn_ids[_i516]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11880,10 +12012,10 @@ uint32_t OpenTxnsResponse::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("txn_ids", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->txn_ids.size()));
-    std::vector<int64_t> ::const_iterator _iter515;
-    for (_iter515 = this->txn_ids.begin(); _iter515 != this->txn_ids.end(); ++_iter515)
+    std::vector<int64_t> ::const_iterator _iter517;
+    for (_iter517 = this->txn_ids.begin(); _iter517 != this->txn_ids.end(); ++_iter517)
     {
-      xfer += oprot->writeI64((*_iter515));
+      xfer += oprot->writeI64((*_iter517));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11899,11 +12031,11 @@ void swap(OpenTxnsResponse &a, OpenTxnsResponse &b) {
   swap(a.txn_ids, b.txn_ids);
 }
 
-OpenTxnsResponse::OpenTxnsResponse(const OpenTxnsResponse& other516) {
-  txn_ids = other516.txn_ids;
+OpenTxnsResponse::OpenTxnsResponse(const OpenTxnsResponse& other518) {
+  txn_ids = other518.txn_ids;
 }
-OpenTxnsResponse& OpenTxnsResponse::operator=(const OpenTxnsResponse& other517) {
-  txn_ids = other517.txn_ids;
+OpenTxnsResponse& OpenTxnsResponse::operator=(const OpenTxnsResponse& other519) {
+  txn_ids = other519.txn_ids;
   return *this;
 }
 void OpenTxnsResponse::printTo(std::ostream& out) const {
@@ -11985,11 +12117,11 @@ void swap(AbortTxnRequest &a, AbortTxnRequest &b) {
   swap(a.txnid, b.txnid);
 }
 
-AbortTxnRequest::AbortTxnRequest(const AbortTxnRequest& other518) {
-  txnid = other518.txnid;
+AbortTxnRequest::AbortTxnRequest(const AbortTxnRequest& other520) {
+  txnid = other520.txnid;
 }
-AbortTxnRequest& AbortTxnRequest::operator=(const AbortTxnRequest& other519) {
-  txnid = other519.txnid;
+AbortTxnRequest& AbortTxnRequest::operator=(const AbortTxnRequest& other521) {
+  txnid = other521.txnid;
   return *this;
 }
 void AbortTxnRequest::printTo(std::ostream& out) const {
@@ -12071,11 +12203,11 @@ void swap(CommitTxnRequest &a, CommitTxnRequest &b) {
   swap(a.txnid, b.txnid);
 }
 
-CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other520) {
-  txnid = other520.txnid;
+CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other522) {
+  txnid = other522.txnid;
 }
-CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other521) {
-  txnid = other521.txnid;
+CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other523) {
+  txnid = other523.txnid;
   return *this;
 }
 void CommitTxnRequest::printTo(std::ostream& out) const {
@@ -12138,9 +12270,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast522;
-          xfer += iprot->readI32(ecast522);
-          this->type = (LockType::type)ecast522;
+          int32_t ecast524;
+          xfer += iprot->readI32(ecast524);
+          this->type = (LockType::type)ecast524;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -12148,9 +12280,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast523;
-          xfer += iprot->readI32(ecast523);
-          this->level = (LockLevel::type)ecast523;
+          int32_t ecast525;
+          xfer += iprot->readI32(ecast525);
+          this->level = (LockLevel::type)ecast525;
           isset_level = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -12240,21 +12372,21 @@ void swap(LockComponent &a, LockComponent &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockComponent::LockComponent(const LockComponent& other524) {
-  type = other524.type;
-  level = other524.level;
-  dbname = other524.dbname;
-  tablename = other524.tablename;
-  partitionname = other524.partitionname;
-  __isset = other524.__isset;
-}
-LockComponent& LockComponent::operator=(const LockComponent& other525) {
-  type = other525.type;
-  level = other525.level;
-  dbname = other525.dbname;
-  tablename = other525.tablename;
-  partitionname = other525.partitionname;
-  __isset = other525.__isset;
+LockComponent::LockComponent(const LockComponent& other526) {
+  type = other526.type;
+  level = other526.level;
+  dbname = other526.dbname;
+  tablename = other526.tablename;
+  partitionname = other526.partitionname;
+  __isset = other526.__isset;
+}
+LockComponent& LockComponent::operator=(const LockComponent& other527) {
+  type = other527.type;
+  level = other527.level;
+  dbname = other527.dbname;
+  tablename = other527.tablename;
+  partitionname = other527.partitionname;
+  __isset = other527.__isset;
   return *this;
 }
 void LockComponent::printTo(std::ostream& out) const {
@@ -12323,14 +12455,14 @@ uint32_t LockRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->component.clear();
-            uint32_t _size526;
-            ::apache::thrift::protocol::TType _etype529;
-            xfer += iprot->readListBegin(_etype529, _size526);
-            this->component.resize(_size526);
-            uint32_t _i530;
-            for (_i530 = 0; _i530 < _size526; ++_i530)
+            uint32_t _size528;
+            ::apache::thrift::protocol::TType _etype531;
+            xfer += iprot->readListBegin(_etype531, _size528);
+            this->component.resize(_size528);
+            uint32_t _i532;
+            for (_i532 = 0; _i532 < _size528; ++_i532)
             {
-              xfer += this->component[_i530].read(iprot);
+              xfer += this->component[_i532].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12397,10 +12529,10 @@ uint32_t LockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const
   xfer += oprot->writeFieldBegin("component", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->component.size()));
-    std::vector<LockComponent> ::const_iterator _iter531;
-    for (_iter531 = this->component.begin(); _iter531 != this->component.end(); ++_iter531)
+    std::vector<LockComponent> ::const_iterator _iter533;
+    for (_iter533 = this->component.begin(); _iter533 != this->component.end(); ++_iter533)
     {
-      xfer += (*_iter531).write(oprot);
+      xfer += (*_iter533).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12439,21 +12571,21 @@ void swap(LockRequest &a, LockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockRequest::LockRequest(const LockRequest& other532) {
-  component = other532.component;
-  txnid = other532.txnid;
-  user = other532.user;
-  hostname = other532.hostname;
-  agentInfo = other532.agentInfo;
-  __isset = other532.__isset;
-}
-LockRequest& LockRequest::operator=(const LockRequest& other533) {
-  component = other533.component;
-  txnid = other533.txnid;
-  user = other533.user;
-  hostname = other533.hostname;
-  agentInfo = other533.agentInfo;
-  __isset = other533.__isset;
+LockRequest::LockRequest(const LockRequest& other534) {
+  component = other534.component;
+  txnid = other534.txnid;
+  user = other534.user;
+  hostname = other534.hostname;
+  agentInfo = other534.agentInfo;
+  __isset = other534.__isset;
+}
+LockRequest& LockRequest::operator=(const LockRequest& other535) {
+  component = other535.component;
+  txnid = other535.txnid;
+  user = other535.user;
+  hostname = other535.hostname;
+  agentInfo = other535.agentInfo;
+  __isset = other535.__isset;
   return *this;
 }
 void LockRequest::printTo(std::ostream& out) const {
@@ -12513,9 +12645,9 @@ uint32_t LockResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast534;
-          xfer += iprot->readI32(ecast534);
-          this->state = (LockState::type)ecast534;
+          int32_t ecast536;
+          xfer += iprot->readI32(ecast536);
+          this->state = (LockState::type)ecast536;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -12561,13 +12693,13 @@ void swap(LockResponse &a, LockResponse &b) {
   swap(a.state, b.state);
 }
 
-LockResponse::LockResponse(const LockResponse& other535) {
-  lockid = other535.lockid;
-  state = other535.state;
+LockResponse::LockResponse(const LockResponse& other537) {
+  lockid = other537.lockid;
+  state = other537.state;
 }
-LockResponse& LockResponse::operator=(const LockResponse& other536) {
-  lockid = other536.lockid;
-  state = other536.state;
+LockResponse& LockResponse::operator=(const LockResponse& other538) {
+  lockid = other538.lockid;
+  state = other538.state;
   return *this;
 }
 void LockResponse::printTo(std::ostream& out) const {
@@ -12689,17 +12821,17 @@ void swap(CheckLockRequest &a, CheckLockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CheckLockRequest::CheckLockRequest(const CheckLockRequest& other537) {
-  lockid = other537.lockid;
-  txnid = other537.txnid;
-  elapsed_ms = other537.elapsed_ms;
-  __isset = other537.__isset;
+CheckLockRequest::CheckLockRequest(const CheckLockRequest& other539) {
+  lockid = other539.lockid;
+  txnid = other539.txnid;
+  elapsed_ms = other539.elapsed_ms;
+  __isset = other539.__isset;
 }
-CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other538) {
-  lockid = other538.lockid;
-  txnid = other538.txnid;
-  elapsed_ms = other538.elapsed_ms;
-  __isset = other538.__isset;
+CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other540) {
+  lockid = other540.lockid;
+  txnid = other540.txnid;
+  elapsed_ms = other540.elapsed_ms;
+  __isset = other540.__isset;
   return *this;
 }
 void CheckLockRequest::printTo(std::ostream& out) const {
@@ -12783,11 +12915,11 @@ void swap(UnlockRequest &a, UnlockRequest &b) {
   swap(a.lockid, b.lockid);
 }
 
-UnlockRequest::UnlockRequest(const UnlockRequest& other539) {
-  lockid = other539.lockid;
+UnlockRequest::UnlockRequest(const UnlockRequest& other541) {
+  lockid = other541.lockid;
 }
-UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other540) {
-  lockid = other540.lockid;
+UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other542) {
+  lockid = other542.lockid;
   return *this;
 }
 void UnlockRequest::printTo(std::ostream& out) const {
@@ -12926,19 +13058,19 @@ void swap(ShowLocksRequest &a, ShowLocksRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other541) {
-  dbname = other541.dbname;
-  tablename = other541.tablename;
-  partname = other541.partname;
-  isExtended = other541.isExtended;
-  __isset = other541.__isset;
+ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other543) {
+  dbname = other543.dbname;
+  tablename = other543.tablename;
+  partname = other543.partname;
+  isExtended = other543.isExtended;
+  __isset = other543.__isset;
 }
-ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other542) {
-  dbname = other542.dbname;
-  tablename = other542.tablename;
-  partname = other542.partname;
-  isExtended = other542.isExtended;
-  __isset = other542.__isset;
+ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other544) {
+  dbname = other544.dbname;
+  tablename = other544.tablename;
+  partname = other544.partname;
+  isExtended = other544.isExtended;
+  __isset = other544.__isset;
   return *this;
 }
 void ShowLocksRequest::printTo(std::ostream& out) const {
@@ -13091,9 +13223,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast543;
-          xfer += iprot->readI32(ecast543);
-          this->state = (LockState::type)ecast543;
+          int32_t ecast545;
+          xfer += iprot->readI32(ecast545);
+          this->state = (LockState::type)ecast545;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -13101,9 +13233,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast544;
-          xfer += iprot->readI32(ecast544);
-          this->type = (LockType::type)ecast544;
+          int32_t ecast546;
+          xfer += iprot->readI32(ecast546);
+          this->type = (LockType::type)ecast546;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -13319,43 +13451,43 @@ void swap(ShowLocksResponseElement &a, ShowLocksResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other545) {
-  lockid = other545.lockid;
-  dbname = other545.dbname;
-  tablename = other545.tablename;
-  partname = other545.partname;
-  state = other545.state;
-  type = other545.type;
-  txnid = other545.txnid;
-  lastheartbeat = other545.lastheartbeat;
-  acquiredat = other545.acquiredat;
-  user = other545.user;
-  hostname = other545.hostname;
-  heartbeatCount = other545.heartbeatCount;
-  agentInfo = other545.agentInfo;
-  blockedByExtId = other545.blockedByExtId;
-  blockedByIntId = other545.blockedByIntId;
-  lockIdInternal = other545.lockIdInternal;
-  __isset = other545.__isset;
-}
-ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other546) {
-  lockid = other546.lockid;
-  dbname = other546.dbname;
-  tablename = other546.tablename;
-  partname = other546.partname;
-  state = other546.state;
-  type = other546.type;
-  txnid = other546.txnid;
-  lastheartbeat = other546.lastheartbeat;
-  acquiredat = other546.acquiredat;
-  user = other546.user;
-  hostname = other546.hostname;
-  heartbeatCount = other546.heartbeatCount;
-  agentInfo = other546.agentInfo;
-  blockedByExtId = other546.blockedByExtId;
-  blockedByIntId = other546.blockedByIntId;
-  lockIdInternal = other546.lockIdInternal;
-  __isset = other546.__isset;
+ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other547) {
+  lockid = other547.lockid;
+  dbname = other547.dbname;
+  tablename = other547.tablename;
+  partname = other547.partname;
+  state = other547.state;
+  type = other547.type;
+  txnid = other547.txnid;
+  lastheartbeat = other547.lastheartbeat;
+  acquiredat = other547.acquiredat;
+  user = other547.user;
+  hostname = other547.hostname;
+  heartbeatCount = other547.heartbeatCount;
+  agentInfo = other547.agentInfo;
+  blockedByExtId = other547.blockedByExtId;
+  blockedByIntId = other547.blockedByIntId;
+  lockIdInternal = other547.lockIdInternal;
+  __isset = other547.__isset;
+}
+ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other548) {
+  lockid = other548.lockid;
+  dbname = other548.dbname;
+  tablename = other548.tablename;
+  partname = other548.partname;
+  state = other548.state;
+  type = other548.type;
+  txnid = other548.txnid;
+  lastheartbeat = other548.lastheartbeat;
+  acquiredat = other548.acquiredat;
+  user = other548.user;
+  hostname = other548.hostname;
+  heartbeatCount = other548.heartbeatCount;
+  agentInfo = other548.agentInfo;
+  blockedByExtId = other548.blockedByExtId;
+  blockedByIntId = other548.blockedByIntId;
+  lockIdInternal = other548.lockIdInternal;
+  __isset = other548.__isset;
   return *this;
 }
 void ShowLocksResponseElement::printTo(std::ostream& out) const {
@@ -13414,14 +13546,14 @@ uint32_t ShowLocksResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->locks.clear();
-            uint32_t _size547;
-            ::apache::thrift::protocol::TType _etype550;
-            xfer += iprot->readListBegin(_etype550, _size547);
-            this->locks.resize(_size547);
-            uint32_t _i551;
-            for (_i551 = 0; _i551 < _size547; ++_i551)
+            uint32_t _size549;
+            ::apache::thrift::protocol::TType _etype552;
+            xfer += iprot->readListBegin(_etype552, _size549);
+            this->locks.resize(_size549);
+            uint32_t _i553;
+            for (_i553 = 0; _i553 < _size549; ++_i553)
             {
-              xfer += this->locks[_i551].read(iprot);
+              xfer += this->locks[_i553].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13450,10 +13582,10 @@ uint32_t ShowLocksResponse::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("locks", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->locks.size()));
-    std::vector<ShowLocksResponseElement> ::const_iterator _iter552;
-    for (_iter552 = this->locks.begin(); _iter552 != this->locks.end(); ++_iter552)
+    std::vector<ShowLocksResponseElement> ::const_iterator _iter554;
+    for (_iter554 = this->locks.begin(); _iter554 != this->locks.end(); ++_iter554)
     {
-      xfer += (*_iter552).write(oprot);
+      xfer += (*_iter554).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13470,13 +13602,13 @@ void swap(ShowLocksResponse &a, ShowLocksResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other553) {
-  locks = other553.locks;
-  __isset = other553.__isset;
+ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other555) {
+  locks = other555.locks;
+  __isset = other555.__isset;
 }
-ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other554) {
-  locks = other554.locks;
-  __isset = other554.__isset;
+ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other556) {
+  locks = other556.locks;
+  __isset = other556.__isset;
   return *this;
 }
 void ShowLocksResponse::printTo(std::ostream& out) const {
@@ -13577,15 +13709,15 @@ void swap(HeartbeatRequest &a, HeartbeatRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other555) {
-  lockid = other555.lockid;
-  txnid = other555.txnid;
-  __isset = other555.__isset;
+HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other557) {
+  lockid = other557.lockid;
+  txnid = other557.txnid;
+  __isset = other557.__isset;
 }
-HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other556) {
-  lockid = other556.lockid;
-  txnid = other556.txnid;
-  __isset = other556.__isset;
+HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other558) {
+  lockid = other558.lockid;
+  txnid = other558.txnid;
+  __isset = other558.__isset;
   return *this;
 }
 void HeartbeatRequest::printTo(std::ostream& out) const {
@@ -13688,13 +13820,13 @@ void swap(HeartbeatTxnRangeRequest &a, HeartbeatTxnRangeRequest &b) {
   swap(a.max, b.max);
 }
 
-HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other557) {
-  min = other557.min;
-  max = other557.max;
+HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other559) {
+  min = other559.min;
+  max = other559.max;
 }
-HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other558) {
-  min = other558.min;
-  max = other558.max;
+HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other560) {
+  min = other560.min;
+  max = other560.max;
   return *this;
 }
 void HeartbeatTxnRangeRequest::printTo(std::ostream& out) const {
@@ -13745,15 +13877,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->aborted.clear();
-            uint32_t _size559;
-            ::apache::thrift::protocol::TType _etype562;
-            xfer += iprot->readSetBegin(_etype562, _size559);
-            uint32_t _i563;
-            for (_i563 = 0; _i563 < _size559; ++_i563)
+            uint32_t _size561;
+            ::apache::thrift::protocol::TType _etype564;
+            xfer += iprot->readSetBegin(_etype564, _size561);
+            uint32_t _i565;
+            for (_i565 = 0; _i565 < _size561; ++_i565)
             {
-              int64_t _elem564;
-              xfer += iprot->readI64(_elem564);
-              this->aborted.insert(_elem564);
+              int64_t _elem566;
+              xfer += iprot->readI64(_elem566);
+              this->aborted.insert(_elem566);
             }
             xfer += iprot->readSetEnd();
           }
@@ -13766,15 +13898,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->nosuch.clear();
-            uint32_t _size565;
-            ::apache::thrift::protocol::TType _etype568;
-            xfer += iprot->readSetBegin(_etype568, _size565);
-            uint32_t _i569;
-            for (_i569 = 0; _i569 < _size565; ++_i569)
+            uint32_t _size567;
+            ::apache::thrift::protocol::TType _etype570;
+            xfer += iprot->readSetBegin(_etype570, _size567);
+            uint32_t _i571;
+            for (_i571 = 0; _i571 < _size567; ++_i571)
             {
-              int64_t _elem570;
-              xfer += iprot->readI64(_elem570);
-              this->nosuch.insert(_elem570);
+              int64_t _elem572;
+              xfer += iprot->readI64(_elem572);
+              this->nosuch.insert(_elem572);
             }
             xfer += iprot->readSetEnd();
           }
@@ -13807,10 +13939,10 @@ uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("aborted", ::apache::thrift::protocol::T_SET, 1);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->aborted.size()));
-    std::set<int64_t> ::const_iterator _iter571;
-    for (_iter571 = this->aborted.begin(); _iter571 != this->aborted.end(); ++_iter571)
+    std::set<int64_t> ::const_iterator _iter573;
+    for (_iter573 = this->aborted.begin(); _iter573 != this->aborted.end(); ++_iter573)
     {
-      xfer += oprot->writeI64((*_iter571));
+      xfer += oprot->writeI64((*_iter573));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -13819,10 +13951,10 @@ uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("nosuch", ::apache::thrift::protocol::T_SET, 2);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->nosuch.size()));
-    std::set<int64_t> ::const_iterator _iter572;
-    for (_iter572 = this->nosuch.begin(); _iter572 != this->nosuch.end(); ++_iter572)
+    std::set<int64_t> ::const_iterator _iter574;
+    for (_iter574 = this->nosuch.begin(); _iter574 != this->nosuch.end(); ++_iter574)
     {
-      xfer += oprot->writeI64((*_iter572));
+      xfer += oprot->writeI64((*_iter574));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -13839,13 +13971,13 @@ void swap(HeartbeatTxnRangeResponse &a, HeartbeatTxnRangeResponse &b) {
   swap(a.nosuch, b.nosuch);
 }
 
-HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other573) {
-  aborted = other573.aborted;
-  nosuch = other573.nosuch;
+HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other575) {
+  aborted = other575.aborted;
+  nosuch = other575.nosuch;
 }
-HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other574) {
-  aborted = other574.aborted;
-  nosuch = other574.nosuch;
+HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other576) {
+  aborted = other576.aborted;
+  nosuch = other576.nosuch;
   return *this;
 }
 void HeartbeatTxnRangeResponse::printTo(std::ostream& out) const {
@@ -13933,9 +14065,9 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast575;
-          xfer += iprot->readI32(ecast575);
-          this->type = (CompactionType::type)ecast575;
+          int32_t ecast577;
+          xfer += iprot->readI32(ecast577);
+          this->type = (CompactionType::type)ecast577;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -14009,21 +14141,21 @@ void swap(CompactionRequest &a, CompactionRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CompactionRequest::CompactionRequest(const CompactionRequest& other576) {
-  dbname = other576.dbname;
-  tablename = other576.tablename;
-  partitionname = other576.partitionname;
-  type = other576.type;
-  runas = other576.runas;
-  __isset = other576.__isset;
-}
-CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other577) {
-  dbname = other577.dbname;
-  tablename = other577.tablename;
-  partitionname = other577.partitionname;
-  type = other577.type;
-  runas = other577.runas;
-  __isset = other577.__isset;
+CompactionRequest::CompactionRequest(const CompactionRequest& other578) {
+  dbname = other578.dbname;
+  tablename = other578.tablename;
+  partitionname = other578.partitionname;
+  type = other578.type;
+  runas = other578.runas;
+  __isset = other578.__isset;
+}
+CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other579) {
+  dbname = other579.dbname;
+  tablename = other579.tablename;
+  partitionname = other579.partitionname;
+  type = other579.type;
+  runas = other579.runas;
+  __isset = other579.__isset;
   return *this;
 }
 void CompactionRequest::printTo(std::ostream& out) const {
@@ -14086,11 +14218,11 @@ void swap(ShowCompactRequest &a, ShowCompactRequest &b) {
   (void) b;
 }
 
-ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other578) {
-  (void) other578;
+ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other580) {
+  (void) other580;
 }
-ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other579) {
-  (void) other579;
+ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other581) {
+  (void) other581;
   return *this;
 }
 void ShowCompactRequest::printTo(std::ostream& out) const {
@@ -14211,9 +14343,9 @@ uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol*
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast580;
-          xfer += iprot->readI32(ecast580);
-          this->type = (CompactionType::type)ecast580;
+          int32_t ecast582;
+          xfer += iprot->readI32(ecast582);
+          this->type = (CompactionType::type)ecast582;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -14386,35 +14518,35 @@ void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other581) {
-  dbname = other581.dbname;
-  tablename = other581.tablename;
-  partitionname = other581.partitionname;
-  type = other581.type;
-  state = other581.state;
-  workerid = other581.workerid;
-  start = other581.start;
-  runAs = other581.runAs;
-  hightestTxnId = other581.hightestTxnId;
-  metaInfo = other581.metaInfo;
-  endTime = other581.endTime;
-  hadoopJobId = other581.hadoopJobId;
-  __isset = other581.__isset;
-}
-ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other582) {
-  dbname = other582.dbname;
-  tablename = other582.tablename;
-  partitionname = other582.partitionname;
-  type = other582.type;
-  state = other582.state;
-  workerid = other582.workerid;
-  start = other582.start;
-  runAs = other582.runAs;
-  hightestTxnId = other582.hightestTxnId;
-  metaInfo = other582.metaInfo;
-  endTime = other582.endTime;
-  hadoopJobId = other582.hadoopJobId;
-  __isset = other582.__isset;
+ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other583) {
+  dbname = other583.dbname;
+  tablename = other583.tablename;
+  partitionname = other583.partitionname;
+  type = other583.type;
+  state = other583.state;
+  workerid = other583.workerid;
+  start = other583.start;
+  runAs = other583.runAs;
+  hightestTxnId = other583.hightestTxnId;
+  metaInfo = other583.metaInfo;
+  endTime = other583.endTime;
+  hadoopJobId = other583.hadoopJobId;
+  __isset = other583.__isset;
+}
+ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other584) {
+  dbname = other584.dbname;
+  tablename = other584.tablename;
+  partitionname = other584.partitionname;
+  type = other584.type;
+  state = other584.state;
+  workerid = other584.workerid;
+  start = other584.start;
+  runAs = other584.runAs;
+  hightestTxnId = other584.hightestTxnId;
+  metaInfo = other584.metaInfo;
+  endTime = other584.endTime;
+  hadoopJobId = other584.hadoopJobId;
+  __isset = other584.__isset;
   return *this;
 }
 void ShowCompactResponseElement::printTo(std::ostream& out) const {
@@ -14470,14 +14602,14 @@ uint32_t ShowCompactResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->compacts.clear();
-            uint32_t _size583;
-            ::apache::thrift::protocol::TType _etype586;
-            xfer += iprot->readListBegin(_etype586, _size583);
-            this->compacts.resize(_size583);
-            uint32_t _i587;
-            for (_i587 = 0; _i587 < _size583; ++_i587)
+            uint32_t _size585;
+            ::apache::thrift::protocol::TType _etype588;
+            xfer += iprot->readListBegin(_etype588, _size585);
+            this->compacts.resize(_size585);
+            uint32_t _i589;
+            for (_i589 = 0; _i589 < _size585; ++_i589)
             {
-              xfer += this->compacts[_i587].read(iprot);
+              xfer += this->compacts[_i589].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14508,10 +14640,10 @@ uint32_t ShowCompactResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->compacts.size()));
-    std::vector<ShowCompactResponseElement> ::const_iterator _iter588;
-    for (_iter588 = this->compacts.begin(); _iter588 != this->compacts.end(); ++_iter588)
+    std::vector<ShowCompactResponseElement> ::const_iterator _iter590;
+    for (_iter590 = this->compacts.begin(); _iter590 != this->compacts.end(); ++_iter590)
     {
-      xfer += (*_iter588).write(oprot);
+      xfer += (*_iter590).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -14527,11 +14659,11 @@ void swap(ShowCompactResponse &a, ShowCompactResponse &b) {
   swap(a.compacts, b.compacts);
 }
 
-ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other589) {
-  compacts = other589.compacts;
+ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other591) {
+  compacts = other591.compacts;
 }
-ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other590) {
-  compacts = other590.compacts;
+ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other592) {
+  compacts = other592.compacts;
   return *this;
 }
 void ShowCompactResponse::printTo(std::ostream& out) const {
@@ -14615,14 +14747,14 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionnames.clear();
-            uint32_t _size591;
-            ::apache::thrift::protocol::TType _etype594;
-            xfer += iprot->readListBegin(_etype594, _size591);
-            this->partitionnames.resize(_size591);
-            uint32_t _i595;
-            for (_i595 = 0; _i595 < _size591; ++_i595)
+            uint32_t _size593;
+            ::apache::thrift::protocol::TType _etype596;
+            xfer += iprot->readListBegin(_etype596, _size593);
+            this->partitionnames.resize(_size593);
+            uint32_t _i597;
+            for (_i597 = 0; _i597 < _size593; ++_i597)
             {
-              xfer += iprot->readString(this->partitionnames[_i595]);
+              xfer += iprot->readString(this->partitionnames[_i597]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14671,10 +14803,10 @@ uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionnames.size()));
-    std::vector<std::string> ::const_iterator _iter596;
-    for (_iter596 = this->partitionnames.begin(); _iter596 != this->partitionnames.end(); ++_iter596)
+    std::vector<std::string> ::const_iterator _iter598;
+    for (_iter598 = this->partitionnames.begin(); _iter598 != this->partitionnames.end(); ++_iter598)
     {
-      xfer += oprot->writeString((*_iter596));
+      xfer += oprot->writeString((*_iter598));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14693,17 +14825,17 @@ void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) {
   swap(a.partitionnames, b.partitionnames);
 }
 
-AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other597) {
-  txnid = other597.txnid;
-  dbname = other597.dbname;
-  tablename = other597.tablename;
-  partitionnames = other597.partitionnames;
+AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other599) {
+  txnid = other599.txnid;
+  dbname = other599.dbname;
+  tablename = other599.tablename;
+  partitionnames = other599.partitionnames;
 }
-AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other598) {
-  txnid = other598.txnid;
-  dbname = other598.dbname;
-  tablename = other598.tablename;
-  partitionnames = other598.partitionnames;
+AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other600) {
+  txnid = other600.txnid;
+  dbname = other600.dbname;
+  tablename = other600.tablename;
+  partitionnames = other600.partitionnames;
   return *this;
 }
 void AddDynamicPartitions::printTo(std::ostream& out) const {
@@ -14808,15 +14940,15 @@ void swap(NotificationEventRequest &a, NotificationEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other599) {
-  lastEvent = other599.lastEvent;
-  maxEvents = other599.maxEvents;
-  __isset = other599.__isset;
+NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other601) {
+  lastEvent = other601.lastEvent;
+  maxEvents = other601.maxEvents;
+  __isset = other601.__isset;
 }
-NotificationEventRequest& NotificationEventRequest::operator=(const No

<TRUNCATED>

[25/50] [abbrv] hive git commit: HIVE-13671: Add PerfLogger to log4j2.properties logger (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Posted by sp...@apache.org.
HIVE-13671: Add PerfLogger to log4j2.properties logger (Prasanth Jayachandran reviewed by Sergey Shelukhin)


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

Branch: refs/heads/java8
Commit: a88050bd9ae1f2cfec87a54e773a83cdb3de325f
Parents: f68b5db
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Wed May 4 21:30:45 2016 -0500
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Wed May 4 21:30:45 2016 -0500

----------------------------------------------------------------------
 common/src/main/resources/hive-log4j2.properties | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a88050bd/common/src/main/resources/hive-log4j2.properties
----------------------------------------------------------------------
diff --git a/common/src/main/resources/hive-log4j2.properties b/common/src/main/resources/hive-log4j2.properties
index 12cd9ac..cf0369a 100644
--- a/common/src/main/resources/hive-log4j2.properties
+++ b/common/src/main/resources/hive-log4j2.properties
@@ -23,6 +23,7 @@ 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
+property.hive.perflogger.log.level = INFO
 
 # list of all appenders
 appenders = console, DRFA
@@ -50,7 +51,7 @@ appender.DRFA.strategy.type = DefaultRolloverStrategy
 appender.DRFA.strategy.max = 30
 
 # list of all loggers
-loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX
+loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX, PerfLogger
 
 logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn
 logger.NIOServerCnxn.level = WARN
@@ -67,6 +68,9 @@ logger.Datastore.level = ERROR
 logger.JPOX.name = JPOX
 logger.JPOX.level = ERROR
 
+logger.PerfLogger.name = org.apache.hadoop.hive.ql.log.PerfLogger
+logger.PerfLogger.level = ${sys:hive.perflogger.log.level}
+
 # root logger
 rootLogger.level = ${sys:hive.log.level}
 rootLogger.appenderRefs = root


[02/50] [abbrv] hive git commit: Merge branch 'master' into llap

Posted by sp...@apache.org.
Merge branch 'master' into llap


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

Branch: refs/heads/java8
Commit: bc75d72b8e5ed8a538cf69d3b0a7556150e4713e
Parents: e073cce b621827
Author: Jason Dere <jd...@hortonworks.com>
Authored: Tue May 3 13:30:39 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Tue May 3 13:30:39 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/common/FileUtils.java    |   19 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    2 +-
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |   43 +-
 .../cli/session/TestHiveSessionImpl.java        |    2 +-
 .../test/resources/testconfiguration.properties |    4 +
 .../org/apache/hive/jdbc/HiveStatement.java     |   20 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |   47 +
 .../hive/metastore/HiveMetaStoreFsImpl.java     |   11 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   27 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   73 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |    7 +-
 .../hive/ql/exec/vector/VectorExtractRow.java   |   12 +-
 .../ql/exec/vector/VectorizationContext.java    |    2 +-
 .../hadoop/hive/ql/history/HiveHistoryImpl.java |    8 +-
 .../ql/io/orc/ConvertTreeReaderFactory.java     | 3750 +++++++++++++
 .../hadoop/hive/ql/io/orc/SchemaEvolution.java  |   29 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |   86 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   27 +-
 .../hadoop/hive/ql/session/OperationLog.java    |    8 +-
 .../hadoop/hive/ql/session/SessionState.java    |    2 +-
 .../clientnegative/orc_replace_columns2.q       |    5 +-
 .../clientnegative/orc_replace_columns2_acid.q  |    5 +-
 .../clientnegative/orc_replace_columns3.q       |    3 +
 .../clientnegative/orc_replace_columns3_acid.q  |    3 +
 .../clientnegative/orc_type_promotion1.q        |    7 +-
 .../clientnegative/orc_type_promotion1_acid.q   |    7 +-
 .../clientnegative/orc_type_promotion2.q        |    5 +-
 .../clientnegative/orc_type_promotion2_acid.q   |    5 +-
 .../clientnegative/orc_type_promotion3.q        |    5 +-
 .../clientnegative/orc_type_promotion3_acid.q   |    5 +-
 .../clientpositive/orc_int_type_promotion.q     |    4 +
 .../clientpositive/orc_schema_evolution.q       |    2 +
 .../schema_evol_orc_acid_mapwork_part.q         |  846 ++-
 .../schema_evol_orc_acid_mapwork_table.q        |  804 ++-
 .../schema_evol_orc_acidvec_mapwork_part.q      |  843 ++-
 .../schema_evol_orc_acidvec_mapwork_table.q     |  801 ++-
 .../schema_evol_orc_nonvec_fetchwork_part.q     |  831 ++-
 .../schema_evol_orc_nonvec_fetchwork_table.q    |  824 ++-
 .../schema_evol_orc_nonvec_mapwork_part.q       |  833 ++-
 ...a_evol_orc_nonvec_mapwork_part_all_complex.q |  162 +
 ...evol_orc_nonvec_mapwork_part_all_primitive.q |  481 ++
 .../schema_evol_orc_nonvec_mapwork_table.q      |  824 ++-
 .../schema_evol_orc_vec_mapwork_part.q          |  831 ++-
 ...hema_evol_orc_vec_mapwork_part_all_complex.q |  162 +
 ...ma_evol_orc_vec_mapwork_part_all_primitive.q |  481 ++
 .../schema_evol_orc_vec_mapwork_table.q         |  819 ++-
 .../schema_evol_text_nonvec_mapwork_part.q      |    5 +-
 ..._evol_text_nonvec_mapwork_part_all_complex.q |    5 +-
 ...vol_text_nonvec_mapwork_part_all_primitive.q |    5 +-
 .../schema_evol_text_nonvec_mapwork_table.q     |    5 +-
 .../schema_evol_text_vec_mapwork_part.q         |    2 +-
 ...ema_evol_text_vec_mapwork_part_all_complex.q |    2 +-
 ...a_evol_text_vec_mapwork_part_all_primitive.q |    2 +-
 .../schema_evol_text_vec_mapwork_table.q        |    2 +-
 .../schema_evol_text_vecrow_mapwork_part.q      |    2 +-
 ..._evol_text_vecrow_mapwork_part_all_complex.q |    2 +-
 ...vol_text_vecrow_mapwork_part_all_primitive.q |    2 +-
 .../schema_evol_text_vecrow_mapwork_table.q     |    2 +-
 .../clientnegative/orc_replace_columns2.q.out   |   13 +-
 .../orc_replace_columns2_acid.q.out             |   13 +-
 .../clientnegative/orc_replace_columns3.q.out   |   11 +-
 .../orc_replace_columns3_acid.q.out             |   11 +-
 .../clientnegative/orc_type_promotion1.q.out    |   13 +-
 .../orc_type_promotion1_acid.q.out              |   13 +-
 .../clientnegative/orc_type_promotion2.q.out    |   13 +-
 .../orc_type_promotion2_acid.q.out              |   13 +-
 .../clientnegative/orc_type_promotion3.q.out    |   11 +-
 .../clientnegative/orc_type_promotion3_acid.q   |   18 +
 .../orc_type_promotion3_acid.q.out              |   11 +-
 .../schema_evol_orc_acid_mapwork_part.q.out     | 4319 ++++++++++++---
 .../schema_evol_orc_acid_mapwork_table.q.out    | 3334 ++++++++++--
 .../schema_evol_orc_acidvec_mapwork_part.q.out  | 4319 ++++++++++++---
 .../schema_evol_orc_acidvec_mapwork_table.q.out | 3334 ++++++++++--
 .../schema_evol_orc_nonvec_fetchwork_part.q.out | 4905 +++++++++++++++--
 ...schema_evol_orc_nonvec_fetchwork_table.q.out | 4367 +++++++++++++++-
 .../schema_evol_orc_nonvec_mapwork_part.q.out   | 4909 +++++++++++++++--
 ...ol_orc_nonvec_mapwork_part_all_complex.q.out |  726 +++
 ..._orc_nonvec_mapwork_part_all_primitive.q.out | 2872 ++++++++++
 .../schema_evol_orc_nonvec_mapwork_table.q.out  | 4367 +++++++++++++++-
 .../schema_evol_orc_vec_mapwork_part.q.out      | 4929 ++++++++++++++++--
 ..._evol_orc_vec_mapwork_part_all_complex.q.out |  726 +++
 ...vol_orc_vec_mapwork_part_all_primitive.q.out | 2887 ++++++++++
 .../schema_evol_orc_vec_mapwork_table.q.out     | 4391 +++++++++++++++-
 .../tez/schema_evol_orc_acid_mapwork_part.q.out | 4319 ++++++++++++---
 .../schema_evol_orc_acid_mapwork_table.q.out    | 3334 ++++++++++--
 .../schema_evol_orc_acidvec_mapwork_part.q.out  | 4319 ++++++++++++---
 .../schema_evol_orc_acidvec_mapwork_table.q.out | 3334 ++++++++++--
 .../schema_evol_orc_nonvec_fetchwork_part.q.out | 4449 ++++++++++++++--
 ...schema_evol_orc_nonvec_fetchwork_table.q.out | 3911 +++++++++++++-
 .../schema_evol_orc_nonvec_mapwork_part.q.out   | 4453 ++++++++++++++--
 ...ol_orc_nonvec_mapwork_part_all_complex.q.out |  669 +++
 ..._orc_nonvec_mapwork_part_all_primitive.q.out | 2587 +++++++++
 .../schema_evol_orc_nonvec_mapwork_table.q.out  | 3911 +++++++++++++-
 .../tez/schema_evol_orc_vec_mapwork_part.q.out  | 4449 ++++++++++++++--
 ..._evol_orc_vec_mapwork_part_all_complex.q.out |  669 +++
 ...vol_orc_vec_mapwork_part_all_primitive.q.out | 2587 +++++++++
 .../tez/schema_evol_orc_vec_mapwork_table.q.out | 3911 +++++++++++++-
 service-rpc/if/TCLIService.thrift               |    6 +
 .../gen/thrift/gen-cpp/TCLIService_types.cpp    |   30 +-
 .../src/gen/thrift/gen-cpp/TCLIService_types.h  |   15 +-
 .../rpc/thrift/TExecuteStatementReq.java        |  109 +-
 .../service/rpc/thrift/TOperationState.java     |    5 +-
 service-rpc/src/gen/thrift/gen-php/Types.php    |   25 +
 .../src/gen/thrift/gen-py/TCLIService/ttypes.py |   18 +-
 .../gen/thrift/gen-rb/t_c_l_i_service_types.rb  |    9 +-
 .../org/apache/hive/service/cli/CLIService.java |   46 +-
 .../service/cli/EmbeddedCLIServiceClient.java   |   19 +-
 .../apache/hive/service/cli/ICLIService.java    |   16 +-
 .../apache/hive/service/cli/OperationState.java |    7 +-
 .../operation/ExecuteStatementOperation.java    |    9 +-
 .../cli/operation/HiveCommandOperation.java     |    5 +
 .../cli/operation/MetadataOperation.java        |    7 +-
 .../hive/service/cli/operation/Operation.java   |   17 +-
 .../service/cli/operation/OperationManager.java |   27 +-
 .../service/cli/operation/SQLOperation.java     |  106 +-
 .../hive/service/cli/session/HiveSession.java   |   28 +-
 .../service/cli/session/HiveSessionImpl.java    |   38 +-
 .../thrift/RetryingThriftCLIServiceClient.java  |   22 +-
 .../service/cli/thrift/ThriftCLIService.java    |   16 +-
 .../cli/thrift/ThriftCLIServiceClient.java      |   32 +-
 .../cli/thrift/ThriftCLIServiceTest.java        |    6 +-
 .../thrift/ThriftCliServiceTestWithCookie.java  |    3 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |  148 -
 .../org/apache/hadoop/hive/io/HdfsUtils.java    |  156 +-
 .../apache/hadoop/hive/shims/HadoopShims.java   |   41 -
 .../hadoop/hive/shims/HadoopShimsSecure.java    |   10 -
 126 files changed, 101719 insertions(+), 10497 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bc75d72b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/bc75d72b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------


[34/50] [abbrv] hive git commit: HIVE-13395 Lost Update problem in ACID (Eugene Koifman, reviewed by Alan Gates)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index c0fa97a..06cd4aa 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -72,7 +72,7 @@ import java.util.regex.Pattern;
  * used to properly sequence operations.  Most notably:
  * 1. various sequence IDs are generated with aid of this mutex
  * 2. ensuring that each (Hive) Transaction state is transitioned atomically.  Transaction state
- *  includes it's actual state (Open, Aborted) as well as it's lock list/component list.  Thus all
+ *  includes its actual state (Open, Aborted) as well as it's lock list/component list.  Thus all
  *  per transaction ops, either start by update/delete of the relevant TXNS row or do S4U on that row.
  *  This allows almost all operations to run at READ_COMMITTED and minimizes DB deadlocks.
  * 3. checkLock() - this is mutexted entirely since we must ensure that while we check if some lock
@@ -126,6 +126,41 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
   static private DataSource connPool;
   static private boolean doRetryOnConnPool = false;
+  
+  private enum OpertaionType {
+    INSERT('i'), UPDATE('u'), DELETE('d');
+    private final char sqlConst;
+    OpertaionType(char sqlConst) {
+      this.sqlConst = sqlConst;
+    }
+    public String toString() {
+      return Character.toString(sqlConst);
+    }
+    public static OpertaionType fromString(char sqlConst) {
+      switch (sqlConst) {
+        case 'i':
+          return INSERT;
+        case 'u':
+          return UPDATE;
+        case 'd':
+          return DELETE;
+        default:
+          throw new IllegalArgumentException(quoteChar(sqlConst));
+      }
+    }
+    //we should instead just pass in OpertaionType from client (HIVE-13622)
+    @Deprecated
+    public static OpertaionType fromLockType(LockType lockType) {
+      switch (lockType) {
+        case SHARED_READ:
+          return INSERT;
+        case SHARED_WRITE:
+          return UPDATE;
+        default:
+          throw new IllegalArgumentException("Unexpected lock type: " + lockType);
+      }
+    }
+  }
 
   /**
    * Number of consecutive deadlocks we have seen
@@ -454,6 +489,31 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     }
   }
 
+  /**
+   * Concurrency/isolation notes:
+   * This is mutexed with {@link #openTxns(OpenTxnRequest)} and other {@link #commitTxn(CommitTxnRequest)}
+   * operations using select4update on NEXT_TXN_ID.  Also, mutexes on TXNX table for specific txnid:X
+   * see more notes below.
+   * In order to prevent lost updates, we need to determine if any 2 transactions overlap.  Each txn
+   * is viewed as an interval [M,N]. M is the txnid and N is taken from the same NEXT_TXN_ID sequence
+   * so that we can compare commit time of txn T with start time of txn S.  This sequence can be thought of
+   * as a logical time counter.  If S.commitTime < T.startTime, T and S do NOT overlap.
+   *
+   * Motivating example:
+   * Suppose we have multi-statment transactions T and S both of which are attempting x = x + 1
+   * In order to prevent lost update problem, the the non-overlapping txns must lock in the snapshot
+   * that they read appropriately.  In particular, if txns do not overlap, then one follows the other
+   * (assumig they write the same entity), and thus the 2nd must see changes of the 1st.  We ensure
+   * this by locking in snapshot after 
+   * {@link #openTxns(OpenTxnRequest)} call is made (see {@link org.apache.hadoop.hive.ql.Driver#acquireLocksAndOpenTxn()})
+   * and mutexing openTxn() with commit().  In other words, once a S.commit() starts we must ensure
+   * that txn T which will be considered a later txn, locks in a snapshot that includes the result
+   * of S's commit (assuming no other txns).
+   * As a counter example, suppose we have S[3,3] and T[4,4] (commitId=txnid means no other transactions
+   * were running in parallel).  If T and S both locked in the same snapshot (for example commit of
+   * txnid:2, which is possible if commitTxn() and openTxnx() is not mutexed)
+   * 'x' would be updated to the same value by both, i.e. lost update. 
+   */
   public void commitTxn(CommitTxnRequest rqst)
     throws NoSuchTxnException, TxnAbortedException,  MetaException {
     long txnid = rqst.getTxnid();
@@ -461,40 +521,116 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       Connection dbConn = null;
       Statement stmt = null;
       ResultSet lockHandle = null;
+      ResultSet commitIdRs = null, rs;
       try {
         lockInternal();
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+        stmt = dbConn.createStatement();
+        /**
+         * This S4U will mutex with other commitTxn() and openTxns(). 
+         * -1 below makes txn intervals look like [3,3] [4,4] if all txns are serial
+         * Note: it's possible to have several txns have the same commit id.  Suppose 3 txns start
+         * at the same time and no new txns start until all 3 commit.
+         * We could've incremented the sequence for commitId is well but it doesn't add anything functionally.
+         */
+        commitIdRs = stmt.executeQuery(addForUpdateClause("select ntxn_next - 1 from NEXT_TXN_ID"));
+        if(!commitIdRs.next()) {
+          throw new IllegalStateException("No rows found in NEXT_TXN_ID");
+        }
+        long commitId = commitIdRs.getLong(1);
         /**
          * Runs at READ_COMMITTED with S4U on TXNS row for "txnid".  S4U ensures that no other
          * operation can change this txn (such acquiring locks). While lock() and commitTxn()
          * should not normally run concurrently (for same txn) but could due to bugs in the client
          * which could then corrupt internal transaction manager state.  Also competes with abortTxn().
          */
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-
         lockHandle = lockTransactionRecord(stmt, txnid, TXN_OPEN);
         if(lockHandle == null) {
           //this also ensures that txn is still there and in expected state (hasn't been timed out)
           ensureValidTxn(dbConn, txnid, stmt);
           shouldNeverHappen(txnid);
         }
-
+        Savepoint undoWriteSetForCurrentTxn = dbConn.setSavepoint();
+        int numCompsWritten = stmt.executeUpdate("insert into WRITE_SET (ws_database, ws_table, ws_partition, ws_txnid, ws_commit_id, ws_operation_type)" +
+          " select tc_database, tc_table, tc_partition, tc_txnid, " + commitId + ", tc_operation_type " +
+          "from TXN_COMPONENTS where tc_txnid=" + txnid + " and tc_operation_type IN(" + quoteChar(OpertaionType.UPDATE.sqlConst) + "," + quoteChar(OpertaionType.DELETE.sqlConst) + ")");
+        if(numCompsWritten == 0) {
+          /**
+           * current txn didn't update/delete anything (may have inserted), so just proceed with commit
+           * 
+           * We only care about commit id for write txns, so for RO (when supported) txns we don't
+           * have to mutex on NEXT_TXN_ID.
+           * Consider: if RO txn is after a W txn, then RO's openTxns() will be mutexed with W's
+           * commitTxn() because both do S4U on NEXT_TXN_ID and thus RO will see result of W txn.
+           * If RO < W, then there is no reads-from relationship.
+           */
+        }
+        else {
+          /**
+           * see if there are any overlapping txns wrote the same element, i.e. have a conflict
+           * Since entire commit operation is mutexed wrt other start/commit ops,
+           * committed.ws_commit_id <= current.ws_commit_id for all txns
+           * thus if committed.ws_commit_id < current.ws_txnid, transactions do NOT overlap
+           * For example, [17,20] is committed, [6,80] is being committed right now - these overlap
+           * [17,20] committed and [21,21] committing now - these do not overlap.
+           * [17,18] committed and [18,19] committing now - these overlap  (here 18 started while 17 was still running)
+           */
+          rs = stmt.executeQuery
+            (addLimitClause(1, "committed.ws_txnid, committed.ws_commit_id, committed.ws_database," +
+              "committed.ws_table, committed.ws_partition, cur.ws_commit_id " + 
+              "from WRITE_SET committed INNER JOIN WRITE_SET cur " +
+            "ON committed.ws_database=cur.ws_database and committed.ws_table=cur.ws_table " +
+              //For partitioned table we always track writes at partition level (never at table)
+              //and for non partitioned - always at table level, thus the same table should never
+              //have entries with partition key and w/o
+            "and (committed.ws_partition=cur.ws_partition or (committed.ws_partition is null and cur.ws_partition is null)) " +
+            "where cur.ws_txnid <= committed.ws_commit_id" + //txns overlap; could replace ws_txnid
+              // with txnid, though any decent DB should infer this
+            " and cur.ws_txnid=" + txnid + //make sure RHS of join only has rows we just inserted as
+              // part of this commitTxn() op
+            " and committed.ws_txnid <> " + txnid + //and LHS only has committed txns
+              //U+U and U+D is a conflict but D+D is not and we don't currently track I in WRITE_SET at all
+              " and (committed.ws_operation_type=" + quoteChar(OpertaionType.UPDATE.sqlConst) +  
+                    " OR cur.ws_operation_type=" + quoteChar(OpertaionType.UPDATE.sqlConst) + ")"));
+          if(rs.next()) {
+            //found a conflict
+            String committedTxn = "[" + JavaUtils.txnIdToString(rs.getLong(1)) + "," + rs.getLong(2) + "]";
+            StringBuilder resource = new StringBuilder(rs.getString(3)).append("/").append(rs.getString(4));
+            String partitionName = rs.getString(5);
+            if(partitionName != null) {
+              resource.append('/').append(partitionName);
+            }
+            String msg = "Aborting [" + JavaUtils.txnIdToString(txnid) + "," + rs.getLong(6) + "]" + " due to a write conflict on " + resource +
+              " committed by " + committedTxn;
+            close(rs);
+            //remove WRITE_SET info for current txn since it's about to abort
+            dbConn.rollback(undoWriteSetForCurrentTxn);
+            LOG.info(msg);
+            //todo: should make abortTxns() write something into TXNS.TXN_META_INFO about this
+            if(abortTxns(dbConn, Collections.singletonList(txnid)) != 1) {
+              throw new IllegalStateException(msg + " FAILED!");
+            }
+            dbConn.commit();
+            close(null, stmt, dbConn);
+            throw new TxnAbortedException(msg);
+          }
+          else {
+            //no conflicting operations, proceed with the rest of commit sequence
+          }
+        }
         // Move the record from txn_components into completed_txn_components so that the compactor
         // knows where to look to compact.
         String s = "insert into COMPLETED_TXN_COMPONENTS select tc_txnid, tc_database, tc_table, " +
           "tc_partition from TXN_COMPONENTS where tc_txnid = " + txnid;
         LOG.debug("Going to execute insert <" + s + ">");
         if (stmt.executeUpdate(s) < 1) {
-          //this can be reasonable for an empty txn START/COMMIT
+          //this can be reasonable for an empty txn START/COMMIT or read-only txn
           LOG.info("Expected to move at least one record from txn_components to " +
             "completed_txn_components when committing txn! " + JavaUtils.txnIdToString(txnid));
         }
-
-        // Always access TXN_COMPONENTS before HIVE_LOCKS;
         s = "delete from TXN_COMPONENTS where tc_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
         stmt.executeUpdate(s);
-        // Always access HIVE_LOCKS before TXNS
         s = "delete from HIVE_LOCKS where hl_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
         stmt.executeUpdate(s);
@@ -510,6 +646,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         throw new MetaException("Unable to update transaction database "
           + StringUtils.stringifyException(e));
       } finally {
+        close(commitIdRs);
         close(lockHandle, stmt, dbConn);
         unlockInternal();
       }
@@ -517,7 +654,50 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       commitTxn(rqst);
     }
   }
-
+  @Override
+  public void performWriteSetGC() {
+    Connection dbConn = null;
+    Statement stmt = null;
+    ResultSet rs = null;
+    try {
+      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+      stmt = dbConn.createStatement();
+      rs = stmt.executeQuery("select ntxn_next - 1 from NEXT_TXN_ID");
+      if(!rs.next()) {
+        throw new IllegalStateException("NEXT_TXN_ID is empty: DB is corrupted");
+      }
+      long highestAllocatedTxnId = rs.getLong(1);
+      close(rs);
+      rs = stmt.executeQuery("select min(txn_id) from TXNS where txn_state=" + quoteChar(TXN_OPEN));
+      if(!rs.next()) {
+        throw new IllegalStateException("Scalar query returned no rows?!?!!");
+      }
+      long commitHighWaterMark;//all currently open txns (if any) have txnid >= than commitHighWaterMark
+      long lowestOpenTxnId = rs.getLong(1);
+      if(rs.wasNull()) {
+        //if here then there are no Open txns and  highestAllocatedTxnId must be
+        //resolved (i.e. committed or aborted), either way
+        //there are no open txns with id <= highestAllocatedTxnId
+        //the +1 is there because "delete ..." below has < (which is correct for the case when
+        //there is an open txn
+        //Concurrency: even if new txn starts (or starts + commits) it is still true that
+        //there are no currently open txns that overlap with any committed txn with 
+        //commitId <= commitHighWaterMark (as set on next line).  So plain READ_COMMITTED is enough.
+        commitHighWaterMark = highestAllocatedTxnId + 1;
+      }
+      else {
+        commitHighWaterMark = lowestOpenTxnId;
+      }
+      int delCnt = stmt.executeUpdate("delete from WRITE_SET where ws_commit_id < " + commitHighWaterMark);
+      LOG.info("Deleted " + delCnt + " obsolete rows from WRTIE_SET");
+      dbConn.commit();
+    } catch (SQLException ex) {
+      LOG.warn("WriteSet GC failed due to " + getMessage(ex), ex);
+    }
+    finally {
+      close(rs, stmt, dbConn);
+    }
+  }
   /**
    * As much as possible (i.e. in absence of retries) we want both operations to be done on the same
    * connection (but separate transactions).  This avoid some flakiness in BONECP where if you
@@ -545,7 +725,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
   /**
    * Note that by definition select for update is divorced from update, i.e. you executeQuery() to read
-   * and then executeUpdate().  One other alternative would be to actually update the row in TXNX but
+   * and then executeUpdate().  One other alternative would be to actually update the row in TXNS but
    * to the same value as before thus forcing db to acquire write lock for duration of the transaction.
    *
    * There is no real reason to return the ResultSet here other than to make sure the reference to it
@@ -616,6 +796,19 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         stmt.executeUpdate(s);
 
         if (txnid > 0) {
+          /**DBTxnManager#acquireLocks() knows if it's I/U/D (that's how it decides what lock to get)
+           * So if we add that to LockRequest we'll know that here 
+           * Should probably add it to LockComponent so that if in the future we decide wo allow 1 LockRequest
+           * to contain LockComponent for multiple operations.
+           * Deriving it from lock info doesn't distinguish between Update and Delete
+           * 
+           * QueryPlan has BaseSemanticAnalyzer which has acidFileSinks list of FileSinkDesc
+           * FileSinkDesc.table is ql.metadata.Table
+           * Table.tableSpec which is TableSpec, which has specType which is SpecType
+           * So maybe this can work to know that this is part of dynamic partition insert in which case
+           * we'll get addDynamicPartitions() call and should not write TXN_COMPONENTS here.
+           * In any case, that's an optimization for now;  will be required when adding multi-stmt txns
+           */
           // For each component in this lock request,
           // add an entry to the txn_components table
           // This must be done before HIVE_LOCKS is accessed
@@ -624,10 +817,11 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
             String tblName = lc.getTablename();
             String partName = lc.getPartitionname();
             s = "insert into TXN_COMPONENTS " +
-              "(tc_txnid, tc_database, tc_table, tc_partition) " +
+              "(tc_txnid, tc_database, tc_table, tc_partition, tc_operation_type) " +
               "values (" + txnid + ", '" + dbName + "', " +
               (tblName == null ? "null" : "'" + tblName + "'") + ", " +
-              (partName == null ? "null" : "'" + partName + "'") + ")";
+              (partName == null ? "null" : "'" + partName + "'")+ "," +
+              quoteString(OpertaionType.fromLockType(lc.getType()).toString()) + ")";
             LOG.debug("Going to execute update <" + s + ">");
             stmt.executeUpdate(s);
           }
@@ -698,9 +892,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         lockInternal();
         if(dbConn.isClosed()) {
           //should only get here if retrying this op
-          dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+          dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         }
-        dbConn.setTransactionIsolation(Connection.TRANSACTION_SERIALIZABLE);
         return checkLock(dbConn, extLockId);
       } catch (SQLException e) {
         LOG.debug("Going to rollback");
@@ -756,7 +949,6 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         //todo: strictly speaking there is a bug here.  heartbeat*() commits but both heartbeat and
         //checkLock() are in the same retry block, so if checkLock() throws, heartbeat is also retired
         //extra heartbeat is logically harmless, but ...
-        dbConn.setTransactionIsolation(Connection.TRANSACTION_SERIALIZABLE);
         return checkLock(dbConn, extLockId);
       } catch (SQLException e) {
         LOG.debug("Going to rollback");
@@ -1162,11 +1354,13 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     throw new RuntimeException("This should never happen: " + JavaUtils.txnIdToString(txnid) + " "
       + JavaUtils.lockIdToString(extLockId) + " " + intLockId);
   }
+
   public void addDynamicPartitions(AddDynamicPartitions rqst)
       throws NoSuchTxnException,  TxnAbortedException, MetaException {
     Connection dbConn = null;
     Statement stmt = null;
     ResultSet lockHandle = null;
+    ResultSet rs = null;
     try {
       try {
         lockInternal();
@@ -1178,18 +1372,35 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           ensureValidTxn(dbConn, rqst.getTxnid(), stmt);
           shouldNeverHappen(rqst.getTxnid());
         }
+        //we should be able to get this from AddDynamicPartitions object longer term; in fact we'd have to
+        //for multi stmt txns if same table is written more than once per tx
+        // MoveTask knows if it's I/U/D
+        // MoveTask calls Hive.loadDynamicPartitions() which calls HiveMetaStoreClient.addDynamicPartitions()
+        // which ends up here so we'd need to add a field to AddDynamicPartitions.
+        String findOperationType = " tc_operation_type from TXN_COMPONENTS where tc_txnid=" + rqst.getTxnid()
+          + " and tc_database=" + quoteString(rqst.getDbname()) + " and tc_table=" + quoteString(rqst.getTablename());
+        //do limit 1 on this; currently they will all have the same operations
+        rs = stmt.executeQuery(addLimitClause(1, findOperationType));
+        if(!rs.next()) {
+          throw new IllegalStateException("Unable to determine tc_operation_type for " + JavaUtils.txnIdToString(rqst.getTxnid()));
+        }
+        OpertaionType ot = OpertaionType.fromString(rs.getString(1).charAt(0));
+        
+        //what if a txn writes the same table > 1 time... let's go with this for now, but really
+        //need to not write this in the first place, i.e. make this delete not needed
+        //see enqueueLockWithRetry() - that's where we write to TXN_COMPONENTS
+        String deleteSql = "delete from TXN_COMPONENTS where tc_txnid=" + rqst.getTxnid() + " and tc_database=" +
+          quoteString(rqst.getDbname()) + " and tc_table=" + quoteString(rqst.getTablename());
+        //we delete the entries made by enqueueLockWithRetry() since those are based on lock information which is
+        //much "wider" than necessary in a lot of cases.  Here on the other hand, we know exactly which
+        //partitions have been written to.  w/o this WRITE_SET would contain entries for partitions not actually
+        //written to
+        stmt.executeUpdate(deleteSql);
         for (String partName : rqst.getPartitionnames()) {
-          StringBuilder buff = new StringBuilder();
-          buff.append("insert into TXN_COMPONENTS (tc_txnid, tc_database, tc_table, tc_partition) values (");
-          buff.append(rqst.getTxnid());
-          buff.append(", '");
-          buff.append(rqst.getDbname());
-          buff.append("', '");
-          buff.append(rqst.getTablename());
-          buff.append("', '");
-          buff.append(partName);
-          buff.append("')");
-          String s = buff.toString();
+          String s =
+            "insert into TXN_COMPONENTS (tc_txnid, tc_database, tc_table, tc_partition, tc_operation_type) values (" +
+              rqst.getTxnid() + "," + quoteString(rqst.getDbname()) + "," + quoteString(rqst.getTablename()) +
+              "," + quoteString(partName) + "," + quoteChar(ot.sqlConst) + ")";
           LOG.debug("Going to execute update <" + s + ">");
           stmt.executeUpdate(s);
         }
@@ -1908,60 +2119,113 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     return txnId != 0;
   }
   /**
+   * Lock acquisition is meant to be fair, so every lock can only block on some lock with smaller
+   * hl_lock_ext_id by only checking earlier locks.
+   *
+   * For any given SQL statment all locks required by it are grouped under single extLockId and are
+   * granted all at once or all locks wait.
+   *
+   * This is expected to run at READ_COMMITTED.
+   *
    * Note: this calls acquire() for (extLockId,intLockId) but extLockId is the same and we either take
    * all locks for given extLockId or none.  Would be more efficient to update state on all locks
-   * at once.  Semantics are the same since this is all part of the same txn@serializable.
+   * at once.  Semantics are the same since this is all part of the same txn.
    *
-   * Lock acquisition is meant to be fair, so every lock can only block on some lock with smaller
-   * hl_lock_ext_id by only checking earlier locks.
+   * If there is a concurrent commitTxn/rollbackTxn, those can only remove rows from HIVE_LOCKS.
+   * If they happen to be for the same txnid, there will be a WW conflict (in MS DB), if different txnid,
+   * checkLock() will in the worst case keep locks in Waiting state a little longer.
    */
-  private LockResponse checkLock(Connection dbConn,
-                                 long extLockId)
+  private LockResponse checkLock(Connection dbConn, long extLockId)
     throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, MetaException, SQLException {
-    if(dbConn.getTransactionIsolation() != Connection.TRANSACTION_SERIALIZABLE) {
-      //longer term we should instead use AUX_TABLE/S4U to serialize all checkLock() operations
-      //that would be less prone to deadlocks
-      throw new IllegalStateException("Unexpected Isolation Level: " + dbConn.getTransactionIsolation());
-    }
-    List<LockInfo> locksBeingChecked = getLockInfoFromLockId(dbConn, extLockId);//being acquired now
+    TxnStore.MutexAPI.LockHandle handle =  null;
+    Statement stmt = null;
+    ResultSet rs = null;
     LockResponse response = new LockResponse();
-    response.setLockid(extLockId);
-
-    LOG.debug("checkLock(): Setting savepoint. extLockId=" + JavaUtils.lockIdToString(extLockId));
-    Savepoint save = dbConn.setSavepoint();//todo: get rid of this
-    StringBuilder query = new StringBuilder("select hl_lock_ext_id, " +
-      "hl_lock_int_id, hl_db, hl_table, hl_partition, hl_lock_state, " +
-      "hl_lock_type, hl_txnid from HIVE_LOCKS where hl_db in (");
-
-    Set<String> strings = new HashSet<String>(locksBeingChecked.size());
-    for (LockInfo info : locksBeingChecked) {
-      strings.add(info.db);
-    }
-    boolean first = true;
-    for (String s : strings) {
-      if (first) first = false;
-      else query.append(", ");
-      query.append('\'');
-      query.append(s);
-      query.append('\'');
-    }
-    query.append(")");
-
-    // If any of the table requests are null, then I need to pull all the
-    // table locks for this db.
-    boolean sawNull = false;
-    strings.clear();
-    for (LockInfo info : locksBeingChecked) {
-      if (info.table == null) {
-        sawNull = true;
-        break;
-      } else {
-        strings.add(info.table);
+    /**
+     * todo: Longer term we should pass this from client somehow - this would be an optimization;  once
+     * that is in place make sure to build and test "writeSet" below using OperationType not LockType
+     */
+    boolean isPartOfDynamicPartitionInsert = true;
+    try {
+      /**
+       * checkLock() must be mutexed against any other checkLock to make sure 2 conflicting locks
+       * are not granted by parallel checkLock() calls.
+       */
+      handle = getMutexAPI().acquireLock(MUTEX_KEY.CheckLock.name());
+      List<LockInfo> locksBeingChecked = getLockInfoFromLockId(dbConn, extLockId);//being acquired now
+      response.setLockid(extLockId);
+
+      LOG.debug("checkLock(): Setting savepoint. extLockId=" + JavaUtils.lockIdToString(extLockId));
+      Savepoint save = dbConn.setSavepoint();//todo: get rid of this
+      StringBuilder query = new StringBuilder("select hl_lock_ext_id, " +
+        "hl_lock_int_id, hl_db, hl_table, hl_partition, hl_lock_state, " +
+        "hl_lock_type, hl_txnid from HIVE_LOCKS where hl_db in (");
+
+      Set<String> strings = new HashSet<String>(locksBeingChecked.size());
+
+      //This the set of entities that the statement represnted by extLockId wants to update
+      List<LockInfo> writeSet = new ArrayList<>();
+
+      for (LockInfo info : locksBeingChecked) {
+        strings.add(info.db);
+        if(!isPartOfDynamicPartitionInsert && info.type == LockType.SHARED_WRITE) {
+          writeSet.add(info);
+        }
       }
-    }
-    if (!sawNull) {
-      query.append(" and (hl_table is null or hl_table in(");
-      first = true;
+      if(!writeSet.isEmpty()) {
+        if(writeSet.get(0).txnId == 0) {
+          //Write operation always start a txn
+          throw new IllegalStateException("Found Write lock for " + JavaUtils.lockIdToString(extLockId) + " but no txnid");
+        }
+        stmt = dbConn.createStatement();
+        StringBuilder sb = new StringBuilder(" ws_database, ws_table, ws_partition, " +
+          "ws_txnid, ws_commit_id " +
+          "from WRITE_SET where ws_commit_id >= " + writeSet.get(0).txnId + " and (");//see commitTxn() for more info on this inequality
+        for(LockInfo info : writeSet) {
+          sb.append("(ws_database = ").append(quoteString(info.db)).append(" and ws_table = ")
+            .append(quoteString(info.table)).append(" and ws_partition ")
+            .append(info.partition == null ? "is null" : "= " + quoteString(info.partition)).append(") or ");
+        }
+        sb.setLength(sb.length() - 4);//nuke trailing " or "
+        sb.append(")");
+        //1 row is sufficient to know we have to kill the query
+        rs = stmt.executeQuery(addLimitClause(1, sb.toString()));
+        if(rs.next()) {
+          /**
+           * if here, it means we found an already committed txn which overlaps with the current one and
+           * it updated the same resource the current txn wants to update.  By First-committer-wins
+           * rule, current txn will not be allowed to commit so  may as well kill it now;  This is just an
+           * optimization to prevent wasting cluster resources to run a query which is known to be DOA.
+           * {@link #commitTxn(CommitTxnRequest)} has the primary responsibility to ensure this.
+           * checkLock() runs at READ_COMMITTED so you could have another (Hive) txn running commitTxn()
+           * in parallel and thus writing to WRITE_SET.  commitTxn() logic is properly mutexed to ensure
+           * that we don't "miss" any WW conflicts. We could've mutexed the checkLock() and commitTxn()
+           * as well but this reduces concurrency for very little gain.
+           * Note that update/delete (which runs as dynamic partition insert) acquires a lock on the table,
+           * but WRITE_SET has entries for actual partitions updated.  Thus this optimization will "miss"
+           * the WW conflict but it will be caught in commitTxn() where actual partitions written are known.
+           * This is OK since we want 2 concurrent updates that update different sets of partitions to both commit.
+           */
+          String resourceName = rs.getString(1) + '/' + rs.getString(2);
+          String partName = rs.getString(3);
+          if(partName != null) {
+            resourceName += '/' + partName;
+          }
+
+          String msg = "Aborting " + JavaUtils.txnIdToString(writeSet.get(0).txnId) +
+            " since a concurrent committed transaction [" + JavaUtils.txnIdToString(rs.getLong(4)) + "," + rs.getLong(5) +
+            "] has already updated resouce '" + resourceName + "'";
+          LOG.info(msg);
+          if(abortTxns(dbConn, Collections.singletonList(writeSet.get(0).txnId)) != 1) {
+            throw new IllegalStateException(msg + " FAILED!");
+          }
+          dbConn.commit();
+          throw new TxnAbortedException(msg);
+        }
+        close(rs, stmt, null);
+      }
+
+      boolean first = true;
       for (String s : strings) {
         if (first) first = false;
         else query.append(", ");
@@ -1969,22 +2233,22 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         query.append(s);
         query.append('\'');
       }
-      query.append("))");
+      query.append(")");
 
-      // If any of the partition requests are null, then I need to pull all
-      // partition locks for this table.
-      sawNull = false;
+      // If any of the table requests are null, then I need to pull all the
+      // table locks for this db.
+      boolean sawNull = false;
       strings.clear();
       for (LockInfo info : locksBeingChecked) {
-        if (info.partition == null) {
+        if (info.table == null) {
           sawNull = true;
           break;
         } else {
-          strings.add(info.partition);
+          strings.add(info.table);
         }
       }
       if (!sawNull) {
-        query.append(" and (hl_partition is null or hl_partition in(");
+        query.append(" and (hl_table is null or hl_table in(");
         first = true;
         for (String s : strings) {
           if (first) first = false;
@@ -1994,14 +2258,35 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           query.append('\'');
         }
         query.append("))");
+
+        // If any of the partition requests are null, then I need to pull all
+        // partition locks for this table.
+        sawNull = false;
+        strings.clear();
+        for (LockInfo info : locksBeingChecked) {
+          if (info.partition == null) {
+            sawNull = true;
+            break;
+          } else {
+            strings.add(info.partition);
+          }
+        }
+        if (!sawNull) {
+          query.append(" and (hl_partition is null or hl_partition in(");
+          first = true;
+          for (String s : strings) {
+            if (first) first = false;
+            else query.append(", ");
+            query.append('\'');
+            query.append(s);
+            query.append('\'');
+          }
+          query.append("))");
+        }
       }
-    }
-    query.append(" and hl_lock_ext_id <= ").append(extLockId);
+      query.append(" and hl_lock_ext_id <= ").append(extLockId);
 
-    LOG.debug("Going to execute query <" + query.toString() + ">");
-    Statement stmt = null;
-    ResultSet rs = null;
-    try {
+      LOG.debug("Going to execute query <" + query.toString() + ">");
       stmt = dbConn.createStatement();
       rs = stmt.executeQuery(query.toString());
       SortedSet<LockInfo> lockSet = new TreeSet<LockInfo>(new LockInfoComparator());
@@ -2117,6 +2402,9 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       response.setState(LockState.ACQUIRED);
     } finally {
       close(rs, stmt, null);
+      if(handle != null) {
+        handle.releaseLocks();
+      }
     }
     return response;
   }
@@ -2158,7 +2446,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     String s = "update HIVE_LOCKS set hl_lock_state = '" + LOCK_ACQUIRED + "', " +
       //if lock is part of txn, heartbeat info is in txn record
       "hl_last_heartbeat = " + (isValidTxn(lockInfo.txnId) ? 0 : now) +
-    ", hl_acquired_at = " + now + " where hl_lock_ext_id = " +
+    ", hl_acquired_at = " + now + ",HL_BLOCKEDBY_EXT_ID=NULL,HL_BLOCKEDBY_INT_ID=null" + " where hl_lock_ext_id = " +
       extLockId + " and hl_lock_int_id = " + lockInfo.intLockId;
     LOG.debug("Going to execute update <" + s + ">");
     int rc = stmt.executeUpdate(s);
@@ -2238,6 +2526,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       //todo: add LIMIT 1 instead of count - should be more efficient
       s = "select count(*) from COMPLETED_TXN_COMPONENTS where CTC_TXNID = " + txnid;
       ResultSet rs2 = stmt.executeQuery(s);
+      //todo: strictly speaking you can commit an empty txn, thus 2nd conjunct is wrong but only
+      //possible for for multi-stmt txns
       boolean alreadyCommitted = rs2.next() && rs2.getInt(1) > 0;
       LOG.debug("Going to rollback");
       dbConn.rollback();

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
index 927e9bc..f9cac18 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
@@ -47,7 +47,7 @@ import java.util.Set;
 @InterfaceStability.Evolving
 public interface TxnStore {
 
-  public static enum MUTEX_KEY {Initiator, Cleaner, HouseKeeper, CompactionHistory}
+  public static enum MUTEX_KEY {Initiator, Cleaner, HouseKeeper, CompactionHistory, CheckLock, WriteSetCleaner}
   // Compactor states (Should really be enum)
   static final public String INITIATED_RESPONSE = "initiated";
   static final public String WORKING_RESPONSE = "working";
@@ -321,6 +321,12 @@ public interface TxnStore {
   public void purgeCompactionHistory() throws MetaException;
 
   /**
+   * WriteSet tracking is used to ensure proper transaction isolation.  This method deletes the 
+   * transaction metadata once it becomes unnecessary.  
+   */
+  public void performWriteSetGC();
+
+  /**
    * Determine if there are enough consecutive failures compacting a table or partition that no
    * new automatic compactions should be scheduled.  User initiated compactions do not do this
    * check.

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
index cc9e583..b829d9d 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
@@ -69,6 +69,8 @@ public class TxnUtils {
    * @return a valid txn list.
    */
   public static ValidTxnList createValidCompactTxnList(GetOpenTxnsInfoResponse txns) {
+    //todo: this could be more efficient: using select min(txn_id) from TXNS where txn_state=" +
+    // quoteChar(TXN_OPEN)  to compute compute HWM...
     long highWater = txns.getTxn_high_water_mark();
     long minOpenTxn = Long.MAX_VALUE;
     long[] exceptions = new long[txns.getOpen_txnsSize()];

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
index 2c1560b..80e3cd6 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
@@ -413,7 +413,7 @@ public class TestCompactionTxnHandler {
     lc.setTablename(tableName);
     LockRequest lr = new LockRequest(Arrays.asList(lc), "me", "localhost");
     lr.setTxnid(txnId);
-    LockResponse lock = txnHandler.lock(new LockRequest(Arrays.asList(lc), "me", "localhost"));
+    LockResponse lock = txnHandler.lock(lr);
     assertEquals(LockState.ACQUIRED, lock.getState());
 
     txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnId, dbName, tableName,
@@ -429,8 +429,8 @@ public class TestCompactionTxnHandler {
       assertEquals(dbName, ci.dbname);
       assertEquals(tableName, ci.tableName);
       switch (i++) {
-      case 0: assertEquals("ds=today", ci.partName); break;
-      case 1: assertEquals("ds=yesterday", ci.partName); break;
+        case 0: assertEquals("ds=today", ci.partName); break;
+        case 1: assertEquals("ds=yesterday", ci.partName); break;
       default: throw new RuntimeException("What?");
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
index 28d0269..1a118a9 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
@@ -483,6 +483,7 @@ public class TestTxnHandler {
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
     res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.ACQUIRED);
   }
@@ -514,6 +515,7 @@ public class TestTxnHandler {
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
     res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.WAITING);
   }
@@ -580,6 +582,7 @@ public class TestTxnHandler {
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
     LockResponse res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.ACQUIRED);
 
@@ -602,6 +605,7 @@ public class TestTxnHandler {
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
     LockResponse res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.ACQUIRED);
 
@@ -611,6 +615,7 @@ public class TestTxnHandler {
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
     res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.WAITING);
 
@@ -633,6 +638,7 @@ public class TestTxnHandler {
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
     LockResponse res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.ACQUIRED);
 
@@ -642,6 +648,7 @@ public class TestTxnHandler {
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
     res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.WAITING);
 
@@ -651,6 +658,7 @@ public class TestTxnHandler {
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
     res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.WAITING);
   }
@@ -682,6 +690,7 @@ public class TestTxnHandler {
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
     res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.WAITING);
   }
@@ -725,6 +734,8 @@ public class TestTxnHandler {
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
+    long txnId = openTxn();
+    req.setTxnid(txnId);
     LockResponse res = txnHandler.lock(req);
     long lockid1 = res.getLockid();
     assertTrue(res.getState() == LockState.ACQUIRED);
@@ -735,11 +746,12 @@ public class TestTxnHandler {
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
     res = txnHandler.lock(req);
     long lockid2 = res.getLockid();
     assertTrue(res.getState() == LockState.WAITING);
 
-    txnHandler.unlock(new UnlockRequest(lockid1));
+    txnHandler.abortTxn(new AbortTxnRequest(txnId));
     res = txnHandler.checkLock(new CheckLockRequest(lockid2));
     assertTrue(res.getState() == LockState.ACQUIRED);
   }
@@ -1070,16 +1082,14 @@ public class TestTxnHandler {
   @Test
   public void showLocks() throws Exception {
     long begining = System.currentTimeMillis();
-    long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
-    req.setTxnid(txnid);
     LockResponse res = txnHandler.lock(req);
 
     // Open txn
-    txnid = openTxn();
+    long txnid = openTxn();
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "mydb");
     comp.setTablename("mytable");
     components = new ArrayList<LockComponent>(1);
@@ -1090,7 +1100,7 @@ public class TestTxnHandler {
 
     // Locks not associated with a txn
     components = new ArrayList<LockComponent>(1);
-    comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "yourdb");
+    comp = new LockComponent(LockType.SHARED_READ, LockLevel.PARTITION, "yourdb");
     comp.setTablename("yourtable");
     comp.setPartitionname("yourpartition");
     components.add(comp);
@@ -1104,14 +1114,13 @@ public class TestTxnHandler {
     for (int i = 0; i < saw.length; i++) saw[i] = false;
     for (ShowLocksResponseElement lock : locks) {
       if (lock.getLockid() == 1) {
-        assertEquals(1, lock.getTxnid());
+        assertEquals(0, lock.getTxnid());
         assertEquals("mydb", lock.getDbname());
         assertNull(lock.getTablename());
         assertNull(lock.getPartname());
         assertEquals(LockState.ACQUIRED, lock.getState());
         assertEquals(LockType.EXCLUSIVE, lock.getType());
-        assertTrue(lock.toString(), 0 == lock.getLastheartbeat() &&
-            lock.getTxnid() != 0);
+        assertTrue(lock.toString(), 0 != lock.getLastheartbeat());
         assertTrue("Expected acquired at " + lock.getAcquiredat() + " to be between " + begining
             + " and " + System.currentTimeMillis(),
             begining <= lock.getAcquiredat() && System.currentTimeMillis() >= lock.getAcquiredat());
@@ -1119,7 +1128,7 @@ public class TestTxnHandler {
         assertEquals("localhost", lock.getHostname());
         saw[0] = true;
       } else if (lock.getLockid() == 2) {
-        assertEquals(2, lock.getTxnid());
+        assertEquals(1, lock.getTxnid());
         assertEquals("mydb", lock.getDbname());
         assertEquals("mytable", lock.getTablename());
         assertNull(lock.getPartname());
@@ -1137,7 +1146,7 @@ public class TestTxnHandler {
         assertEquals("yourtable", lock.getTablename());
         assertEquals("yourpartition", lock.getPartname());
         assertEquals(LockState.ACQUIRED, lock.getState());
-        assertEquals(LockType.SHARED_WRITE, lock.getType());
+        assertEquals(LockType.SHARED_READ, lock.getType());
         assertTrue(lock.toString(), begining <= lock.getLastheartbeat() &&
             System.currentTimeMillis() >= lock.getLastheartbeat());
         assertTrue(begining <= lock.getAcquiredat() &&

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index 1de3309..52dadb7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -377,7 +377,7 @@ public enum ErrorMsg {
       "instantiated, check hive.txn.manager"),
   TXN_NO_SUCH_TRANSACTION(10262, "No record of transaction {0} could be found, " +
       "may have timed out", true),
-  TXN_ABORTED(10263, "Transaction manager has aborted the transaction {0}.", true),
+  TXN_ABORTED(10263, "Transaction manager has aborted the transaction {0}.  Reason: {1}", true),
   DBTXNMGR_REQUIRES_CONCURRENCY(10264,
       "To use DbTxnManager you must set hive.support.concurrency=true"),
   TXNMGR_NOT_ACID(10265, "This command is not allowed on an ACID table {0}.{1} with a non-ACID transaction manager", true),

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
index 7fa57d6..18ed864 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
@@ -172,8 +172,9 @@ public class DbLockManager implements HiveLockManager{
       LOG.error("Metastore could not find " + JavaUtils.txnIdToString(lock.getTxnid()));
       throw new LockException(e, ErrorMsg.TXN_NO_SUCH_TRANSACTION, JavaUtils.txnIdToString(lock.getTxnid()));
     } catch (TxnAbortedException e) {
-      LOG.error("Transaction " + JavaUtils.txnIdToString(lock.getTxnid()) + " already aborted.");
-      throw new LockException(e, ErrorMsg.TXN_ABORTED, JavaUtils.txnIdToString(lock.getTxnid()));
+      LockException le = new LockException(e, ErrorMsg.TXN_ABORTED, JavaUtils.txnIdToString(lock.getTxnid()), e.getMessage());
+      LOG.error(le.getMessage());
+      throw le;
     } catch (TException e) {
       throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(),
           e);

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index 3aec8eb..9c2a346 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -107,6 +107,8 @@ public class DbTxnManager extends HiveTxnManagerImpl {
 
   @Override
   public long openTxn(String user) throws LockException {
+    //todo: why don't we lock the snapshot here???  Instead of having client make an explicit call
+    //whenever it chooses
     init();
     if(isTxnOpen()) {
       throw new LockException("Transaction already opened. " + JavaUtils.txnIdToString(txnId));
@@ -132,8 +134,17 @@ public class DbTxnManager extends HiveTxnManagerImpl {
 
   @Override
   public void acquireLocks(QueryPlan plan, Context ctx, String username) throws LockException {
-    acquireLocks(plan, ctx, username, true);
-    startHeartbeat();
+    try {
+      acquireLocks(plan, ctx, username, true);
+      startHeartbeat();
+    }
+    catch(LockException e) {
+      if(e.getCause() instanceof TxnAbortedException) {
+        txnId = 0;
+        statementId = -1;
+      }
+      throw e;
+    }
   }
 
   /**
@@ -157,7 +168,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
     // For each source to read, get a shared lock
     for (ReadEntity input : plan.getInputs()) {
       if (!input.needsLock() || input.isUpdateOrDelete()) {
-        // We don't want to acquire readlocks during update or delete as we'll be acquiring write
+        // We don't want to acquire read locks during update or delete as we'll be acquiring write
         // locks instead.
         continue;
       }
@@ -320,8 +331,9 @@ public class DbTxnManager extends HiveTxnManagerImpl {
       LOG.error("Metastore could not find " + JavaUtils.txnIdToString(txnId));
       throw new LockException(e, ErrorMsg.TXN_NO_SUCH_TRANSACTION, JavaUtils.txnIdToString(txnId));
     } catch (TxnAbortedException e) {
-      LOG.error("Transaction " + JavaUtils.txnIdToString(txnId) + " aborted");
-      throw new LockException(e, ErrorMsg.TXN_ABORTED, JavaUtils.txnIdToString(txnId));
+      LockException le = new LockException(e, ErrorMsg.TXN_ABORTED, JavaUtils.txnIdToString(txnId), e.getMessage());
+      LOG.error(le.getMessage());
+      throw le;
     } catch (TException e) {
       throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(),
           e);
@@ -389,8 +401,9 @@ public class DbTxnManager extends HiveTxnManagerImpl {
         LOG.error("Unable to find transaction " + JavaUtils.txnIdToString(txnId));
         throw new LockException(e, ErrorMsg.TXN_NO_SUCH_TRANSACTION, JavaUtils.txnIdToString(txnId));
       } catch (TxnAbortedException e) {
-        LOG.error("Transaction aborted " + JavaUtils.txnIdToString(txnId));
-        throw new LockException(e, ErrorMsg.TXN_ABORTED, JavaUtils.txnIdToString(txnId));
+        LockException le = new LockException(e, ErrorMsg.TXN_ABORTED, JavaUtils.txnIdToString(txnId), e.getMessage());
+        LOG.error(le.getMessage());
+        throw le;
       } catch (TException e) {
         throw new LockException(
             ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg() + "(" + JavaUtils.txnIdToString(txnId)

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidWriteSetService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidWriteSetService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidWriteSetService.java
new file mode 100644
index 0000000..9085a6a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidWriteSetService.java
@@ -0,0 +1,61 @@
+package org.apache.hadoop.hive.ql.txn;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.HouseKeeperServiceBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Periodically cleans WriteSet tracking information used in Transaction management
+ */
+public class AcidWriteSetService extends HouseKeeperServiceBase {
+  private static final Logger LOG = LoggerFactory.getLogger(AcidWriteSetService.class);
+  @Override
+  protected long getStartDelayMs() {
+    return 0;
+  }
+  @Override
+  protected long getIntervalMs() {
+    return hiveConf.getTimeVar(HiveConf.ConfVars.WRITE_SET_REAPER_INTERVAL, TimeUnit.MILLISECONDS);
+  }
+  @Override
+  protected Runnable getScheduedAction(HiveConf hiveConf, AtomicInteger isAliveCounter) {
+    return new WriteSetReaper(hiveConf, isAliveCounter);
+  }
+  @Override
+  public String getServiceDescription() {
+    return "Periodically cleans obsolete WriteSet tracking information used in Transaction management";
+  }
+  private static final class WriteSetReaper implements Runnable {
+    private final TxnStore txnHandler;
+    private final AtomicInteger isAliveCounter;
+    private WriteSetReaper(HiveConf hiveConf, AtomicInteger isAliveCounter) {
+      txnHandler = TxnUtils.getTxnStore(hiveConf);
+      this.isAliveCounter = isAliveCounter;
+    }
+    @Override
+    public void run() {
+      TxnStore.MutexAPI.LockHandle handle = null;
+      try {
+        handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.WriteSetCleaner.name());
+        long startTime = System.currentTimeMillis();
+        txnHandler.performWriteSetGC();
+        int count = isAliveCounter.incrementAndGet();
+        LOG.info("cleaner ran for " + (System.currentTimeMillis() - startTime)/1000 + "seconds.  isAliveCounter=" + count);
+      }
+      catch(Throwable t) {
+        LOG.error("Serious error in {}", Thread.currentThread().getName(), ": {}" + t.getMessage(), t);
+      }
+      finally {
+        if(handle != null) {
+          handle.releaseLocks();
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
index 947f17c..caab10d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
@@ -81,7 +81,7 @@ public abstract class HouseKeeperServiceBase implements HouseKeeperService {
    */
   protected abstract long getStartDelayMs();
   /**
-   * Determines how fequently the service is running its task.
+   * Determines how frequently the service is running its task.
    */
   protected abstract long getIntervalMs();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
index abbe5d4..949cbd5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
@@ -147,7 +147,7 @@ public class Initiator extends CompactorThread {
               if (compactionNeeded != null) requestCompaction(ci, runAs, compactionNeeded);
             } catch (Throwable t) {
               LOG.error("Caught exception while trying to determine if we should compact " +
-                  ci + ".  Marking clean to avoid repeated failures, " +
+                  ci + ".  Marking failed to avoid repeated failures, " +
                   "" + StringUtils.stringifyException(t));
               txnHandler.markFailed(ci);
             }

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
index 6238e2b..767c10c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
@@ -182,7 +182,7 @@ public class Worker extends CompactorThread {
           txnHandler.markCompacted(ci);
         } catch (Exception e) {
           LOG.error("Caught exception while trying to compact " + ci +
-              ".  Marking clean to avoid repeated failures, " + StringUtils.stringifyException(e));
+              ".  Marking failed to avoid repeated failures, " + StringUtils.stringifyException(e));
           txnHandler.markFailed(ci);
         }
       } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
index 1030987..472da0b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
@@ -669,7 +669,7 @@ public class TestTxnCommands2 {
     t.run();
   }
 
-  private static void runHouseKeeperService(HouseKeeperService houseKeeperService, HiveConf conf) throws Exception {
+  public static void runHouseKeeperService(HouseKeeperService houseKeeperService, HiveConf conf) throws Exception {
     int lastCount = houseKeeperService.getIsAliveCounter();
     houseKeeperService.start(conf);
     int maxIter = 10;

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java b/ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java
index f87dd14..83a2ba3 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java
@@ -65,6 +65,26 @@ public class TestAcidUtils {
     assertEquals("/tmp/delta_0000100_0000200_0007/bucket_00023",
       AcidUtils.createFilename(p, options).toString());
   }
+  @Test
+  public void testCreateFilenameLargeIds() throws Exception {
+    Path p = new Path("/tmp");
+    Configuration conf = new Configuration();
+    AcidOutputFormat.Options options = new AcidOutputFormat.Options(conf)
+      .setOldStyle(true).bucket(123456789);
+    assertEquals("/tmp/123456789_0",
+      AcidUtils.createFilename(p, options).toString());
+    options.bucket(23)
+      .minimumTransactionId(1234567880)
+      .maximumTransactionId(1234567890)
+      .writingBase(true)
+      .setOldStyle(false);
+    assertEquals("/tmp/base_1234567890/bucket_00023",
+      AcidUtils.createFilename(p, options).toString());
+    options.writingBase(false);
+    assertEquals("/tmp/delta_1234567880_1234567890_0000/bucket_00023",
+      AcidUtils.createFilename(p, options).toString());
+  }
+  
 
   @Test
   public void testParsing() throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java
index 3a6e76e..22f7482 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.ql.Context;
@@ -500,6 +501,12 @@ public class TestDbTxnManager {
       partCols.add(fs);
       t.setPartCols(partCols);
     }
+    Map<String, String> tblProps = t.getParameters();
+    if(tblProps == null) {
+      tblProps = new HashMap<>();
+    }
+    tblProps.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, "true");
+    t.setParameters(tblProps);
     return t;
   }
 


[09/50] [abbrv] hive git commit: HIVE-13683 Remove erroneously included patch file (Alan Gates)

Posted by sp...@apache.org.
HIVE-13683 Remove erroneously included patch file (Alan Gates)


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

Branch: refs/heads/java8
Commit: 70fe31088639ebfdd114e026d8a332540dfbe3b2
Parents: 0b5c27f
Author: Alan Gates <ga...@hortonworks.com>
Authored: Tue May 3 15:51:44 2016 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Tue May 3 15:53:19 2016 -0700

----------------------------------------------------------------------
 HIVE-13509.2.patch | 478 ------------------------------------------------
 1 file changed, 478 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/70fe3108/HIVE-13509.2.patch
----------------------------------------------------------------------
diff --git a/HIVE-13509.2.patch b/HIVE-13509.2.patch
deleted file mode 100644
index 930b1f7..0000000
--- a/HIVE-13509.2.patch
+++ /dev/null
@@ -1,478 +0,0 @@
-diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
-index 6b03fcb..d165e7e 100644
---- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
-+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
-@@ -208,4 +208,7 @@ private HCatConstants() { // restrict instantiation
-    */
-   public static final String HCAT_INPUT_BAD_RECORD_MIN_KEY = "hcat.input.bad.record.min";
-   public static final int HCAT_INPUT_BAD_RECORD_MIN_DEFAULT = 2;
-+
-+  public static final String HCAT_INPUT_IGNORE_INVALID_PATH_KEY = "hcat.input.ignore.invalid.path";
-+  public static final boolean HCAT_INPUT_IGNORE_INVALID_PATH_DEFAULT = false;
- }
-diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java
-index adfaf4e..dbbdd61 100644
---- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java
-+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java
-@@ -21,11 +21,11 @@
- 
- import java.io.IOException;
- import java.util.ArrayList;
-+import java.util.Iterator;
- import java.util.LinkedList;
- import java.util.Map;
- import java.util.HashMap;
- import java.util.List;
--
- import org.apache.hadoop.conf.Configuration;
- import org.apache.hadoop.fs.FileSystem;
- import org.apache.hadoop.fs.Path;
-@@ -127,7 +127,10 @@ public static void setOutputSchema(Job job, HCatSchema hcatSchema)
-     //For each matching partition, call getSplits on the underlying InputFormat
-     for (PartInfo partitionInfo : partitionInfoList) {
-       jobConf = HCatUtil.getJobConfFromContext(jobContext);
--      setInputPath(jobConf, partitionInfo.getLocation());
-+      List<String> setInputPath = setInputPath(jobConf, partitionInfo.getLocation());
-+      if (setInputPath.isEmpty()) {
-+        continue;
-+      }
-       Map<String, String> jobProperties = partitionInfo.getJobProperties();
- 
-       HCatUtil.copyJobPropertiesToJobConf(jobProperties, jobConf);
-@@ -281,7 +284,7 @@ private static InputJobInfo getJobInfo(Configuration conf)
-     return (InputJobInfo) HCatUtil.deserialize(jobString);
-   }
- 
--  private void setInputPath(JobConf jobConf, String location)
-+  private List<String> setInputPath(JobConf jobConf, String location)
-     throws IOException {
- 
-     // ideally we should just call FileInputFormat.setInputPaths() here - but
-@@ -322,19 +325,33 @@ private void setInputPath(JobConf jobConf, String location)
-     }
-     pathStrings.add(location.substring(pathStart, length));
- 
--    Path[] paths = StringUtils.stringToPath(pathStrings.toArray(new String[0]));
-     String separator = "";
-     StringBuilder str = new StringBuilder();
- 
--    for (Path path : paths) {
-+    boolean ignoreInvalidPath =jobConf.getBoolean(HCatConstants.HCAT_INPUT_IGNORE_INVALID_PATH_KEY,
-+        HCatConstants.HCAT_INPUT_IGNORE_INVALID_PATH_DEFAULT);
-+    Iterator<String> pathIterator = pathStrings.iterator();
-+    while (pathIterator.hasNext()) {
-+      String pathString = pathIterator.next();
-+      if (ignoreInvalidPath && org.apache.commons.lang.StringUtils.isBlank(pathString)) {
-+        continue;
-+      }
-+      Path path = new Path(pathString);
-       FileSystem fs = path.getFileSystem(jobConf);
-+      if (ignoreInvalidPath && !fs.exists(path)) {
-+        pathIterator.remove();
-+        continue;
-+      }
-       final String qualifiedPath = fs.makeQualified(path).toString();
-       str.append(separator)
-         .append(StringUtils.escapeString(qualifiedPath));
-       separator = StringUtils.COMMA_STR;
-     }
- 
--    jobConf.set("mapred.input.dir", str.toString());
-+    if (!ignoreInvalidPath || !pathStrings.isEmpty()) {
-+      jobConf.set("mapred.input.dir", str.toString());
-+    }
-+    return pathStrings;
-   }
- 
- }
-diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java
-index 2440cb5..4e23fa2 100644
---- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java
-+++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java
-@@ -66,6 +66,7 @@
- import org.apache.pig.data.Tuple;
- import org.apache.pig.impl.logicalLayer.schema.Schema;
- import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema;
-+import org.apache.pig.impl.util.PropertiesUtil;
- import org.joda.time.DateTime;
- import org.junit.After;
- import org.junit.Before;
-@@ -102,6 +103,7 @@
-           add("testReadPartitionedBasic");
-           add("testProjectionsBasic");
-           add("testColumnarStorePushdown2");
-+          add("testReadMissingPartitionBasicNeg");
-         }});
-       }};
- 
-@@ -438,6 +440,59 @@ public void testReadPartitionedBasic() throws IOException, CommandNeedRetryExcep
-   }
- 
-   @Test
-+  public void testReadMissingPartitionBasicNeg() throws IOException, CommandNeedRetryException {
-+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
-+    PigServer server = new PigServer(ExecType.LOCAL);
-+
-+    File removedPartitionDir = new File(TEST_WAREHOUSE_DIR + "/" + PARTITIONED_TABLE + "/bkt=0");
-+    if (!removeDirectory(removedPartitionDir)) {
-+      System.out.println("Test did not run because its environment could not be set.");
-+      return;
-+    }
-+    driver.run("select * from " + PARTITIONED_TABLE);
-+    ArrayList<String> valuesReadFromHiveDriver = new ArrayList<String>();
-+    driver.getResults(valuesReadFromHiveDriver);
-+    assertTrue(valuesReadFromHiveDriver.size() == 6);
-+
-+    server.registerQuery("W = load '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatLoader();");
-+    Schema dumpedWSchema = server.dumpSchema("W");
-+    List<FieldSchema> Wfields = dumpedWSchema.getFields();
-+    assertEquals(3, Wfields.size());
-+    assertTrue(Wfields.get(0).alias.equalsIgnoreCase("a"));
-+    assertTrue(Wfields.get(0).type == DataType.INTEGER);
-+    assertTrue(Wfields.get(1).alias.equalsIgnoreCase("b"));
-+    assertTrue(Wfields.get(1).type == DataType.CHARARRAY);
-+    assertTrue(Wfields.get(2).alias.equalsIgnoreCase("bkt"));
-+    assertTrue(Wfields.get(2).type == DataType.CHARARRAY);
-+
-+    try {
-+      Iterator<Tuple> WIter = server.openIterator("W");
-+      fail("Should failed in retriving an invalid partition");
-+    } catch (IOException ioe) {
-+      // expected
-+    }
-+  }
-+
-+  private static boolean removeDirectory(File dir) {
-+    boolean success = false;
-+    if (dir.isDirectory()) {
-+      File[] files = dir.listFiles();
-+      if (files != null && files.length > 0) {
-+        for (File file : files) {
-+          success = removeDirectory(file);
-+          if (!success) {
-+            return false;
-+          }
-+        }
-+      }
-+      success = dir.delete();
-+    } else {
-+        success = dir.delete();
-+    }
-+    return success;
-+  }
-+
-+  @Test
-   public void testProjectionsBasic() throws IOException {
-     assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
- 
-diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderWithProps.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderWithProps.java
-new file mode 100644
-index 0000000..41fe79b
---- /dev/null
-+++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderWithProps.java
-@@ -0,0 +1,305 @@
-+/**
-+ * Licensed to the Apache Software Foundation (ASF) under one
-+ * or more contributor license agreements.  See the NOTICE file
-+ * distributed with this work for additional information
-+ * regarding copyright ownership.  The ASF licenses this file
-+ * to you under the Apache License, Version 2.0 (the
-+ * "License"); you may not use this file except in compliance
-+ * with the License.  You may obtain a copy of the License at
-+ *
-+ *     http://www.apache.org/licenses/LICENSE-2.0
-+ *
-+ * Unless required by applicable law or agreed to in writing,
-+ * software distributed under the License is distributed on an
-+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-+ * KIND, either express or implied.  See the License for the
-+ * specific language governing permissions and limitations
-+ * under the License.
-+ */
-+package org.apache.hive.hcatalog.pig;
-+
-+import java.io.File;
-+import java.io.FileWriter;
-+import java.io.IOException;
-+import java.io.PrintWriter;
-+import java.io.RandomAccessFile;
-+import java.sql.Date;
-+import java.sql.Timestamp;
-+import java.util.ArrayList;
-+import java.util.Collection;
-+import java.util.HashMap;
-+import java.util.HashSet;
-+import java.util.Iterator;
-+import java.util.List;
-+import java.util.Map;
-+import java.util.Properties;
-+import java.util.Set;
-+
-+import org.apache.commons.io.FileUtils;
-+import org.apache.hadoop.fs.FileSystem;
-+import org.apache.hadoop.fs.FileUtil;
-+import org.apache.hadoop.fs.Path;
-+import org.apache.hadoop.hive.cli.CliSessionState;
-+import org.apache.hadoop.hive.conf.HiveConf;
-+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-+import org.apache.hadoop.hive.ql.Driver;
-+import org.apache.hadoop.hive.ql.WindowsPathUtil;
-+import org.apache.hadoop.hive.ql.io.IOConstants;
-+import org.apache.hadoop.hive.ql.io.StorageFormats;
-+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-+import org.apache.hadoop.hive.ql.session.SessionState;
-+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
-+import org.apache.hadoop.mapreduce.Job;
-+import org.apache.hadoop.util.Shell;
-+import org.apache.hive.hcatalog.HcatTestUtils;
-+import org.apache.hive.hcatalog.common.HCatUtil;
-+import org.apache.hive.hcatalog.common.HCatConstants;
-+import org.apache.hive.hcatalog.data.Pair;
-+import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
-+import org.apache.pig.ExecType;
-+import org.apache.pig.PigRunner;
-+import org.apache.pig.PigServer;
-+import org.apache.pig.ResourceStatistics;
-+import org.apache.pig.tools.pigstats.OutputStats;
-+import org.apache.pig.tools.pigstats.PigStats;
-+import org.apache.pig.data.DataType;
-+import org.apache.pig.data.Tuple;
-+import org.apache.pig.impl.logicalLayer.schema.Schema;
-+import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema;
-+import org.apache.pig.impl.util.PropertiesUtil;
-+import org.joda.time.DateTime;
-+import org.junit.After;
-+import org.junit.Before;
-+import org.junit.Test;
-+import org.junit.runner.RunWith;
-+import org.junit.runners.Parameterized;
-+import org.slf4j.Logger;
-+import org.slf4j.LoggerFactory;
-+
-+import static org.junit.Assert.*;
-+import static org.junit.Assume.assumeTrue;
-+
-+@RunWith(Parameterized.class)
-+public class TestHCatLoaderWithProps {
-+  private static final Logger LOG = LoggerFactory.getLogger(TestHCatLoaderWithProps.class);
-+  private static final String TEST_DATA_DIR = HCatUtil.makePathASafeFileName(System.getProperty("java.io.tmpdir") +
-+      File.separator + TestHCatLoaderWithProps.class.getCanonicalName() + "-" + System.currentTimeMillis());
-+  private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse";
-+  private static final String BASIC_FILE_NAME = TEST_DATA_DIR + "/basic.input.data";
-+
-+  private static final String BASIC_TABLE = "junit_unparted_basic";
-+  private static final String PARTITIONED_TABLE = "junit_parted_basic";
-+
-+  private Driver driver;
-+  private Map<Integer, Pair<Integer, String>> basicInputData;
-+
-+  private static final Map<String, Set<String>> DISABLED_STORAGE_FORMATS =
-+      new HashMap<String, Set<String>>() {{
-+        put(IOConstants.PARQUETFILE, new HashSet<String>() {{
-+          add("testReadMissingPartitionBasic");
-+        }});
-+      }};
-+
-+  private final String storageFormat;
-+
-+  @Parameterized.Parameters
-+  public static Collection<Object[]> generateParameters() {
-+    return StorageFormats.names();
-+  }
-+
-+  public TestHCatLoaderWithProps(String storageFormat) {
-+    this.storageFormat = storageFormat;
-+  }
-+
-+  private void dropTable(String tablename) throws IOException, CommandNeedRetryException {
-+    dropTable(tablename, driver);
-+  }
-+
-+  static void dropTable(String tablename, Driver driver) throws IOException, CommandNeedRetryException {
-+    driver.run("drop table if exists " + tablename);
-+  }
-+
-+  private void createTable(String tablename, String schema, String partitionedBy) throws IOException, CommandNeedRetryException {
-+    createTable(tablename, schema, partitionedBy, driver, storageFormat);
-+  }
-+
-+  static void createTable(String tablename, String schema, String partitionedBy, Driver driver, String storageFormat)
-+      throws IOException, CommandNeedRetryException {
-+    String createTable;
-+    createTable = "create table " + tablename + "(" + schema + ") ";
-+    if ((partitionedBy != null) && (!partitionedBy.trim().isEmpty())) {
-+      createTable = createTable + "partitioned by (" + partitionedBy + ") ";
-+    }
-+    createTable = createTable + "stored as " +storageFormat;
-+    executeStatementOnDriver(createTable, driver);
-+  }
-+
-+  private void createTable(String tablename, String schema) throws IOException, CommandNeedRetryException {
-+    createTable(tablename, schema, null);
-+  }
-+
-+  /**
-+   * Execute Hive CLI statement
-+   * @param cmd arbitrary statement to execute
-+   */
-+  static void executeStatementOnDriver(String cmd, Driver driver) throws IOException, CommandNeedRetryException {
-+    LOG.debug("Executing: " + cmd);
-+    CommandProcessorResponse cpr = driver.run(cmd);
-+    if(cpr.getResponseCode() != 0) {
-+      throw new IOException("Failed to execute \"" + cmd + "\". Driver returned " + cpr.getResponseCode() + " Error: " + cpr.getErrorMessage());
-+    }
-+  }
-+
-+  @Before
-+  public void setup() throws Exception {
-+    File f = new File(TEST_WAREHOUSE_DIR);
-+    if (f.exists()) {
-+      FileUtil.fullyDelete(f);
-+    }
-+    if (!(new File(TEST_WAREHOUSE_DIR).mkdirs())) {
-+      throw new RuntimeException("Could not create " + TEST_WAREHOUSE_DIR);
-+    }
-+
-+    HiveConf hiveConf = new HiveConf(this.getClass());
-+    hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-+    hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-+    hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-+    hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR);
-+    hiveConf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict");
-+
-+    if (Shell.WINDOWS) {
-+      WindowsPathUtil.convertPathsFromWindowsToHdfs(hiveConf);
-+    }
-+
-+    driver = new Driver(hiveConf);
-+    SessionState.start(new CliSessionState(hiveConf));
-+
-+    createTable(BASIC_TABLE, "a int, b string");
-+    createTable(PARTITIONED_TABLE, "a int, b string", "bkt string");
-+
-+    int LOOP_SIZE = 3;
-+    String[] input = new String[LOOP_SIZE * LOOP_SIZE];
-+    basicInputData = new HashMap<Integer, Pair<Integer, String>>();
-+    int k = 0;
-+    for (int i = 1; i <= LOOP_SIZE; i++) {
-+      String si = i + "";
-+      for (int j = 1; j <= LOOP_SIZE; j++) {
-+        String sj = "S" + j + "S";
-+        input[k] = si + "\t" + sj;
-+        basicInputData.put(k, new Pair<Integer, String>(i, sj));
-+        k++;
-+      }
-+    }
-+    HcatTestUtils.createTestDataFile(BASIC_FILE_NAME, input);
-+
-+    PigServer server = new PigServer(ExecType.LOCAL);
-+    server.setBatchOn();
-+    int i = 0;
-+    server.registerQuery("A = load '" + BASIC_FILE_NAME + "' as (a:int, b:chararray);", ++i);
-+
-+    server.registerQuery("store A into '" + BASIC_TABLE + "' using org.apache.hive.hcatalog.pig.HCatStorer();", ++i);
-+    server.registerQuery("B = foreach A generate a,b;", ++i);
-+    server.registerQuery("B2 = filter B by a < 2;", ++i);
-+    server.registerQuery("store B2 into '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatStorer('bkt=0');", ++i);
-+
-+    server.registerQuery("C = foreach A generate a,b;", ++i);
-+    server.registerQuery("C2 = filter C by a >= 2;", ++i);
-+    server.registerQuery("store C2 into '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatStorer('bkt=1');", ++i);
-+
-+    server.executeBatch();
-+  }
-+
-+  @After
-+  public void tearDown() throws Exception {
-+    try {
-+      if (driver != null) {
-+        dropTable(BASIC_TABLE);
-+        dropTable(PARTITIONED_TABLE);
-+      }
-+    } finally {
-+      FileUtils.deleteDirectory(new File(TEST_DATA_DIR));
-+    }
-+  }
-+
-+  @Test
-+  public void testReadMissingPartitionBasic() throws IOException, CommandNeedRetryException {
-+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
-+    Properties pigProperties = PropertiesUtil.loadDefaultProperties();
-+    pigProperties.setProperty("hcat.input.ignore.invalid.path", "true");
-+    PigServer server = new PigServer(ExecType.LOCAL, pigProperties);
-+
-+    File removedPartitionDir = new File(TEST_WAREHOUSE_DIR + "/" + PARTITIONED_TABLE + "/bkt=0");
-+    if (!removeDirectory(removedPartitionDir)) {
-+      System.out.println("Test did not run because its environment could not be set.");
-+      return;
-+    }
-+    driver.run("select * from " + PARTITIONED_TABLE);
-+    ArrayList<String> valuesReadFromHiveDriver = new ArrayList<String>();
-+    driver.getResults(valuesReadFromHiveDriver);
-+    assertTrue(valuesReadFromHiveDriver.size() == 6);
-+
-+    server.registerQuery("W = load '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatLoader();");
-+    Schema dumpedWSchema = server.dumpSchema("W");
-+    List<FieldSchema> Wfields = dumpedWSchema.getFields();
-+    assertEquals(3, Wfields.size());
-+    assertTrue(Wfields.get(0).alias.equalsIgnoreCase("a"));
-+    assertTrue(Wfields.get(0).type == DataType.INTEGER);
-+    assertTrue(Wfields.get(1).alias.equalsIgnoreCase("b"));
-+    assertTrue(Wfields.get(1).type == DataType.CHARARRAY);
-+    assertTrue(Wfields.get(2).alias.equalsIgnoreCase("bkt"));
-+    assertTrue(Wfields.get(2).type == DataType.CHARARRAY);
-+
-+    Iterator<Tuple> WIter = server.openIterator("W");
-+    Collection<Pair<Integer, String>> valuesRead = new ArrayList<Pair<Integer, String>>();
-+    while (WIter.hasNext()) {
-+      Tuple t = WIter.next();
-+      assertTrue(t.size() == 3);
-+      assertNotNull(t.get(0));
-+      assertNotNull(t.get(1));
-+      assertNotNull(t.get(2));
-+      assertTrue(t.get(0).getClass() == Integer.class);
-+      assertTrue(t.get(1).getClass() == String.class);
-+      assertTrue(t.get(2).getClass() == String.class);
-+      valuesRead.add(new Pair<Integer, String>((Integer) t.get(0), (String) t.get(1)));
-+      // the returned partition value is always 1
-+      assertEquals("1", t.get(2));
-+    }
-+    assertEquals(valuesReadFromHiveDriver.size(), valuesRead.size());
-+
-+    server.registerQuery("P1 = load '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatLoader();");
-+    server.registerQuery("P1filter = filter P1 by bkt == '0';");
-+    Iterator<Tuple> P1Iter = server.openIterator("P1filter");
-+    assertFalse(P1Iter.hasNext());
-+
-+    server.registerQuery("P2 = load '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatLoader();");
-+    server.registerQuery("P2filter = filter P2 by bkt == '1';");
-+    Iterator<Tuple> P2Iter = server.openIterator("P2filter");
-+    int count2 = 0;
-+    while (P2Iter.hasNext()) {
-+      Tuple t = P2Iter.next();
-+      assertEquals("1", t.get(2));
-+      assertTrue(((Integer) t.get(0)) > 1);
-+      count2++;
-+    }
-+    assertEquals(6, count2);
-+  }
-+
-+  private static boolean removeDirectory(File dir) {
-+    boolean success = false;
-+    if (dir.isDirectory()) {
-+      File[] files = dir.listFiles();
-+      if (files != null && files.length > 0) {
-+        for (File file : files) {
-+          success = removeDirectory(file);
-+          if (!success) {
-+            return false;
-+          }
-+        }
-+      }
-+      success = dir.delete();
-+    } else {
-+        success = dir.delete();
-+    }
-+    return success;
-+  }
-+}


[29/50] [abbrv] hive git commit: HIVE-13639: CBO rule to pull up constants through Union (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/union_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_view.q.out b/ql/src/test/results/clientpositive/union_view.q.out
index badd209..530739e 100644
--- a/ql/src/test/results/clientpositive/union_view.q.out
+++ b/ql/src/test/results/clientpositive/union_view.q.out
@@ -358,12 +358,12 @@ STAGE PLANS:
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                 Union
                   Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '1' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -382,12 +382,12 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '1' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -406,12 +406,12 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '1' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -471,12 +471,12 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '2' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '2' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -495,12 +495,12 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Union
                   Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '2' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '2' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -519,12 +519,12 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '2' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '2' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -584,12 +584,12 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '3' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '3' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -608,12 +608,12 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '3' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '3' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -632,12 +632,12 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Union
                   Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '3' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '3' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -701,12 +701,12 @@ STAGE PLANS:
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: value (type: string), ds (type: string)
-                outputColumnNames: _col1, _col2
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                 Union
                   Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: _col1 (type: string), _col2 (type: string)
+                    expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: _col1, _col2
                     Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
@@ -723,12 +723,12 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: value (type: string), ds (type: string)
-                outputColumnNames: _col1, _col2
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Union
                   Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: _col1 (type: string), _col2 (type: string)
+                    expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: _col1, _col2
                     Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
@@ -745,12 +745,12 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: value (type: string), ds (type: string)
-                outputColumnNames: _col1, _col2
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Union
                   Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: _col1 (type: string), _col2 (type: string)
+                    expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: _col1, _col2
                     Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
@@ -1226,12 +1226,12 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '4' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '4' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -1250,12 +1250,12 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '4' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '4' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
@@ -1274,12 +1274,12 @@ STAGE PLANS:
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: value (type: string)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                 Union
                   Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: 86 (type: int), _col1 (type: string), '4' (type: string)
+                    expressions: 86 (type: int), _col0 (type: string), '4' (type: string)
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator


[12/50] [abbrv] hive git commit: Merge branch 'master' into llap

Posted by sp...@apache.org.
Merge branch 'master' into llap

Conflicts:
	llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
	llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
	llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java


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

Branch: refs/heads/java8
Commit: e057909732b40b581fcad3f61fb798600f01ecdf
Parents: 4847f65 8729966
Author: Jason Dere <jd...@hortonworks.com>
Authored: Wed May 4 00:17:12 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Wed May 4 00:17:12 2016 -0700

----------------------------------------------------------------------
 HIVE-13509.2.patch                              |  478 --
 .../ext/LlapTaskUmbilicalExternalClient.java    |   18 +-
 .../daemon/rpc/LlapDaemonProtocolProtos.java    | 7000 +++++++++++-------
 .../org/apache/hadoop/hive/llap/DaemonId.java   |   41 +
 .../hive/llap/security/LlapTokenIdentifier.java |   39 +-
 .../hive/llap/security/LlapTokenProvider.java   |    2 +-
 .../apache/hadoop/hive/llap/tez/Converters.java |   84 +-
 .../src/protobuf/LlapDaemonProtocol.proto       |   70 +-
 .../hadoop/hive/llap/tez/TestConverters.java    |   51 +-
 .../hadoop/hive/llap/LlapBaseInputFormat.java   |   32 +-
 .../hive/llap/daemon/ContainerRunner.java       |    9 +-
 .../llap/daemon/impl/ContainerRunnerImpl.java   |  135 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |   52 +-
 .../daemon/impl/LlapProtocolServerImpl.java     |   41 +-
 .../hive/llap/daemon/impl/LlapTokenChecker.java |  137 +
 .../llap/daemon/impl/QueryFragmentInfo.java     |   23 +-
 .../hadoop/hive/llap/daemon/impl/QueryInfo.java |   26 +-
 .../hive/llap/daemon/impl/QueryTracker.java     |   97 +-
 .../hadoop/hive/llap/daemon/impl/Scheduler.java |    2 +
 .../llap/daemon/impl/TaskExecutorService.java   |   17 +-
 .../llap/daemon/impl/TaskRunnerCallable.java    |   77 +-
 .../hive/llap/security/LlapSecurityHelper.java  |   15 +-
 .../hive/llap/security/SecretManager.java       |   19 +-
 .../hive/llap/daemon/MiniLlapCluster.java       |    2 +-
 .../daemon/impl/TaskExecutorTestHelpers.java    |   44 +-
 .../impl/TestLlapDaemonProtocolServerImpl.java  |    2 +-
 .../llap/daemon/impl/TestLlapTokenChecker.java  |   96 +
 .../TestFirstInFirstOutComparator.java          |   27 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |   31 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |    5 +
 .../hive/ql/exec/tez/TezSessionState.java       |    3 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |   12 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    1 -
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |   16 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |   22 +
 .../hive/ql/lockmgr/TestDbTxnManager2.java      |  114 +
 .../dynpart_sort_optimization_acid.q.out        |  120 +-
 37 files changed, 5479 insertions(+), 3481 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
----------------------------------------------------------------------
diff --cc llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
index fe2fd7c,0000000..6e2c85d
mode 100644,000000..100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
@@@ -1,413 -1,0 +1,421 @@@
 +package org.apache.hadoop.hive.llap.ext;
 +
 +import java.io.IOException;
 +import java.net.InetSocketAddress;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.concurrent.atomic.AtomicLong;
 +import java.util.concurrent.ConcurrentHashMap;
 +import java.util.concurrent.ConcurrentMap;
 +import java.util.concurrent.ScheduledThreadPoolExecutor;
 +import java.util.concurrent.TimeUnit;
 +
 +import com.google.common.base.Preconditions;
 +import org.apache.commons.collections4.ListUtils;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.hive.conf.HiveConf;
 +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
++import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
++import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier;
 +import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
++import org.apache.hadoop.hive.llap.tez.Converters;
 +import org.apache.hadoop.hive.llap.tez.LlapProtocolClientProxy;
 +import org.apache.hadoop.hive.llap.tezplugins.helpers.LlapTaskUmbilicalServer;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.ipc.ProtocolSignature;
 +import org.apache.hadoop.security.token.Token;
 +import org.apache.hadoop.service.AbstractService;
 +import org.apache.hadoop.yarn.api.records.ContainerId;
 +import org.apache.hadoop.yarn.util.ConverterUtils;
 +import org.apache.tez.common.security.JobTokenIdentifier;
 +import org.apache.tez.dag.api.TezException;
 +import org.apache.tez.dag.records.TezTaskAttemptID;
 +import org.apache.tez.runtime.api.Event;
 +import org.apache.tez.runtime.api.impl.EventType;
 +import org.apache.tez.runtime.api.impl.TezEvent;
 +import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
 +import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +
 +public class LlapTaskUmbilicalExternalClient extends AbstractService {
 +
 +  private static final Logger LOG = LoggerFactory.getLogger(LlapTaskUmbilicalExternalClient.class);
 +
 +  private final LlapProtocolClientProxy communicator;
 +  private volatile LlapTaskUmbilicalServer llapTaskUmbilicalServer;
 +  private final Configuration conf;
 +  private final LlapTaskUmbilicalProtocol umbilical;
 +
 +  protected final String tokenIdentifier;
 +  protected final Token<JobTokenIdentifier> sessionToken;
 +
 +  private final ConcurrentMap<String, PendingEventData> pendingEvents = new ConcurrentHashMap<>();
 +  private final ConcurrentMap<String, TaskHeartbeatInfo> registeredTasks= new ConcurrentHashMap<String, TaskHeartbeatInfo>();
 +  private LlapTaskUmbilicalExternalResponder responder = null;
 +  private final ScheduledThreadPoolExecutor timer;
 +  private final long connectionTimeout;
 +
 +  private static class TaskHeartbeatInfo {
 +    final String taskAttemptId;
 +    final String hostname;
 +    final int port;
 +    final AtomicLong lastHeartbeat = new AtomicLong();
 +
 +    public TaskHeartbeatInfo(String taskAttemptId, String hostname, int port) {
 +      this.taskAttemptId = taskAttemptId;
 +      this.hostname = hostname;
 +      this.port = port;
 +      this.lastHeartbeat.set(System.currentTimeMillis());
 +    }
 +  }
 +
 +  private static class PendingEventData {
 +    final TaskHeartbeatInfo heartbeatInfo;
 +    final List<TezEvent> tezEvents;
 +
 +    public PendingEventData(TaskHeartbeatInfo heartbeatInfo, List<TezEvent> tezEvents) {
 +      this.heartbeatInfo = heartbeatInfo;
 +      this.tezEvents = tezEvents;
 +    }
 +  }
 +
 +  // TODO KKK Work out the details of the tokenIdentifier, and the session token.
 +  // It may just be possible to create one here - since Shuffle is not involved, and this is only used
 +  // for communication from LLAP-Daemons to the server. It will need to be sent in as part
 +  // of the job submission request.
 +  public LlapTaskUmbilicalExternalClient(Configuration conf, String tokenIdentifier,
 +      Token<JobTokenIdentifier> sessionToken, LlapTaskUmbilicalExternalResponder responder) {
 +    super(LlapTaskUmbilicalExternalClient.class.getName());
 +    this.conf = conf;
 +    this.umbilical = new LlapTaskUmbilicalExternalImpl();
 +    this.tokenIdentifier = tokenIdentifier;
 +    this.sessionToken = sessionToken;
 +    this.responder = responder;
 +    this.timer = new ScheduledThreadPoolExecutor(1);
 +    this.connectionTimeout = HiveConf.getTimeVar(conf,
 +        HiveConf.ConfVars.LLAP_DAEMON_AM_LIVENESS_CONNECTION_TIMEOUT_MS, TimeUnit.MILLISECONDS);
 +    // TODO. No support for the LLAP token yet. Add support for configurable threads, however 1 should always be enough.
 +    this.communicator = new LlapProtocolClientProxy(1, conf, null);
 +    this.communicator.init(conf);
 +  }
 +
 +  @Override
 +  public void serviceStart() throws IOException {
 +    // If we use a single server for multiple external clients, then consider using more than one handler.
 +    int numHandlers = 1;
 +    llapTaskUmbilicalServer = new LlapTaskUmbilicalServer(conf, umbilical, numHandlers, tokenIdentifier, sessionToken);
 +    communicator.start();
 +  }
 +
 +  @Override
 +  public void serviceStop() {
 +    llapTaskUmbilicalServer.shutdownServer();
 +    timer.shutdown();
 +    if (this.communicator != null) {
 +      this.communicator.stop();
 +    }
 +  }
 +
 +  public InetSocketAddress getAddress() {
 +    return llapTaskUmbilicalServer.getAddress();
 +  }
 +
 +
 +  /**
 +   * Submit the work for actual execution.
 +   * @param submitWorkRequestProto
 +   */
 +  public void submitWork(final SubmitWorkRequestProto submitWorkRequestProto, String llapHost, int llapPort, List<TezEvent> tezEvents) {
 +    // Register the pending events to be sent for this spec.
-     String fragmentId = submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString();
++    SignableVertexSpec vertex = submitWorkRequestProto.getWorkSpec().getVertex();
++    VertexIdentifier vId = vertex.getVertexIdentifier();
++    TezTaskAttemptID attemptId = Converters.createTaskAttemptId(
++        vId, submitWorkRequestProto.getFragmentNumber(), submitWorkRequestProto.getAttemptNumber());
++    final String fragmentId = attemptId.toString();
++
 +    PendingEventData pendingEventData = new PendingEventData(
 +        new TaskHeartbeatInfo(fragmentId, llapHost, llapPort),
 +        tezEvents);
 +    pendingEvents.putIfAbsent(fragmentId, pendingEventData);
 +
 +    // Setup timer task to check for hearbeat timeouts
 +    timer.scheduleAtFixedRate(new HeartbeatCheckTask(),
 +        connectionTimeout, connectionTimeout, TimeUnit.MILLISECONDS);
 +
 +    // Send out the actual SubmitWorkRequest
 +    communicator.sendSubmitWork(submitWorkRequestProto, llapHost, llapPort,
 +        new LlapProtocolClientProxy.ExecuteRequestCallback<LlapDaemonProtocolProtos.SubmitWorkResponseProto>() {
 +
 +          @Override
 +          public void setResponse(LlapDaemonProtocolProtos.SubmitWorkResponseProto response) {
 +            if (response.hasSubmissionState()) {
 +              if (response.getSubmissionState().equals(LlapDaemonProtocolProtos.SubmissionStateProto.REJECTED)) {
-                 String msg = "Fragment: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString() + " rejected. Server Busy.";
++                String msg = "Fragment: " + fragmentId + " rejected. Server Busy.";
 +                LOG.info(msg);
 +                if (responder != null) {
 +                  Throwable err = new RuntimeException(msg);
-                   responder.submissionFailed(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), err);
++                  responder.submissionFailed(fragmentId, err);
 +                }
 +                return;
 +              }
 +            }
 +          }
 +
 +          @Override
 +          public void indicateError(Throwable t) {
-             String msg = "Failed to submit: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString();
++            String msg = "Failed to submit: " + fragmentId;
 +            LOG.error(msg, t);
 +            Throwable err = new RuntimeException(msg, t);
-             responder.submissionFailed(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), err);
++            responder.submissionFailed(fragmentId, err);
 +          }
 +        });
 +
 +
 +
 +
 +
 +//    // TODO Also send out information saying that the fragment is finishable - if that is not already included in the main fragment.
 +//    // This entire call is only required if we're doing more than scans. MRInput has no dependencies and is always finishable
 +//    QueryIdentifierProto queryIdentifier = QueryIdentifierProto
 +//        .newBuilder()
 +//        .setAppIdentifier(submitWorkRequestProto.getApplicationIdString()).setDagIdentifier(submitWorkRequestProto.getFragmentSpec().getDagId())
 +//        .build();
 +//    LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto sourceStateUpdatedRequest =
 +//        LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto.newBuilder().setQueryIdentifier(queryIdentifier).setState(
 +//            LlapDaemonProtocolProtos.SourceStateProto.S_SUCCEEDED).
 +//            setSrcName(TODO)
 +//    communicator.sendSourceStateUpdate(LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto.newBuilder().setQueryIdentifier(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString()).set);
 +
 +
 +  }
 +
 +  private void updateHeartbeatInfo(String taskAttemptId) {
 +    int updateCount = 0;
 +
 +    PendingEventData pendingEventData = pendingEvents.get(taskAttemptId);
 +    if (pendingEventData != null) {
 +      pendingEventData.heartbeatInfo.lastHeartbeat.set(System.currentTimeMillis());
 +      updateCount++;
 +    }
 +
 +    TaskHeartbeatInfo heartbeatInfo = registeredTasks.get(taskAttemptId);
 +    if (heartbeatInfo != null) {
 +      heartbeatInfo.lastHeartbeat.set(System.currentTimeMillis());
 +      updateCount++;
 +    }
 +
 +    if (updateCount == 0) {
 +      LOG.warn("No tasks found for heartbeat from taskAttemptId " + taskAttemptId);
 +    }
 +  }
 +
 +  private void updateHeartbeatInfo(String hostname, int port) {
 +    int updateCount = 0;
 +
 +    for (String key : pendingEvents.keySet()) {
 +      PendingEventData pendingEventData = pendingEvents.get(key);
 +      if (pendingEventData != null) {
 +        if (pendingEventData.heartbeatInfo.hostname.equals(hostname)
 +            && pendingEventData.heartbeatInfo.port == port) {
 +          pendingEventData.heartbeatInfo.lastHeartbeat.set(System.currentTimeMillis());
 +          updateCount++;
 +        }
 +      }
 +    }
 +
 +    for (String key : registeredTasks.keySet()) {
 +      TaskHeartbeatInfo heartbeatInfo = registeredTasks.get(key);
 +      if (heartbeatInfo != null) {
 +        if (heartbeatInfo.hostname.equals(hostname)
 +            && heartbeatInfo.port == port) {
 +          heartbeatInfo.lastHeartbeat.set(System.currentTimeMillis());
 +          updateCount++;
 +        }
 +      }
 +    }
 +
 +    if (updateCount == 0) {
 +      LOG.info("No tasks found for heartbeat from hostname " + hostname + ", port " + port);
 +    }
 +  }
 +
 +  private class HeartbeatCheckTask implements Runnable {
 +    public void run() {
 +      long currentTime = System.currentTimeMillis();
 +      List<String> timedOutTasks = new ArrayList<String>();
 +
 +      // Check both pending and registered tasks for timeouts
 +      for (String key : pendingEvents.keySet()) {
 +        PendingEventData pendingEventData = pendingEvents.get(key);
 +        if (pendingEventData != null) {
 +          if (currentTime - pendingEventData.heartbeatInfo.lastHeartbeat.get() >= connectionTimeout) {
 +            timedOutTasks.add(key);
 +          }
 +        }
 +      }
 +      for (String timedOutTask : timedOutTasks) {
 +        LOG.info("Pending taskAttemptId " + timedOutTask + " timed out");
 +        responder.heartbeatTimeout(timedOutTask);
 +        pendingEvents.remove(timedOutTask);
 +        // TODO: Do we need to tell the LLAP daemon we are no longer interested in this task?
 +      }
 +
 +      timedOutTasks.clear();
 +      for (String key : registeredTasks.keySet()) {
 +        TaskHeartbeatInfo heartbeatInfo = registeredTasks.get(key);
 +        if (heartbeatInfo != null) {
 +          if (currentTime - heartbeatInfo.lastHeartbeat.get() >= connectionTimeout) {
 +            timedOutTasks.add(key);
 +          }
 +        }
 +      }
 +      for (String timedOutTask : timedOutTasks) {
 +        LOG.info("Running taskAttemptId " + timedOutTask + " timed out");
 +        responder.heartbeatTimeout(timedOutTask);
 +        registeredTasks.remove(timedOutTask);
 +        // TODO: Do we need to tell the LLAP daemon we are no longer interested in this task?
 +      }
 +    }
 +  }
 +
 +  public interface LlapTaskUmbilicalExternalResponder {
 +    void submissionFailed(String fragmentId, Throwable throwable);
 +    void heartbeat(TezHeartbeatRequest request);
 +    void taskKilled(TezTaskAttemptID taskAttemptId);
 +    void heartbeatTimeout(String fragmentId);
 +  }
 +
 +
 +
 +  // TODO Ideally, the server should be shared across all client sessions running on the same node.
 +  private class LlapTaskUmbilicalExternalImpl implements  LlapTaskUmbilicalProtocol {
 +
 +    @Override
 +    public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
 +      // Expecting only a single instance of a task to be running.
 +      return true;
 +    }
 +
 +    @Override
 +    public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOException,
 +        TezException {
 +      // Keep-alive information. The client should be informed and will have to take care of re-submitting the work.
 +      // Some parts of fault tolerance go here.
 +
 +      // This also provides completion information, and a possible notification when task actually starts running (first heartbeat)
 +
 +      if (LOG.isDebugEnabled()) {
 +        LOG.debug("Received heartbeat from container, request=" + request);
 +      }
 +
 +      // Incoming events can be ignored until the point when shuffle needs to be handled, instead of just scans.
 +      TezHeartbeatResponse response = new TezHeartbeatResponse();
 +
 +      response.setLastRequestId(request.getRequestId());
 +      // Assuming TaskAttemptId and FragmentIdentifierString are the same. Verify this.
 +      TezTaskAttemptID taskAttemptId = request.getCurrentTaskAttemptID();
 +      String taskAttemptIdString = taskAttemptId.toString();
 +
 +      updateHeartbeatInfo(taskAttemptIdString);
 +
 +      List<TezEvent> tezEvents = null;
 +      PendingEventData pendingEventData = pendingEvents.remove(taskAttemptIdString);
 +      if (pendingEventData == null) {
 +        tezEvents = Collections.emptyList();
 +
 +        // If this heartbeat was not from a pending event and it's not in our list of registered tasks,
 +        if (!registeredTasks.containsKey(taskAttemptIdString)) {
 +          LOG.info("Unexpected heartbeat from " + taskAttemptIdString);
 +          response.setShouldDie(); // Do any of the other fields need to be set?
 +          return response;
 +        }
 +      } else {
 +        tezEvents = pendingEventData.tezEvents;
 +        // Tasks removed from the pending list should then be added to the registered list.
 +        registeredTasks.put(taskAttemptIdString, pendingEventData.heartbeatInfo);
 +      }
 +
 +      response.setLastRequestId(request.getRequestId());
 +      // Irrelevant from eventIds. This can be tracked in the AM itself, instead of polluting the task.
 +      // Also since we have all the MRInput events here - they'll all be sent in together.
 +      response.setNextFromEventId(0); // Irrelevant. See comment above.
 +      response.setNextPreRoutedEventId(0); //Irrelevant. See comment above.
 +      response.setEvents(tezEvents);
 +
 +      List<TezEvent> inEvents = request.getEvents();
 +      if (LOG.isDebugEnabled()) {
 +        LOG.debug("Heartbeat from " + taskAttemptIdString +
 +            " events: " + (inEvents != null ? inEvents.size() : -1));
 +      }
 +      for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
 +        EventType eventType = tezEvent.getEventType();
 +        switch (eventType) {
 +          case TASK_ATTEMPT_COMPLETED_EVENT:
 +            LOG.debug("Task completed event for " + taskAttemptIdString);
 +            registeredTasks.remove(taskAttemptIdString);
 +            break;
 +          case TASK_ATTEMPT_FAILED_EVENT:
 +            LOG.debug("Task failed event for " + taskAttemptIdString);
 +            registeredTasks.remove(taskAttemptIdString);
 +            break;
 +          case TASK_STATUS_UPDATE_EVENT:
 +            // If we want to handle counters
 +            LOG.debug("Task update event for " + taskAttemptIdString);
 +            break;
 +          default:
 +            LOG.warn("Unhandled event type " + eventType);
 +            break;
 +        }
 +      }
 +
 +      // Pass the request on to the responder
 +      try {
 +        if (responder != null) {
 +          responder.heartbeat(request);
 +        }
 +      } catch (Exception err) {
 +        LOG.error("Error during responder execution", err);
 +      }
 +
 +      return response;
 +    }
 +
 +    @Override
 +    public void nodeHeartbeat(Text hostname, int port) throws IOException {
 +      updateHeartbeatInfo(hostname.toString(), port);
 +      // No need to propagate to this to the responder
 +    }
 +
 +    @Override
 +    public void taskKilled(TezTaskAttemptID taskAttemptId) throws IOException {
 +      String taskAttemptIdString = taskAttemptId.toString();
 +      LOG.error("Task killed - " + taskAttemptIdString);
 +      registeredTasks.remove(taskAttemptIdString);
 +
 +      try {
 +        if (responder != null) {
 +          responder.taskKilled(taskAttemptId);
 +        }
 +      } catch (Exception err) {
 +        LOG.error("Error during responder execution", err);
 +      }
 +    }
 +
 +    @Override
 +    public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
 +      return 0;
 +    }
 +
 +    @Override
 +    public ProtocolSignature getProtocolSignature(String protocol, long clientVersion,
 +                                                  int clientMethodsHash) throws IOException {
 +      return ProtocolSignature.getProtocolSignature(this, protocol,
 +          clientVersion, clientMethodsHash);
 +    }
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
----------------------------------------------------------------------
diff --cc llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
index 8db2f88,0000000..988002f
mode 100644,000000..100644
--- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
@@@ -1,476 -1,0 +1,480 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hive.llap;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +import java.util.Set;
 +import java.util.concurrent.LinkedBlockingQueue;
 +
 +import java.sql.SQLException;
 +import java.sql.Connection;
 +import java.sql.ResultSet;
 +import java.sql.Statement;
 +import java.sql.DriverManager;
 +
 +import java.io.IOException;
 +import java.io.DataInput;
 +import java.io.DataOutput;
 +import java.io.DataInputStream;
 +import java.io.ByteArrayInputStream;
 +import java.net.InetAddress;
 +import java.net.InetSocketAddress;
 +import java.net.Socket;
 +import java.nio.ByteBuffer;
 +
 +import org.apache.commons.collections4.ListUtils;
 +
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.hive.conf.HiveConf;
 +import org.apache.hadoop.hive.llap.LlapBaseRecordReader;
 +import org.apache.hadoop.hive.llap.LlapBaseRecordReader.ReaderEvent;
 +import org.apache.hadoop.hive.llap.LlapInputSplit;
 +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
++import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
++import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary;
 +import org.apache.hadoop.hive.llap.ext.LlapTaskUmbilicalExternalClient;
 +import org.apache.hadoop.hive.llap.ext.LlapTaskUmbilicalExternalClient.LlapTaskUmbilicalExternalResponder;
 +import org.apache.hadoop.hive.llap.registry.ServiceInstance;
 +import org.apache.hadoop.hive.llap.registry.ServiceInstanceSet;
 +import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService;
 +import org.apache.hadoop.hive.llap.tez.Converters;
 +import org.apache.hadoop.io.DataInputBuffer;
 +import org.apache.hadoop.io.DataOutputBuffer;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.io.WritableComparable;
 +import org.apache.hadoop.io.NullWritable;
 +import org.apache.hadoop.io.Writable;
 +import org.apache.hadoop.mapred.JobConf;
 +import org.apache.hadoop.mapred.InputFormat;
 +import org.apache.hadoop.mapred.InputSplit;
 +import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
 +import org.apache.hadoop.mapred.SplitLocationInfo;
 +import org.apache.hadoop.mapred.FileSplit;
 +import org.apache.hadoop.mapred.RecordReader;
 +import org.apache.hadoop.mapred.Reporter;
 +import org.apache.hadoop.security.Credentials;
 +import org.apache.hadoop.security.UserGroupInformation;
 +import org.apache.hadoop.security.token.Token;
 +import org.apache.hadoop.security.token.TokenIdentifier;
 +import org.apache.hadoop.util.Progressable;
 +import org.apache.hadoop.yarn.api.ApplicationConstants;
 +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 +import org.apache.hadoop.yarn.api.records.ApplicationId;
 +import org.apache.hadoop.yarn.api.records.ContainerId;
 +
 +import org.apache.tez.common.security.JobTokenIdentifier;
 +import org.apache.tez.common.security.TokenCache;
 +import org.apache.tez.dag.records.TezTaskAttemptID;
 +import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
 +import org.apache.tez.runtime.api.impl.EventType;
 +import org.apache.tez.runtime.api.impl.TaskSpec;
 +import org.apache.tez.runtime.api.impl.TezEvent;
 +import org.apache.tez.runtime.api.impl.TezEvent;
 +import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
 +import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
 +
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.google.common.base.Preconditions;
 +import com.google.common.collect.Lists;
 +import com.google.protobuf.ByteString;
 +
 +
 +public class LlapBaseInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
 +
 +  private static final Logger LOG = LoggerFactory.getLogger(LlapBaseInputFormat.class);
 +
 +  private static String driverName = "org.apache.hive.jdbc.HiveDriver";
 +  private String url;  // "jdbc:hive2://localhost:10000/default"
 +  private String user; // "hive",
 +  private String pwd;  // ""
 +  private String query;
 +
 +  public static final String URL_KEY = "llap.if.hs2.connection";
 +  public static final String QUERY_KEY = "llap.if.query";
 +  public static final String USER_KEY = "llap.if.user";
 +  public static final String PWD_KEY = "llap.if.pwd";
 +
 +  public final String SPLIT_QUERY = "select get_splits(\"%s\",%d)";
 +
 +  private Connection con;
 +  private Statement stmt;
 +
 +  public LlapBaseInputFormat(String url, String user, String pwd, String query) {
 +    this.url = url;
 +    this.user = user;
 +    this.pwd = pwd;
 +    this.query = query;
 +  }
 +
 +  public LlapBaseInputFormat() {}
 +
 +
 +  @Override
 +  public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
 +
 +    LlapInputSplit llapSplit = (LlapInputSplit) split;
 +
 +    // Set conf to use LLAP user rather than current user for LLAP Zk registry.
 +    HiveConf.setVar(job, HiveConf.ConfVars.LLAP_ZK_REGISTRY_USER, llapSplit.getLlapUser());
 +    SubmitWorkInfo submitWorkInfo = SubmitWorkInfo.fromBytes(llapSplit.getPlanBytes());
 +
 +    ServiceInstance serviceInstance = getServiceInstance(job, llapSplit);
 +    String host = serviceInstance.getHost();
 +    int llapSubmitPort = serviceInstance.getRpcPort();
 +
 +    LOG.info("Found service instance for host " + host + " with rpc port " + llapSubmitPort
 +        + " and outputformat port " + serviceInstance.getOutputFormatPort());
 +
 +    LlapRecordReaderTaskUmbilicalExternalResponder umbilicalResponder =
 +        new LlapRecordReaderTaskUmbilicalExternalResponder();
 +    LlapTaskUmbilicalExternalClient llapClient =
 +      new LlapTaskUmbilicalExternalClient(job, submitWorkInfo.getTokenIdentifier(),
 +          submitWorkInfo.getToken(), umbilicalResponder);
 +    llapClient.init(job);
 +    llapClient.start();
 +
 +    SubmitWorkRequestProto submitWorkRequestProto =
 +      constructSubmitWorkRequestProto(submitWorkInfo, llapSplit.getSplitNum(),
 +          llapClient.getAddress(), submitWorkInfo.getToken());
 +
 +    TezEvent tezEvent = new TezEvent();
 +    DataInputBuffer dib = new DataInputBuffer();
 +    dib.reset(llapSplit.getFragmentBytes(), 0, llapSplit.getFragmentBytes().length);
 +    tezEvent.readFields(dib);
 +    List<TezEvent> tezEventList = Lists.newArrayList();
 +    tezEventList.add(tezEvent);
 +
 +    llapClient.submitWork(submitWorkRequestProto, host, llapSubmitPort, tezEventList);
 +
 +    String id = HiveConf.getVar(job, HiveConf.ConfVars.HIVEQUERYID) + "_" + llapSplit.getSplitNum();
 +
 +    HiveConf conf = new HiveConf();
 +    Socket socket = new Socket(host,
 +        serviceInstance.getOutputFormatPort());
 +
 +    LOG.debug("Socket connected");
 +
 +    socket.getOutputStream().write(id.getBytes());
 +    socket.getOutputStream().write(0);
 +    socket.getOutputStream().flush();
 +
 +    LOG.info("Registered id: " + id);
 +
 +    LlapBaseRecordReader recordReader = new LlapBaseRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
 +    umbilicalResponder.setRecordReader(recordReader);
 +    return recordReader;
 +  }
 +
 +  @Override
 +  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
 +    List<InputSplit> ins = new ArrayList<InputSplit>();
 +
 +    if (url == null) url = job.get(URL_KEY);
 +    if (query == null) query = job.get(QUERY_KEY);
 +    if (user == null) user = job.get(USER_KEY);
 +    if (pwd == null) pwd = job.get(PWD_KEY);
 +
 +    if (url == null || query == null) {
 +      throw new IllegalStateException();
 +    }
 +
 +    try {
 +      Class.forName(driverName);
 +    } catch (ClassNotFoundException e) {
 +      throw new IOException(e);
 +    }
 +
 +    try {
 +      con = DriverManager.getConnection(url,user,pwd);
 +      stmt = con.createStatement();
 +      String sql = String.format(SPLIT_QUERY, query, numSplits);
 +      ResultSet res = stmt.executeQuery(sql);
 +      while (res.next()) {
 +        // deserialize split
 +        DataInput in = new DataInputStream(res.getBinaryStream(1));
 +        InputSplitWithLocationInfo is = new LlapInputSplit();
 +        is.readFields(in);
 +        ins.add(is);
 +      }
 +
 +      res.close();
 +      stmt.close();
 +    } catch (Exception e) {
 +      throw new IOException(e);
 +    }
 +    return ins.toArray(new InputSplit[ins.size()]);
 +  }
 +
 +  public void close() {
 +    try {
 +      con.close();
 +    } catch (Exception e) {
 +      // ignore
 +    }
 +  }
 +
 +  private ServiceInstance getServiceInstance(JobConf job, LlapInputSplit llapSplit) throws IOException {
 +    LlapRegistryService registryService = LlapRegistryService.getClient(job);
 +    String host = llapSplit.getLocations()[0];
 +
 +    ServiceInstance serviceInstance = getServiceInstanceForHost(registryService, host);
 +    if (serviceInstance == null) {
 +      throw new IOException("No service instances found for " + host + " in registry");
 +    }
 +
 +    return serviceInstance;
 +  }
 +
 +  private ServiceInstance getServiceInstanceForHost(LlapRegistryService registryService, String host) throws IOException {
 +    InetAddress address = InetAddress.getByName(host);
 +    ServiceInstanceSet instanceSet = registryService.getInstances();
 +    ServiceInstance serviceInstance = null;
 +
 +    // The name used in the service registry may not match the host name we're using.
 +    // Try hostname/canonical hostname/host address
 +
 +    String name = address.getHostName();
 +    LOG.info("Searching service instance by hostname " + name);
 +    serviceInstance = selectServiceInstance(instanceSet.getByHost(name));
 +    if (serviceInstance != null) {
 +      return serviceInstance;
 +    }
 +
 +    name = address.getCanonicalHostName();
 +    LOG.info("Searching service instance by canonical hostname " + name);
 +    serviceInstance = selectServiceInstance(instanceSet.getByHost(name));
 +    if (serviceInstance != null) {
 +      return serviceInstance;
 +    }
 +
 +    name = address.getHostAddress();
 +    LOG.info("Searching service instance by address " + name);
 +    serviceInstance = selectServiceInstance(instanceSet.getByHost(name));
 +    if (serviceInstance != null) {
 +      return serviceInstance;
 +    }
 +
 +    return serviceInstance;
 +  }
 +
 +  private ServiceInstance selectServiceInstance(Set<ServiceInstance> serviceInstances) {
 +    if (serviceInstances == null || serviceInstances.isEmpty()) {
 +      return null;
 +    }
 +
 +    // Get the first live service instance
 +    for (ServiceInstance serviceInstance : serviceInstances) {
 +      if (serviceInstance.isAlive()) {
 +        return serviceInstance;
 +      }
 +    }
 +
 +    LOG.info("No live service instances were found");
 +    return null;
 +  }
 +
 +  private SubmitWorkRequestProto constructSubmitWorkRequestProto(SubmitWorkInfo submitWorkInfo,
 +      int taskNum,
 +      InetSocketAddress address,
 +      Token<JobTokenIdentifier> token) throws
 +        IOException {
 +    TaskSpec taskSpec = submitWorkInfo.getTaskSpec();
 +    ApplicationId appId = submitWorkInfo.getFakeAppId();
 +
-     SubmitWorkRequestProto.Builder builder = SubmitWorkRequestProto.newBuilder();
++    int attemptId = taskSpec.getTaskAttemptID().getId();
 +    // This works, assuming the executor is running within YARN.
-     LOG.info("Setting user in submitWorkRequest to: " +
-         System.getenv(ApplicationConstants.Environment.USER.name()));
-     builder.setUser(System.getenv(ApplicationConstants.Environment.USER.name()));
-     builder.setApplicationIdString(appId.toString());
-     builder.setAppAttemptNumber(0);
-     builder.setTokenIdentifier(appId.toString());
++    String user = System.getenv(ApplicationConstants.Environment.USER.name());
++    LOG.info("Setting user in submitWorkRequest to: " + user);
++    SignableVertexSpec svs = Converters.convertTaskSpecToProto(
++        taskSpec, attemptId, appId.toString(), null, user); // TODO signatureKeyId
 +
 +    ContainerId containerId =
 +      ContainerId.newInstance(ApplicationAttemptId.newInstance(appId, 0), taskNum);
-     builder.setContainerIdString(containerId.toString());
 +
-     builder.setAmHost(address.getHostName());
-     builder.setAmPort(address.getPort());
++
 +    Credentials taskCredentials = new Credentials();
 +    // Credentials can change across DAGs. Ideally construct only once per DAG.
 +    // TODO Figure out where credentials will come from. Normally Hive sets up
 +    // URLs on the tez dag, for which Tez acquires credentials.
 +
 +    //    taskCredentials.addAll(getContext().getCredentials());
 +
 +    //    Preconditions.checkState(currentQueryIdentifierProto.getDagIdentifier() ==
 +    //        taskSpec.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getId());
 +    //    ByteBuffer credentialsBinary = credentialMap.get(currentQueryIdentifierProto);
 +    //    if (credentialsBinary == null) {
 +    //      credentialsBinary = serializeCredentials(getContext().getCredentials());
 +    //      credentialMap.putIfAbsent(currentQueryIdentifierProto, credentialsBinary.duplicate());
 +    //    } else {
 +    //      credentialsBinary = credentialsBinary.duplicate();
 +    //    }
 +    //    builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
 +    Credentials credentials = new Credentials();
 +    TokenCache.setSessionToken(token, credentials);
 +    ByteBuffer credentialsBinary = serializeCredentials(credentials);
-     builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
- 
- 
-     builder.setFragmentSpec(Converters.convertTaskSpecToProto(taskSpec));
 +
 +    FragmentRuntimeInfo.Builder runtimeInfo = FragmentRuntimeInfo.newBuilder();
 +    runtimeInfo.setCurrentAttemptStartTime(System.currentTimeMillis());
 +    runtimeInfo.setWithinDagPriority(0);
 +    runtimeInfo.setDagStartTime(submitWorkInfo.getCreationTime());
 +    runtimeInfo.setFirstAttemptStartTime(submitWorkInfo.getCreationTime());
 +    runtimeInfo.setNumSelfAndUpstreamTasks(taskSpec.getVertexParallelism());
 +    runtimeInfo.setNumSelfAndUpstreamCompletedTasks(0);
 +
++    SubmitWorkRequestProto.Builder builder = SubmitWorkRequestProto.newBuilder();
 +
++    builder.setWorkSpec(VertexOrBinary.newBuilder().setVertex(svs).build());
++    // TODO work spec signature
++    builder.setFragmentNumber(taskSpec.getTaskAttemptID().getTaskID().getId());
++    builder.setAttemptNumber(0);
++    builder.setContainerIdString(containerId.toString());
++    builder.setAmHost(address.getHostName());
++    builder.setAmPort(address.getPort());
++    builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
 +    builder.setFragmentRuntimeInfo(runtimeInfo.build());
++
 +    return builder.build();
 +  }
 +
 +  private ByteBuffer serializeCredentials(Credentials credentials) throws IOException {
 +    Credentials containerCredentials = new Credentials();
 +    containerCredentials.addAll(credentials);
 +    DataOutputBuffer containerTokens_dob = new DataOutputBuffer();
 +    containerCredentials.writeTokenStorageToStream(containerTokens_dob);
 +    return ByteBuffer.wrap(containerTokens_dob.getData(), 0, containerTokens_dob.getLength());
 +  }
 +
 +  private static class LlapRecordReaderTaskUmbilicalExternalResponder implements LlapTaskUmbilicalExternalResponder {
 +    protected LlapBaseRecordReader recordReader = null;
 +    protected LinkedBlockingQueue<ReaderEvent> queuedEvents = new LinkedBlockingQueue<ReaderEvent>();
 +
 +    public LlapRecordReaderTaskUmbilicalExternalResponder() {
 +    }
 +
 +    @Override
 +    public void submissionFailed(String fragmentId, Throwable throwable) {
 +      try {
 +        sendOrQueueEvent(ReaderEvent.errorEvent(
 +            "Received submission failed event for fragment ID " + fragmentId));
 +      } catch (Exception err) {
 +        LOG.error("Error during heartbeat responder:", err);
 +      }
 +    }
 +
 +    @Override
 +    public void heartbeat(TezHeartbeatRequest request) {
 +      TezTaskAttemptID taskAttemptId = request.getCurrentTaskAttemptID();
 +      List<TezEvent> inEvents = request.getEvents();
 +      for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
 +        EventType eventType = tezEvent.getEventType();
 +        try {
 +          switch (eventType) {
 +            case TASK_ATTEMPT_COMPLETED_EVENT:
 +              sendOrQueueEvent(ReaderEvent.doneEvent());
 +              break;
 +            case TASK_ATTEMPT_FAILED_EVENT:
 +              TaskAttemptFailedEvent taskFailedEvent = (TaskAttemptFailedEvent) tezEvent.getEvent();
 +              sendOrQueueEvent(ReaderEvent.errorEvent(taskFailedEvent.getDiagnostics()));
 +              break;
 +            case TASK_STATUS_UPDATE_EVENT:
 +              // If we want to handle counters
 +              break;
 +            default:
 +              LOG.warn("Unhandled event type " + eventType);
 +              break;
 +          }
 +        } catch (Exception err) {
 +          LOG.error("Error during heartbeat responder:", err);
 +        }
 +      }
 +    }
 +
 +    @Override
 +    public void taskKilled(TezTaskAttemptID taskAttemptId) {
 +      try {
 +        sendOrQueueEvent(ReaderEvent.errorEvent(
 +            "Received task killed event for task ID " + taskAttemptId));
 +      } catch (Exception err) {
 +        LOG.error("Error during heartbeat responder:", err);
 +      }
 +    }
 +
 +    @Override
 +    public void heartbeatTimeout(String taskAttemptId) {
 +      try {
 +        sendOrQueueEvent(ReaderEvent.errorEvent(
 +            "Timed out waiting for heartbeat for task ID " + taskAttemptId));
 +      } catch (Exception err) {
 +        LOG.error("Error during heartbeat responder:", err);
 +      }
 +    }
 +
 +    public synchronized LlapBaseRecordReader getRecordReader() {
 +      return recordReader;
 +    }
 +
 +    public synchronized void setRecordReader(LlapBaseRecordReader recordReader) {
 +      this.recordReader = recordReader;
 +
 +      if (recordReader == null) {
 +        return;
 +      }
 +
 +      // If any events were queued by the responder, give them to the record reader now.
 +      while (!queuedEvents.isEmpty()) {
 +        ReaderEvent readerEvent = queuedEvents.poll();
 +        LOG.debug("Sending queued event to record reader: " + readerEvent.getEventType());
 +        recordReader.handleEvent(readerEvent);
 +      }
 +    }
 +
 +    /**
 +     * Send the ReaderEvents to the record reader, if it is registered to this responder.
 +     * If there is no registered record reader, add them to a list of pending reader events
 +     * since we don't want to drop these events.
 +     * @param readerEvent
 +     */
 +    protected synchronized void sendOrQueueEvent(ReaderEvent readerEvent) {
 +      LlapBaseRecordReader recordReader = getRecordReader();
 +      if (recordReader != null) {
 +        recordReader.handleEvent(readerEvent);
 +      } else {
 +        if (LOG.isDebugEnabled()) {
 +          LOG.debug("No registered record reader, queueing event " + readerEvent.getEventType()
 +              + " with message " + readerEvent.getMessage());
 +        }
 +
 +        try {
 +          queuedEvents.put(readerEvent);
 +        } catch (Exception err) {
 +          throw new RuntimeException("Unexpected exception while queueing reader event", err);
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Clear the list of queued reader events if we are not interested in sending any pending events to any registering record reader.
 +     */
 +    public void clearQueuedEvents() {
 +      queuedEvents.clear();
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --cc llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
index d8367ce,2bfe3ed..2524dc2
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
@@@ -263,13 -267,12 +267,12 @@@ public class ContainerRunnerImpl extend
          new QueryIdentifier(request.getQueryIdentifier().getAppIdentifier(),
              request.getQueryIdentifier().getDagIdentifier());
      LOG.info("Processing queryComplete notification for {}", queryIdentifier);
-     List<QueryFragmentInfo> knownFragments =
-         queryTracker
-             .queryComplete(queryIdentifier, request.getDeleteDelay());
-     LOG.info("Pending fragment count for completed query {} = {}", queryIdentifier,
+     List<QueryFragmentInfo> knownFragments = queryTracker.queryComplete(
+         queryIdentifier, request.getDeleteDelay(), false);
+     LOG.info("DBG: Pending fragment count for completed query {} = {}", queryIdentifier,
          knownFragments.size());
      for (QueryFragmentInfo fragmentInfo : knownFragments) {
 -      LOG.info("DBG: Issuing killFragment for completed query {} {}", queryIdentifier,
 +      LOG.info("Issuing killFragment for completed query {} {}", queryIdentifier,
            fragmentInfo.getFragmentIdentifierString());
        executorService.killFragment(fragmentInfo.getFragmentIdentifierString());
      }

http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------
diff --cc llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
index 4a33373,3093de7..8594ee1
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
@@@ -134,15 -135,20 +135,18 @@@ public class TaskRunnerCallable extend
      this.memoryAvailable = memoryAvailable;
      this.confParams = confParams;
      this.jobToken = TokenCache.getSessionToken(credentials);
-     this.taskSpec = Converters.getTaskSpecfromProto(request.getFragmentSpec());
+     // TODO: support binary spec here or above
+     this.vertex = request.getWorkSpec().getVertex();
+     this.taskSpec = Converters.getTaskSpecfromProto(
+         vertex, request.getFragmentNumber(), request.getAttemptNumber(), attemptId);
      this.amReporter = amReporter;
      // Register with the AMReporter when the callable is setup. Unregister once it starts running.
 -    if (jobToken != null) {
      this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
-         request.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
+         vertex.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
 -    }
      this.metrics = metrics;
-     this.requestId = request.getFragmentSpec().getFragmentIdentifierString();
+     this.requestId = taskSpec.getTaskAttemptID().toString();
      // TODO Change this to the queryId/Name when that's available.
-     this.queryId = request.getFragmentSpec().getDagName();
+     this.queryId = vertex.getDagName();
      this.killedTaskHandler = killedTaskHandler;
      this.fragmentCompletionHanler = fragmentCompleteHandler;
      this.tezHadoopShim = tezHadoopShim;

http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------


[05/50] [abbrv] hive git commit: HIVE-11848 - tables in subqueries don't get locked (Eugene Koifman, reviewed by Wei Zheng)

Posted by sp...@apache.org.
HIVE-11848 - tables in subqueries don't get locked (Eugene Koifman, reviewed by Wei Zheng)


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

Branch: refs/heads/java8
Commit: 47bf055c02990272753105b917b487c5bbfe9208
Parents: 868e5e1
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Tue May 3 13:33:42 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Tue May 3 13:53:02 2016 -0700

----------------------------------------------------------------------
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  | 16 +++++++++-
 .../hive/ql/lockmgr/TestDbTxnManager2.java      | 33 ++++++++++++++++++++
 2 files changed, 48 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/47bf055c/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
index b8771d2..33fbffe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
@@ -329,7 +329,9 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
     // Walk through all our inputs and set them to note that this read is part of an update or a
     // delete.
     for (ReadEntity input : inputs) {
-      input.setUpdateOrDelete(true);
+      if(isWritten(input)) {
+        input.setUpdateOrDelete(true);
+      }
     }
 
     if (inputIsPartitioned(inputs)) {
@@ -377,6 +379,18 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
     }
   }
 
+  /**
+   * Check that {@code readEntity} is also being written
+   */
+  private boolean isWritten(Entity readEntity) {
+    for(Entity writeEntity : outputs) {
+      //make sure to compare them as Entity, i.e. that it's the same table or partition, etc
+      if(writeEntity.toString().equalsIgnoreCase(readEntity.toString())) {
+        return true;
+      }
+    }
+    return false;
+  }
   private String operation() {
     if (updating()) return "update";
     else if (deleting()) return "delete";

http://git-wip-us.apache.org/repos/asf/hive/blob/47bf055c/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index 836b507..6e2cf30 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -71,6 +71,39 @@ public class TestDbTxnManager2 {
     TxnDbUtil.prepDb();
   }
   @Test
+  public void testLocksInSubquery() throws Exception {
+    checkCmdOnDriver(driver.run("create table if not exists T (a int, b int)"));
+    checkCmdOnDriver(driver.run("create table if not exists S (a int, b int) clustered by(b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"));
+    checkCmdOnDriver(driver.run("create table if not exists R (a int, b int) clustered by(b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"));
+
+    checkCmdOnDriver(driver.compileAndRespond("delete from S where a in (select a from T where b = 1)"));
+    txnMgr.openTxn("one");
+    txnMgr.acquireLocks(driver.getPlan(), ctx, "one");
+    List<ShowLocksResponseElement> locks = getLocks();
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "T", null, locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "S", null, locks.get(1));
+    txnMgr.rollbackTxn();
+
+    checkCmdOnDriver(driver.compileAndRespond("update S set a = 7 where a in (select a from T where b = 1)"));
+    txnMgr.openTxn("one");
+    txnMgr.acquireLocks(driver.getPlan(), ctx, "one");
+    locks = getLocks();
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "T", null, locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "S", null, locks.get(1));
+    txnMgr.rollbackTxn();
+
+    checkCmdOnDriver(driver.compileAndRespond("insert into R select * from S where a in (select a from T where b = 1)"));
+    txnMgr.openTxn("three");
+    txnMgr.acquireLocks(driver.getPlan(), ctx, "three");
+    locks = getLocks();
+    Assert.assertEquals("Unexpected lock count", 3, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "T", null, locks.get(0));
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "S", null, locks.get(1));
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "R", null, locks.get(2));
+  }
+  @Test
   public void createTable() throws Exception {
     CommandProcessorResponse cpr = driver.compileAndRespond("create table if not exists T (a int, b int)");
     checkCmdOnDriver(cpr);


[28/50] [abbrv] hive git commit: HIVE-13653 : improve config error messages for LLAP cache size/etc (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by sp...@apache.org.
HIVE-13653 : improve config error messages for LLAP cache size/etc (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/java8
Commit: f41d693b5b984ea55b01394af0dbb6c7121db90a
Parents: 96f2dc7
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu May 5 10:41:47 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu May 5 10:41:47 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/llap/cache/BuddyAllocator.java  | 43 +++++++++++++++-----
 1 file changed, 32 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f41d693b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
index d78c1e0..1d5a7db 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
@@ -44,6 +44,8 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
   // We don't know the acceptable size for Java array, so we'll use 1Gb boundary.
   // That is guaranteed to fit any maximum allocation.
   private static final int MAX_ARENA_SIZE = 1024*1024*1024;
+  // Don't try to operate with less than MIN_SIZE allocator space, it will just give you grief.
+  private static final int MIN_TOTAL_MEMORY_SIZE = 64*1024*1024;
 
 
   public BuddyAllocator(Configuration conf, MemoryManager mm, LlapDaemonCacheMetrics metrics) {
@@ -51,8 +53,19 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
         (int)HiveConf.getSizeVar(conf, ConfVars.LLAP_ALLOCATOR_MIN_ALLOC),
         (int)HiveConf.getSizeVar(conf, ConfVars.LLAP_ALLOCATOR_MAX_ALLOC),
         HiveConf.getIntVar(conf, ConfVars.LLAP_ALLOCATOR_ARENA_COUNT),
-        HiveConf.getSizeVar(conf, ConfVars.LLAP_IO_MEMORY_MAX_SIZE),
-        mm, metrics);
+        getMaxTotalMemorySize(conf), mm, metrics);
+  }
+
+  private static long getMaxTotalMemorySize(Configuration conf) {
+    long maxSize = HiveConf.getSizeVar(conf, ConfVars.LLAP_IO_MEMORY_MAX_SIZE);
+    if (maxSize > MIN_TOTAL_MEMORY_SIZE || HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST)) {
+      return maxSize;
+    }
+    throw new RuntimeException("Allocator space is too small for reasonable operation; "
+        + ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname + "=" + maxSize + ", but at least "
+        + MIN_TOTAL_MEMORY_SIZE + " is required. If you cannot spare any memory, you can "
+        + "disable LLAP IO entirely via " + ConfVars.LLAP_IO_ENABLED.varname + "; or set "
+        + ConfVars.LLAP_IO_MEMORY_MODE.varname + " to 'none'");
   }
 
   @VisibleForTesting
@@ -69,16 +82,19 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
           + ", arena size " + arenaSizeVal + ". total size " + maxSizeVal);
     }
 
+    String minName = ConfVars.LLAP_ALLOCATOR_MIN_ALLOC.varname,
+        maxName = ConfVars.LLAP_ALLOCATOR_MAX_ALLOC.varname;
     if (minAllocation < 8) {
-      throw new AssertionError("Min allocation must be at least 8 bytes: " + minAllocation);
+      throw new RuntimeException(minName + " must be at least 8 bytes: " + minAllocation);
     }
-    if (maxSizeVal < arenaSizeVal || maxAllocation < minAllocation) {
-      throw new AssertionError("Inconsistent sizes of cache, arena and allocations: "
-          + minAllocation + ", " + maxAllocation + ", " + arenaSizeVal + ", " + maxSizeVal);
+    if (maxSizeVal < maxAllocation || maxAllocation < minAllocation) {
+      throw new RuntimeException("Inconsistent sizes; expecting " + minName + " <= " + maxName
+          + " <= " + ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname + "; configured with min="
+          + minAllocation + ", max=" + maxAllocation + " and total=" + maxSizeVal);
     }
     if ((Integer.bitCount(minAllocation) != 1) || (Integer.bitCount(maxAllocation) != 1)) {
-      throw new AssertionError("Allocation sizes must be powers of two: "
-          + minAllocation + ", " + maxAllocation);
+      throw new RuntimeException("Allocation sizes must be powers of two; configured with "
+          + minName + "=" + minAllocation + ", " + maxName + "=" + maxAllocation);
     }
     if ((arenaSizeVal % maxAllocation) > 0) {
       long oldArenaSize = arenaSizeVal;
@@ -94,8 +110,8 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
           + " to be divisible by arena size " + arenaSize);
     }
     if ((maxSizeVal / arenaSize) > Integer.MAX_VALUE) {
-      throw new AssertionError(
-          "Too many arenas needed to allocate the cache: " + arenaSize + "," + maxSizeVal);
+      throw new RuntimeException(
+          "Too many arenas needed to allocate the cache: " + arenaSize + ", " + maxSizeVal);
     }
     maxSize = maxSizeVal;
     memoryManager.updateMaxSize(maxSize);
@@ -280,7 +296,12 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
     private FreeList[] freeLists;
 
     void init() {
-      data = isDirect ? ByteBuffer.allocateDirect(arenaSize) : ByteBuffer.allocate(arenaSize);
+      try {
+        data = isDirect ? ByteBuffer.allocateDirect(arenaSize) : ByteBuffer.allocate(arenaSize);
+      } catch (OutOfMemoryError oom) {
+        throw new OutOfMemoryError("Cannot allocate " + arenaSize + " bytes: " + oom.getMessage()
+            + "; make sure your xmx and process size are set correctly.");
+      }
       int maxMinAllocs = 1 << (arenaSizeLog2 - minAllocLog2);
       headers = new byte[maxMinAllocs];
       int allocLog2Diff = maxAllocLog2 - minAllocLog2, freeListCount = allocLog2Diff + 1;


[22/50] [abbrv] hive git commit: HIVE-13669 : LLAP: io.enabled config is ignored on the server side (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by sp...@apache.org.
HIVE-13669 : LLAP: io.enabled config is ignored on the server side (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/java8
Commit: 652f88ad973ebe1668b5663617259795cc007953
Parents: 212077b
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed May 4 14:55:01 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed May 4 14:55:01 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java     | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/652f88ad/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index d23a44a..e662de9 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -322,8 +322,9 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       fnLocalizer.init();
       fnLocalizer.startLocalizeAllFunctions();
     }
-    LlapProxy.initializeLlapIo(conf);
-
+    if (isIoEnabled()) {
+      LlapProxy.initializeLlapIo(conf);
+    }
   }
 
   @Override


[21/50] [abbrv] hive git commit: HIVE-13351: Support drop Primary Key/Foreign Key constraints (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
HIVE-13351: Support drop Primary Key/Foreign Key constraints (Hari Subramaniyan, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/java8
Commit: 212077b8ae4aed130d8fea38febfc86c2bc55bbb
Parents: b04dc95
Author: Hari Subramaniyan <ha...@apache.org>
Authored: Wed May 4 12:26:38 2016 -0700
Committer: Hari Subramaniyan <ha...@apache.org>
Committed: Wed May 4 12:26:38 2016 -0700

----------------------------------------------------------------------
 metastore/if/hive_metastore.thrift              |    8 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2431 ++++++++++--------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  133 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 2180 ++++++++--------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   52 +
 .../metastore/api/DropConstraintRequest.java    |  591 +++++
 .../hive/metastore/api/ThriftHiveMetastore.java | 1966 ++++++++++----
 .../gen-php/metastore/ThriftHiveMetastore.php   |  242 ++
 .../src/gen/thrift/gen-php/metastore/Types.php  |  121 +
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  212 ++
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   97 +
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   23 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   63 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   29 +
 .../hive/metastore/HiveMetaStoreClient.java     |    6 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    3 +
 .../hadoop/hive/metastore/ObjectStore.java      |   46 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |    2 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |    6 +
 .../DummyRawStoreControlledCommit.java          |    6 +
 .../DummyRawStoreForJdoConnection.java          |    6 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   21 +-
 .../hadoop/hive/ql/hooks/WriteEntity.java       |    3 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    9 +
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   13 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    9 +
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    2 +
 .../hadoop/hive/ql/plan/AlterTableDesc.java     |   25 +-
 .../hadoop/hive/ql/plan/HiveOperation.java      |    2 +
 .../clientnegative/drop_invalid_constraint1.q   |    3 +
 .../clientnegative/drop_invalid_constraint2.q   |    2 +
 .../clientnegative/drop_invalid_constraint3.q   |    2 +
 .../clientnegative/drop_invalid_constraint4.q   |    3 +
 .../clientpositive/create_with_constraints.q    |   12 +
 .../drop_invalid_constraint1.q.out              |   15 +
 .../drop_invalid_constraint2.q.out              |   11 +
 .../drop_invalid_constraint3.q.out              |   11 +
 .../drop_invalid_constraint4.q.out              |   19 +
 .../create_with_constraints.q.out               |   68 +
 service/src/gen/thrift/gen-py/__init__.py       |    0
 42 files changed, 5925 insertions(+), 2540 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index acebf7a..c8d78b6 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -487,6 +487,11 @@ struct ForeignKeysResponse {
   1: required list<SQLForeignKey> foreignKeys
 }
 
+struct DropConstraintRequest {
+  1: required string dbname, 
+  2: required string tablename,
+  3: required string constraintname
+}
 
 // Return type for get_partitions_by_expr
 struct PartitionsByExprResult {
@@ -993,6 +998,9 @@ service ThriftHiveMetastore extends fb303.FacebookService
       throws (1:AlreadyExistsException o1,
               2:InvalidObjectException o2, 3:MetaException o3,
               4:NoSuchObjectException o4)
+  void drop_constraint(1:DropConstraintRequest req)
+      throws(1:NoSuchObjectException o1, 2:MetaException o3)
+
   // drops the table and all the partitions associated with it if the table has partitions
   // delete data (including partitions) if deleteData is set to true
   void drop_table(1:string dbname, 2:string name, 3:bool deleteData)


[33/50] [abbrv] hive git commit: HIVE-13395 Lost Update problem in ACID (Eugene Koifman, reviewed by Alan Gates)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index e94af55..c956d78 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -17,7 +17,13 @@
  */
 package org.apache.hadoop.hive.ql.lockmgr;
 
-import junit.framework.Assert;
+import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.ql.TestTxnCommands2;
+import org.apache.hadoop.hive.ql.txn.AcidWriteSetService;
+import org.junit.After;
+import org.junit.Assert;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.LockState;
 import org.apache.hadoop.hive.metastore.api.LockType;
@@ -29,23 +35,32 @@ import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.junit.After;
 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.List;
 
 /**
  * See additional tests in {@link org.apache.hadoop.hive.ql.lockmgr.TestDbTxnManager}
  * Tests here are "end-to-end"ish and simulate concurrent queries.
+ * 
+ * The general approach is to use an instance of Driver to use Driver.run() to create tables
+ * Use Driver.compile() to generate QueryPlan which can then be passed to HiveTxnManager.acquireLocks().
+ * Same HiveTxnManager is used to openTxn()/commitTxn() etc.  This can exercise almost the entire
+ * code path that CLI would but with the advantage that you can create a 2nd HiveTxnManager and then
+ * simulate interleaved transactional/locking operations but all from within a single thread.
+ * The later not only controls concurrency precisely but is the only way to run in UT env with DerbyDB.
  */
 public class TestDbTxnManager2 {
   private static HiveConf conf = new HiveConf(Driver.class);
   private HiveTxnManager txnMgr;
   private Context ctx;
   private Driver driver;
+  TxnStore txnHandler;
 
   @BeforeClass
   public static void setUpClass() throws Exception {
@@ -60,15 +75,17 @@ public class TestDbTxnManager2 {
     driver.init();
     TxnDbUtil.cleanDb();
     TxnDbUtil.prepDb();
-    txnMgr = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    SessionState ss = SessionState.get();
+    ss.initTxnMgr(conf);
+    txnMgr = ss.getTxnMgr();
     Assert.assertTrue(txnMgr instanceof DbTxnManager);
+    txnHandler = TxnUtils.getTxnStore(conf);
+
   }
   @After
   public void tearDown() throws Exception {
     driver.close();
     if (txnMgr != null) txnMgr.closeTxnManager();
-    TxnDbUtil.cleanDb();
-    TxnDbUtil.prepDb();
   }
   @Test
   public void testLocksInSubquery() throws Exception {
@@ -192,22 +209,24 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(cpr);
     cpr = driver.compileAndRespond("update temp.T7 set a = 5 where b = 6");
     checkCmdOnDriver(cpr);
+    txnMgr.openTxn("Fifer");
     txnMgr.acquireLocks(driver.getPlan(), ctx, "Fifer");
-    List<HiveLock> updateLocks = ctx.getHiveLocks();
-    cpr = driver.compileAndRespond("drop database if exists temp");
-    LockState lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Fiddler", false);//gets SS lock on T7
+    checkCmdOnDriver(driver.compileAndRespond("drop database if exists temp"));
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    //txnMgr2.openTxn("Fiddler");
+    ((DbTxnManager)txnMgr2).acquireLocks(driver.getPlan(), ctx, "Fiddler", false);//gets SS lock on T7
     List<ShowLocksResponseElement> locks = getLocks();
     Assert.assertEquals("Unexpected lock count", 2, locks.size());
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "temp", "T7", null, locks.get(0));
     checkLock(LockType.EXCLUSIVE, LockState.WAITING, "temp", null, null, locks.get(1));
-    txnMgr.getLockManager().releaseLocks(updateLocks);
-    lockState = ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(1).getLockid());
+    txnMgr.commitTxn();
+    ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(1).getLockid());
     locks = getLocks();
     Assert.assertEquals("Unexpected lock count", 1, locks.size());
     checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "temp", null, null, locks.get(0));
     List<HiveLock> xLock = new ArrayList<HiveLock>(0);
     xLock.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
-    txnMgr.getLockManager().releaseLocks(xLock);
+    txnMgr2.getLockManager().releaseLocks(xLock);
   }
   @Test
   public void updateSelectUpdate() throws Exception {
@@ -215,29 +234,27 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(cpr);
     cpr = driver.compileAndRespond("delete from T8 where b = 89");
     checkCmdOnDriver(cpr);
+    txnMgr.openTxn("Fifer");
     txnMgr.acquireLocks(driver.getPlan(), ctx, "Fifer");//gets SS lock on T8
-    List<HiveLock> deleteLocks = ctx.getHiveLocks();
     cpr = driver.compileAndRespond("select a from T8");//gets S lock on T8
     checkCmdOnDriver(cpr);
-    txnMgr.acquireLocks(driver.getPlan(), ctx, "Fiddler");
-    cpr = driver.compileAndRespond("update T8 set a = 1 where b = 1");
-    checkCmdOnDriver(cpr);
-    LockState lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);//waits for SS lock on T8 from fifer
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnMgr2.openTxn("Fiddler");
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "Fiddler");
+    checkCmdOnDriver(driver.compileAndRespond("update T8 set a = 1 where b = 1"));
+    ((DbTxnManager) txnMgr2).acquireLocks(driver.getPlan(), ctx, "Practical", false);//waits for SS lock on T8 from fifer
     List<ShowLocksResponseElement> locks = getLocks();
     Assert.assertEquals("Unexpected lock count", 3, locks.size());
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "T8", null, locks.get(0));
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "T8", null, locks.get(1));
     checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "T8", null, locks.get(2));
-    txnMgr.getLockManager().releaseLocks(deleteLocks);
-    lockState = ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());
+    txnMgr.rollbackTxn();
+    ((DbLockManager)txnMgr2.getLockManager()).checkLock(locks.get(2).getLockid());
     locks = getLocks();
     Assert.assertEquals("Unexpected lock count", 2, locks.size());
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "T8", null, locks.get(0));
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "T8", null, locks.get(1));
-    List<HiveLock> relLocks = new ArrayList<HiveLock>(2);
-    relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
-    relLocks.add(new DbLockManager.DbHiveLock(locks.get(1).getLockid()));
-    txnMgr.getLockManager().releaseLocks(relLocks);
+    txnMgr2.commitTxn();
     cpr = driver.run("drop table if exists T6");
     locks = getLocks();
     Assert.assertEquals("Unexpected number of locks found", 0, locks.size());
@@ -617,12 +634,12 @@ public class TestDbTxnManager2 {
     txnMgr.getLockManager().releaseLocks(relLocks);
   }
 
-  private void checkLock(LockType type, LockState state, String db, String table, String partition, ShowLocksResponseElement l) {
-    Assert.assertEquals(l.toString(),l.getType(), type);
-    Assert.assertEquals(l.toString(),l.getState(), state);
-    Assert.assertEquals(l.toString(), normalizeCase(l.getDbname()), normalizeCase(db));
-    Assert.assertEquals(l.toString(), normalizeCase(l.getTablename()), normalizeCase(table));
-    Assert.assertEquals(l.toString(), normalizeCase(l.getPartname()), normalizeCase(partition));
+  private void checkLock(LockType expectedType, LockState expectedState, String expectedDb, String expectedTable, String expectedPartition, ShowLocksResponseElement actual) {
+    Assert.assertEquals(actual.toString(), expectedType, actual.getType());
+    Assert.assertEquals(actual.toString(), expectedState,actual.getState());
+    Assert.assertEquals(actual.toString(), normalizeCase(expectedDb), normalizeCase(actual.getDbname()));
+    Assert.assertEquals(actual.toString(), normalizeCase(expectedTable), normalizeCase(actual.getTablename()));
+    Assert.assertEquals(actual.toString(), normalizeCase(expectedPartition), normalizeCase(actual.getPartname()));
   }
   private void checkCmdOnDriver(CommandProcessorResponse cpr) {
     Assert.assertTrue(cpr.toString(), cpr.getResponseCode() == 0);
@@ -637,4 +654,541 @@ public class TestDbTxnManager2 {
     ShowLocksResponse rsp = ((DbLockManager)txnMgr.getLockManager()).getLocks();
     return rsp.getLocks();
   }
+
+  /**
+   * txns update same resource but do not overlap in time - no conflict
+   */
+  @Test
+  public void testWriteSetTracking1() throws Exception {
+    CommandProcessorResponse cpr = driver.run("create table if not exists TAB_PART (a int, b int) " +
+      "partitioned by (p string) clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+
+    checkCmdOnDriver(driver.compileAndRespond("select * from TAB_PART"));
+    txnMgr.openTxn("Nicholas");
+    checkCmdOnDriver(driver.compileAndRespond("update TAB_PART set b = 7 where p = 'blah'"));
+    txnMgr.acquireLocks(driver.getPlan(), ctx, "Nicholas");
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnMgr.commitTxn();
+    txnMgr2.openTxn("Alexandra");
+    checkCmdOnDriver(driver.compileAndRespond("update TAB_PART set b = 7 where p = 'blah'"));
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "Nicholas");
+    txnMgr2.commitTxn();
+  }
+  /**
+   * txns overlap in time but do not update same resource - no conflict
+   */
+  @Test
+  public void testWriteSetTracking2() throws Exception {
+    CommandProcessorResponse cpr = driver.run("create table if not exists TAB_PART (a int, b int) " +
+      "partitioned by (p string) clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    cpr = driver.run("create table if not exists TAB2 (a int, b int) partitioned by (p string) " +
+      "clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnMgr.openTxn("Peter");
+    checkCmdOnDriver(driver.compileAndRespond("update TAB_PART set b = 7 where p = 'blah'"));
+    txnMgr.acquireLocks(driver.getPlan(), ctx, "Peter");
+    txnMgr2.openTxn("Catherine");
+    List<ShowLocksResponseElement> locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    //note that "update" uses dynamic partitioning thus lock is on the table not partition
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", null, locks.get(0));
+    txnMgr.commitTxn();
+    checkCmdOnDriver(driver.compileAndRespond("update TAB2 set b = 9 where p = 'doh'"));
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "Catherine");
+    txnMgr2.commitTxn();
+  }
+
+  /**
+   * txns overlap and update the same resource - can't commit 2nd txn
+   */
+  @Test
+  public void testWriteSetTracking3() throws Exception {
+    CommandProcessorResponse cpr = driver.run("create table if not exists TAB_PART (a int, b int) " +
+      "partitioned by (p string) clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+
+    txnMgr.openTxn("Known");
+    txnMgr2.openTxn("Unknown");
+    checkCmdOnDriver(driver.compileAndRespond("update TAB_PART set b = 7 where p = 'blah'"));
+    txnMgr.acquireLocks(driver.getPlan(), ctx, "Known");
+    List<ShowLocksResponseElement> locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", null, locks.get(0));
+    checkCmdOnDriver(driver.compileAndRespond("update TAB_PART set b = 7 where p = 'blah'"));
+    ((DbTxnManager)txnMgr2).acquireLocks(driver.getPlan(), ctx, "Unknown", false);
+    locks = getLocks(txnMgr2);//should not matter which txnMgr is used here
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", null, locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB_PART", null, locks.get(1));
+    txnMgr.commitTxn();
+    LockException expectedException = null;
+    try {
+      txnMgr2.commitTxn();
+    }
+    catch (LockException e) {
+      expectedException = e;
+    }
+    Assert.assertTrue("Didn't get exception", expectedException != null);
+    Assert.assertEquals("Got wrong message code", ErrorMsg.TXN_ABORTED, expectedException.getCanonicalErrorMsg());
+    Assert.assertEquals("Exception msg didn't match", 
+      "Aborting [txnid:2,2] due to a write conflict on default/tab_part committed by [txnid:1,2]",
+      expectedException.getCause().getMessage());
+  }
+  /**
+   * txns overlap, update same resource, simulate multi-stmt txn case
+   * Also tests that we kill txn when it tries to acquire lock if we already know it will not be committed
+   */
+  @Test
+  public void testWriteSetTracking4() throws Exception {
+    Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    CommandProcessorResponse cpr = driver.run("create table if not exists TAB_PART (a int, b int) " +
+      "partitioned by (p string) clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    cpr = driver.run("create table if not exists TAB2 (a int, b int) partitioned by (p string) " +
+      "clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    
+    txnMgr.openTxn("Long Running");
+    checkCmdOnDriver(driver.compileAndRespond("select a from  TAB_PART where p = 'blah'"));
+    txnMgr.acquireLocks(driver.getPlan(), ctx, "Long Running");
+    List<ShowLocksResponseElement> locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    //for some reason this just locks the table; if I alter table to add this partition, then 
+    //we end up locking both table and partition with share_read.  (Plan has 2 ReadEntities)...?
+    //same for other locks below
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB_PART", null, locks.get(0));
+
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnMgr2.openTxn("Short Running");
+    checkCmdOnDriver(driver.compileAndRespond("update TAB2 set b = 7 where p = 'blah'"));//no such partition
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "Short Running");
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB_PART", null, locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", null, locks.get(1));
+    //update stmt has p=blah, thus nothing is actually update and we generate empty dyn part list
+    Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(),
+      "default", "tab2", Collections.EMPTY_LIST));
+    txnMgr2.commitTxn();
+    //Short Running updated nothing, so we expect 0 rows in WRITE_SET
+    Assert.assertEquals( 0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+
+    txnMgr2.openTxn("T3");
+    checkCmdOnDriver(driver.compileAndRespond("update TAB2 set b = 7 where p = 'two'"));//pretend this partition exists
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "T3");
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB_PART", null, locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", null, locks.get(1));//since TAB2 is empty
+    //update stmt has p=blah, thus nothing is actually update and we generate empty dyn part list
+    Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(),
+      "default", "tab2", Collections.singletonList("p=two")));//simulate partition update
+    txnMgr2.commitTxn();
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    
+    AcidWriteSetService houseKeeper = new AcidWriteSetService();
+    TestTxnCommands2.runHouseKeeperService(houseKeeper, conf);
+    //since T3 overlaps with Long Running (still open) GC does nothing
+    Assert.assertEquals(1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    checkCmdOnDriver(driver.compileAndRespond("update TAB2 set b = 17 where a = 1"));//no rows match
+    txnMgr.acquireLocks(driver.getPlan(), ctx, "Long Running");
+    //so generate empty Dyn Part call
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(),
+      "default", "tab2", Collections.EMPTY_LIST));     
+    txnMgr.commitTxn();
+
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 0, locks.size());
+    TestTxnCommands2.runHouseKeeperService(houseKeeper, conf);
+    Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+  }
+  /**
+   * overlapping txns updating the same resource but 1st one rolls back; 2nd commits
+   * @throws Exception
+   */
+  @Test
+  public void testWriteSetTracking5() throws Exception {
+    Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    CommandProcessorResponse cpr = driver.run("create table if not exists TAB_PART (a int, b int) " +
+      "partitioned by (p string) clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+
+    txnMgr.openTxn("Known");
+    txnMgr2.openTxn("Unknown");
+    checkCmdOnDriver(driver.compileAndRespond("update TAB_PART set b = 7 where p = 'blah'"));
+    txnMgr.acquireLocks(driver.getPlan(), ctx, "Known");
+    List<ShowLocksResponseElement> locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", null, locks.get(0));
+    checkCmdOnDriver(driver.compileAndRespond("update TAB_PART set b = 7 where p = 'blah'"));
+    ((DbTxnManager)txnMgr2).acquireLocks(driver.getPlan(), ctx, "Unknown", false);
+    locks = getLocks(txnMgr2);//should not matter which txnMgr is used here
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", null, locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB_PART", null, locks.get(1));
+    txnMgr.rollbackTxn();
+    Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    txnMgr2.commitTxn();//since conflicting txn rolled back, commit succeeds
+    Assert.assertEquals(1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+  }
+  /**
+   * check that read query concurrent with txn works ok
+   */
+  @Test
+  public void testWriteSetTracking6() throws Exception {
+    Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    CommandProcessorResponse cpr = driver.run("create table if not exists TAB2(a int, b int) clustered " +
+      "by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    checkCmdOnDriver(driver.compileAndRespond("select * from TAB2 where a = 113"));
+    txnMgr.acquireLocks(driver.getPlan(), ctx, "Works");
+    List<ShowLocksResponseElement> locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB2", null, locks.get(0));
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnMgr2.openTxn("Horton");
+    checkCmdOnDriver(driver.compileAndRespond("update TAB2 set b = 17 where a = 101"));
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "Horton");
+    Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB2", null, locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", null, locks.get(1));
+    txnMgr2.commitTxn();//no conflict
+    Assert.assertEquals(1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB2", null, locks.get(0));
+    TestTxnCommands2.runHouseKeeperService(new AcidWriteSetService(), conf);
+    Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+  }
+
+  /**
+   * 2 concurrent txns update different partitions of the same table and succeed
+   * @throws Exception
+   */
+  @Test
+  public void testWriteSetTracking7() throws Exception {
+    Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
+    CommandProcessorResponse cpr = driver.run("create table if not exists tab2 (a int, b int) " +
+      "partitioned by (p string) clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    checkCmdOnDriver(driver.run("insert into tab2 partition(p)(a,b,p) values(1,1,'one'),(2,2,'two')"));//txnid:1
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+
+    //test with predicates such that partition pruning works
+    txnMgr2.openTxn("T2");
+    checkCmdOnDriver(driver.compileAndRespond("update tab2 set b = 7 where p='two'"));
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2");
+    List<ShowLocksResponseElement> locks = getLocks(txnMgr2);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", "p=two", locks.get(0));
+
+    //now start concurrent txn
+    txnMgr.openTxn("T3");
+    checkCmdOnDriver(driver.compileAndRespond("update tab2 set b = 7 where p='one'"));
+    ((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", "p=one", locks.get(1));
+    
+    //this simulates the completion of txnid:2
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab2",
+      Collections.singletonList("p=two")));
+    txnMgr2.commitTxn();//txnid:2
+    
+    locks = getLocks(txnMgr2);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", "p=one", locks.get(0));
+    //completion of txnid:3
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab2",
+      Collections.singletonList("p=one")));
+    txnMgr.commitTxn();//txnid:3
+    //now both txns concurrently updated TAB2 but different partitions.
+    
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=one' and ws_operation_type='u'"));
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='u'"));
+    //2 from txnid:1, 1 from txnid:2, 1 from txnid:3
+    Assert.assertEquals("COMPLETED_TXN_COMPONENTS mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      4, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_table='tab2' and ctc_partition is not null"));
+    
+    //================
+    //test with predicates such that partition pruning doesn't kick in
+    cpr = driver.run("create table if not exists tab1 (a int, b int) partitioned by (p string) " +
+      "clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    checkCmdOnDriver(driver.run("insert into tab1 partition(p)(a,b,p) values(1,1,'one'),(2,2,'two')"));//txnid:4
+    txnMgr2.openTxn("T5");
+    checkCmdOnDriver(driver.compileAndRespond("update tab1 set b = 7 where b=1"));
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "T5");
+    locks = getLocks(txnMgr2);
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+
+    //now start concurrent txn
+    txnMgr.openTxn("T6");
+    checkCmdOnDriver(driver.compileAndRespond("update tab1 set b = 7 where b = 2"));
+    ((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T6", false);
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 4, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+    checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=two", locks.get(2));
+    checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=one", locks.get(3));
+
+    //this simulates the completion of txnid:5
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=one")));
+    txnMgr2.commitTxn();//txnid:5
+
+    ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());//retest WAITING locks (both have same ext id)
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+    //completion of txnid:6
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two")));
+    txnMgr.commitTxn();//txnid:6
+
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=one' and ws_operation_type='u' and ws_table='tab1'"));
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='u' and ws_table='tab1'"));
+    //2 from insert + 1 for each update stmt
+    Assert.assertEquals("COMPLETED_TXN_COMPONENTS mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      4, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_table='tab1' and ctc_partition is not null"));
+  }
+  /**
+   * Concurrent updates with partition pruning predicate and w/o one
+   */
+  @Test
+  public void testWriteSetTracking8() throws Exception {
+    CommandProcessorResponse cpr = driver.run("create table if not exists tab1 (a int, b int) partitioned by (p string) " +
+      "clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    checkCmdOnDriver(driver.run("insert into tab1 partition(p)(a,b,p) values(1,1,'one'),(2,2,'two')"));//txnid:1
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnMgr2.openTxn("T2");
+    checkCmdOnDriver(driver.compileAndRespond("update tab1 set b = 7 where b=1"));
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2");
+    List<ShowLocksResponseElement> locks = getLocks(txnMgr2);
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+
+    //now start concurrent txn
+    txnMgr.openTxn("T3");
+    checkCmdOnDriver(driver.compileAndRespond("update tab1 set b = 7 where p='two'"));
+    ((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 3, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+    checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=two", locks.get(2));
+
+    //this simulates the completion of txnid:2
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=one")));
+    txnMgr2.commitTxn();//txnid:2
+
+    ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());//retest WAITING locks (both have same ext id)
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    //completion of txnid:3
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two")));
+    txnMgr.commitTxn();//txnid:3
+
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=one' and ws_operation_type='u' and ws_table='tab1'"));
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='u' and ws_table='tab1'"));
+    Assert.assertEquals("COMPLETED_TXN_COMPONENTS mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      4, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_table='tab1' and ctc_partition is not null"));
+  }
+  /**
+   * Concurrent update/delete of different partitions - should pass
+   */
+  @Test
+  public void testWriteSetTracking9() throws Exception {
+    CommandProcessorResponse cpr = driver.run("create table if not exists tab1 (a int, b int) partitioned by (p string) " +
+      "clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    checkCmdOnDriver(driver.run("insert into tab1 partition(p)(a,b,p) values(1,1,'one'),(2,2,'two')"));//txnid:1
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnMgr2.openTxn("T2");
+    checkCmdOnDriver(driver.compileAndRespond("update tab1 set b = 7 where b=1"));
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2");
+    List<ShowLocksResponseElement> locks = getLocks(txnMgr2);
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+
+    //now start concurrent txn
+    txnMgr.openTxn("T3");
+    checkCmdOnDriver(driver.compileAndRespond("delete from tab1 where p='two' and b=2"));
+    ((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 3, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+    checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=two", locks.get(2));
+
+    //this simulates the completion of txnid:2
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=one")));
+    txnMgr2.commitTxn();//txnid:2
+
+    ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());//retest WAITING locks (both have same ext id)
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    //completion of txnid:3
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two")));
+    txnMgr.commitTxn();//txnid:3
+
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=one' and ws_operation_type='u' and ws_table='tab1'"));
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='u' and ws_table='tab1'"));
+    Assert.assertEquals("COMPLETED_TXN_COMPONENTS mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      4, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_table='tab1' and ctc_partition is not null"));
+  }
+  /**
+   * Concurrent update/delete of same partition - should fail to commit
+   */
+  @Test
+  public void testWriteSetTracking10() throws Exception {
+    CommandProcessorResponse cpr = driver.run("create table if not exists tab1 (a int, b int) partitioned by (p string) " +
+      "clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    checkCmdOnDriver(driver.run("insert into tab1 partition(p)(a,b,p) values(1,1,'one'),(2,2,'two')"));//txnid:1
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnMgr2.openTxn("T2");
+    checkCmdOnDriver(driver.compileAndRespond("update tab1 set b = 7 where b=2"));
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2");
+    List<ShowLocksResponseElement> locks = getLocks(txnMgr2);
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+
+    //now start concurrent txn
+    txnMgr.openTxn("T3");
+    checkCmdOnDriver(driver.compileAndRespond("delete from tab1 where p='two' and b=2"));
+    ((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 3, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+    checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=two", locks.get(2));
+
+    //this simulates the completion of txnid:2
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two")));
+    txnMgr2.commitTxn();//txnid:2
+
+    ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());//retest WAITING locks (both have same ext id)
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    //completion of txnid:3
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two")));
+    LockException exception = null;
+    try {
+      txnMgr.commitTxn();//txnid:3
+    }
+    catch(LockException e) {
+      exception = e;
+    }
+    Assert.assertNotEquals("Expected exception", null, exception);
+    Assert.assertEquals("Exception msg doesn't match",
+      "Aborting [txnid:3,3] due to a write conflict on default/tab1/p=two committed by [txnid:2,3]",
+      exception.getCause().getMessage());
+
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='u' and ws_table='tab1'"));
+    Assert.assertEquals("COMPLETED_TXN_COMPONENTS mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      3, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_table='tab1' and ctc_partition is not null"));
+  }
+  /**
+   * Concurrent delte/detele of same partition - should pass
+   * This test doesn't work yet, because we don't yet pass in operation type
+   * 
+   * todo: Concurrent insert/update of same partition - should pass
+   */
+  @Ignore("HIVE-13622")
+  @Test
+  public void testWriteSetTracking11() throws Exception {
+    CommandProcessorResponse cpr = driver.run("create table if not exists tab1 (a int, b int) partitioned by (p string) " +
+      "clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    checkCmdOnDriver(driver.run("insert into tab1 partition(p)(a,b,p) values(1,1,'one'),(2,2,'two')"));//txnid:1
+    HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnMgr2.openTxn("T2");
+    checkCmdOnDriver(driver.compileAndRespond("delete from tab1 where b=2"));
+    txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2");
+    List<ShowLocksResponseElement> locks = getLocks(txnMgr2);
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+
+    //now start concurrent txn
+    txnMgr.openTxn("T3");
+    checkCmdOnDriver(driver.compileAndRespond("delete from tab1 where p='two' and b=2"));
+    ((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 3, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+    checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=two", locks.get(2));
+
+    //this simulates the completion of txnid:2
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two")));
+    txnMgr2.commitTxn();//txnid:2
+
+    ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());//retest WAITING locks (both have same ext id)
+    locks = getLocks(txnMgr);
+    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    //completion of txnid:3
+    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two")));
+    LockException exception = null;
+    try {
+      txnMgr.commitTxn();//txnid:3
+    }
+    catch(LockException e) {
+      exception = e;
+    }
+    Assert.assertNotEquals("Expected exception", null, exception);
+    Assert.assertEquals("Exception msg doesn't match",
+      "Aborting [txnid:3,3] due to a write conflict on default/tab1/p=two committed by [txnid:2,3]",
+      exception.getCause().getMessage());
+
+    //todo: this currently fails since we don't yet set operation type properly
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='d' and ws_table='tab1'"));
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='d' and ws_table='tab1'"));
+    Assert.assertEquals("COMPLETED_TXN_COMPONENTS mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      4, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_table='tab1' and ctc_partition is not null"));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
index a247065..1578bfb 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.hive.metastore.api.LockLevel;
 import org.apache.hadoop.hive.metastore.api.LockRequest;
 import org.apache.hadoop.hive.metastore.api.LockResponse;
 import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.OpenTxnRequest;
+import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
@@ -261,6 +263,8 @@ public class TestCleaner extends CompactorTest {
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
+    OpenTxnsResponse resp = txnHandler.openTxns(new OpenTxnRequest(1, "Dracula", "Transylvania"));
+    req.setTxnid(resp.getTxn_ids().get(0));
     LockResponse res = txnHandler.lock(req);
 
     startCleaner();


[37/50] [abbrv] hive git commit: Merge branch 'master' into llap

Posted by sp...@apache.org.
Merge branch 'master' into llap


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

Branch: refs/heads/java8
Commit: 763e6969d0e78806db0fc875830395c783f18b0c
Parents: 03ee048 0927187
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu May 5 13:03:53 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu May 5 13:03:53 2016 -0700

----------------------------------------------------------------------
 .../src/main/resources/hive-log4j2.properties   |    6 +-
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     |  108 +-
 .../main/java/org/apache/hive/hplsql/Exec.java  |   67 +-
 .../java/org/apache/hive/hplsql/Expression.java |   31 +-
 .../java/org/apache/hive/hplsql/Select.java     |   31 +-
 .../java/org/apache/hive/hplsql/Signal.java     |    2 +-
 .../main/java/org/apache/hive/hplsql/Stmt.java  |  154 +-
 hplsql/src/main/resources/hplsql-site.xml       |    2 -
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |    5 +
 .../apache/hive/hplsql/TestHplsqlOffline.java   |   20 +
 hplsql/src/test/queries/local/if3_bteq.sql      |    3 +
 .../test/queries/offline/create_table_td.sql    |   45 +
 hplsql/src/test/queries/offline/delete_all.sql  |    1 +
 hplsql/src/test/queries/offline/select.sql      |   42 +
 .../test/queries/offline/select_teradata.sql    |   12 +
 hplsql/src/test/results/db/select_into.out.txt  |    3 +-
 hplsql/src/test/results/db/select_into2.out.txt |    4 +-
 hplsql/src/test/results/local/if3_bteq.out.txt  |    3 +
 hplsql/src/test/results/local/lang.out.txt      |   10 +-
 .../results/offline/create_table_mssql.out.txt  |   39 +-
 .../results/offline/create_table_mssql2.out.txt |   13 +-
 .../results/offline/create_table_mysql.out.txt  |    5 +-
 .../results/offline/create_table_ora.out.txt    |   65 +-
 .../results/offline/create_table_ora2.out.txt   |    9 +-
 .../results/offline/create_table_pg.out.txt     |    7 +-
 .../results/offline/create_table_td.out.txt     |   31 +
 .../src/test/results/offline/delete_all.out.txt |    2 +
 hplsql/src/test/results/offline/select.out.txt  |   34 +
 .../src/test/results/offline/select_db2.out.txt |    3 +-
 .../results/offline/select_teradata.out.txt     |   10 +
 .../hadoop/hive/llap/cache/BuddyAllocator.java  |   43 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |    5 +-
 metastore/if/hive_metastore.thrift              |    8 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2431 ++++++++++--------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  133 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 2180 ++++++++--------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   52 +
 .../metastore/api/DropConstraintRequest.java    |  591 +++++
 .../hive/metastore/api/ThriftHiveMetastore.java | 1966 ++++++++++----
 .../gen-php/metastore/ThriftHiveMetastore.php   |  242 ++
 .../src/gen/thrift/gen-php/metastore/Types.php  |  121 +
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  212 ++
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   97 +
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   23 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   63 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   29 +
 .../hive/metastore/HiveMetaStoreClient.java     |    6 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    3 +
 .../hadoop/hive/metastore/ObjectStore.java      |   46 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |    2 +
 .../hive/metastore/RetryingMetaStoreClient.java |   17 +-
 .../hadoop/hive/metastore/hbase/HBaseStore.java |    6 +
 .../DummyRawStoreControlledCommit.java          |    6 +
 .../DummyRawStoreForJdoConnection.java          |    6 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   21 +-
 .../persistence/HybridHashTableContainer.java   |   60 +-
 .../ql/exec/persistence/KeyValueContainer.java  |    4 +
 .../ql/exec/vector/VectorizationContext.java    |    7 +
 .../hadoop/hive/ql/hooks/WriteEntity.java       |    3 +-
 .../serde/AbstractParquetMapInspector.java      |    4 +-
 .../serde/ParquetHiveArrayInspector.java        |    4 +-
 .../ql/io/parquet/write/DataWritableWriter.java |   67 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   12 +-
 .../rules/HiveReduceExpressionsRule.java        |  125 +
 .../rules/HiveSortLimitPullUpConstantsRule.java |  157 ++
 .../rules/HiveUnionPullUpConstantsRule.java     |  133 +
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    5 +
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   13 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    9 +
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    2 +
 .../hadoop/hive/ql/plan/AlterTableDesc.java     |   25 +-
 .../hadoop/hive/ql/plan/HiveOperation.java      |    2 +
 .../ql/io/parquet/TestDataWritableWriter.java   |   29 +
 .../serde/TestAbstractParquetMapInspector.java  |    4 +-
 .../serde/TestParquetHiveArrayInspector.java    |    4 +-
 .../clientnegative/drop_invalid_constraint1.q   |    3 +
 .../clientnegative/drop_invalid_constraint2.q   |    2 +
 .../clientnegative/drop_invalid_constraint3.q   |    2 +
 .../clientnegative/drop_invalid_constraint4.q   |    3 +
 .../test/queries/clientpositive/cbo_input26.q   |   54 +
 .../queries/clientpositive/cbo_union_view.q     |   19 +
 .../clientpositive/create_with_constraints.q    |   12 +
 .../parquet_array_map_emptynullvals.q           |   20 +
 .../vector_non_constant_in_expr.q               |    4 +
 .../drop_invalid_constraint1.q.out              |   15 +
 .../drop_invalid_constraint2.q.out              |   11 +
 .../drop_invalid_constraint3.q.out              |   11 +
 .../drop_invalid_constraint4.q.out              |   19 +
 .../results/clientpositive/cbo_input26.q.out    |  596 +++++
 .../results/clientpositive/cbo_union_view.q.out |  228 ++
 .../create_with_constraints.q.out               |   68 +
 .../results/clientpositive/groupby_ppd.q.out    |   28 +-
 .../clientpositive/load_dyn_part14.q.out        |    6 +-
 .../parquet_array_map_emptynullvals.q.out       |   87 +
 .../results/clientpositive/perf/query66.q.out   |  328 +--
 .../results/clientpositive/perf/query75.q.out   |  692 ++---
 .../clientpositive/spark/load_dyn_part14.q.out  |    6 +-
 .../clientpositive/spark/union_remove_25.q.out  |   16 +-
 .../clientpositive/spark/union_view.q.out       |   60 +-
 .../clientpositive/union_remove_25.q.out        |   20 +-
 .../results/clientpositive/union_view.q.out     |   60 +-
 .../vector_non_constant_in_expr.q.out           |   36 +
 service/src/gen/thrift/gen-py/__init__.py       |    0
 105 files changed, 8771 insertions(+), 3392 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/763e6969/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------


[11/50] [abbrv] hive git commit: HIVE-13674: usingTezAm field not required in LLAP SubmitWorkRequestProto

Posted by sp...@apache.org.
HIVE-13674: usingTezAm field not required in LLAP SubmitWorkRequestProto


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

Branch: refs/heads/java8
Commit: 4847f652804f476bbc969716fe7643d8b20eba8c
Parents: bc75d72
Author: Jason Dere <jd...@hortonworks.com>
Authored: Tue May 3 18:38:07 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Tue May 3 18:38:07 2016 -0700

----------------------------------------------------------------------
 .../ext/LlapTaskUmbilicalExternalClient.java    |   4 +-
 .../daemon/rpc/LlapDaemonProtocolProtos.java    | 230 ++++++-------------
 .../src/protobuf/LlapDaemonProtocol.proto       |   8 -
 .../hadoop/hive/llap/LlapBaseInputFormat.java   |   1 -
 .../llap/daemon/impl/TaskRunnerCallable.java    |   3 -
 5 files changed, 71 insertions(+), 175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4847f652/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java b/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
index 8598bc8..fe2fd7c 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
@@ -123,12 +123,10 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
 
 
   /**
-   * Submit the work for actual execution. This should always have the usingTezAm flag disabled
+   * Submit the work for actual execution.
    * @param submitWorkRequestProto
    */
   public void submitWork(final SubmitWorkRequestProto submitWorkRequestProto, String llapHost, int llapPort, List<TezEvent> tezEvents) {
-    Preconditions.checkArgument(submitWorkRequestProto.getUsingTezAm() == false);
-
     // Register the pending events to be sent for this spec.
     String fragmentId = submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString();
     PendingEventData pendingEventData = new PendingEventData(

http://git-wip-us.apache.org/repos/asf/hive/blob/4847f652/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
----------------------------------------------------------------------
diff --git a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
index 653e7e0..6a20031 100644
--- a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
+++ b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
@@ -1,5 +1,5 @@
 // Generated by the protocol buffer compiler.  DO NOT EDIT!
-// source: LlapDaemonProtocol.proto
+// source: llap-common/src/protobuf/LlapDaemonProtocol.proto
 
 package org.apache.hadoop.hive.llap.daemon.rpc;
 
@@ -7334,16 +7334,6 @@ public final class LlapDaemonProtocolProtos {
      * <code>optional .FragmentRuntimeInfo fragment_runtime_info = 10;</code>
      */
     org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfoOrBuilder getFragmentRuntimeInfoOrBuilder();
-
-    // optional bool usingTezAm = 11 [default = true];
-    /**
-     * <code>optional bool usingTezAm = 11 [default = true];</code>
-     */
-    boolean hasUsingTezAm();
-    /**
-     * <code>optional bool usingTezAm = 11 [default = true];</code>
-     */
-    boolean getUsingTezAm();
   }
   /**
    * Protobuf type {@code SubmitWorkRequestProto}
@@ -7462,11 +7452,6 @@ public final class LlapDaemonProtocolProtos {
               bitField0_ |= 0x00000200;
               break;
             }
-            case 88: {
-              bitField0_ |= 0x00000400;
-              usingTezAm_ = input.readBool();
-              break;
-            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -7814,22 +7799,6 @@ public final class LlapDaemonProtocolProtos {
       return fragmentRuntimeInfo_;
     }
 
-    // optional bool usingTezAm = 11 [default = true];
-    public static final int USINGTEZAM_FIELD_NUMBER = 11;
-    private boolean usingTezAm_;
-    /**
-     * <code>optional bool usingTezAm = 11 [default = true];</code>
-     */
-    public boolean hasUsingTezAm() {
-      return ((bitField0_ & 0x00000400) == 0x00000400);
-    }
-    /**
-     * <code>optional bool usingTezAm = 11 [default = true];</code>
-     */
-    public boolean getUsingTezAm() {
-      return usingTezAm_;
-    }
-
     private void initFields() {
       containerIdString_ = "";
       amHost_ = "";
@@ -7841,7 +7810,6 @@ public final class LlapDaemonProtocolProtos {
       appAttemptNumber_ = 0;
       fragmentSpec_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.getDefaultInstance();
       fragmentRuntimeInfo_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo.getDefaultInstance();
-      usingTezAm_ = true;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -7885,9 +7853,6 @@ public final class LlapDaemonProtocolProtos {
       if (((bitField0_ & 0x00000200) == 0x00000200)) {
         output.writeMessage(10, fragmentRuntimeInfo_);
       }
-      if (((bitField0_ & 0x00000400) == 0x00000400)) {
-        output.writeBool(11, usingTezAm_);
-      }
       getUnknownFields().writeTo(output);
     }
 
@@ -7937,10 +7902,6 @@ public final class LlapDaemonProtocolProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(10, fragmentRuntimeInfo_);
       }
-      if (((bitField0_ & 0x00000400) == 0x00000400)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(11, usingTezAm_);
-      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -8014,11 +7975,6 @@ public final class LlapDaemonProtocolProtos {
         result = result && getFragmentRuntimeInfo()
             .equals(other.getFragmentRuntimeInfo());
       }
-      result = result && (hasUsingTezAm() == other.hasUsingTezAm());
-      if (hasUsingTezAm()) {
-        result = result && (getUsingTezAm()
-            == other.getUsingTezAm());
-      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -8072,10 +8028,6 @@ public final class LlapDaemonProtocolProtos {
         hash = (37 * hash) + FRAGMENT_RUNTIME_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getFragmentRuntimeInfo().hashCode();
       }
-      if (hasUsingTezAm()) {
-        hash = (37 * hash) + USINGTEZAM_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getUsingTezAm());
-      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -8215,8 +8167,6 @@ public final class LlapDaemonProtocolProtos {
           fragmentRuntimeInfoBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000200);
-        usingTezAm_ = true;
-        bitField0_ = (bitField0_ & ~0x00000400);
         return this;
       }
 
@@ -8293,10 +8243,6 @@ public final class LlapDaemonProtocolProtos {
         } else {
           result.fragmentRuntimeInfo_ = fragmentRuntimeInfoBuilder_.build();
         }
-        if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
-          to_bitField0_ |= 0x00000400;
-        }
-        result.usingTezAm_ = usingTezAm_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -8353,9 +8299,6 @@ public final class LlapDaemonProtocolProtos {
         if (other.hasFragmentRuntimeInfo()) {
           mergeFragmentRuntimeInfo(other.getFragmentRuntimeInfo());
         }
-        if (other.hasUsingTezAm()) {
-          setUsingTezAm(other.getUsingTezAm());
-        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -9089,39 +9032,6 @@ public final class LlapDaemonProtocolProtos {
         return fragmentRuntimeInfoBuilder_;
       }
 
-      // optional bool usingTezAm = 11 [default = true];
-      private boolean usingTezAm_ = true;
-      /**
-       * <code>optional bool usingTezAm = 11 [default = true];</code>
-       */
-      public boolean hasUsingTezAm() {
-        return ((bitField0_ & 0x00000400) == 0x00000400);
-      }
-      /**
-       * <code>optional bool usingTezAm = 11 [default = true];</code>
-       */
-      public boolean getUsingTezAm() {
-        return usingTezAm_;
-      }
-      /**
-       * <code>optional bool usingTezAm = 11 [default = true];</code>
-       */
-      public Builder setUsingTezAm(boolean value) {
-        bitField0_ |= 0x00000400;
-        usingTezAm_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bool usingTezAm = 11 [default = true];</code>
-       */
-      public Builder clearUsingTezAm() {
-        bitField0_ = (bitField0_ & ~0x00000400);
-        usingTezAm_ = true;
-        onChanged();
-        return this;
-      }
-
       // @@protoc_insertion_point(builder_scope:SubmitWorkRequestProto)
     }
 
@@ -14455,74 +14365,74 @@ public final class LlapDaemonProtocolProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n\030LlapDaemonProtocol.proto\"9\n\020UserPayloa" +
-      "dProto\022\024\n\014user_payload\030\001 \001(\014\022\017\n\007version\030" +
-      "\002 \001(\005\"j\n\025EntityDescriptorProto\022\022\n\nclass_" +
-      "name\030\001 \001(\t\022\'\n\014user_payload\030\002 \001(\0132\021.UserP" +
-      "ayloadProto\022\024\n\014history_text\030\003 \001(\014\"x\n\013IOS" +
-      "pecProto\022\035\n\025connected_vertex_name\030\001 \001(\t\022" +
-      "-\n\rio_descriptor\030\002 \001(\0132\026.EntityDescripto" +
-      "rProto\022\033\n\023physical_edge_count\030\003 \001(\005\"z\n\023G" +
-      "roupInputSpecProto\022\022\n\ngroup_name\030\001 \001(\t\022\026" +
-      "\n\016group_vertices\030\002 \003(\t\0227\n\027merged_input_d",
-      "escriptor\030\003 \001(\0132\026.EntityDescriptorProto\"" +
-      "\353\002\n\021FragmentSpecProto\022\"\n\032fragment_identi" +
-      "fier_string\030\001 \001(\t\022\020\n\010dag_name\030\002 \001(\t\022\016\n\006d" +
-      "ag_id\030\013 \001(\005\022\023\n\013vertex_name\030\003 \001(\t\0224\n\024proc" +
-      "essor_descriptor\030\004 \001(\0132\026.EntityDescripto" +
-      "rProto\022!\n\013input_specs\030\005 \003(\0132\014.IOSpecProt" +
-      "o\022\"\n\014output_specs\030\006 \003(\0132\014.IOSpecProto\0221\n" +
-      "\023grouped_input_specs\030\007 \003(\0132\024.GroupInputS" +
-      "pecProto\022\032\n\022vertex_parallelism\030\010 \001(\005\022\027\n\017" +
-      "fragment_number\030\t \001(\005\022\026\n\016attempt_number\030",
-      "\n \001(\005\"\344\001\n\023FragmentRuntimeInfo\022#\n\033num_sel" +
-      "f_and_upstream_tasks\030\001 \001(\005\022-\n%num_self_a" +
-      "nd_upstream_completed_tasks\030\002 \001(\005\022\033\n\023wit" +
-      "hin_dag_priority\030\003 \001(\005\022\026\n\016dag_start_time" +
-      "\030\004 \001(\003\022 \n\030first_attempt_start_time\030\005 \001(\003" +
-      "\022\"\n\032current_attempt_start_time\030\006 \001(\003\"F\n\024" +
-      "QueryIdentifierProto\022\026\n\016app_identifier\030\001" +
-      " \001(\t\022\026\n\016dag_identifier\030\002 \001(\005\"\320\002\n\026SubmitW" +
-      "orkRequestProto\022\033\n\023container_id_string\030\001" +
-      " \001(\t\022\017\n\007am_host\030\002 \001(\t\022\017\n\007am_port\030\003 \001(\005\022\030",
-      "\n\020token_identifier\030\004 \001(\t\022\032\n\022credentials_" +
-      "binary\030\005 \001(\014\022\014\n\004user\030\006 \001(\t\022\035\n\025applicatio" +
-      "n_id_string\030\007 \001(\t\022\032\n\022app_attempt_number\030" +
-      "\010 \001(\005\022)\n\rfragment_spec\030\t \001(\0132\022.FragmentS" +
-      "pecProto\0223\n\025fragment_runtime_info\030\n \001(\0132" +
-      "\024.FragmentRuntimeInfo\022\030\n\nusingTezAm\030\013 \001(" +
-      "\010:\004true\"J\n\027SubmitWorkResponseProto\022/\n\020su" +
-      "bmission_state\030\001 \001(\0162\025.SubmissionStatePr" +
-      "oto\"\205\001\n\036SourceStateUpdatedRequestProto\022/" +
-      "\n\020query_identifier\030\001 \001(\0132\025.QueryIdentifi",
-      "erProto\022\020\n\010src_name\030\002 \001(\t\022 \n\005state\030\003 \001(\016" +
-      "2\021.SourceStateProto\"!\n\037SourceStateUpdate" +
-      "dResponseProto\"w\n\031QueryCompleteRequestPr" +
-      "oto\022\020\n\010query_id\030\001 \001(\t\022/\n\020query_identifie" +
-      "r\030\002 \001(\0132\025.QueryIdentifierProto\022\027\n\014delete" +
-      "_delay\030\004 \001(\003:\0010\"\034\n\032QueryCompleteResponse" +
-      "Proto\"t\n\035TerminateFragmentRequestProto\022/" +
-      "\n\020query_identifier\030\001 \001(\0132\025.QueryIdentifi" +
-      "erProto\022\"\n\032fragment_identifier_string\030\002 " +
-      "\001(\t\" \n\036TerminateFragmentResponseProto\"\026\n",
-      "\024GetTokenRequestProto\"&\n\025GetTokenRespons" +
-      "eProto\022\r\n\005token\030\001 \001(\014*2\n\020SourceStateProt" +
-      "o\022\017\n\013S_SUCCEEDED\020\001\022\r\n\tS_RUNNING\020\002*E\n\024Sub" +
-      "missionStateProto\022\014\n\010ACCEPTED\020\001\022\014\n\010REJEC" +
-      "TED\020\002\022\021\n\rEVICTED_OTHER\020\0032\316\002\n\022LlapDaemonP" +
-      "rotocol\022?\n\nsubmitWork\022\027.SubmitWorkReques" +
-      "tProto\032\030.SubmitWorkResponseProto\022W\n\022sour" +
-      "ceStateUpdated\022\037.SourceStateUpdatedReque" +
-      "stProto\032 .SourceStateUpdatedResponseProt" +
-      "o\022H\n\rqueryComplete\022\032.QueryCompleteReques",
-      "tProto\032\033.QueryCompleteResponseProto\022T\n\021t" +
-      "erminateFragment\022\036.TerminateFragmentRequ" +
-      "estProto\032\037.TerminateFragmentResponseProt" +
-      "o2]\n\026LlapManagementProtocol\022C\n\022getDelega" +
-      "tionToken\022\025.GetTokenRequestProto\032\026.GetTo" +
-      "kenResponseProtoBH\n&org.apache.hadoop.hi" +
-      "ve.llap.daemon.rpcB\030LlapDaemonProtocolPr" +
-      "otos\210\001\001\240\001\001"
+      "\n1llap-common/src/protobuf/LlapDaemonPro" +
+      "tocol.proto\"9\n\020UserPayloadProto\022\024\n\014user_" +
+      "payload\030\001 \001(\014\022\017\n\007version\030\002 \001(\005\"j\n\025Entity" +
+      "DescriptorProto\022\022\n\nclass_name\030\001 \001(\t\022\'\n\014u" +
+      "ser_payload\030\002 \001(\0132\021.UserPayloadProto\022\024\n\014" +
+      "history_text\030\003 \001(\014\"x\n\013IOSpecProto\022\035\n\025con" +
+      "nected_vertex_name\030\001 \001(\t\022-\n\rio_descripto" +
+      "r\030\002 \001(\0132\026.EntityDescriptorProto\022\033\n\023physi" +
+      "cal_edge_count\030\003 \001(\005\"z\n\023GroupInputSpecPr" +
+      "oto\022\022\n\ngroup_name\030\001 \001(\t\022\026\n\016group_vertice",
+      "s\030\002 \003(\t\0227\n\027merged_input_descriptor\030\003 \001(\013" +
+      "2\026.EntityDescriptorProto\"\353\002\n\021FragmentSpe" +
+      "cProto\022\"\n\032fragment_identifier_string\030\001 \001" +
+      "(\t\022\020\n\010dag_name\030\002 \001(\t\022\016\n\006dag_id\030\013 \001(\005\022\023\n\013" +
+      "vertex_name\030\003 \001(\t\0224\n\024processor_descripto" +
+      "r\030\004 \001(\0132\026.EntityDescriptorProto\022!\n\013input" +
+      "_specs\030\005 \003(\0132\014.IOSpecProto\022\"\n\014output_spe" +
+      "cs\030\006 \003(\0132\014.IOSpecProto\0221\n\023grouped_input_" +
+      "specs\030\007 \003(\0132\024.GroupInputSpecProto\022\032\n\022ver" +
+      "tex_parallelism\030\010 \001(\005\022\027\n\017fragment_number",
+      "\030\t \001(\005\022\026\n\016attempt_number\030\n \001(\005\"\344\001\n\023Fragm" +
+      "entRuntimeInfo\022#\n\033num_self_and_upstream_" +
+      "tasks\030\001 \001(\005\022-\n%num_self_and_upstream_com" +
+      "pleted_tasks\030\002 \001(\005\022\033\n\023within_dag_priorit" +
+      "y\030\003 \001(\005\022\026\n\016dag_start_time\030\004 \001(\003\022 \n\030first" +
+      "_attempt_start_time\030\005 \001(\003\022\"\n\032current_att" +
+      "empt_start_time\030\006 \001(\003\"F\n\024QueryIdentifier" +
+      "Proto\022\026\n\016app_identifier\030\001 \001(\t\022\026\n\016dag_ide" +
+      "ntifier\030\002 \001(\005\"\266\002\n\026SubmitWorkRequestProto" +
+      "\022\033\n\023container_id_string\030\001 \001(\t\022\017\n\007am_host",
+      "\030\002 \001(\t\022\017\n\007am_port\030\003 \001(\005\022\030\n\020token_identif" +
+      "ier\030\004 \001(\t\022\032\n\022credentials_binary\030\005 \001(\014\022\014\n" +
+      "\004user\030\006 \001(\t\022\035\n\025application_id_string\030\007 \001" +
+      "(\t\022\032\n\022app_attempt_number\030\010 \001(\005\022)\n\rfragme" +
+      "nt_spec\030\t \001(\0132\022.FragmentSpecProto\0223\n\025fra" +
+      "gment_runtime_info\030\n \001(\0132\024.FragmentRunti" +
+      "meInfo\"J\n\027SubmitWorkResponseProto\022/\n\020sub" +
+      "mission_state\030\001 \001(\0162\025.SubmissionStatePro" +
+      "to\"\205\001\n\036SourceStateUpdatedRequestProto\022/\n" +
+      "\020query_identifier\030\001 \001(\0132\025.QueryIdentifie",
+      "rProto\022\020\n\010src_name\030\002 \001(\t\022 \n\005state\030\003 \001(\0162" +
+      "\021.SourceStateProto\"!\n\037SourceStateUpdated" +
+      "ResponseProto\"w\n\031QueryCompleteRequestPro" +
+      "to\022\020\n\010query_id\030\001 \001(\t\022/\n\020query_identifier" +
+      "\030\002 \001(\0132\025.QueryIdentifierProto\022\027\n\014delete_" +
+      "delay\030\004 \001(\003:\0010\"\034\n\032QueryCompleteResponseP" +
+      "roto\"t\n\035TerminateFragmentRequestProto\022/\n" +
+      "\020query_identifier\030\001 \001(\0132\025.QueryIdentifie" +
+      "rProto\022\"\n\032fragment_identifier_string\030\002 \001" +
+      "(\t\" \n\036TerminateFragmentResponseProto\"\026\n\024",
+      "GetTokenRequestProto\"&\n\025GetTokenResponse" +
+      "Proto\022\r\n\005token\030\001 \001(\014*2\n\020SourceStateProto" +
+      "\022\017\n\013S_SUCCEEDED\020\001\022\r\n\tS_RUNNING\020\002*E\n\024Subm" +
+      "issionStateProto\022\014\n\010ACCEPTED\020\001\022\014\n\010REJECT" +
+      "ED\020\002\022\021\n\rEVICTED_OTHER\020\0032\316\002\n\022LlapDaemonPr" +
+      "otocol\022?\n\nsubmitWork\022\027.SubmitWorkRequest" +
+      "Proto\032\030.SubmitWorkResponseProto\022W\n\022sourc" +
+      "eStateUpdated\022\037.SourceStateUpdatedReques" +
+      "tProto\032 .SourceStateUpdatedResponseProto" +
+      "\022H\n\rqueryComplete\022\032.QueryCompleteRequest",
+      "Proto\032\033.QueryCompleteResponseProto\022T\n\021te" +
+      "rminateFragment\022\036.TerminateFragmentReque" +
+      "stProto\032\037.TerminateFragmentResponseProto" +
+      "2]\n\026LlapManagementProtocol\022C\n\022getDelegat" +
+      "ionToken\022\025.GetTokenRequestProto\032\026.GetTok" +
+      "enResponseProtoBH\n&org.apache.hadoop.hiv" +
+      "e.llap.daemon.rpcB\030LlapDaemonProtocolPro" +
+      "tos\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -14576,7 +14486,7 @@ public final class LlapDaemonProtocolProtos {
           internal_static_SubmitWorkRequestProto_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_SubmitWorkRequestProto_descriptor,
-              new java.lang.String[] { "ContainerIdString", "AmHost", "AmPort", "TokenIdentifier", "CredentialsBinary", "User", "ApplicationIdString", "AppAttemptNumber", "FragmentSpec", "FragmentRuntimeInfo", "UsingTezAm", });
+              new java.lang.String[] { "ContainerIdString", "AmHost", "AmPort", "TokenIdentifier", "CredentialsBinary", "User", "ApplicationIdString", "AppAttemptNumber", "FragmentSpec", "FragmentRuntimeInfo", });
           internal_static_SubmitWorkResponseProto_descriptor =
             getDescriptor().getMessageTypes().get(8);
           internal_static_SubmitWorkResponseProto_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hive/blob/4847f652/llap-common/src/protobuf/LlapDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/llap-common/src/protobuf/LlapDaemonProtocol.proto b/llap-common/src/protobuf/LlapDaemonProtocol.proto
index e964c5f..944c96c 100644
--- a/llap-common/src/protobuf/LlapDaemonProtocol.proto
+++ b/llap-common/src/protobuf/LlapDaemonProtocol.proto
@@ -91,7 +91,6 @@ message SubmitWorkRequestProto {
   optional int32 app_attempt_number = 8;
   optional FragmentSpecProto fragment_spec = 9;
   optional FragmentRuntimeInfo fragment_runtime_info = 10;
-  optional bool usingTezAm = 11 [default = true];
 }
 
 enum SubmissionStateProto {
@@ -137,18 +136,11 @@ message GetTokenResponseProto {
   optional bytes token = 1;
 }
 
-message SendEventsRequestProto {
-}
-
-message SendEventsResponseProto {
-}
-
 service LlapDaemonProtocol {
   rpc submitWork(SubmitWorkRequestProto) returns (SubmitWorkResponseProto);
   rpc sourceStateUpdated(SourceStateUpdatedRequestProto) returns (SourceStateUpdatedResponseProto);
   rpc queryComplete(QueryCompleteRequestProto) returns (QueryCompleteResponseProto);
   rpc terminateFragment(TerminateFragmentRequestProto) returns (TerminateFragmentResponseProto);
-  rpc sendEvents(SendEventsRequestProto) return (SendEventsResponseProto);
 }
 
 service LlapManagementProtocol {

http://git-wip-us.apache.org/repos/asf/hive/blob/4847f652/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
index 10d14c0..8db2f88 100644
--- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
@@ -346,7 +346,6 @@ public class LlapBaseInputFormat<V extends WritableComparable> implements InputF
     runtimeInfo.setNumSelfAndUpstreamCompletedTasks(0);
 
 
-    builder.setUsingTezAm(false);
     builder.setFragmentRuntimeInfo(runtimeInfo.build());
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4847f652/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
index efd6f0a..4a33373 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
@@ -108,7 +108,6 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
   private final String queryId;
   private final HadoopShim tezHadoopShim;
   private boolean shouldRunTask = true;
-  private final boolean withTezAm;
   final Stopwatch runtimeWatch = new Stopwatch();
   final Stopwatch killtimerWatch = new Stopwatch();
   private final AtomicBoolean isStarted = new AtomicBoolean(false);
@@ -137,8 +136,6 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
     this.jobToken = TokenCache.getSessionToken(credentials);
     this.taskSpec = Converters.getTaskSpecfromProto(request.getFragmentSpec());
     this.amReporter = amReporter;
-    this.withTezAm = request.getUsingTezAm();
-    LOG.warn("ZZZ: DBG: usingTezAm=" + withTezAm);
     // Register with the AMReporter when the callable is setup. Unregister once it starts running.
     this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
         request.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());


[17/50] [abbrv] hive git commit: HIVE-13351: Support drop Primary Key/Foreign Key constraints (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index d392f67..3b3e05e 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -251,6 +251,8 @@ class ForeignKeysRequest;
 
 class ForeignKeysResponse;
 
+class DropConstraintRequest;
+
 class PartitionsByExprResult;
 
 class PartitionsByExprRequest;
@@ -3779,6 +3781,56 @@ inline std::ostream& operator<<(std::ostream& out, const ForeignKeysResponse& ob
 }
 
 
+class DropConstraintRequest {
+ public:
+
+  DropConstraintRequest(const DropConstraintRequest&);
+  DropConstraintRequest& operator=(const DropConstraintRequest&);
+  DropConstraintRequest() : dbname(), tablename(), constraintname() {
+  }
+
+  virtual ~DropConstraintRequest() throw();
+  std::string dbname;
+  std::string tablename;
+  std::string constraintname;
+
+  void __set_dbname(const std::string& val);
+
+  void __set_tablename(const std::string& val);
+
+  void __set_constraintname(const std::string& val);
+
+  bool operator == (const DropConstraintRequest & rhs) const
+  {
+    if (!(dbname == rhs.dbname))
+      return false;
+    if (!(tablename == rhs.tablename))
+      return false;
+    if (!(constraintname == rhs.constraintname))
+      return false;
+    return true;
+  }
+  bool operator != (const DropConstraintRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const DropConstraintRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(DropConstraintRequest &a, DropConstraintRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const DropConstraintRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
 class PartitionsByExprResult {
  public:
 

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropConstraintRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropConstraintRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropConstraintRequest.java
new file mode 100644
index 0000000..4519dac
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropConstraintRequest.java
@@ -0,0 +1,591 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class DropConstraintRequest implements org.apache.thrift.TBase<DropConstraintRequest, DropConstraintRequest._Fields>, java.io.Serializable, Cloneable, Comparable<DropConstraintRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DropConstraintRequest");
+
+  private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TABLENAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tablename", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField CONSTRAINTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("constraintname", org.apache.thrift.protocol.TType.STRING, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new DropConstraintRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new DropConstraintRequestTupleSchemeFactory());
+  }
+
+  private String dbname; // required
+  private String tablename; // required
+  private String constraintname; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    DBNAME((short)1, "dbname"),
+    TABLENAME((short)2, "tablename"),
+    CONSTRAINTNAME((short)3, "constraintname");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // DBNAME
+          return DBNAME;
+        case 2: // TABLENAME
+          return TABLENAME;
+        case 3: // CONSTRAINTNAME
+          return CONSTRAINTNAME;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.DBNAME, new org.apache.thrift.meta_data.FieldMetaData("dbname", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLENAME, new org.apache.thrift.meta_data.FieldMetaData("tablename", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CONSTRAINTNAME, new org.apache.thrift.meta_data.FieldMetaData("constraintname", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DropConstraintRequest.class, metaDataMap);
+  }
+
+  public DropConstraintRequest() {
+  }
+
+  public DropConstraintRequest(
+    String dbname,
+    String tablename,
+    String constraintname)
+  {
+    this();
+    this.dbname = dbname;
+    this.tablename = tablename;
+    this.constraintname = constraintname;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public DropConstraintRequest(DropConstraintRequest other) {
+    if (other.isSetDbname()) {
+      this.dbname = other.dbname;
+    }
+    if (other.isSetTablename()) {
+      this.tablename = other.tablename;
+    }
+    if (other.isSetConstraintname()) {
+      this.constraintname = other.constraintname;
+    }
+  }
+
+  public DropConstraintRequest deepCopy() {
+    return new DropConstraintRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.dbname = null;
+    this.tablename = null;
+    this.constraintname = null;
+  }
+
+  public String getDbname() {
+    return this.dbname;
+  }
+
+  public void setDbname(String dbname) {
+    this.dbname = dbname;
+  }
+
+  public void unsetDbname() {
+    this.dbname = null;
+  }
+
+  /** Returns true if field dbname is set (has been assigned a value) and false otherwise */
+  public boolean isSetDbname() {
+    return this.dbname != null;
+  }
+
+  public void setDbnameIsSet(boolean value) {
+    if (!value) {
+      this.dbname = null;
+    }
+  }
+
+  public String getTablename() {
+    return this.tablename;
+  }
+
+  public void setTablename(String tablename) {
+    this.tablename = tablename;
+  }
+
+  public void unsetTablename() {
+    this.tablename = null;
+  }
+
+  /** Returns true if field tablename is set (has been assigned a value) and false otherwise */
+  public boolean isSetTablename() {
+    return this.tablename != null;
+  }
+
+  public void setTablenameIsSet(boolean value) {
+    if (!value) {
+      this.tablename = null;
+    }
+  }
+
+  public String getConstraintname() {
+    return this.constraintname;
+  }
+
+  public void setConstraintname(String constraintname) {
+    this.constraintname = constraintname;
+  }
+
+  public void unsetConstraintname() {
+    this.constraintname = null;
+  }
+
+  /** Returns true if field constraintname is set (has been assigned a value) and false otherwise */
+  public boolean isSetConstraintname() {
+    return this.constraintname != null;
+  }
+
+  public void setConstraintnameIsSet(boolean value) {
+    if (!value) {
+      this.constraintname = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case DBNAME:
+      if (value == null) {
+        unsetDbname();
+      } else {
+        setDbname((String)value);
+      }
+      break;
+
+    case TABLENAME:
+      if (value == null) {
+        unsetTablename();
+      } else {
+        setTablename((String)value);
+      }
+      break;
+
+    case CONSTRAINTNAME:
+      if (value == null) {
+        unsetConstraintname();
+      } else {
+        setConstraintname((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DBNAME:
+      return getDbname();
+
+    case TABLENAME:
+      return getTablename();
+
+    case CONSTRAINTNAME:
+      return getConstraintname();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case DBNAME:
+      return isSetDbname();
+    case TABLENAME:
+      return isSetTablename();
+    case CONSTRAINTNAME:
+      return isSetConstraintname();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof DropConstraintRequest)
+      return this.equals((DropConstraintRequest)that);
+    return false;
+  }
+
+  public boolean equals(DropConstraintRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_dbname = true && this.isSetDbname();
+    boolean that_present_dbname = true && that.isSetDbname();
+    if (this_present_dbname || that_present_dbname) {
+      if (!(this_present_dbname && that_present_dbname))
+        return false;
+      if (!this.dbname.equals(that.dbname))
+        return false;
+    }
+
+    boolean this_present_tablename = true && this.isSetTablename();
+    boolean that_present_tablename = true && that.isSetTablename();
+    if (this_present_tablename || that_present_tablename) {
+      if (!(this_present_tablename && that_present_tablename))
+        return false;
+      if (!this.tablename.equals(that.tablename))
+        return false;
+    }
+
+    boolean this_present_constraintname = true && this.isSetConstraintname();
+    boolean that_present_constraintname = true && that.isSetConstraintname();
+    if (this_present_constraintname || that_present_constraintname) {
+      if (!(this_present_constraintname && that_present_constraintname))
+        return false;
+      if (!this.constraintname.equals(that.constraintname))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_dbname = true && (isSetDbname());
+    list.add(present_dbname);
+    if (present_dbname)
+      list.add(dbname);
+
+    boolean present_tablename = true && (isSetTablename());
+    list.add(present_tablename);
+    if (present_tablename)
+      list.add(tablename);
+
+    boolean present_constraintname = true && (isSetConstraintname());
+    list.add(present_constraintname);
+    if (present_constraintname)
+      list.add(constraintname);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(DropConstraintRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetDbname()).compareTo(other.isSetDbname());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDbname()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbname, other.dbname);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTablename()).compareTo(other.isSetTablename());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTablename()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tablename, other.tablename);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetConstraintname()).compareTo(other.isSetConstraintname());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetConstraintname()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.constraintname, other.constraintname);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("DropConstraintRequest(");
+    boolean first = true;
+
+    sb.append("dbname:");
+    if (this.dbname == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.dbname);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tablename:");
+    if (this.tablename == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tablename);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("constraintname:");
+    if (this.constraintname == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.constraintname);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetDbname()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbname' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTablename()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tablename' is unset! Struct:" + toString());
+    }
+
+    if (!isSetConstraintname()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'constraintname' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class DropConstraintRequestStandardSchemeFactory implements SchemeFactory {
+    public DropConstraintRequestStandardScheme getScheme() {
+      return new DropConstraintRequestStandardScheme();
+    }
+  }
+
+  private static class DropConstraintRequestStandardScheme extends StandardScheme<DropConstraintRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DropConstraintRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // DBNAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.dbname = iprot.readString();
+              struct.setDbnameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TABLENAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tablename = iprot.readString();
+              struct.setTablenameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // CONSTRAINTNAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.constraintname = iprot.readString();
+              struct.setConstraintnameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, DropConstraintRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.dbname != null) {
+        oprot.writeFieldBegin(DBNAME_FIELD_DESC);
+        oprot.writeString(struct.dbname);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tablename != null) {
+        oprot.writeFieldBegin(TABLENAME_FIELD_DESC);
+        oprot.writeString(struct.tablename);
+        oprot.writeFieldEnd();
+      }
+      if (struct.constraintname != null) {
+        oprot.writeFieldBegin(CONSTRAINTNAME_FIELD_DESC);
+        oprot.writeString(struct.constraintname);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class DropConstraintRequestTupleSchemeFactory implements SchemeFactory {
+    public DropConstraintRequestTupleScheme getScheme() {
+      return new DropConstraintRequestTupleScheme();
+    }
+  }
+
+  private static class DropConstraintRequestTupleScheme extends TupleScheme<DropConstraintRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, DropConstraintRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.dbname);
+      oprot.writeString(struct.tablename);
+      oprot.writeString(struct.constraintname);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, DropConstraintRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.dbname = iprot.readString();
+      struct.setDbnameIsSet(true);
+      struct.tablename = iprot.readString();
+      struct.setTablenameIsSet(true);
+      struct.constraintname = iprot.readString();
+      struct.setConstraintnameIsSet(true);
+    }
+  }
+
+}
+


[49/50] [abbrv] hive git commit: HIVE-13710 : LLAP registry ACL check causes error due to namespacing (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by sp...@apache.org.
HIVE-13710 : LLAP registry ACL check causes error due to namespacing (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/java8
Commit: 92a9ae80003beb2f94d94f7ed0a1cdf168534930
Parents: f089f2e
Author: Sergey Shelukhin <se...@apache.org>
Authored: Fri May 6 12:27:03 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Fri May 6 12:27:28 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/92a9ae80/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
index fde70e7..cffa493 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
@@ -328,7 +328,7 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
     if (ix > 0) {
       pathToCheck = pathToCheck.substring(0, ix);
     }
-    List<ACL> acls = zooKeeperClient.usingNamespace(null).getACL().forPath(pathToCheck);
+    List<ACL> acls = zooKeeperClient.getACL().forPath(pathToCheck);
     if (acls == null || acls.isEmpty()) {
       // Can there be no ACLs? There's some access (to get ACLs), so assume it means free for all.
       throw new SecurityException("No ACLs on "  + pathToCheck);


[47/50] [abbrv] hive git commit: HIVE-13707: Fix TestLlapOutputFormat

Posted by sp...@apache.org.
HIVE-13707: Fix TestLlapOutputFormat


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

Branch: refs/heads/java8
Commit: 89ec219e170d7ce59bf223cdce2fb521ba0a1518
Parents: 763e696
Author: Jason Dere <jd...@hortonworks.com>
Authored: Fri May 6 10:05:49 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Fri May 6 10:05:49 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/llap/TestLlapOutputFormat.java  | 21 +++++++++++---------
 1 file changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/89ec219e/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
index 907d5b0..1d592fb 100644
--- a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hive.llap;
 
 import org.junit.Assert;
 import org.junit.Test;
-import org.junit.Before;
-import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
 
 import java.net.Socket;
 
@@ -60,18 +60,22 @@ public class TestLlapOutputFormat {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestLlapOutputFormat.class);
 
-  private LlapOutputFormatService service;
+  private static LlapOutputFormatService service;
 
-  @Before
-  public void setUp() throws IOException {
+  @BeforeClass
+  public static void setUp() throws Exception {
     LOG.debug("Setting up output service");
+    Configuration conf = new Configuration();
+    // Pick random avail port
+    HiveConf.setIntVar(conf, HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT, 0);
+    LlapOutputFormatService.initializeAndStart(conf);
     service = LlapOutputFormatService.get();
     LlapProxy.setDaemon(true);
     LOG.debug("Output service up");
   }
 
-  @After
-  public void tearDown() throws IOException, InterruptedException {
+  @AfterClass
+  public static void tearDown() throws IOException, InterruptedException {
     LOG.debug("Tearing down service");
     service.stop();
     LOG.debug("Tearing down complete");
@@ -87,8 +91,7 @@ public class TestLlapOutputFormat {
       LlapOutputFormat format = new LlapOutputFormat();
 
       HiveConf conf = new HiveConf();
-      Socket socket = new Socket("localhost",
-          conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
+      Socket socket = new Socket("localhost", service.getPort());
 
       LOG.debug("Socket connected");
 


[36/50] [abbrv] hive git commit: HIVE-13695: LlapOutputFormatService port should be able to be set via conf

Posted by sp...@apache.org.
HIVE-13695: LlapOutputFormatService port should be able to be set via conf


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

Branch: refs/heads/java8
Commit: 03ee0481a518585a4a92875d88c560ff525d75d4
Parents: 2a03f1f
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu May 5 12:56:20 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu May 5 12:56:20 2016 -0700

----------------------------------------------------------------------
 .../hive/llap/daemon/impl/LlapDaemon.java       |  6 +++
 .../hive/llap/daemon/MiniLlapCluster.java       |  3 ++
 .../hive/llap/LlapOutputFormatService.java      | 44 +++++++++++++-------
 3 files changed, 38 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/03ee0481/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index 223c390..b3c1abf 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -132,6 +132,10 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
         "Work dirs must be specified");
     Preconditions.checkArgument(shufflePort == 0 || (shufflePort > 1024 && shufflePort < 65536),
         "Shuffle Port must be betwee 1024 and 65535, or 0 for automatic selection");
+    int outputFormatServicePort = HiveConf.getIntVar(daemonConf, HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
+    Preconditions.checkArgument(outputFormatServicePort == 0
+        || (outputFormatServicePort > 1024 && outputFormatServicePort < 65536),
+        "OutputFormatService Port must be between 1024 and 65535, or 0 for automatic selection");
     String hosts = HiveConf.getTrimmedVar(daemonConf, ConfVars.LLAP_DAEMON_SERVICE_HOSTS);
     if (hosts.startsWith("@")) {
       String zkHosts = HiveConf.getTrimmedVar(daemonConf, ConfVars.HIVE_ZOOKEEPER_QUORUM);
@@ -165,6 +169,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
         ", rpcListenerPort=" + srvPort +
         ", mngListenerPort=" + mngPort +
         ", webPort=" + webPort +
+        ", outputFormatSvcPort=" + outputFormatServicePort +
         ", workDirs=" + Arrays.toString(localDirs) +
         ", shufflePort=" + shufflePort +
         ", executorMemory=" + executorMemoryBytes +
@@ -335,6 +340,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     this.shufflePort.set(ShuffleHandler.get().getPort());
     getConfig()
         .setInt(ConfVars.LLAP_DAEMON_YARN_SHUFFLE_PORT.varname, ShuffleHandler.get().getPort());
+    LlapOutputFormatService.initializeAndStart(getConfig());
     super.serviceStart();
 
     // Setup the actual ports in the configuration.

http://git-wip-us.apache.org/repos/asf/hive/blob/03ee0481/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
index dde5be0..e394191 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
@@ -166,6 +166,7 @@ public class MiniLlapCluster extends AbstractService {
     int mngPort = 0;
     int shufflePort = 0;
     int webPort = 0;
+    int outputFormatServicePort = 0;
     boolean usePortsFromConf = conf.getBoolean("minillap.usePortsFromConf", false);
     LOG.info("MiniLlap configured to use ports from conf: {}", usePortsFromConf);
     if (usePortsFromConf) {
@@ -173,7 +174,9 @@ public class MiniLlapCluster extends AbstractService {
       mngPort = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_MANAGEMENT_RPC_PORT);
       shufflePort = conf.getInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, ShuffleHandler.DEFAULT_SHUFFLE_PORT);
       webPort = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_WEB_PORT);
+      outputFormatServicePort = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
     }
+    HiveConf.setIntVar(conf, ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT, outputFormatServicePort);
 
     if (ownZkCluster) {
       miniZooKeeperCluster = new MiniZooKeeperCluster();

http://git-wip-us.apache.org/repos/asf/hive/blob/03ee0481/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
index 6adbf7c..f852041 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
@@ -25,6 +25,7 @@ import java.net.InetSocketAddress;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.BytesWritable;
@@ -39,9 +40,13 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
+
+import com.google.common.base.Preconditions;
+
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.ChannelFuture;
@@ -67,9 +72,12 @@ public class LlapOutputFormatService {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapOutputFormat.class);
 
-  private static LlapOutputFormatService service;
+  private static final AtomicBoolean started = new AtomicBoolean(false);
+  private static final AtomicBoolean initing = new AtomicBoolean(false);
+  private static LlapOutputFormatService INSTANCE;
+
   private final Map<String, RecordWriter> writers;
-  private final HiveConf conf;
+  private final Configuration conf;
   private static final int WAIT_TIME = 5;
   private static final int MAX_QUERY_ID_LENGTH = 256;
 
@@ -78,23 +86,29 @@ public class LlapOutputFormatService {
   private ChannelFuture listeningChannelFuture;
   private int port;
 
-  private LlapOutputFormatService() throws IOException {
+  private LlapOutputFormatService(Configuration conf) throws IOException {
     writers = new HashMap<String, RecordWriter>();
-    conf = new HiveConf();
+    this.conf = conf;
   }
 
-  public static LlapOutputFormatService get() throws IOException {
-    if (service == null) {
-      service = new LlapOutputFormatService();
-      service.start();
+  public static void initializeAndStart(Configuration conf) throws Exception {
+    if (!initing.getAndSet(true)) {
+      INSTANCE = new LlapOutputFormatService(conf);
+      INSTANCE.start();
+      started.set(true);
     }
-    return service;
+  }
+
+  public static LlapOutputFormatService get() throws IOException {
+    Preconditions.checkState(started.get(),
+        "LlapOutputFormatService must be started before invoking get");
+    return INSTANCE;
   }
 
   public void start() throws IOException {
     LOG.info("Starting LlapOutputFormatService");
 
-    int portFromConf = conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
+    int portFromConf = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
     eventLoopGroup = new NioEventLoopGroup(1);
     serverBootstrap = new ServerBootstrap();
     serverBootstrap.group(eventLoopGroup);
@@ -125,10 +139,10 @@ public class LlapOutputFormatService {
 
   public <K,V> RecordWriter<K, V> getWriter(String id) throws IOException, InterruptedException {
     RecordWriter writer = null;
-    synchronized(service) {
+    synchronized(INSTANCE) {
       while ((writer = writers.get(id)) == null) {
         LOG.info("Waiting for writer for: "+id);
-        service.wait();
+        INSTANCE.wait();
       }
     }
     LOG.info("Returning writer for: "+id);
@@ -147,7 +161,7 @@ public class LlapOutputFormatService {
     }
 
     private void registerReader(ChannelHandlerContext ctx, String id) {
-      synchronized(service) {
+      synchronized(INSTANCE) {
         LOG.debug("registering socket for: "+id);
         int bufSize = 128 * 1024; // configable?
         OutputStream stream = new ChannelOutputStream(ctx, id, bufSize);
@@ -157,7 +171,7 @@ public class LlapOutputFormatService {
         // Add listener to handle any cleanup for when the connection is closed
         ctx.channel().closeFuture().addListener(new LlapOutputFormatChannelCloseListener(id));
 
-        service.notifyAll();
+        INSTANCE.notifyAll();
       }
     }
   }
@@ -173,7 +187,7 @@ public class LlapOutputFormatService {
     public void operationComplete(ChannelFuture future) throws Exception {
       RecordWriter writer = null;
 
-      synchronized (service) {
+      synchronized (INSTANCE) {
         writer = writers.remove(id);
       }
 


[40/50] [abbrv] hive git commit: HIVE-13393: Beeline: Print help message for the --incremental option (Vaibhav Gumashta reviewed by Thejas Nair)

Posted by sp...@apache.org.
HIVE-13393: Beeline: Print help message for the --incremental option (Vaibhav Gumashta reviewed by Thejas Nair)


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

Branch: refs/heads/java8
Commit: 794f161c136c4d99693eb60222c0f17b10948e0d
Parents: 4eb9603
Author: Vaibhav Gumashta <vg...@hortonworks.com>
Authored: Thu May 5 15:12:38 2016 -0700
Committer: Vaibhav Gumashta <vg...@hortonworks.com>
Committed: Thu May 5 15:12:38 2016 -0700

----------------------------------------------------------------------
 beeline/src/main/resources/BeeLine.properties | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/794f161c/beeline/src/main/resources/BeeLine.properties
----------------------------------------------------------------------
diff --git a/beeline/src/main/resources/BeeLine.properties b/beeline/src/main/resources/BeeLine.properties
index a118c09..bc40685 100644
--- a/beeline/src/main/resources/BeeLine.properties
+++ b/beeline/src/main/resources/BeeLine.properties
@@ -171,7 +171,14 @@ cmd-usage: Usage: java org.apache.hive.cli.beeline.BeeLine \n \
 \  --silent=[true/false]           be more silent\n \
 \  --autosave=[true/false]         automatically save preferences\n \
 \  --outputformat=[table/vertical/csv2/tsv2/dsv/csv/tsv]  format mode for result display\n \
-\                                  Note that csv, and tsv are deprecated - use csv2, tsv2 instead\n\
+\                                  Note that csv, and tsv are deprecated - use csv2, tsv2 instead\n \
+\  --incremental=[true/false]      Defaults to false. When set to false, the entire result set\n \
+\                                  is fetched and buffered before being displayed, yielding optimal\n \
+\                                  display column sizing. When set to true, result rows are displayed\n \
+\                                  immediately as they are fetched, yielding lower latency and\n \
+\                                  memory usage at the price of extra display column padding.\n \
+\                                  Setting --incremental=true is recommended if you encounter an OutOfMemory\n \
+\                                  on the client side (due to the fetched result set size being large).\n \
 \  --truncateTable=[true/false]    truncate table column when it exceeds length\n \
 \  --delimiterForDSV=DELIMITER     specify the delimiter for delimiter-separated values output format (default: |)\n \
 \  --isolation=LEVEL               set the transaction isolation level\n \


[08/50] [abbrv] hive git commit: HIVE-13442 : LLAP: refactor submit API to be amenable to signing (Sergey Shelukhin, reviewed by Siddharth Seth)

Posted by sp...@apache.org.
HIVE-13442 : LLAP: refactor submit API to be amenable to signing (Sergey Shelukhin, reviewed by Siddharth Seth)


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

Branch: refs/heads/java8
Commit: 0b5c27fdd4fbf8861d4eefc207c2da3a6ceac23d
Parents: b70efa4
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue May 3 15:02:24 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue May 3 15:23:39 2016 -0700

----------------------------------------------------------------------
 .../daemon/rpc/LlapDaemonProtocolProtos.java    | 6836 +++++++++++-------
 .../apache/hadoop/hive/llap/tez/Converters.java |   84 +-
 .../src/protobuf/LlapDaemonProtocol.proto       |   69 +-
 .../hadoop/hive/llap/tez/TestConverters.java    |   51 +-
 .../llap/daemon/impl/ContainerRunnerImpl.java   |   88 +-
 .../llap/daemon/impl/QueryFragmentInfo.java     |   23 +-
 .../hadoop/hive/llap/daemon/impl/QueryInfo.java |    9 +-
 .../hive/llap/daemon/impl/QueryTracker.java     |   18 +-
 .../llap/daemon/impl/TaskExecutorService.java   |    8 +-
 .../llap/daemon/impl/TaskRunnerCallable.java    |   77 +-
 .../daemon/impl/TaskExecutorTestHelpers.java    |   42 +-
 .../TestFirstInFirstOutComparator.java          |   27 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |   31 +-
 13 files changed, 4504 insertions(+), 2859 deletions(-)
----------------------------------------------------------------------



[19/50] [abbrv] hive git commit: HIVE-13351: Support drop Primary Key/Foreign Key constraints (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 11d3322..990be15 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -41,6 +41,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void create_table(const Table& tbl) = 0;
   virtual void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) = 0;
   virtual void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys) = 0;
+  virtual void drop_constraint(const DropConstraintRequest& req) = 0;
   virtual void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) = 0;
   virtual void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) = 0;
   virtual void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern) = 0;
@@ -256,6 +257,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void create_table_with_constraints(const Table& /* tbl */, const std::vector<SQLPrimaryKey> & /* primaryKeys */, const std::vector<SQLForeignKey> & /* foreignKeys */) {
     return;
   }
+  void drop_constraint(const DropConstraintRequest& /* req */) {
+    return;
+  }
   void drop_table(const std::string& /* dbname */, const std::string& /* name */, const bool /* deleteData */) {
     return;
   }
@@ -3032,6 +3036,118 @@ class ThriftHiveMetastore_create_table_with_constraints_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_drop_constraint_args__isset {
+  _ThriftHiveMetastore_drop_constraint_args__isset() : req(false) {}
+  bool req :1;
+} _ThriftHiveMetastore_drop_constraint_args__isset;
+
+class ThriftHiveMetastore_drop_constraint_args {
+ public:
+
+  ThriftHiveMetastore_drop_constraint_args(const ThriftHiveMetastore_drop_constraint_args&);
+  ThriftHiveMetastore_drop_constraint_args& operator=(const ThriftHiveMetastore_drop_constraint_args&);
+  ThriftHiveMetastore_drop_constraint_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_drop_constraint_args() throw();
+  DropConstraintRequest req;
+
+  _ThriftHiveMetastore_drop_constraint_args__isset __isset;
+
+  void __set_req(const DropConstraintRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_drop_constraint_args & rhs) const
+  {
+    if (!(req == rhs.req))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_drop_constraint_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_drop_constraint_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_drop_constraint_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_drop_constraint_pargs() throw();
+  const DropConstraintRequest* req;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_constraint_result__isset {
+  _ThriftHiveMetastore_drop_constraint_result__isset() : o1(false), o3(false) {}
+  bool o1 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_drop_constraint_result__isset;
+
+class ThriftHiveMetastore_drop_constraint_result {
+ public:
+
+  ThriftHiveMetastore_drop_constraint_result(const ThriftHiveMetastore_drop_constraint_result&);
+  ThriftHiveMetastore_drop_constraint_result& operator=(const ThriftHiveMetastore_drop_constraint_result&);
+  ThriftHiveMetastore_drop_constraint_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_drop_constraint_result() throw();
+  NoSuchObjectException o1;
+  MetaException o3;
+
+  _ThriftHiveMetastore_drop_constraint_result__isset __isset;
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o3(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_drop_constraint_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o3 == rhs.o3))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_drop_constraint_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_drop_constraint_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_constraint_presult__isset {
+  _ThriftHiveMetastore_drop_constraint_presult__isset() : o1(false), o3(false) {}
+  bool o1 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_drop_constraint_presult__isset;
+
+class ThriftHiveMetastore_drop_constraint_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_drop_constraint_presult() throw();
+  NoSuchObjectException o1;
+  MetaException o3;
+
+  _ThriftHiveMetastore_drop_constraint_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_drop_table_args__isset {
   _ThriftHiveMetastore_drop_table_args__isset() : dbname(false), name(false), deleteData(false) {}
   bool dbname :1;
@@ -18851,6 +18967,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys);
   void send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys);
   void recv_create_table_with_constraints();
+  void drop_constraint(const DropConstraintRequest& req);
+  void send_drop_constraint(const DropConstraintRequest& req);
+  void recv_drop_constraint();
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   void send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   void recv_drop_table();
@@ -19261,6 +19380,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_create_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_create_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_create_table_with_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_drop_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -19411,6 +19531,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["create_table"] = &ThriftHiveMetastoreProcessor::process_create_table;
     processMap_["create_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_create_table_with_environment_context;
     processMap_["create_table_with_constraints"] = &ThriftHiveMetastoreProcessor::process_create_table_with_constraints;
+    processMap_["drop_constraint"] = &ThriftHiveMetastoreProcessor::process_drop_constraint;
     processMap_["drop_table"] = &ThriftHiveMetastoreProcessor::process_drop_table;
     processMap_["drop_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context;
     processMap_["get_tables"] = &ThriftHiveMetastoreProcessor::process_get_tables;
@@ -19752,6 +19873,15 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys);
   }
 
+  void drop_constraint(const DropConstraintRequest& req) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->drop_constraint(req);
+    }
+    ifaces_[i]->drop_constraint(req);
+  }
+
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -21045,6 +21175,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys);
   int32_t send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys);
   void recv_create_table_with_constraints(const int32_t seqid);
+  void drop_constraint(const DropConstraintRequest& req);
+  int32_t send_drop_constraint(const DropConstraintRequest& req);
+  void recv_drop_constraint(const int32_t seqid);
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   int32_t send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   void recv_drop_table(const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index fa87e34..2d13e77 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -117,6 +117,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("create_table_with_constraints\n");
   }
 
+  void drop_constraint(const DropConstraintRequest& req) {
+    // Your implementation goes here
+    printf("drop_constraint\n");
+  }
+
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) {
     // Your implementation goes here
     printf("drop_table\n");


[42/50] [abbrv] hive git commit: HIVE-13656 : need to set direct memory limit higher in LlapServiceDriver for certain edge case configurations (Sergey Shelukhin, reviewed by Vikram Dixit K and Siddharth Seth)

Posted by sp...@apache.org.
HIVE-13656 : need to set direct memory limit higher in LlapServiceDriver for certain edge case configurations (Sergey Shelukhin, reviewed by Vikram Dixit K and Siddharth Seth)


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

Branch: refs/heads/java8
Commit: 3517a99edde061596d62b41339bacb5aac0e8290
Parents: eb2c54b
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu May 5 17:01:47 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu May 5 17:02:36 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/llap/cli/LlapServiceDriver.java | 21 +++++++++++---------
 llap-server/src/main/resources/package.py       |  6 +++++-
 2 files changed, 17 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3517a99e/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
index de6d9b8..006f70f 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
@@ -236,20 +236,22 @@ public class LlapServiceDriver {
           String.valueOf(options.getIoThreads()));
     }
 
+    long cache = -1, xmx = -1;
     if (options.getCache() != -1) {
-      conf.set(HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname,
-          Long.toString(options.getCache()));
+      cache = options.getCache();
+      conf.set(HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname, Long.toString(cache));
       propsDirectOptions.setProperty(HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname,
-          Long.toString(options.getCache()));
+          Long.toString(cache));
     }
 
     if (options.getXmx() != -1) {
       // Needs more explanation here
-      // Xmx is not the max heap value in JDK8
-      // You need to subtract 50% of the survivor fraction from this, to get actual usable memory before it goes into GC
-      long xmx = (long) (options.getXmx() / (1024 * 1024));
+      // Xmx is not the max heap value in JDK8. You need to subtract 50% of the survivor fraction
+      // from this, to get actual usable  memory before it goes into GC
+      xmx = (long) (options.getXmx() / (1024 * 1024));
       conf.setLong(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname, xmx);
-      propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname, String.valueOf(xmx));
+      propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname,
+          String.valueOf(xmx));
     }
 
     if (options.getLlapQueueName() != null && !options.getLlapQueueName().isEmpty()) {
@@ -258,8 +260,6 @@ public class LlapServiceDriver {
           .setProperty(ConfVars.LLAP_DAEMON_QUEUE_NAME.varname, options.getLlapQueueName());
     }
 
-
-
     URL logger = conf.getResource(LlapDaemon.LOG4j2_PROPERTIES_FILE);
 
     if (null == logger) {
@@ -460,6 +460,9 @@ public class LlapServiceDriver {
     configs.put(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
         conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, -1));
 
+    long maxDirect = (xmx > 0 && cache > 0 && xmx < cache * 1.25) ? (long)(cache * 1.25) : -1;
+    configs.put("max_direct_memory", Long.toString(maxDirect));
+
     FSDataOutputStream os = lfs.create(new Path(tmpDir, "config.json"));
     OutputStreamWriter w = new OutputStreamWriter(os);
     configs.write(w);

http://git-wip-us.apache.org/repos/asf/hive/blob/3517a99e/llap-server/src/main/resources/package.py
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/package.py b/llap-server/src/main/resources/package.py
index 63c0ef1..94c9d1a 100644
--- a/llap-server/src/main/resources/package.py
+++ b/llap-server/src/main/resources/package.py
@@ -101,6 +101,10 @@ def main(args):
 		return
 	config = json_parse(open(join(input, "config.json")).read())
 	java_home = config["java.home"]
+	max_direct_memory = config["max_direct_memory"]
+	daemon_args = args.args
+	if max_direct_memory > 0:
+		daemon_args = " -XX:MaxDirectMemorySize=%s %s" % (max_direct_memory, daemon_args)
 	resource = LlapResource(config)
 	# 5% container failure every monkey_interval seconds
 	monkey_percentage = 5 # 5%
@@ -114,7 +118,7 @@ def main(args):
 		"hadoop_home" : os.getenv("HADOOP_HOME"),
 		"java_home" : java_home,
 		"name" : resource.clusterName,
-		"daemon_args" : args.args,
+		"daemon_args" : daemon_args,
 		"daemon_loglevel" : args.loglevel,
 		"queue.string" : resource.queueString,
 		"monkey_interval" : args.chaosmonkey,


[16/50] [abbrv] hive git commit: HIVE-13351: Support drop Primary Key/Foreign Key constraints (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 051c1f2..2a81c4b 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -80,6 +80,8 @@ public class ThriftHiveMetastore {
 
     public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException;
 
+    public void drop_constraint(DropConstraintRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
     public void drop_table(String dbname, String name, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
 
     public void drop_table_with_environment_context(String dbname, String name, boolean deleteData, EnvironmentContext environment_context) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
@@ -376,6 +378,8 @@ public class ThriftHiveMetastore {
 
     public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void drop_constraint(DropConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void drop_table(String dbname, String name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void drop_table_with_environment_context(String dbname, String name, boolean deleteData, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -1218,6 +1222,32 @@ public class ThriftHiveMetastore {
       return;
     }
 
+    public void drop_constraint(DropConstraintRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_drop_constraint(req);
+      recv_drop_constraint();
+    }
+
+    public void send_drop_constraint(DropConstraintRequest req) throws org.apache.thrift.TException
+    {
+      drop_constraint_args args = new drop_constraint_args();
+      args.setReq(req);
+      sendBase("drop_constraint", args);
+    }
+
+    public void recv_drop_constraint() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      drop_constraint_result result = new drop_constraint_result();
+      receiveBase(result, "drop_constraint");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o3 != null) {
+        throw result.o3;
+      }
+      return;
+    }
+
     public void drop_table(String dbname, String name, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
     {
       send_drop_table(dbname, name, deleteData);
@@ -5535,6 +5565,38 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void drop_constraint(DropConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      drop_constraint_call method_call = new drop_constraint_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class drop_constraint_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private DropConstraintRequest req;
+      public drop_constraint_call(DropConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.req = req;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("drop_constraint", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        drop_constraint_args args = new drop_constraint_args();
+        args.setReq(req);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_drop_constraint();
+      }
+    }
+
     public void drop_table(String dbname, String name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       drop_table_call method_call = new drop_table_call(dbname, name, deleteData, resultHandler, this, ___protocolFactory, ___transport);
@@ -10078,6 +10140,7 @@ public class ThriftHiveMetastore {
       processMap.put("create_table", new create_table());
       processMap.put("create_table_with_environment_context", new create_table_with_environment_context());
       processMap.put("create_table_with_constraints", new create_table_with_constraints());
+      processMap.put("drop_constraint", new drop_constraint());
       processMap.put("drop_table", new drop_table());
       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
       processMap.put("get_tables", new get_tables());
@@ -10720,6 +10783,32 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class drop_constraint<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drop_constraint_args> {
+      public drop_constraint() {
+        super("drop_constraint");
+      }
+
+      public drop_constraint_args getEmptyArgsInstance() {
+        return new drop_constraint_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public drop_constraint_result getResult(I iface, drop_constraint_args args) throws org.apache.thrift.TException {
+        drop_constraint_result result = new drop_constraint_result();
+        try {
+          iface.drop_constraint(args.req);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o3) {
+          result.o3 = o3;
+        }
+        return result;
+      }
+    }
+
     public static class drop_table<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drop_table_args> {
       public drop_table() {
         super("drop_table");
@@ -13964,6 +14053,7 @@ public class ThriftHiveMetastore {
       processMap.put("create_table", new create_table());
       processMap.put("create_table_with_environment_context", new create_table_with_environment_context());
       processMap.put("create_table_with_constraints", new create_table_with_constraints());
+      processMap.put("drop_constraint", new drop_constraint());
       processMap.put("drop_table", new drop_table());
       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
       processMap.put("get_tables", new get_tables());
@@ -15307,20 +15397,20 @@ public class ThriftHiveMetastore {
       }
     }
 
-    public static class drop_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_table_args, Void> {
-      public drop_table() {
-        super("drop_table");
+    public static class drop_constraint<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_constraint_args, Void> {
+      public drop_constraint() {
+        super("drop_constraint");
       }
 
-      public drop_table_args getEmptyArgsInstance() {
-        return new drop_table_args();
+      public drop_constraint_args getEmptyArgsInstance() {
+        return new drop_constraint_args();
       }
 
       public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
         return new AsyncMethodCallback<Void>() { 
           public void onComplete(Void o) {
-            drop_table_result result = new drop_table_result();
+            drop_constraint_result result = new drop_constraint_result();
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -15332,7 +15422,7 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            drop_table_result result = new drop_table_result();
+            drop_constraint_result result = new drop_constraint_result();
             if (e instanceof NoSuchObjectException) {
                         result.o1 = (NoSuchObjectException) e;
                         result.setO1IsSet(true);
@@ -15363,25 +15453,25 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, drop_table_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.drop_table(args.dbname, args.name, args.deleteData,resultHandler);
+      public void start(I iface, drop_constraint_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.drop_constraint(args.req,resultHandler);
       }
     }
 
-    public static class drop_table_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_table_with_environment_context_args, Void> {
-      public drop_table_with_environment_context() {
-        super("drop_table_with_environment_context");
+    public static class drop_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_table_args, Void> {
+      public drop_table() {
+        super("drop_table");
       }
 
-      public drop_table_with_environment_context_args getEmptyArgsInstance() {
-        return new drop_table_with_environment_context_args();
+      public drop_table_args getEmptyArgsInstance() {
+        return new drop_table_args();
       }
 
       public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
         return new AsyncMethodCallback<Void>() { 
           public void onComplete(Void o) {
-            drop_table_with_environment_context_result result = new drop_table_with_environment_context_result();
+            drop_table_result result = new drop_table_result();
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -15393,7 +15483,7 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            drop_table_with_environment_context_result result = new drop_table_with_environment_context_result();
+            drop_table_result result = new drop_table_result();
             if (e instanceof NoSuchObjectException) {
                         result.o1 = (NoSuchObjectException) e;
                         result.setO1IsSet(true);
@@ -15424,259 +15514,25 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, drop_table_with_environment_context_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.drop_table_with_environment_context(args.dbname, args.name, args.deleteData, args.environment_context,resultHandler);
-      }
-    }
-
-    public static class get_tables<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_tables_args, List<String>> {
-      public get_tables() {
-        super("get_tables");
-      }
-
-      public get_tables_args getEmptyArgsInstance() {
-        return new get_tables_args();
-      }
-
-      public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<List<String>>() { 
-          public void onComplete(List<String> o) {
-            get_tables_result result = new get_tables_result();
-            result.success = o;
-            try {
-              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-              return;
-            } catch (Exception e) {
-              LOGGER.error("Exception writing to internal frame buffer", e);
-            }
-            fb.close();
-          }
-          public void onError(Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TBase msg;
-            get_tables_result result = new get_tables_result();
-            if (e instanceof MetaException) {
-                        result.o1 = (MetaException) e;
-                        result.setO1IsSet(true);
-                        msg = result;
-            }
-             else 
-            {
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-              return;
-            } catch (Exception ex) {
-              LOGGER.error("Exception writing to internal frame buffer", ex);
-            }
-            fb.close();
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, get_tables_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
-        iface.get_tables(args.db_name, args.pattern,resultHandler);
-      }
-    }
-
-    public static class get_table_meta<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_meta_args, List<TableMeta>> {
-      public get_table_meta() {
-        super("get_table_meta");
-      }
-
-      public get_table_meta_args getEmptyArgsInstance() {
-        return new get_table_meta_args();
-      }
-
-      public AsyncMethodCallback<List<TableMeta>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<List<TableMeta>>() { 
-          public void onComplete(List<TableMeta> o) {
-            get_table_meta_result result = new get_table_meta_result();
-            result.success = o;
-            try {
-              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-              return;
-            } catch (Exception e) {
-              LOGGER.error("Exception writing to internal frame buffer", e);
-            }
-            fb.close();
-          }
-          public void onError(Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TBase msg;
-            get_table_meta_result result = new get_table_meta_result();
-            if (e instanceof MetaException) {
-                        result.o1 = (MetaException) e;
-                        result.setO1IsSet(true);
-                        msg = result;
-            }
-             else 
-            {
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-              return;
-            } catch (Exception ex) {
-              LOGGER.error("Exception writing to internal frame buffer", ex);
-            }
-            fb.close();
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, get_table_meta_args args, org.apache.thrift.async.AsyncMethodCallback<List<TableMeta>> resultHandler) throws TException {
-        iface.get_table_meta(args.db_patterns, args.tbl_patterns, args.tbl_types,resultHandler);
-      }
-    }
-
-    public static class get_all_tables<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_all_tables_args, List<String>> {
-      public get_all_tables() {
-        super("get_all_tables");
-      }
-
-      public get_all_tables_args getEmptyArgsInstance() {
-        return new get_all_tables_args();
-      }
-
-      public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<List<String>>() { 
-          public void onComplete(List<String> o) {
-            get_all_tables_result result = new get_all_tables_result();
-            result.success = o;
-            try {
-              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-              return;
-            } catch (Exception e) {
-              LOGGER.error("Exception writing to internal frame buffer", e);
-            }
-            fb.close();
-          }
-          public void onError(Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TBase msg;
-            get_all_tables_result result = new get_all_tables_result();
-            if (e instanceof MetaException) {
-                        result.o1 = (MetaException) e;
-                        result.setO1IsSet(true);
-                        msg = result;
-            }
-             else 
-            {
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-              return;
-            } catch (Exception ex) {
-              LOGGER.error("Exception writing to internal frame buffer", ex);
-            }
-            fb.close();
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, get_all_tables_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
-        iface.get_all_tables(args.db_name,resultHandler);
-      }
-    }
-
-    public static class get_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_args, Table> {
-      public get_table() {
-        super("get_table");
-      }
-
-      public get_table_args getEmptyArgsInstance() {
-        return new get_table_args();
-      }
-
-      public AsyncMethodCallback<Table> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Table>() { 
-          public void onComplete(Table o) {
-            get_table_result result = new get_table_result();
-            result.success = o;
-            try {
-              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-              return;
-            } catch (Exception e) {
-              LOGGER.error("Exception writing to internal frame buffer", e);
-            }
-            fb.close();
-          }
-          public void onError(Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TBase msg;
-            get_table_result result = new get_table_result();
-            if (e instanceof MetaException) {
-                        result.o1 = (MetaException) e;
-                        result.setO1IsSet(true);
-                        msg = result;
-            }
-            else             if (e instanceof NoSuchObjectException) {
-                        result.o2 = (NoSuchObjectException) e;
-                        result.setO2IsSet(true);
-                        msg = result;
-            }
-             else 
-            {
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-              return;
-            } catch (Exception ex) {
-              LOGGER.error("Exception writing to internal frame buffer", ex);
-            }
-            fb.close();
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, get_table_args args, org.apache.thrift.async.AsyncMethodCallback<Table> resultHandler) throws TException {
-        iface.get_table(args.dbname, args.tbl_name,resultHandler);
+      public void start(I iface, drop_table_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.drop_table(args.dbname, args.name, args.deleteData,resultHandler);
       }
     }
 
-    public static class get_table_objects_by_name<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_objects_by_name_args, List<Table>> {
-      public get_table_objects_by_name() {
-        super("get_table_objects_by_name");
+    public static class drop_table_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_table_with_environment_context_args, Void> {
+      public drop_table_with_environment_context() {
+        super("drop_table_with_environment_context");
       }
 
-      public get_table_objects_by_name_args getEmptyArgsInstance() {
-        return new get_table_objects_by_name_args();
+      public drop_table_with_environment_context_args getEmptyArgsInstance() {
+        return new drop_table_with_environment_context_args();
       }
 
-      public AsyncMethodCallback<List<Table>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<List<Table>>() { 
-          public void onComplete(List<Table> o) {
-            get_table_objects_by_name_result result = new get_table_objects_by_name_result();
-            result.success = o;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            drop_table_with_environment_context_result result = new drop_table_with_environment_context_result();
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -15688,19 +15544,14 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            get_table_objects_by_name_result result = new get_table_objects_by_name_result();
-            if (e instanceof MetaException) {
-                        result.o1 = (MetaException) e;
+            drop_table_with_environment_context_result result = new drop_table_with_environment_context_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof InvalidOperationException) {
-                        result.o2 = (InvalidOperationException) e;
-                        result.setO2IsSet(true);
-                        msg = result;
-            }
-            else             if (e instanceof UnknownDBException) {
-                        result.o3 = (UnknownDBException) e;
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
                         result.setO3IsSet(true);
                         msg = result;
             }
@@ -15724,25 +15575,25 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, get_table_objects_by_name_args args, org.apache.thrift.async.AsyncMethodCallback<List<Table>> resultHandler) throws TException {
-        iface.get_table_objects_by_name(args.dbname, args.tbl_names,resultHandler);
+      public void start(I iface, drop_table_with_environment_context_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.drop_table_with_environment_context(args.dbname, args.name, args.deleteData, args.environment_context,resultHandler);
       }
     }
 
-    public static class get_table_names_by_filter<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_names_by_filter_args, List<String>> {
-      public get_table_names_by_filter() {
-        super("get_table_names_by_filter");
+    public static class get_tables<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_tables_args, List<String>> {
+      public get_tables() {
+        super("get_tables");
       }
 
-      public get_table_names_by_filter_args getEmptyArgsInstance() {
-        return new get_table_names_by_filter_args();
+      public get_tables_args getEmptyArgsInstance() {
+        return new get_tables_args();
       }
 
       public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
         return new AsyncMethodCallback<List<String>>() { 
           public void onComplete(List<String> o) {
-            get_table_names_by_filter_result result = new get_table_names_by_filter_result();
+            get_tables_result result = new get_tables_result();
             result.success = o;
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -15755,22 +15606,12 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            get_table_names_by_filter_result result = new get_table_names_by_filter_result();
+            get_tables_result result = new get_tables_result();
             if (e instanceof MetaException) {
                         result.o1 = (MetaException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof InvalidOperationException) {
-                        result.o2 = (InvalidOperationException) e;
-                        result.setO2IsSet(true);
-                        msg = result;
-            }
-            else             if (e instanceof UnknownDBException) {
-                        result.o3 = (UnknownDBException) e;
-                        result.setO3IsSet(true);
-                        msg = result;
-            }
              else 
             {
               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
@@ -15791,25 +15632,26 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, get_table_names_by_filter_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
-        iface.get_table_names_by_filter(args.dbname, args.filter, args.max_tables,resultHandler);
+      public void start(I iface, get_tables_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
+        iface.get_tables(args.db_name, args.pattern,resultHandler);
       }
     }
 
-    public static class alter_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_table_args, Void> {
-      public alter_table() {
-        super("alter_table");
+    public static class get_table_meta<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_meta_args, List<TableMeta>> {
+      public get_table_meta() {
+        super("get_table_meta");
       }
 
-      public alter_table_args getEmptyArgsInstance() {
-        return new alter_table_args();
+      public get_table_meta_args getEmptyArgsInstance() {
+        return new get_table_meta_args();
       }
 
-      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<List<TableMeta>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
-            alter_table_result result = new alter_table_result();
+        return new AsyncMethodCallback<List<TableMeta>>() { 
+          public void onComplete(List<TableMeta> o) {
+            get_table_meta_result result = new get_table_meta_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -15821,17 +15663,12 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            alter_table_result result = new alter_table_result();
-            if (e instanceof InvalidOperationException) {
-                        result.o1 = (InvalidOperationException) e;
+            get_table_meta_result result = new get_table_meta_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof MetaException) {
-                        result.o2 = (MetaException) e;
-                        result.setO2IsSet(true);
-                        msg = result;
-            }
              else 
             {
               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
@@ -15852,25 +15689,26 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, alter_table_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.alter_table(args.dbname, args.tbl_name, args.new_tbl,resultHandler);
+      public void start(I iface, get_table_meta_args args, org.apache.thrift.async.AsyncMethodCallback<List<TableMeta>> resultHandler) throws TException {
+        iface.get_table_meta(args.db_patterns, args.tbl_patterns, args.tbl_types,resultHandler);
       }
     }
 
-    public static class alter_table_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_table_with_environment_context_args, Void> {
-      public alter_table_with_environment_context() {
-        super("alter_table_with_environment_context");
+    public static class get_all_tables<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_all_tables_args, List<String>> {
+      public get_all_tables() {
+        super("get_all_tables");
       }
 
-      public alter_table_with_environment_context_args getEmptyArgsInstance() {
-        return new alter_table_with_environment_context_args();
+      public get_all_tables_args getEmptyArgsInstance() {
+        return new get_all_tables_args();
       }
 
-      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
-            alter_table_with_environment_context_result result = new alter_table_with_environment_context_result();
+        return new AsyncMethodCallback<List<String>>() { 
+          public void onComplete(List<String> o) {
+            get_all_tables_result result = new get_all_tables_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -15882,17 +15720,12 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            alter_table_with_environment_context_result result = new alter_table_with_environment_context_result();
-            if (e instanceof InvalidOperationException) {
-                        result.o1 = (InvalidOperationException) e;
+            get_all_tables_result result = new get_all_tables_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof MetaException) {
-                        result.o2 = (MetaException) e;
-                        result.setO2IsSet(true);
-                        msg = result;
-            }
              else 
             {
               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
@@ -15913,25 +15746,26 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, alter_table_with_environment_context_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.alter_table_with_environment_context(args.dbname, args.tbl_name, args.new_tbl, args.environment_context,resultHandler);
+      public void start(I iface, get_all_tables_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
+        iface.get_all_tables(args.db_name,resultHandler);
       }
     }
 
-    public static class alter_table_with_cascade<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_table_with_cascade_args, Void> {
-      public alter_table_with_cascade() {
-        super("alter_table_with_cascade");
+    public static class get_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_args, Table> {
+      public get_table() {
+        super("get_table");
       }
 
-      public alter_table_with_cascade_args getEmptyArgsInstance() {
-        return new alter_table_with_cascade_args();
+      public get_table_args getEmptyArgsInstance() {
+        return new get_table_args();
       }
 
-      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Table> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
-            alter_table_with_cascade_result result = new alter_table_with_cascade_result();
+        return new AsyncMethodCallback<Table>() { 
+          public void onComplete(Table o) {
+            get_table_result result = new get_table_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -15943,14 +15777,14 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            alter_table_with_cascade_result result = new alter_table_with_cascade_result();
-            if (e instanceof InvalidOperationException) {
-                        result.o1 = (InvalidOperationException) e;
+            get_table_result result = new get_table_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof MetaException) {
-                        result.o2 = (MetaException) e;
+            else             if (e instanceof NoSuchObjectException) {
+                        result.o2 = (NoSuchObjectException) e;
                         result.setO2IsSet(true);
                         msg = result;
             }
@@ -15974,25 +15808,25 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, alter_table_with_cascade_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.alter_table_with_cascade(args.dbname, args.tbl_name, args.new_tbl, args.cascade,resultHandler);
+      public void start(I iface, get_table_args args, org.apache.thrift.async.AsyncMethodCallback<Table> resultHandler) throws TException {
+        iface.get_table(args.dbname, args.tbl_name,resultHandler);
       }
     }
 
-    public static class add_partition<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_partition_args, Partition> {
-      public add_partition() {
-        super("add_partition");
+    public static class get_table_objects_by_name<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_objects_by_name_args, List<Table>> {
+      public get_table_objects_by_name() {
+        super("get_table_objects_by_name");
       }
 
-      public add_partition_args getEmptyArgsInstance() {
-        return new add_partition_args();
+      public get_table_objects_by_name_args getEmptyArgsInstance() {
+        return new get_table_objects_by_name_args();
       }
 
-      public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<List<Table>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Partition>() { 
-          public void onComplete(Partition o) {
-            add_partition_result result = new add_partition_result();
+        return new AsyncMethodCallback<List<Table>>() { 
+          public void onComplete(List<Table> o) {
+            get_table_objects_by_name_result result = new get_table_objects_by_name_result();
             result.success = o;
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -16005,19 +15839,19 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            add_partition_result result = new add_partition_result();
-            if (e instanceof InvalidObjectException) {
-                        result.o1 = (InvalidObjectException) e;
+            get_table_objects_by_name_result result = new get_table_objects_by_name_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof AlreadyExistsException) {
-                        result.o2 = (AlreadyExistsException) e;
+            else             if (e instanceof InvalidOperationException) {
+                        result.o2 = (InvalidOperationException) e;
                         result.setO2IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof MetaException) {
-                        result.o3 = (MetaException) e;
+            else             if (e instanceof UnknownDBException) {
+                        result.o3 = (UnknownDBException) e;
                         result.setO3IsSet(true);
                         msg = result;
             }
@@ -16041,25 +15875,25 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, add_partition_args args, org.apache.thrift.async.AsyncMethodCallback<Partition> resultHandler) throws TException {
-        iface.add_partition(args.new_part,resultHandler);
+      public void start(I iface, get_table_objects_by_name_args args, org.apache.thrift.async.AsyncMethodCallback<List<Table>> resultHandler) throws TException {
+        iface.get_table_objects_by_name(args.dbname, args.tbl_names,resultHandler);
       }
     }
 
-    public static class add_partition_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_partition_with_environment_context_args, Partition> {
-      public add_partition_with_environment_context() {
-        super("add_partition_with_environment_context");
+    public static class get_table_names_by_filter<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_names_by_filter_args, List<String>> {
+      public get_table_names_by_filter() {
+        super("get_table_names_by_filter");
       }
 
-      public add_partition_with_environment_context_args getEmptyArgsInstance() {
-        return new add_partition_with_environment_context_args();
+      public get_table_names_by_filter_args getEmptyArgsInstance() {
+        return new get_table_names_by_filter_args();
       }
 
-      public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Partition>() { 
-          public void onComplete(Partition o) {
-            add_partition_with_environment_context_result result = new add_partition_with_environment_context_result();
+        return new AsyncMethodCallback<List<String>>() { 
+          public void onComplete(List<String> o) {
+            get_table_names_by_filter_result result = new get_table_names_by_filter_result();
             result.success = o;
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -16072,19 +15906,19 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            add_partition_with_environment_context_result result = new add_partition_with_environment_context_result();
-            if (e instanceof InvalidObjectException) {
-                        result.o1 = (InvalidObjectException) e;
+            get_table_names_by_filter_result result = new get_table_names_by_filter_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof AlreadyExistsException) {
-                        result.o2 = (AlreadyExistsException) e;
+            else             if (e instanceof InvalidOperationException) {
+                        result.o2 = (InvalidOperationException) e;
                         result.setO2IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof MetaException) {
-                        result.o3 = (MetaException) e;
+            else             if (e instanceof UnknownDBException) {
+                        result.o3 = (UnknownDBException) e;
                         result.setO3IsSet(true);
                         msg = result;
             }
@@ -16108,27 +15942,25 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, add_partition_with_environment_context_args args, org.apache.thrift.async.AsyncMethodCallback<Partition> resultHandler) throws TException {
-        iface.add_partition_with_environment_context(args.new_part, args.environment_context,resultHandler);
+      public void start(I iface, get_table_names_by_filter_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
+        iface.get_table_names_by_filter(args.dbname, args.filter, args.max_tables,resultHandler);
       }
     }
 
-    public static class add_partitions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_partitions_args, Integer> {
-      public add_partitions() {
-        super("add_partitions");
+    public static class alter_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_table_args, Void> {
+      public alter_table() {
+        super("alter_table");
       }
 
-      public add_partitions_args getEmptyArgsInstance() {
-        return new add_partitions_args();
+      public alter_table_args getEmptyArgsInstance() {
+        return new alter_table_args();
       }
 
-      public AsyncMethodCallback<Integer> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Integer>() { 
-          public void onComplete(Integer o) {
-            add_partitions_result result = new add_partitions_result();
-            result.success = o;
-            result.setSuccessIsSet(true);
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            alter_table_result result = new alter_table_result();
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -16140,20 +15972,15 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            add_partitions_result result = new add_partitions_result();
-            if (e instanceof InvalidObjectException) {
-                        result.o1 = (InvalidObjectException) e;
+            alter_table_result result = new alter_table_result();
+            if (e instanceof InvalidOperationException) {
+                        result.o1 = (InvalidOperationException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof AlreadyExistsException) {
-                        result.o2 = (AlreadyExistsException) e;
-                        result.setO2IsSet(true);
-                        msg = result;
-            }
             else             if (e instanceof MetaException) {
-                        result.o3 = (MetaException) e;
-                        result.setO3IsSet(true);
+                        result.o2 = (MetaException) e;
+                        result.setO2IsSet(true);
                         msg = result;
             }
              else 
@@ -16176,27 +16003,25 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, add_partitions_args args, org.apache.thrift.async.AsyncMethodCallback<Integer> resultHandler) throws TException {
-        iface.add_partitions(args.new_parts,resultHandler);
+      public void start(I iface, alter_table_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.alter_table(args.dbname, args.tbl_name, args.new_tbl,resultHandler);
       }
     }
 
-    public static class add_partitions_pspec<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_partitions_pspec_args, Integer> {
-      public add_partitions_pspec() {
-        super("add_partitions_pspec");
+    public static class alter_table_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_table_with_environment_context_args, Void> {
+      public alter_table_with_environment_context() {
+        super("alter_table_with_environment_context");
       }
 
-      public add_partitions_pspec_args getEmptyArgsInstance() {
-        return new add_partitions_pspec_args();
+      public alter_table_with_environment_context_args getEmptyArgsInstance() {
+        return new alter_table_with_environment_context_args();
       }
 
-      public AsyncMethodCallback<Integer> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Integer>() { 
-          public void onComplete(Integer o) {
-            add_partitions_pspec_result result = new add_partitions_pspec_result();
-            result.success = o;
-            result.setSuccessIsSet(true);
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            alter_table_with_environment_context_result result = new alter_table_with_environment_context_result();
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -16208,20 +16033,15 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            add_partitions_pspec_result result = new add_partitions_pspec_result();
-            if (e instanceof InvalidObjectException) {
-                        result.o1 = (InvalidObjectException) e;
+            alter_table_with_environment_context_result result = new alter_table_with_environment_context_result();
+            if (e instanceof InvalidOperationException) {
+                        result.o1 = (InvalidOperationException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof AlreadyExistsException) {
-                        result.o2 = (AlreadyExistsException) e;
-                        result.setO2IsSet(true);
-                        msg = result;
-            }
             else             if (e instanceof MetaException) {
-                        result.o3 = (MetaException) e;
-                        result.setO3IsSet(true);
+                        result.o2 = (MetaException) e;
+                        result.setO2IsSet(true);
                         msg = result;
             }
              else 
@@ -16244,26 +16064,25 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, add_partitions_pspec_args args, org.apache.thrift.async.AsyncMethodCallback<Integer> resultHandler) throws TException {
-        iface.add_partitions_pspec(args.new_parts,resultHandler);
+      public void start(I iface, alter_table_with_environment_context_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.alter_table_with_environment_context(args.dbname, args.tbl_name, args.new_tbl, args.environment_context,resultHandler);
       }
     }
 
-    public static class append_partition<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, append_partition_args, Partition> {
-      public append_partition() {
-        super("append_partition");
+    public static class alter_table_with_cascade<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_table_with_cascade_args, Void> {
+      public alter_table_with_cascade() {
+        super("alter_table_with_cascade");
       }
 
-      public append_partition_args getEmptyArgsInstance() {
-        return new append_partition_args();
+      public alter_table_with_cascade_args getEmptyArgsInstance() {
+        return new alter_table_with_cascade_args();
       }
 
-      public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Partition>() { 
-          public void onComplete(Partition o) {
-            append_partition_result result = new append_partition_result();
-            result.success = o;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            alter_table_with_cascade_result result = new alter_table_with_cascade_result();
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -16275,20 +16094,15 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            append_partition_result result = new append_partition_result();
-            if (e instanceof InvalidObjectException) {
-                        result.o1 = (InvalidObjectException) e;
+            alter_table_with_cascade_result result = new alter_table_with_cascade_result();
+            if (e instanceof InvalidOperationException) {
+                        result.o1 = (InvalidOperationException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof AlreadyExistsException) {
-                        result.o2 = (AlreadyExistsException) e;
-                        result.setO2IsSet(true);
-                        msg = result;
-            }
             else             if (e instanceof MetaException) {
-                        result.o3 = (MetaException) e;
-                        result.setO3IsSet(true);
+                        result.o2 = (MetaException) e;
+                        result.setO2IsSet(true);
                         msg = result;
             }
              else 
@@ -16311,25 +16125,25 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, append_partition_args args, org.apache.thrift.async.AsyncMethodCallback<Partition> resultHandler) throws TException {
-        iface.append_partition(args.db_name, args.tbl_name, args.part_vals,resultHandler);
+      public void start(I iface, alter_table_with_cascade_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.alter_table_with_cascade(args.dbname, args.tbl_name, args.new_tbl, args.cascade,resultHandler);
       }
     }
 
-    public static class add_partitions_req<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_partitions_req_args, AddPartitionsResult> {
-      public add_partitions_req() {
-        super("add_partitions_req");
+    public static class add_partition<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_partition_args, Partition> {
+      public add_partition() {
+        super("add_partition");
       }
 
-      public add_partitions_req_args getEmptyArgsInstance() {
-        return new add_partitions_req_args();
+      public add_partition_args getEmptyArgsInstance() {
+        return new add_partition_args();
       }
 
-      public AsyncMethodCallback<AddPartitionsResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<AddPartitionsResult>() { 
-          public void onComplete(AddPartitionsResult o) {
-            add_partitions_req_result result = new add_partitions_req_result();
+        return new AsyncMethodCallback<Partition>() { 
+          public void onComplete(Partition o) {
+            add_partition_result result = new add_partition_result();
             result.success = o;
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -16342,7 +16156,7 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            add_partitions_req_result result = new add_partitions_req_result();
+            add_partition_result result = new add_partition_result();
             if (e instanceof InvalidObjectException) {
                         result.o1 = (InvalidObjectException) e;
                         result.setO1IsSet(true);
@@ -16378,25 +16192,25 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, add_partitions_req_args args, org.apache.thrift.async.AsyncMethodCallback<AddPartitionsResult> resultHandler) throws TException {
-        iface.add_partitions_req(args.request,resultHandler);
+      public void start(I iface, add_partition_args args, org.apache.thrift.async.AsyncMethodCallback<Partition> resultHandler) throws TException {
+        iface.add_partition(args.new_part,resultHandler);
       }
     }
 
-    public static class append_partition_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, append_partition_with_environment_context_args, Partition> {
-      public append_partition_with_environment_context() {
-        super("append_partition_with_environment_context");
+    public static class add_partition_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_partition_with_environment_context_args, Partition> {
+      public add_partition_with_environment_context() {
+        super("add_partition_with_environment_context");
       }
 
-      public append_partition_with_environment_context_args getEmptyArgsInstance() {
-        return new append_partition_with_environment_context_args();
+      public add_partition_with_environment_context_args getEmptyArgsInstance() {
+        return new add_partition_with_environment_context_args();
       }
 
       public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
         return new AsyncMethodCallback<Partition>() { 
           public void onComplete(Partition o) {
-            append_partition_with_environment_context_result result = new append_partition_with_environment_context_result();
+            add_partition_with_environment_context_result result = new add_partition_with_environment_context_result();
             result.success = o;
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -16409,7 +16223,7 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            append_partition_with_environment_context_result result = new append_partition_with_environment_context_result();
+            add_partition_with_environment_context_result result = new add_partition_with_environment_context_result();
             if (e instanceof InvalidObjectException) {
                         result.o1 = (InvalidObjectException) e;
                         result.setO1IsSet(true);
@@ -16445,26 +16259,27 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, append_partition_with_environment_context_args args, org.apache.thrift.async.AsyncMethodCallback<Partition> resultHandler) throws TException {
-        iface.append_partition_with_environment_context(args.db_name, args.tbl_name, args.part_vals, args.environment_context,resultHandler);
+      public void start(I iface, add_partition_with_environment_context_args args, org.apache.thrift.async.AsyncMethodCallback<Partition> resultHandler) throws TException {
+        iface.add_partition_with_environment_context(args.new_part, args.environment_context,resultHandler);
       }
     }
 
-    public static class append_partition_by_name<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, append_partition_by_name_args, Partition> {
-      public append_partition_by_name() {
-        super("append_partition_by_name");
+    public static class add_partitions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_partitions_args, Integer> {
+      public add_partitions() {
+        super("add_partitions");
       }
 
-      public append_partition_by_name_args getEmptyArgsInstance() {
-        return new append_partition_by_name_args();
+      public add_partitions_args getEmptyArgsInstance() {
+        return new add_partitions_args();
       }
 
-      public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Integer> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Partition>() { 
-          public void onComplete(Partition o) {
-            append_partition_by_name_result result = new append_partition_by_name_result();
+        return new AsyncMethodCallback<Integer>() { 
+          public void onComplete(Integer o) {
+            add_partitions_result result = new add_partitions_result();
             result.success = o;
+            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -16476,7 +16291,7 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            append_partition_by_name_result result = new append_partition_by_name_result();
+            add_partitions_result result = new add_partitions_result();
             if (e instanceof InvalidObjectException) {
                         result.o1 = (InvalidObjectException) e;
                         result.setO1IsSet(true);
@@ -16512,26 +16327,27 @@ public class ThriftHiveMetastore {
         return false;
       }
 
-      public void start(I iface, append_partition_by_name_args args, org.apache.thrift.async.AsyncMethodCallback<Partition> resultHandler) throws TException {
-        iface.append_partition_by_name(args.db_name, args.tbl_name, args.part_name,resultHandler);
+      public void start(I iface, add_partitions_args args, org.apache.thrift.async.AsyncMethodCallback<Integer> resultHandler) throws TException {
+        iface.add_partitions(args.new_parts,resultHandler);
       }
     }
 
-    public static class append_partition_by_name_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, append_partition_by_name_with_environment_context_args, Partition> {
-      public append_partition_by_name_with_environment_context() {
-        super("append_partition_by_name_with_environment_context");
+    public static class add_partitions_pspec<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_partitions_pspec_args, Integer> {
+      public add_partitions_pspec() {
+        super("add_partitions_pspec");
       }
 
-      public append_partition_by_name_with_environment_context_args getEmptyArgsInstance() {
-        return new append_partition_by_name_with_environment_context_args();
+      public add_partitions_pspec_args getEmptyArgsInstance() {
+        return new add_partitions_pspec_args();
       }
 
-      public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Integer> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Partition>() { 
-          public void onComplete(Partition o) {
-            append_partition_by_name_with_environment_context_result result = new append_partition_by_name_with_environment_context_result();
+        return new AsyncMethodCallback<Integer>() { 
+          public void onComplete(Integer o) {
+            add_partitions_pspec_result result = new add_partitions_pspec_result();
             result.success = o;
+            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -16543,7 +16359,342 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            append_partition_by_name_with_environment_context_result result = new append_partition_by_name_with_environment_context_result();
+            add_partitions_pspec_result result = new add_partitions_pspec_result();
+            if (e instanceof InvalidObjectException) {
+                        result.o1 = (InvalidObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof AlreadyExistsException) {
+                        result.o2 = (AlreadyExistsException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, add_partitions_pspec_args args, org.apache.thrift.async.AsyncMethodCallback<Integer> resultHandler) throws TException {
+        iface.add_partitions_pspec(args.new_parts,resultHandler);
+      }
+    }
+
+    public static class append_partition<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, append_partition_args, Partition> {
+      public append_partition() {
+        super("append_partition");
+      }
+
+      public append_partition_args getEmptyArgsInstance() {
+        return new append_partition_args();
+      }
+
+      public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Partition>() { 
+          public void onComplete(Partition o) {
+            append_partition_result result = new append_partition_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            append_partition_result result = new append_partition_result();
+            if (e instanceof InvalidObjectException) {
+                        result.o1 = (InvalidObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof AlreadyExistsException) {
+                        result.o2 = (AlreadyExistsException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, append_partition_args args, org.apache.thrift.async.AsyncMethodCallback<Partition> resultHandler) throws TException {
+        iface.append_partition(args.db_name, args.tbl_name, args.part_vals,resultHandler);
+      }
+    }
+
+    public static class add_partitions_req<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_partitions_req_args, AddPartitionsResult> {
+      public add_partitions_req() {
+        super("add_partitions_req");
+      }
+
+      public add_partitions_req_args getEmptyArgsInstance() {
+        return new add_partitions_req_args();
+      }
+
+      public AsyncMethodCallback<AddPartitionsResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<AddPartitionsResult>() { 
+          public void onComplete(AddPartitionsResult o) {
+            add_partitions_req_result result = new add_partitions_req_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            add_partitions_req_result result = new add_partitions_req_result();
+            if (e instanceof InvalidObjectException) {
+                        result.o1 = (InvalidObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof AlreadyExistsException) {
+                        result.o2 = (AlreadyExistsException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, add_partitions_req_args args, org.apache.thrift.async.AsyncMethodCallback<AddPartitionsResult> resultHandler) throws TException {
+        iface.add_partitions_req(args.request,resultHandler);
+      }
+    }
+
+    public static class append_partition_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, append_partition_with_environment_context_args, Partition> {
+      public append_partition_with_environment_context() {
+        super("append_partition_with_environment_context");
+      }
+
+      public append_partition_with_environment_context_args getEmptyArgsInstance() {
+        return new append_partition_with_environment_context_args();
+      }
+
+      public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Partition>() { 
+          public void onComplete(Partition o) {
+            append_partition_with_environment_context_result result = new append_partition_with_environment_context_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            append_partition_with_environment_context_result result = new append_partition_with_environment_context_result();
+            if (e instanceof InvalidObjectException) {
+                        result.o1 = (InvalidObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof AlreadyExistsException) {
+                        result.o2 = (AlreadyExistsException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, append_partition_with_environment_context_args args, org.apache.thrift.async.AsyncMethodCallback<Partition> resultHandler) throws TException {
+        iface.append_partition_with_environment_context(args.db_name, args.tbl_name, args.part_vals, args.environment_context,resultHandler);
+      }
+    }
+
+    public static class append_partition_by_name<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, append_partition_by_name_args, Partition> {
+      public append_partition_by_name() {
+        super("append_partition_by_name");
+      }
+
+      public append_partition_by_name_args getEmptyArgsInstance() {
+        return new append_partition_by_name_args();
+      }
+
+      public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Partition>() { 
+          public void onComplete(Partition o) {
+            append_partition_by_name_result result = new append_partition_by_name_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            append_partition_by_name_result result = new append_partition_by_name_result();
+            if (e instanceof InvalidObjectException) {
+                        result.o1 = (InvalidObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof AlreadyExistsException) {
+                        result.o2 = (AlreadyExistsException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, append_partition_by_name_args args, org.apache.thrift.async.AsyncMethodCallback<Partition> resultHandler) throws TException {
+        iface.append_partition_by_name(args.db_name, args.tbl_name, args.part_name,resultHandler);
+      }
+    }
+
+    public static class append_partition_by_name_with_environment_context<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, append_partition_by_name_with_environment_context_args, Partition> {
+      public append_partition_by_name_with_environment_context() {
+        super("append_partition_by_name_with_environment_context");
+      }
+
+      public append_partition_by_name_with_environment_context_args getEmptyArgsInstance() {
+        return new append_partition_by_name_with_environment_context_args();
+      }
+
+      public AsyncMethodCallback<Partition> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Partition>() { 
+          public void onComplete(Partition o) {
+            append_partition_by_name_with_environment_context_result result = new append_partition_by_name_with_environment_context_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            append_partition_by_name_with_environment_context_result result = new append_partition_by_name_with_environment_context_result();
             if (e instanceof InvalidObjectException) {
                         result.o1 = (InvalidObjectException) e;
                         result.setO1IsSet(true);
@@ -42513,6 +42664,835 @@ public class ThriftHiveMetastore {
 
   }
 
+  public static class drop_constraint_args implements org.apache.thrift.TBase<drop_constraint_args, drop_constraint_args._Fields>, java.io.Serializable, Cloneable, Comparable<drop_constraint_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("drop_constraint_args");
+
+    private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new drop_constraint_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new drop_constraint_argsTupleSchemeFactory());
+    }
+
+    private DropConstraintRequest req; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      REQ((short)1, "req");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // REQ
+            return REQ;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DropConstraintRequest.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(drop_constraint_args.class, metaDataMap);
+    }
+
+    public drop_constraint_args() {
+    }
+
+    public drop_constraint_args(
+      DropConstraintRequest req)
+    {
+      this();
+      this.req = req;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public drop_constraint_args(drop_constraint_args other) {
+      if (other.isSetReq()) {
+        this.req = new DropConstraintRequest(other.req);
+      }
+    }
+
+    public drop_constraint_args deepCopy() {
+      return new drop_constraint_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.req = null;
+    }
+
+    public DropConstraintRequest getReq() {
+      return this.req;
+    }
+
+    public void setReq(DropConstraintRequest req) {
+      this.req = req;
+    }
+
+    public void unsetReq() {
+      this.req = null;
+    }
+
+    /** Returns true if field req is set (has been assigned a value) and false otherwise */
+    public boolean isSetReq() {
+      return this.req != null;
+    }
+
+    public void setReqIsSet(boolean value) {
+      if (!value) {
+        this.req = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case REQ:
+        if (value == null) {
+          unsetReq();
+        } else {
+          setReq((DropConstraintRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case REQ:
+        return getReq();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case REQ:
+        return isSetReq();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof drop_constraint_args)
+        return this.equals((drop_constraint_args)that);
+      return false;
+    }
+
+    public boolean equals(drop_constraint_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_req = true && this.isSetReq();
+      boolean that_present_req = true && that.isSetReq();
+      if (this_present_req || that_present_req) {
+        if (!(this_present_req && that_present_req))
+          return false;
+        if (!this.req.equals(that.req))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_req = true && (isSetReq());
+      list.add(present_req);
+      if (present_req)
+        list.add(req);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(drop_constraint_args other) {
+      if (!getCl

<TRUNCATED>

[13/50] [abbrv] hive git commit: HIVE-13516: Adding BTEQ .IF, .QUIT, ERRORCODE to HPL/SQL (Dmitry Tolpeko reviewed by Alan Gates

Posted by sp...@apache.org.
HIVE-13516: Adding BTEQ .IF, .QUIT, ERRORCODE to HPL/SQL (Dmitry Tolpeko reviewed by Alan Gates


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

Branch: refs/heads/java8
Commit: 2d33d091b61dce092543970e62f41b63af1f32d1
Parents: 8729966
Author: Dmitry Tolpeko <dm...@gmail.com>
Authored: Wed May 4 03:13:18 2016 -0700
Committer: Dmitry Tolpeko <dm...@gmail.com>
Committed: Wed May 4 03:13:18 2016 -0700

----------------------------------------------------------------------
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     | 108 ++++++++++---
 .../main/java/org/apache/hive/hplsql/Exec.java  |  67 +++++++-
 .../java/org/apache/hive/hplsql/Expression.java |  31 ++--
 .../java/org/apache/hive/hplsql/Select.java     |  31 ++--
 .../java/org/apache/hive/hplsql/Signal.java     |   2 +-
 .../main/java/org/apache/hive/hplsql/Stmt.java  | 154 ++++++++++++-------
 hplsql/src/main/resources/hplsql-site.xml       |   2 -
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |   5 +
 .../apache/hive/hplsql/TestHplsqlOffline.java   |  20 +++
 hplsql/src/test/queries/local/if3_bteq.sql      |   3 +
 .../test/queries/offline/create_table_td.sql    |  45 ++++++
 hplsql/src/test/queries/offline/delete_all.sql  |   1 +
 hplsql/src/test/queries/offline/select.sql      |  42 +++++
 .../test/queries/offline/select_teradata.sql    |  12 ++
 hplsql/src/test/results/db/select_into.out.txt  |   3 +-
 hplsql/src/test/results/db/select_into2.out.txt |   4 +-
 hplsql/src/test/results/local/if3_bteq.out.txt  |   3 +
 hplsql/src/test/results/local/lang.out.txt      |  10 +-
 .../results/offline/create_table_mssql.out.txt  |  39 ++---
 .../results/offline/create_table_mssql2.out.txt |  13 +-
 .../results/offline/create_table_mysql.out.txt  |   5 +-
 .../results/offline/create_table_ora.out.txt    |  65 ++++----
 .../results/offline/create_table_ora2.out.txt   |   9 +-
 .../results/offline/create_table_pg.out.txt     |   7 +-
 .../results/offline/create_table_td.out.txt     |  31 ++++
 .../src/test/results/offline/delete_all.out.txt |   2 +
 hplsql/src/test/results/offline/select.out.txt  |  34 ++++
 .../src/test/results/offline/select_db2.out.txt |   3 +-
 .../results/offline/select_teradata.out.txt     |  10 ++
 29 files changed, 589 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
----------------------------------------------------------------------
diff --git a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4 b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
index b84116f..5ce0e23 100644
--- a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
+++ b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
@@ -30,7 +30,7 @@ single_block_stmt :                                      // Single BEGIN END blo
        T_BEGIN block exception_block? block_end
      | stmt T_SEMICOLON?
      ;
-	 
+
 block_end :
        {!_input.LT(2).getText().equalsIgnoreCase("TRANSACTION")}? T_END 
      ;
@@ -48,6 +48,7 @@ stmt :
      | begin_transaction_stmt
      | break_stmt
      | call_stmt
+     | collect_stats_stmt
      | close_stmt
      | cmp_stmt
      | copy_from_ftp_stmt
@@ -83,6 +84,7 @@ stmt :
      | merge_stmt
      | open_stmt
      | print_stmt
+     | quit_stmt
      | raise_stmt
      | resignal_stmt
      | return_stmt
@@ -181,9 +183,9 @@ declare_block_inplace :
      
 declare_stmt_item :
        declare_cursor_item
-     | declare_var_item 
      | declare_condition_item  
      | declare_handler_item
+     | declare_var_item 
      | declare_temporary_table_item
      ;
 
@@ -213,15 +215,19 @@ declare_handler_item :     // Condition handler declaration
      ;
      
 declare_temporary_table_item :     // DECLARE TEMPORARY TABLE statement
-       T_GLOBAL? T_TEMPORARY T_TABLE ident (T_AS? T_OPEN_P select_stmt T_CLOSE_P | T_AS? select_stmt | T_OPEN_P create_table_columns T_CLOSE_P) create_table_options?
+       T_GLOBAL? T_TEMPORARY T_TABLE ident create_table_preoptions? create_table_definition
      ;
      
 create_table_stmt :
-       T_CREATE T_TABLE (T_IF T_NOT T_EXISTS)? table_name T_OPEN_P create_table_columns T_CLOSE_P create_table_options?
+       T_CREATE T_TABLE (T_IF T_NOT T_EXISTS)? table_name create_table_preoptions? create_table_definition
      ;
      
 create_local_temp_table_stmt :
-       T_CREATE (T_LOCAL T_TEMPORARY | (T_SET | T_MULTISET)? T_VOLATILE) T_TABLE ident create_table_preoptions? T_OPEN_P create_table_columns T_CLOSE_P create_table_options?
+       T_CREATE (T_LOCAL T_TEMPORARY | (T_SET | T_MULTISET)? T_VOLATILE) T_TABLE ident create_table_preoptions? create_table_definition
+     ;
+     
+create_table_definition :
+      (T_AS? T_OPEN_P select_stmt T_CLOSE_P | T_AS? select_stmt | T_OPEN_P create_table_columns T_CLOSE_P) create_table_options?
      ;
      
 create_table_columns :         
@@ -262,7 +268,7 @@ create_table_preoptions :
      ;     
      
 create_table_preoptions_item :
-       T_NO? T_LOG
+       T_NO? (T_LOG | T_FALLBACK)
      ;
      
 create_table_options :
@@ -273,6 +279,7 @@ create_table_options_item :
        T_ON T_COMMIT (T_DELETE | T_PRESERVE) T_ROWS 
      | create_table_options_ora_item
      | create_table_options_db2_item  
+     | create_table_options_td_item
      | create_table_options_hive_item  
      | create_table_options_mssql_item
      | create_table_options_mysql_item       
@@ -296,6 +303,11 @@ create_table_options_db2_item :
      | T_DEFINITION T_ONLY
      ;
      
+create_table_options_td_item :
+       T_UNIQUE? T_PRIMARY T_INDEX T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P
+     | T_WITH T_DATA
+     ;
+    
 create_table_options_hive_item :
        create_table_hive_row_format
      ;
@@ -379,7 +391,7 @@ dtype :                  // Data types
      | T_VARCHAR
      | T_VARCHAR2
      | T_XML
-     | L_ID ('%' (T_TYPE | T_ROWTYPE))?             // User-defined or derived data type
+     | ident ('%' (T_TYPE | T_ROWTYPE))?             // User-defined or derived data type
      ;
      
 dtype_len :             // Data type length or size specification
@@ -450,7 +462,9 @@ create_routine_params :
        T_OPEN_P T_CLOSE_P
      | T_OPEN_P create_routine_param_item (T_COMMA create_routine_param_item)* T_CLOSE_P
      | {!_input.LT(1).getText().equalsIgnoreCase("IS") &&
-        !_input.LT(1).getText().equalsIgnoreCase("AS")}? 
+        !_input.LT(1).getText().equalsIgnoreCase("AS") &&
+		!(_input.LT(1).getText().equalsIgnoreCase("DYNAMIC") && _input.LT(2).getText().equalsIgnoreCase("RESULT"))
+		}? 
        create_routine_param_item (T_COMMA create_routine_param_item)* 
      ;
      
@@ -484,6 +498,7 @@ exec_stmt :             // EXEC, EXECUTE IMMEDIATE statement
 if_stmt :               // IF statement 
        if_plsql_stmt
      | if_tsql_stmt 
+     | if_bteq_stmt
      ;
 
 if_plsql_stmt : 
@@ -494,13 +509,17 @@ if_tsql_stmt :
        T_IF bool_expr single_block_stmt (T_ELSE single_block_stmt)?  
      ;
      
+if_bteq_stmt :
+       '.' T_IF bool_expr T_THEN single_block_stmt 
+     ;
+
 elseif_block :
        (T_ELSIF | T_ELSEIF) bool_expr T_THEN block
      ;
 
 else_block :
        T_ELSE block
-     ;
+     ; 
      
 include_stmt :          // INCLUDE statement
        T_INCLUDE (file_name | expr)
@@ -571,6 +590,14 @@ fetch_stmt :            // FETCH cursor statement
        T_FETCH T_FROM? L_ID T_INTO L_ID (T_COMMA L_ID)*
      ;
      
+collect_stats_stmt :
+       T_COLLECT (T_STATISTICS | T_STATS) T_ON table_name collect_stats_clause?
+     ;
+     
+collect_stats_clause :
+       T_COLUMN T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P
+     ;
+     
 close_stmt :            // CLOSE cursor statement
        T_CLOSE L_ID
      ;
@@ -652,6 +679,10 @@ print_stmt :            // PRINT statement
        T_PRINT expr
      | T_PRINT T_OPEN_P expr T_CLOSE_P
      ;
+
+quit_stmt :
+       '.'? T_QUIT expr?
+     ;
      
 raise_stmt :
        T_RAISE
@@ -761,7 +792,7 @@ fullselect_set_clause :
      ;
   
 subselect_stmt : 
-       (T_SELECT | T_SEL) select_list into_clause? from_clause? where_clause? group_by_clause? having_clause? order_by_clause? select_options?
+       (T_SELECT | T_SEL) select_list into_clause? from_clause? where_clause? group_by_clause? (having_clause | qualify_clause)? order_by_clause? select_options?
      ;
 
 select_list :           
@@ -834,6 +865,8 @@ from_table_values_row:
 from_alias_clause :
        {!_input.LT(1).getText().equalsIgnoreCase("EXEC") &&
         !_input.LT(1).getText().equalsIgnoreCase("EXECUTE") && 
+        !_input.LT(1).getText().equalsIgnoreCase("INNER") &&
+        !_input.LT(1).getText().equalsIgnoreCase("LEFT") &&
         !_input.LT(1).getText().equalsIgnoreCase("GROUP") &&
         !_input.LT(1).getText().equalsIgnoreCase("ORDER") &&
         !_input.LT(1).getText().equalsIgnoreCase("LIMIT") &&
@@ -856,6 +889,10 @@ group_by_clause :
 having_clause :           
        T_HAVING bool_expr
      ;     
+     
+qualify_clause :           
+       T_QUALIFY bool_expr
+     ;  
 
 order_by_clause :
        T_ORDER T_BY expr (T_ASC | T_DESC)? (T_COMMA expr (T_ASC | T_DESC)?)*
@@ -879,7 +916,7 @@ update_assignment :
      ;
 
 update_table :
-       (table_name | (T_OPEN_P select_stmt T_CLOSE_P)) (T_AS? ident)?
+       (table_name from_clause? | T_OPEN_P select_stmt T_CLOSE_P) (T_AS? ident)?
      ;     
      
 update_upsert :
@@ -905,9 +942,14 @@ merge_action :
      | T_DELETE
      ;
      
-delete_stmt :                             // DELETE statement
-       T_DELETE T_FROM? table_name (T_AS? ident)? where_clause?
+delete_stmt :                             
+       T_DELETE T_FROM? table_name delete_alias? (where_clause | T_ALL)?
      ;
+	
+delete_alias :
+       {!_input.LT(1).getText().equalsIgnoreCase("ALL")}?
+       T_AS? ident
+	 ;
  
 describe_stmt :
        (T_DESCRIBE | T_DESC) T_TABLE? table_name 
@@ -928,6 +970,7 @@ bool_expr_atom :
 bool_expr_unary :
       expr T_IS T_NOT? T_NULL
     | expr T_BETWEEN expr T_AND expr
+    | T_NOT? T_EXISTS T_OPEN_P select_stmt T_CLOSE_P
     | bool_expr_single_in
     | bool_expr_multi_in
     ;
@@ -967,6 +1010,7 @@ expr :
      | expr T_DIV expr  
      | expr T_ADD expr  
      | expr T_SUB expr   
+     | T_OPEN_P select_stmt T_CLOSE_P 
      | T_OPEN_P expr T_CLOSE_P 
      | expr_interval 
      | expr_concat
@@ -997,6 +1041,8 @@ interval_item :
      | T_DAYS
      | T_MICROSECOND 
      | T_MICROSECONDS  
+     | T_SECOND 
+     | T_SECONDS  
      ;
      
 expr_concat :                  // String concatenation operator
@@ -1141,8 +1187,7 @@ timestamp_literal :                       // TIMESTAMP 'YYYY-MM-DD HH:MI:SS.FFF'
      ;
      
 ident :
-       L_ID
-     | non_reserved_words
+       (L_ID | non_reserved_words) ('.' (L_ID | non_reserved_words))* 
      ;
      
 string :                                   // String literal (single or double quoted)
@@ -1207,7 +1252,9 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_CLOSE 
      | T_CLUSTERED
      | T_CMP
+     | T_COLLECT
      | T_COLLECTION  
+     | T_COLUMN
      | T_COMMENT     
      | T_CONSTANT     
      | T_COPY
@@ -1229,6 +1276,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_CURRENT_TIMESTAMP
      | T_CURRENT_USER
      | T_CURSOR  
+     | T_DATA
      | T_DATABASE
      | T_DATE   
      | T_DATETIME     
@@ -1270,12 +1318,13 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_EXCEPTION  
      | T_EXCLUSIVE     
      | T_EXISTS
-     | T_EXIT       
+     | T_EXIT 
+     | T_FALLBACK     
      | T_FALSE     
      | T_FETCH  
      | T_FIELDS
      | T_FILE    
-     | T_FILES	 
+     | T_FILES 
      | T_FIRST_VALUE     
      | T_FLOAT        
      | T_FOR  
@@ -1390,7 +1439,9 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_PROC
      | T_PROCEDURE 
      | T_PWD     
+     | T_QUALIFY
      | T_QUERY_BAND
+     | T_QUIT
      | T_QUOTED_IDENTIFIER
      | T_RAISE
      | T_RANK  
@@ -1416,6 +1467,8 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_ROW_COUNT
      | T_ROW_NUMBER
      | T_SCHEMA
+     | T_SECOND
+     | T_SECONDS
      | T_SECURITY
      | T_SEGMENT
      | T_SEL          
@@ -1434,7 +1487,9 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_SQLEXCEPTION 
      | T_SQLINSERT
      | T_SQLSTATE
-     | T_SQLWARNING   
+     | T_SQLWARNING  
+     | T_STATS
+     | T_STATISTICS     
      | T_STEP    
      | T_STDEV     
      | T_STORAGE
@@ -1523,7 +1578,9 @@ T_CLIENT          : C L I E N T ;
 T_CLOSE           : C L O S E ;
 T_CLUSTERED       : C L U S T E R E D;
 T_CMP             : C M P ; 
+T_COLLECT         : C O L L E C T ; 
 T_COLLECTION      : C O L L E C T I O N ; 
+T_COLUMN          : C O L U M N ;
 T_COMMENT         : C O M M E N T;
 T_CONSTANT        : C O N S T A N T ;
 T_COMMIT          : C O M M I T ; 
@@ -1541,7 +1598,8 @@ T_CS              : C S;
 T_CURRENT         : C U R R E N T ;
 T_CURRENT_SCHEMA  : C U R R E N T '_' S C H E M A ;
 T_CURSOR          : C U R S O R ;
-T_DATABASE        : D A T A B A S E;
+T_DATABASE        : D A T A B A S E ;
+T_DATA            : D A T A ;
 T_DATE            : D A T E ;
 T_DATETIME        : D A T E T I M E ; 
 T_DAY             : D A Y ;
@@ -1582,6 +1640,7 @@ T_EXCEPTION       : E X C E P T I O N ;
 T_EXCLUSIVE       : E X C L U S I V E ; 
 T_EXISTS          : E X I S T S ; 
 T_EXIT            : E X I T ;
+T_FALLBACK        : F A L L B A C K ;
 T_FALSE           : F A L S E ;
 T_FETCH           : F E T C H ;
 T_FIELDS          : F I E L D S ; 
@@ -1694,8 +1753,10 @@ T_PRESERVE        : P R E S E R V E ;
 T_PRIMARY         : P R I M A R Y ;
 T_PRINT           : P R I N T ; 
 T_PROC            : P R O C ;
-T_PROCEDURE       : P R O C E D U R E;
+T_PROCEDURE       : P R O C E D U R E ;
+T_QUALIFY         : Q U A L I F Y ;
 T_QUERY_BAND      : Q U E R Y '_' B A N D ; 
+T_QUIT            : Q U I T ; 
 T_QUOTED_IDENTIFIER : Q U O T E D '_' I D E N T I F I E R ;
 T_RAISE           : R A I S E ;
 T_REAL            : R E A L ; 
@@ -1722,6 +1783,8 @@ T_RS              : R S ;
 T_PWD             : P W D ; 
 T_TRIM            : T R I M ;
 T_SCHEMA          : S C H E M A ;
+T_SECOND          : S E C O N D ;
+T_SECONDS         : S E C O N D S;
 T_SECURITY        : S E C U R I T Y ; 
 T_SEGMENT         : S E G M E N T ; 
 T_SEL             : S E L ;
@@ -1742,6 +1805,8 @@ T_SQLEXCEPTION    : S Q L E X C E P T I O N ;
 T_SQLINSERT       : S Q L I N S E R T ;
 T_SQLSTATE        : S Q L S T A T E ;
 T_SQLWARNING      : S Q L W A R N I N G ;
+T_STATS           : S T A T S ; 
+T_STATISTICS      : S T A T I S T I C S ;
 T_STEP            : S T E P ; 
 T_STORAGE         : S T O R A G E ; 
 T_STRING          : S T R I N G ;
@@ -1836,7 +1901,7 @@ T_CLOSE_SB     : ']' ;
 T_SEMICOLON    : ';' ;
 T_SUB          : '-' ;
 
-L_ID        : L_ID_PART (L_BLANK* '.' L_BLANK* L_ID_PART)*             // Identifier
+L_ID        : L_ID_PART                                                // Identifier
             ;
 L_S_STRING  : '\'' (('\'' '\'') | ('\\' '\'') | ~('\''))* '\''         // Single quoted string literal
             ;
@@ -1859,6 +1924,7 @@ L_LABEL     : ([a-zA-Z] | L_DIGIT | '_')* ':'
 fragment
 L_ID_PART  :
              [a-zA-Z] ([a-zA-Z] | L_DIGIT | '_')*                           // Identifier part
+            | '$' '{' .*? '}'
             | ('_' | '@' | ':' | '#' | '$') ([a-zA-Z] | L_DIGIT | '_' | '@' | ':' | '#' | '$')+     // (at least one char must follow special char)
             | '"' .*? '"'                                                   // Quoted identifiers
             | '[' .*? ']'

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
index 02605a8..67cf2ae 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
@@ -40,6 +40,7 @@ import org.antlr.v4.runtime.ParserRuleContext;
 import org.antlr.v4.runtime.Token;
 import org.antlr.v4.runtime.misc.NotNull;
 import org.antlr.v4.runtime.tree.ParseTree;
+import org.antlr.v4.runtime.tree.TerminalNode;
 import org.apache.commons.io.FileUtils;
 import org.apache.hive.hplsql.Var.Type;
 import org.apache.hive.hplsql.functions.*;
@@ -50,7 +51,8 @@ import org.apache.hive.hplsql.functions.*;
  */
 public class Exec extends HplsqlBaseVisitor<Integer> {
   
-  public static final String VERSION = "HPL/SQL 0.3.17";
+  public static final String VERSION = "HPL/SQL 0.3.31";
+  public static final String ERRORCODE = "ERRORCODE";
   public static final String SQLCODE = "SQLCODE";
   public static final String SQLSTATE = "SQLSTATE";
   public static final String HOSTCODE = "HOSTCODE";
@@ -665,9 +667,14 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
    * Set SQLCODE
    */
   public void setSqlCode(int sqlcode) {
+    Long code = new Long(sqlcode);
     Var var = findVariable(SQLCODE);
     if (var != null) {
-      var.setValue(new Long(sqlcode));
+      var.setValue(code);
+    }
+    var = findVariable(ERRORCODE);
+    if (var != null) {
+      var.setValue(code);
     }
   }
   
@@ -783,6 +790,7 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
     new FunctionMisc(this).register(function);
     new FunctionString(this).register(function);
     new FunctionOra(this).register(function);
+    addVariable(new Var(ERRORCODE, Var.Type.BIGINT, 0L));
     addVariable(new Var(SQLCODE, Var.Type.BIGINT, 0L));
     addVariable(new Var(SQLSTATE, Var.Type.STRING, "00000"));
     addVariable(new Var(HOSTCODE, Var.Type.BIGINT, 0L)); 
@@ -942,9 +950,10 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
    */
   Integer getProgramReturnCode() {
     Integer rc = 0;
-    if(!signals.empty()) {
+    if (!signals.empty()) {
       Signal sig = signals.pop();
-      if(sig.type == Signal.Type.LEAVE_ROUTINE && sig.value != null) {
+      if ((sig.type == Signal.Type.LEAVE_PROGRAM || sig.type == Signal.Type.LEAVE_ROUTINE) && 
+        sig.value != null) {
         try {
           rc = Integer.parseInt(sig.value);
         }
@@ -1133,7 +1142,7 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
     String scale = null;
     Var default_ = null;
     if (ctx.dtype().T_ROWTYPE() != null) {
-      row = meta.getRowDataType(ctx, exec.conf.defaultConnection, ctx.dtype().L_ID().getText());
+      row = meta.getRowDataType(ctx, exec.conf.defaultConnection, ctx.dtype().ident().getText());
       if (row == null) {
         type = Var.DERIVED_ROWTYPE;
       }
@@ -1184,7 +1193,7 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   String getDataType(HplsqlParser.Declare_var_itemContext ctx) {
     String type = null;
     if (ctx.dtype().T_TYPE() != null) {
-      type = meta.getDataType(ctx, exec.conf.defaultConnection, ctx.dtype().L_ID().getText());
+      type = meta.getDataType(ctx, exec.conf.defaultConnection, ctx.dtype().ident().getText());
       if (type == null) {
         type = Var.DERIVED_TYPE; 
       }
@@ -1349,6 +1358,11 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   }
   
   @Override 
+  public Integer visitCreate_table_options_td_item(HplsqlParser.Create_table_options_td_itemContext ctx) { 
+    return 0; 
+  }
+  
+  @Override 
   public Integer visitCreate_table_options_mssql_item(HplsqlParser.Create_table_options_mssql_itemContext ctx) { 
     return 0; 
   }
@@ -1678,6 +1692,14 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   }
   
   /**
+   * IF statement (BTEQ syntax)
+   */
+  @Override  
+  public Integer visitIf_bteq_stmt(HplsqlParser.If_bteq_stmtContext ctx) { 
+    return exec.stmt.ifBteq(ctx); 
+  }
+  
+  /**
    * USE statement
    */
   @Override 
@@ -1786,6 +1808,14 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
 	  return exec.stmt.print(ctx); 
   }
   
+  /** 
+   * QUIT statement 
+   */
+  @Override 
+  public Integer visitQuit_stmt(HplsqlParser.Quit_stmtContext ctx) { 
+    return exec.stmt.quit(ctx); 
+  }
+  
   /**
    * SIGNAL statement
    */
@@ -2290,6 +2320,31 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   }
   
   /**
+   * Append the text preserving the formatting (space symbols) between tokens
+   */
+  void append(StringBuilder str, String appendStr, Token start, Token stop) {
+    String spaces = start.getInputStream().getText(new org.antlr.v4.runtime.misc.Interval(start.getStartIndex(), stop.getStopIndex()));
+    spaces = spaces.substring(start.getText().length(), spaces.length() - stop.getText().length());
+    str.append(spaces);
+    str.append(appendStr);
+  }
+  
+  void append(StringBuilder str, TerminalNode start, TerminalNode stop) {
+    String text = start.getSymbol().getInputStream().getText(new org.antlr.v4.runtime.misc.Interval(start.getSymbol().getStartIndex(), stop.getSymbol().getStopIndex()));
+    str.append(text);
+  }
+   
+  /**
+   * Get the first non-null node
+   */
+  TerminalNode nvl(TerminalNode t1, TerminalNode t2) {
+    if (t1 != null) {
+      return t1;
+    }
+    return t2;
+  }
+  
+  /**
    * Evaluate the expression and pop value from the stack
    */
   Var evalPop(ParserRuleContext ctx) {

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java b/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
index 33ef490..c10f702 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
@@ -74,8 +74,14 @@ public class Expression {
     StringBuilder sql = new StringBuilder();
     if (ctx.T_OPEN_P() != null) {
       sql.append("(");
-      sql.append(evalPop(ctx.expr(0)).toString());
-      sql.append(")");      
+      if (ctx.select_stmt() != null) {
+        exec.append(sql, evalPop(ctx.select_stmt()).toString(), ctx.T_OPEN_P().getSymbol(), ctx.select_stmt().getStart());
+        exec.append(sql, ctx.T_CLOSE_P().getText(), ctx.select_stmt().stop, ctx.T_CLOSE_P().getSymbol()); 
+      }
+      else {
+        sql.append(evalPop(ctx.expr(0)).toString());
+        sql.append(")");
+      }
     }
     else if (ctx.T_MUL() != null) {
       sql.append(evalPop(ctx.expr(0)).toString());
@@ -232,6 +238,11 @@ public class Expression {
       sql.append(" " + ctx.T_AND().getText() + " ");
       sql.append(evalPop(ctx.expr(2)).toString());
     }
+    else if (ctx.T_EXISTS() != null) {
+      exec.append(sql, exec.nvl(ctx.T_NOT(), ctx.T_EXISTS()), ctx.T_OPEN_P());
+      exec.append(sql, evalPop(ctx.select_stmt()).toString(), ctx.T_OPEN_P().getSymbol(), ctx.select_stmt().getStart());
+      exec.append(sql, ctx.T_CLOSE_P().getText(), ctx.select_stmt().stop, ctx.T_CLOSE_P().getSymbol());
+    }
     else if (ctx.bool_expr_single_in() != null) {
       singleInClauseSql(ctx.bool_expr_single_in(), sql);
     }
@@ -245,14 +256,12 @@ public class Expression {
   /**
    * Single value IN clause in executable SQL statement
    */
-  public void singleInClauseSql(HplsqlParser.Bool_expr_single_inContext ctx, StringBuilder sql) {
-    sql.append(evalPop(ctx.expr(0)).toString());
-    if (ctx.T_NOT() != null) {
-      sql.append(" " + ctx.T_NOT().getText());
-    }
-    sql.append(" " + ctx.T_IN().getText() + " (");
+  public void singleInClauseSql(HplsqlParser.Bool_expr_single_inContext ctx, StringBuilder sql) {    
+    sql.append(evalPop(ctx.expr(0)).toString() + " ");
+    exec.append(sql, exec.nvl(ctx.T_NOT(), ctx.T_IN()), ctx.T_OPEN_P());
     if (ctx.select_stmt() != null) {
-      sql.append(evalPop(ctx.select_stmt()));
+      exec.append(sql, evalPop(ctx.select_stmt()).toString(), ctx.T_OPEN_P().getSymbol(), ctx.select_stmt().getStart());
+      exec.append(sql, ctx.T_CLOSE_P().getText(), ctx.select_stmt().stop, ctx.T_CLOSE_P().getSymbol());
     }
     else {
       int cnt = ctx.expr().size();
@@ -262,8 +271,8 @@ public class Expression {
           sql.append(", ");
         }
       }
-    }
-    sql.append(")");
+      sql.append(")");
+    }    
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Select.java b/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
index 4bee252..589e984 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Stack;
 
 import org.antlr.v4.runtime.ParserRuleContext;
+import org.antlr.v4.runtime.Token;
 import org.antlr.v4.runtime.misc.Interval;
 
 public class Select {
@@ -196,26 +197,38 @@ public class Select {
   
   public Integer subselect(HplsqlParser.Subselect_stmtContext ctx) {
     StringBuilder sql = new StringBuilder();
-    if (ctx.T_SELECT() != null) {
-      sql.append(ctx.T_SELECT().getText());
+    sql.append(ctx.start.getText());
+    exec.append(sql, evalPop(ctx.select_list()).toString(), ctx.start, ctx.select_list().getStart());
+    Token last = ctx.select_list().stop;
+    if (ctx.into_clause() != null) {
+      last = ctx.into_clause().stop;
     }
-    sql.append(" " + evalPop(ctx.select_list()));
     if (ctx.from_clause() != null) {
-      sql.append(" " + evalPop(ctx.from_clause()));
-    } else if (conf.dualTable != null) {
+      exec.append(sql, evalPop(ctx.from_clause()).toString(), last, ctx.from_clause().getStart());
+      last = ctx.from_clause().stop;
+    } 
+    else if (conf.dualTable != null) {
       sql.append(" FROM " + conf.dualTable);
     }
     if (ctx.where_clause() != null) {
-      sql.append(" " + evalPop(ctx.where_clause()));
+      exec.append(sql, evalPop(ctx.where_clause()).toString(), last, ctx.where_clause().getStart());
+      last = ctx.where_clause().stop;
     }
     if (ctx.group_by_clause() != null) {
-      sql.append(" " + getText(ctx.group_by_clause()));
+      exec.append(sql, getText(ctx.group_by_clause()), last, ctx.group_by_clause().getStart());
+      last = ctx.group_by_clause().stop;
     }
     if (ctx.having_clause() != null) {
-      sql.append(" " + getText(ctx.having_clause()));
+      exec.append(sql, getText(ctx.having_clause()), last, ctx.having_clause().getStart());
+      last = ctx.having_clause().stop;
+    }
+    if (ctx.qualify_clause() != null) {
+      exec.append(sql, getText(ctx.qualify_clause()), last, ctx.qualify_clause().getStart());
+      last = ctx.qualify_clause().stop;
     }
     if (ctx.order_by_clause() != null) {
-      sql.append(" " + getText(ctx.order_by_clause()));
+      exec.append(sql, getText(ctx.order_by_clause()), last, ctx.order_by_clause().getStart());
+      last = ctx.order_by_clause().stop;
     }
     if (ctx.select_options() != null) {
       Var opt = evalPop(ctx.select_options());

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java b/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java
index 2c8cfc1..ddefcd8 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java
@@ -22,7 +22,7 @@ package org.apache.hive.hplsql;
  * Signals and exceptions
  */
 public class Signal {
-  public enum Type { LEAVE_LOOP, LEAVE_ROUTINE, SQLEXCEPTION, NOTFOUND, UNSUPPORTED_OPERATION, USERDEFINED };
+  public enum Type { LEAVE_LOOP, LEAVE_ROUTINE, LEAVE_PROGRAM, SQLEXCEPTION, NOTFOUND, UNSUPPORTED_OPERATION, USERDEFINED };
   Type type;
   String value = "";
   Exception exception = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
index d35f994..17d2195 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
@@ -25,9 +25,8 @@ import java.util.Stack;
 import java.util.UUID;
 
 import org.antlr.v4.runtime.ParserRuleContext;
+import org.antlr.v4.runtime.Token;
 import org.apache.hive.hplsql.Var.Type;
-import org.apache.hive.hplsql.HplsqlParser.Create_table_columns_itemContext;
-import org.apache.hive.hplsql.HplsqlParser.Create_table_columnsContext;
 
 /**
  * HPL/SQL statements execution
@@ -130,30 +129,13 @@ public class Stmt {
   public Integer createTable(HplsqlParser.Create_table_stmtContext ctx) { 
     trace(ctx, "CREATE TABLE");
     StringBuilder sql = new StringBuilder();
-    sql.append(exec.getText(ctx, ctx.T_CREATE().getSymbol(), ctx.T_TABLE().getSymbol()));
-    sql.append(" " + evalPop(ctx.table_name()) + " (");
-    int cnt = ctx.create_table_columns().create_table_columns_item().size();
-    int cols = 0;
-    for (int i = 0; i < cnt; i++) {
-      Create_table_columns_itemContext col = ctx.create_table_columns().create_table_columns_item(i);
-      if (col.create_table_column_cons() != null) {
-        continue;
-      }
-      if (cols > 0) {
-        sql.append(",\n");
-      }
-      sql.append(evalPop(col.column_name()));
-      sql.append(" ");
-      sql.append(exec.evalPop(col.dtype(), col.dtype_len()));
-      cols++;
-    }
-    sql.append("\n)");
-    if (ctx.create_table_options() != null) {
-      String opt = evalPop(ctx.create_table_options()).toString();
-      if (opt != null) {
-        sql.append(" " + opt);
-      }
+    exec.append(sql, ctx.T_CREATE(), ctx.T_TABLE());
+    exec.append(sql, evalPop(ctx.table_name()).toString(), ctx.T_TABLE().getSymbol(), ctx.table_name().getStart());
+    Token last = ctx.table_name().getStop();
+    if (ctx.create_table_preoptions() != null) {
+      last = ctx.create_table_preoptions().stop;
     }
+    sql.append(createTableDefinition(ctx.create_table_definition(), last));
     trace(ctx, sql.toString());
     Query query = exec.executeSql(ctx, sql.toString(), exec.conf.defaultConnection);
     if (query.error()) {
@@ -166,6 +148,40 @@ public class Stmt {
   }  
 
   /**
+   * Get CREATE TABLE definition (columns or query)
+   */
+  String createTableDefinition(HplsqlParser.Create_table_definitionContext ctx, Token last) { 
+    StringBuilder sql = new StringBuilder();
+    HplsqlParser.Create_table_columnsContext colCtx = ctx.create_table_columns();
+    if (colCtx != null) {
+      int cnt = colCtx.create_table_columns_item().size();
+      for (int i = 0; i < cnt; i++) {
+        HplsqlParser.Create_table_columns_itemContext col = colCtx.create_table_columns_item(i);
+        if (col.create_table_column_cons() != null) {
+          last = col.getStop();
+          continue;
+        }
+        exec.append(sql, evalPop(col.column_name()).toString(), last, col.column_name().getStop());
+        exec.append(sql, exec.evalPop(col.dtype(), col.dtype_len()), col.column_name().getStop(), col.dtype().getStart());
+        last = col.getStop();
+      }
+      exec.append(sql, ctx.T_CLOSE_P().getText(), last, ctx.T_CLOSE_P().getSymbol());
+    }
+    else {
+      exec.append(sql, evalPop(ctx.select_stmt()).toString(), last, ctx.select_stmt().getStart());
+      exec.append(sql, ctx.T_CLOSE_P().getText(), ctx.select_stmt().stop, ctx.T_CLOSE_P().getSymbol());
+    }
+    HplsqlParser.Create_table_optionsContext options = ctx.create_table_options();
+    if (options != null) {
+      String opt = evalPop(options).toString();
+      if (opt != null) {
+        sql.append(" " + opt);
+      }
+    }
+    return sql.toString(); 
+  }  
+
+  /**
    * CREATE TABLE options for Hive
    */
   public Integer createTableHiveOptions(HplsqlParser.Create_table_options_hive_itemContext ctx) {
@@ -196,7 +212,6 @@ public class Stmt {
    * CREATE TABLE options for MySQL
    */
   public Integer createTableMysqlOptions(HplsqlParser.Create_table_options_mysql_itemContext ctx) {
-    StringBuilder sql = new StringBuilder();
     if (ctx.T_COMMENT() != null) {
       evalString(ctx.T_COMMENT().getText() + " " + evalPop(ctx.expr()).toSqlString());
     }
@@ -207,11 +222,8 @@ public class Stmt {
    * DECLARE TEMPORARY TABLE statement 
    */
   public Integer declareTemporaryTable(HplsqlParser.Declare_temporary_table_itemContext ctx) { 
-    String name = ctx.ident().getText();
-    if (trace) {
-      trace(ctx, "DECLARE TEMPORARY TABLE " + name);
-    }
-    return createTemporaryTable(ctx, ctx.create_table_columns(), name);
+    trace(ctx, "DECLARE TEMPORARY TABLE");
+    return createTemporaryTable(ctx.ident(), ctx.create_table_definition(), ctx.create_table_preoptions());
   }
   
   /**
@@ -256,37 +268,45 @@ public class Stmt {
    * CREATE LOCAL TEMPORARY | VOLATILE TABLE statement 
    */
   public Integer createLocalTemporaryTable(HplsqlParser.Create_local_temp_table_stmtContext ctx) { 
-    String name = ctx.ident().getText();
-    if (trace) {
-      trace(ctx, "CREATE LOCAL TEMPORARY TABLE " + name);
-    }
-    return createTemporaryTable(ctx, ctx.create_table_columns(), name);
+    trace(ctx, "CREATE LOCAL TEMPORARY TABLE");
+    return createTemporaryTable(ctx.ident(), ctx.create_table_definition(), ctx.create_table_preoptions());
    }
   
   /**
    * Create a temporary table statement 
    */
-  public Integer createTemporaryTable(ParserRuleContext ctx, Create_table_columnsContext colCtx, String name) { 
+  public Integer createTemporaryTable(HplsqlParser.IdentContext identCtx, HplsqlParser.Create_table_definitionContext defCtx,
+                                      HplsqlParser.Create_table_preoptionsContext optCtx) { 
+    StringBuilder sql = new StringBuilder();
+    String name = identCtx.getText();
     String managedName = null;
-    String sql = null;
-    String columns = exec.getFormattedText(colCtx);
+    Token last = identCtx.getStop();
+    if (optCtx != null) {
+      last = optCtx.stop;
+    }
     if (conf.tempTables == Conf.TempTables.NATIVE) {
-      sql = "CREATE TEMPORARY TABLE " + name + "\n(" + columns + "\n)";
-    } else if (conf.tempTables == Conf.TempTables.MANAGED) {
+      sql.append("CREATE TEMPORARY TABLE " + name);
+      sql.append(createTableDefinition(defCtx, last));
+    } 
+    else if (conf.tempTables == Conf.TempTables.MANAGED) {
       managedName = name + "_" + UUID.randomUUID().toString().replace("-","");
       if (!conf.tempTablesSchema.isEmpty()) {
         managedName = conf.tempTablesSchema + "." + managedName;
       }      
-      sql = "CREATE TABLE " + managedName + "\n(" + columns + "\n)";
+      sql.append("CREATE TABLE " + managedName);
+      sql.append(createTableDefinition(defCtx, last));
       if (!conf.tempTablesLocation.isEmpty()) {
-        sql += "\nLOCATION '" + conf.tempTablesLocation + "/" + managedName + "'";
+        sql.append("\nLOCATION '" + conf.tempTablesLocation + "/" + managedName + "'");
       }
       if (trace) {
-        trace(ctx, "Managed table name: " + managedName);
+        trace(null, "Managed table name: " + managedName);
       }
     }  
+    if (trace) {
+      trace(null, sql.toString()); 
+    }
     if (sql != null) {
-      Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+      Query query = exec.executeSql(null, sql.toString(), exec.conf.defaultConnection);
       if (query.error()) {
         exec.signal(query);
         return 1;
@@ -606,6 +626,19 @@ public class Stmt {
   }
   
   /**
+   * IF statement (BTEQ syntax)
+   */
+  public Integer ifBteq(HplsqlParser.If_bteq_stmtContext ctx) {
+    trace(ctx, "IF");
+    visit(ctx.bool_expr());
+    if (exec.stackPop().isTrue()) {
+      trace(ctx, "IF TRUE executed");
+      visit(ctx.single_block_stmt());
+    }
+    return 0; 
+  }
+  
+  /**
    * Assignment from SELECT statement 
    */
   public Integer assignFromSelect(HplsqlParser.Assignment_stmt_select_itemContext ctx) { 
@@ -1103,13 +1136,17 @@ public class Stmt {
     trace(ctx, "DELETE");
     String table = evalPop(ctx.table_name()).toString();
     StringBuilder sql = new StringBuilder();
-    sql.append("DELETE FROM ");
-    sql.append(table);
-    if (ctx.where_clause() != null) {
-      boolean oldBuildSql = exec.buildSql; 
-      exec.buildSql = true;
-      sql.append(" " + evalPop(ctx.where_clause()).toString());
-      exec.buildSql = oldBuildSql;
+    if (ctx.T_ALL() == null) {
+      sql.append("DELETE FROM " + table);
+      if (ctx.where_clause() != null) {
+        boolean oldBuildSql = exec.buildSql; 
+        exec.buildSql = true;
+        sql.append(" " + evalPop(ctx.where_clause()).toString());
+        exec.buildSql = oldBuildSql;
+      }
+    }
+    else {
+      sql.append("TRUNCATE TABLE " + table);
     }
     trace(ctx, sql.toString());
     Query query = exec.executeSql(ctx, sql.toString(), exec.conf.defaultConnection);
@@ -1150,6 +1187,19 @@ public class Stmt {
 	  return 0; 
   }
   
+  /**
+   * QUIT Statement 
+   */
+  public Integer quit(HplsqlParser.Quit_stmtContext ctx) { 
+    trace(ctx, "QUIT");
+    String rc = null;
+    if (ctx.expr() != null) {
+      rc = evalPop(ctx.expr()).toString();
+    }
+    exec.signal(Signal.Type.LEAVE_PROGRAM, rc); 
+    return 0; 
+  }
+  
   /** 
    * SET current schema 
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/main/resources/hplsql-site.xml
----------------------------------------------------------------------
diff --git a/hplsql/src/main/resources/hplsql-site.xml b/hplsql/src/main/resources/hplsql-site.xml
index 7e2d92d..05fe857 100644
--- a/hplsql/src/main/resources/hplsql-site.xml
+++ b/hplsql/src/main/resources/hplsql-site.xml
@@ -12,7 +12,6 @@
 <property>
   <name>hplsql.conn.init.hiveconn</name>
   <value>
-     set mapred.job.queue.name=default;
      set hive.execution.engine=mr; 
      use default;
   </value>
@@ -36,7 +35,6 @@
 <property>
   <name>hplsql.conn.init.hive2conn</name>
   <value>
-     set mapred.job.queue.name=default;
      set hive.execution.engine=mr; 
      use default;
   </value>

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
----------------------------------------------------------------------
diff --git a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
index 80915ea..9b5a956 100644
--- a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
+++ b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
@@ -222,6 +222,11 @@ public class TestHplsqlLocal {
   public void testIf2() throws Exception {
     run("if2");
   }
+  
+  @Test
+  public void testIf3Bteq() throws Exception {
+    run("if3_bteq");
+  }
 
   @Test
   public void testInclude() throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlOffline.java
----------------------------------------------------------------------
diff --git a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlOffline.java b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlOffline.java
index 59b7bff..3e897be 100644
--- a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlOffline.java
+++ b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlOffline.java
@@ -64,16 +64,36 @@ public class TestHplsqlOffline {
   }
   
   @Test
+  public void testCreateTableTd() throws Exception {
+    run("create_table_td");
+  }
+  
+  @Test
+  public void testDeleteAll() throws Exception {
+    run("delete_all");
+  }
+  
+  @Test
   public void testInsertMysql() throws Exception {
     run("insert_mysql");
   }
 
   @Test
+  public void testSelect() throws Exception {
+    run("select");
+  }
+  
+  @Test
   public void testSelectDb2() throws Exception {
     run("select_db2");
   }
   
   @Test
+  public void testSelectTeradata() throws Exception {
+    run("select_teradata");
+  }
+  
+  @Test
   public void testUpdate() throws Exception {
     run("update");
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/queries/local/if3_bteq.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/if3_bteq.sql b/hplsql/src/test/queries/local/if3_bteq.sql
new file mode 100644
index 0000000..12a39a9
--- /dev/null
+++ b/hplsql/src/test/queries/local/if3_bteq.sql
@@ -0,0 +1,3 @@
+.if errorcode = 0 then .quit errorcode 
+
+print 'Failed: must not be executed';

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/queries/offline/create_table_td.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/offline/create_table_td.sql b/hplsql/src/test/queries/offline/create_table_td.sql
new file mode 100644
index 0000000..2572bb6
--- /dev/null
+++ b/hplsql/src/test/queries/offline/create_table_td.sql
@@ -0,0 +1,45 @@
+CREATE TABLE tab, NO LOG, NO FALLBACK
+  (
+    SOURCE_ID  INT,
+    RUN_ID     INT,
+    STATUS     CHAR,
+    LOAD_START timestamp(0),
+    LOAD_END   timestamp(0)
+  );
+
+CREATE TABLE ctl, NO LOG, NO FALLBACK
+AS
+(
+        SELECT
+                EBC.SOURCE_ID,
+                MAX(EBC.RUN_ID) AS RUN_ID,
+                EBC.STATUS,
+                EBC.LOAD_START,
+                EBC.LOAD_END
+        FROM
+                EBC
+        WHERE
+                EBC.SOURCE_ID = 451 AND
+                EBC.STATUS = 'R'
+        GROUP BY
+                1,3,4,5
+);
+  
+CREATE SET VOLATILE TABLE ctl2, NO LOG, NO FALLBACK
+AS
+(
+        SELECT
+                EBC.SOURCE_ID,
+                MAX(EBC.RUN_ID) AS RUN_ID,
+                EBC.STATUS,
+                EBC.LOAD_START,
+                EBC.LOAD_END
+        FROM
+                EBC
+        WHERE
+                EBC.SOURCE_ID = 451 AND
+                EBC.STATUS = 'R'
+        GROUP BY
+                1,3,4,5
+) WITH DATA PRIMARY INDEX (LOAD_START,LOAD_END)
+  ON COMMIT PRESERVE ROWS ;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/queries/offline/delete_all.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/offline/delete_all.sql b/hplsql/src/test/queries/offline/delete_all.sql
new file mode 100644
index 0000000..e89fd48
--- /dev/null
+++ b/hplsql/src/test/queries/offline/delete_all.sql
@@ -0,0 +1 @@
+DELETE FROM TEST1_DB.WK_WRK ALL;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/queries/offline/select.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/offline/select.sql b/hplsql/src/test/queries/offline/select.sql
new file mode 100644
index 0000000..0b6912e
--- /dev/null
+++ b/hplsql/src/test/queries/offline/select.sql
@@ -0,0 +1,42 @@
+SELECT * FROM a where 1=1 and not exists (select * from b)--abc;
+
+SELECT * 
+ FROM a 
+ where not exists 
+  (
+    select * from b
+  );
+  
+SELECT 
+         *
+         FROM
+         tab
+         WHERE FILE_DATE > (
+                           SELECT 
+                           MAX(FILE_DATE) AS MX_C_FILE_DT
+                           FROM tab
+                           WHERE FLAG = 'C' 
+                           AND IND = 'C'
+                           AND FILE_DATE < 
+                                          ( SELECT 
+                                            CAST( LOAD_START AS DATE) 
+                                            FROM 
+                                            tab  
+                                            WHERE
+                				              SOURCE_ID = 451 AND
+                				              BATCH = 'R'
+                		                   )
+		                  );
+                          
+SELECT 
+*
+FROM
+ DLTA_POC
+  LEFT OUTER JOIN TEST3_DB.TET ORG
+   ON DLTA_POC.YS_NO = ORG.EM_CODE_A
+   AND DLTA_POC.AREA_NO = ORG.AREA_CODE_2
+   AND DLTA_POC.GNT_POC = ORG.GEN_CD
+
+  LEFT OUTER JOIN TEST.LOCATION LOC
+   ON DLTA_POC.SE_KEY_POC = LOC.LOC_ID
+   AND LOC.LOCATION_END_DT = DATE '9999-12-31' ;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/queries/offline/select_teradata.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/offline/select_teradata.sql b/hplsql/src/test/queries/offline/select_teradata.sql
new file mode 100644
index 0000000..69522b8
--- /dev/null
+++ b/hplsql/src/test/queries/offline/select_teradata.sql
@@ -0,0 +1,12 @@
+SELECT branch_code,
+    branch_no,
+    c_no,
+    cd_type
+FROM EMPLOYEE
+    WHERE S_CODE = 'C'
+        AND (branch_no) NOT IN (
+            SELECT branch_code
+            FROM DEPARTMENT
+            WHERE branch_code = 'ABC'
+        )
+QUALIFY ROW_NUMBER() OVER (PARTITION BY c_no ORDER BY cd_type) = 1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/db/select_into.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/select_into.out.txt b/hplsql/src/test/results/db/select_into.out.txt
index 3f4ae31..6e4a69c 100644
--- a/hplsql/src/test/results/db/select_into.out.txt
+++ b/hplsql/src/test/results/db/select_into.out.txt
@@ -6,7 +6,8 @@ Ln:5 DECLARE v_dec DECIMAL
 Ln:6 DECLARE v_dec0 DECIMAL
 Ln:7 DECLARE v_str STRING
 Ln:9 SELECT
-Ln:9 SELECT CAST(1 AS BIGINT), CAST(1 AS INT), CAST(1 AS SMALLINT), CAST(1 AS TINYINT), CAST(1.1 AS DECIMAL(18,2)), CAST(1.1 AS DECIMAL(18,0)) FROM src LIMIT 1
+Ln:9 SELECT CAST(1 AS BIGINT), CAST(1 AS INT), CAST(1 AS SMALLINT), CAST(1 AS TINYINT), CAST(1.1 AS DECIMAL(18,2)), CAST(1.1 AS DECIMAL(18,0))  
+FROM src LIMIT 1
 Ln:9 SELECT completed successfully
 Ln:9 SELECT INTO statement executed
 Ln:9 COLUMN: _c0, bigint

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/db/select_into2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/select_into2.out.txt b/hplsql/src/test/results/db/select_into2.out.txt
index 03e67ad..582fdfb 100644
--- a/hplsql/src/test/results/db/select_into2.out.txt
+++ b/hplsql/src/test/results/db/select_into2.out.txt
@@ -2,7 +2,9 @@ Ln:1 DECLARE v_float float
 Ln:2 DECLARE v_double double
 Ln:3 DECLARE v_double2 double precision
 Ln:5 SELECT
-Ln:5 select cast(1.1 as float), cast(1.1 as double), cast(1.1 as double) from src LIMIT 1
+Ln:5 select
+  cast(1.1 as float), cast(1.1 as double), cast(1.1 as double)
+from src LIMIT 1
 Ln:5 SELECT completed successfully
 Ln:5 SELECT INTO statement executed
 Ln:5 COLUMN: _c0, float

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/local/if3_bteq.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/if3_bteq.out.txt b/hplsql/src/test/results/local/if3_bteq.out.txt
new file mode 100644
index 0000000..47f3010
--- /dev/null
+++ b/hplsql/src/test/results/local/if3_bteq.out.txt
@@ -0,0 +1,3 @@
+Ln:1 IF
+Ln:1 IF TRUE executed
+Ln:1 QUIT
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/local/lang.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/lang.out.txt b/hplsql/src/test/results/local/lang.out.txt
index 0047ec4..b3c460a 100644
--- a/hplsql/src/test/results/local/lang.out.txt
+++ b/hplsql/src/test/results/local/lang.out.txt
@@ -7,19 +7,19 @@
 -1.0
 Ln:19 DECLARE abc int
 Ln:20 DECLARE abc.abc int
-Ln:21 DECLARE abc . abc1 int
+Ln:21 DECLARE abc.abc1 int
 Ln:22 DECLARE "abc" int
 Ln:23 DECLARE "abc".abc int
 Ln:24 DECLARE "abc"."abc" int
-Ln:25 DECLARE "abc" . "abc1" int
+Ln:25 DECLARE "abc"."abc1" int
 Ln:26 DECLARE [abc] int
 Ln:27 DECLARE [abc].abc int
 Ln:28 DECLARE [abc].[abc] int
-Ln:29 DECLARE [abc] . [abc1] int
+Ln:29 DECLARE [abc].[abc1] int
 Ln:30 DECLARE `abc` int
 Ln:31 DECLARE `abc`.abc int
 Ln:32 DECLARE `abc`.`abc` int
-Ln:33 DECLARE `abc` . `abc1` int
+Ln:33 DECLARE `abc`.`abc1` int
 Ln:34 DECLARE :new.abc int
 Ln:35 DECLARE @abc int
 Ln:36 DECLARE _abc int
@@ -31,4 +31,4 @@ Ln:40 DECLARE abc_9 int
 2
 0
 -2
-0
+0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/create_table_mssql.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/create_table_mssql.out.txt b/hplsql/src/test/results/offline/create_table_mssql.out.txt
index 43b0aa7..29d03d6 100644
--- a/hplsql/src/test/results/offline/create_table_mssql.out.txt
+++ b/hplsql/src/test/results/offline/create_table_mssql.out.txt
@@ -1,24 +1,27 @@
 Ln:1 CREATE TABLE
-Ln:1 CREATE TABLE mssql_t1 (d1 TIMESTAMP,
-nc1 STRING,
-n1 DECIMAL(3,0),
-n2 DECIMAL(3),
-n3 DECIMAL,
-v1 STRING,
-nv1 STRING,
-nv2 STRING
+Ln:1 CREATE TABLE mssql_t1 (
+  d1   TIMESTAMP,
+  nc1  STRING,
+  n1   DECIMAL(3,0),
+  n2   DECIMAL(3),
+  n3   DECIMAL,
+  v1   STRING,
+  nv1  STRING,
+  nv2  STRING
 )
 Ln:12 CREATE TABLE
-Ln:12 CREATE TABLE `mssql_t2` (`i1` INT,
-`v1` VARCHAR(350),
-`v2` STRING,
-`b1` TINYINT,
-`d1` TIMESTAMP
+Ln:12 CREATE TABLE `mssql_t2`(
+ `i1` INT,
+ `v1` VARCHAR(350),
+ `v2` STRING,
+ `b1` TINYINT,
+ `d1` TIMESTAMP
 )
 Ln:31 CREATE TABLE
-Ln:31 CREATE TABLE `default`.`mssql_t3` (`v1` VARCHAR(50),
-`s2` SMALLINT,
-`sd1` TIMESTAMP,
-`i1` INT,
-`v2` VARCHAR(100)
+Ln:31 CREATE TABLE `default`.`mssql_t3`(
+ `v1`  VARCHAR(50),
+ `s2`  SMALLINT,
+ `sd1` TIMESTAMP,
+ `i1`  INT,
+ `v2`  VARCHAR(100)
 )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/create_table_mssql2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/create_table_mssql2.out.txt b/hplsql/src/test/results/offline/create_table_mssql2.out.txt
index a765c4a..8341411 100644
--- a/hplsql/src/test/results/offline/create_table_mssql2.out.txt
+++ b/hplsql/src/test/results/offline/create_table_mssql2.out.txt
@@ -1,10 +1,11 @@
 Ln:1 USE
 Ln:1 SQL statement: USE `mic.gr`
 Ln:14 CREATE TABLE
-Ln:14 CREATE TABLE `downloads` (`id` int,
-`fileName` char(255),
-`fileType` char(10),
-`downloads` int,
-`fromDate` char(40),
-`untilDate` char(40)
+Ln:14 CREATE TABLE `downloads`(
+	`id` int,
+	`fileName` char(255),
+	`fileType` char(10),
+	`downloads` int,
+	`fromDate` char(40),
+	`untilDate` char(40)
 )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/create_table_mysql.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/create_table_mysql.out.txt b/hplsql/src/test/results/offline/create_table_mysql.out.txt
index b835135..d07796f 100644
--- a/hplsql/src/test/results/offline/create_table_mysql.out.txt
+++ b/hplsql/src/test/results/offline/create_table_mysql.out.txt
@@ -1,4 +1,5 @@
 Ln:1 CREATE TABLE
-Ln:1 CREATE TABLE `users` (`id` int,
-`name` STRING
+Ln:1 CREATE TABLE IF NOT EXISTS `users` (
+  `id` int,
+  `name` STRING
 ) COMMENT 'users table'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/create_table_ora.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/create_table_ora.out.txt b/hplsql/src/test/results/offline/create_table_ora.out.txt
index cf30c0f..972e00a 100644
--- a/hplsql/src/test/results/offline/create_table_ora.out.txt
+++ b/hplsql/src/test/results/offline/create_table_ora.out.txt
@@ -1,42 +1,49 @@
 Ln:1 CREATE TABLE
-Ln:1 CREATE TABLE ora_t1 (n1 DECIMAL(3,0),
-v1 STRING
+Ln:1 CREATE TABLE ora_t1 (
+  n1   DECIMAL(3,0),
+  v1   STRING
 )
 Ln:6 CREATE TABLE
-Ln:6 CREATE TABLE `USER`.`EMP` (`EMPNO` DECIMAL(4,0),
-`ENAME` STRING,
-`JOB` STRING,
-`MGR` DECIMAL(4,0),
-`HIREDATE` DATE,
-`SAL` DECIMAL(7,2),
-`COMM` DECIMAL(7,2),
-`DEPTNO` DECIMAL(2,0)
-)
+Ln:6 CREATE TABLE `USER`.`EMP` 
+   (    `EMPNO` DECIMAL(4,0), 
+        `ENAME` STRING, 
+        `JOB` STRING, 
+        `MGR` DECIMAL(4,0), 
+        `HIREDATE` DATE, 
+        `SAL` DECIMAL(7,2), 
+        `COMM` DECIMAL(7,2), 
+        `DEPTNO` DECIMAL(2,0)
+   )
 Ln:21 CREATE TABLE
-Ln:21 CREATE TABLE language (id DECIMAL(7),
-cd CHAR(2),
-description STRING
+Ln:21 CREATE TABLE language (
+  id              DECIMAL(7),
+  cd              CHAR(2),
+  description     STRING
 )
 Ln:26 CREATE TABLE
-Ln:26 CREATE TABLE author (id DECIMAL(7),
-first_name STRING,
-last_name STRING,
-date_of_birth DATE,
-year_of_birth DECIMAL(7),
-distinguished DECIMAL(1)
+Ln:26 CREATE TABLE author (
+  id              DECIMAL(7),
+  first_name      STRING,
+  last_name       STRING,
+  date_of_birth   DATE,
+  year_of_birth   DECIMAL(7),
+  distinguished   DECIMAL(1)
 )
 Ln:34 CREATE TABLE
-Ln:34 CREATE TABLE book (id DECIMAL(7),
-author_id DECIMAL(7),
-title STRING,
-published_in DECIMAL(7),
-language_id DECIMAL(7)
+Ln:34 CREATE TABLE book (
+  id              DECIMAL(7),
+  author_id       DECIMAL(7),
+  title           STRING,
+  published_in    DECIMAL(7),
+  language_id     DECIMAL(7)
 )
 Ln:43 CREATE TABLE
-Ln:43 CREATE TABLE book_store (name STRING
+Ln:43 CREATE TABLE book_store (
+  name            STRING
 )
 Ln:46 CREATE TABLE
-Ln:46 CREATE TABLE book_to_book_store (name STRING,
-book_id INTEGER,
-stock INTEGER
+Ln:46 CREATE TABLE book_to_book_store (
+  name            STRING,
+  book_id         INTEGER,
+  stock           INTEGER
 )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/create_table_ora2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/create_table_ora2.out.txt b/hplsql/src/test/results/offline/create_table_ora2.out.txt
index 5d4e107..03f54e8 100644
--- a/hplsql/src/test/results/offline/create_table_ora2.out.txt
+++ b/hplsql/src/test/results/offline/create_table_ora2.out.txt
@@ -1,5 +1,6 @@
 Ln:1 CREATE TABLE
-Ln:1 CREATE TABLE `default`.`AUDIT_LOGS` (`RUN_ID` STRING,
-`FILE_NAME` STRING,
-`RUN_DATE` DATE
-)
\ No newline at end of file
+Ln:1 CREATE TABLE `default`.`AUDIT_LOGS`
+ ( `RUN_ID` STRING,
+   `FILE_NAME` STRING,
+   `RUN_DATE` DATE
+ )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/create_table_pg.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/create_table_pg.out.txt b/hplsql/src/test/results/offline/create_table_pg.out.txt
index cad5488..095eb12 100644
--- a/hplsql/src/test/results/offline/create_table_pg.out.txt
+++ b/hplsql/src/test/results/offline/create_table_pg.out.txt
@@ -1,5 +1,6 @@
 Ln:1 CREATE TABLE
-Ln:1 create table i1 (c1 SMALLINT,
-c2 INT,
-c3 BIGINT
+Ln:1 create table i1 (
+  c1 SMALLINT,
+  c2 INT,
+  c3 BIGINT
 )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/create_table_td.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/create_table_td.out.txt b/hplsql/src/test/results/offline/create_table_td.out.txt
new file mode 100644
index 0000000..9b9d561
--- /dev/null
+++ b/hplsql/src/test/results/offline/create_table_td.out.txt
@@ -0,0 +1,31 @@
+Ln:1 CREATE TABLE
+Ln:1 CREATE TABLE tab
+  (
+    SOURCE_ID  INT,
+    RUN_ID     INT,
+    STATUS     CHAR,
+    LOAD_START timestamp(0),
+    LOAD_END   timestamp(0)
+  )
+Ln:10 CREATE TABLE
+Ln:10 CREATE TABLE ctl
+AS
+(
+        SELECT
+                EBC.SOURCE_ID, MAX(EBC.RUN_ID) AS RUN_ID, EBC.STATUS, EBC.LOAD_START, EBC.LOAD_END
+        FROM EBC
+        WHERE EBC.SOURCE_ID = 451 AND EBC.STATUS = 'R'
+        GROUP BY
+                1,3,4,5
+)
+Ln:28 CREATE LOCAL TEMPORARY TABLE
+CREATE TEMPORARY TABLE ctl2
+AS
+(
+        SELECT
+                EBC.SOURCE_ID, MAX(EBC.RUN_ID) AS RUN_ID, EBC.STATUS, EBC.LOAD_START, EBC.LOAD_END
+        FROM EBC
+        WHERE EBC.SOURCE_ID = 451 AND EBC.STATUS = 'R'
+        GROUP BY
+                1,3,4,5
+)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/delete_all.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/delete_all.out.txt b/hplsql/src/test/results/offline/delete_all.out.txt
new file mode 100644
index 0000000..0cecc95
--- /dev/null
+++ b/hplsql/src/test/results/offline/delete_all.out.txt
@@ -0,0 +1,2 @@
+Ln:1 DELETE
+Ln:1 TRUNCATE TABLE TEST1_DB.WK_WRK
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/select.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/select.out.txt b/hplsql/src/test/results/offline/select.out.txt
new file mode 100644
index 0000000..529f0b5
--- /dev/null
+++ b/hplsql/src/test/results/offline/select.out.txt
@@ -0,0 +1,34 @@
+Ln:1 SELECT
+Ln:1 SELECT * FROM a where 1 = 1 and not exists (select * from b)
+Ln:1 Not executed - offline mode set
+Ln:3 SELECT
+Ln:3 SELECT * 
+ FROM a 
+ where not exists 
+  (
+    select * from b
+  )
+Ln:3 Not executed - offline mode set
+Ln:10 SELECT
+Ln:10 SELECT 
+         *
+         FROM tab
+         WHERE FILE_DATE > (
+                           SELECT 
+                           MAX(FILE_DATE) AS MX_C_FILE_DT
+                           FROM tab
+                           WHERE FLAG = 'C' AND IND = 'C' AND FILE_DATE < ( SELECT 
+                                            CAST( LOAD_START AS DATE) 
+                                            FROM tab  
+                                            WHERE SOURCE_ID = 451 AND BATCH = 'R'
+                		                   )
+		                  )
+Ln:10 Not executed - offline mode set
+Ln:31 SELECT
+Ln:31 SELECT 
+*
+FROM DLTA_POC LEFT OUTER JOIN TEST3_DB.TET ORG ON DLTA_POC.YS_NO = ORG.EM_CODE_A
+   AND DLTA_POC.AREA_NO = ORG.AREA_CODE_2
+   AND DLTA_POC.GNT_POC = ORG.GEN_CD LEFT OUTER JOIN TEST.LOCATION LOC ON DLTA_POC.SE_KEY_POC = LOC.LOC_ID
+   AND LOC.LOCATION_END_DT = DATE '9999-12-31'
+Ln:31 Not executed - offline mode set
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/select_db2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/select_db2.out.txt b/hplsql/src/test/results/offline/select_db2.out.txt
index 1d64e8a..bb5b455 100644
--- a/hplsql/src/test/results/offline/select_db2.out.txt
+++ b/hplsql/src/test/results/offline/select_db2.out.txt
@@ -2,5 +2,6 @@ Ln:1 SELECT
 Ln:1 select coalesce(max(info_id) + 1, 0) from sproc_info
 Ln:1 Not executed - offline mode set
 Ln:3 SELECT
-Ln:3 select cd, cd + inc days, cd - inc days + coalesce(inc, 0) days from (select date '2015-09-02' as cd, 3 as inc from sysibm.sysdummy1)
+Ln:3 select cd, cd + inc days, cd - inc days + coalesce(inc, 0) days
+from (select date '2015-09-02' as cd, 3 as inc from sysibm.sysdummy1)
 Ln:3 Not executed - offline mode set
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2d33d091/hplsql/src/test/results/offline/select_teradata.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/offline/select_teradata.out.txt b/hplsql/src/test/results/offline/select_teradata.out.txt
new file mode 100644
index 0000000..34ab433
--- /dev/null
+++ b/hplsql/src/test/results/offline/select_teradata.out.txt
@@ -0,0 +1,10 @@
+Ln:1 SELECT
+Ln:1 SELECT branch_code, branch_no, c_no, cd_type
+FROM EMPLOYEE
+    WHERE S_CODE = 'C' AND (branch_no) NOT IN (
+            SELECT branch_code
+            FROM DEPARTMENT
+            WHERE branch_code = 'ABC'
+        )
+QUALIFY ROW_NUMBER() OVER (PARTITION BY c_no ORDER BY cd_type) = 1
+Ln:1 Not executed - offline mode set
\ No newline at end of file


[20/50] [abbrv] hive git commit: HIVE-13351: Support drop Primary Key/Foreign Key constraints (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 690c895..2734a1c 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size749;
-            ::apache::thrift::protocol::TType _etype752;
-            xfer += iprot->readListBegin(_etype752, _size749);
-            this->success.resize(_size749);
-            uint32_t _i753;
-            for (_i753 = 0; _i753 < _size749; ++_i753)
+            uint32_t _size751;
+            ::apache::thrift::protocol::TType _etype754;
+            xfer += iprot->readListBegin(_etype754, _size751);
+            this->success.resize(_size751);
+            uint32_t _i755;
+            for (_i755 = 0; _i755 < _size751; ++_i755)
             {
-              xfer += iprot->readString(this->success[_i753]);
+              xfer += iprot->readString(this->success[_i755]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter754;
-      for (_iter754 = this->success.begin(); _iter754 != this->success.end(); ++_iter754)
+      std::vector<std::string> ::const_iterator _iter756;
+      for (_iter756 = this->success.begin(); _iter756 != this->success.end(); ++_iter756)
       {
-        xfer += oprot->writeString((*_iter754));
+        xfer += oprot->writeString((*_iter756));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size755;
-            ::apache::thrift::protocol::TType _etype758;
-            xfer += iprot->readListBegin(_etype758, _size755);
-            (*(this->success)).resize(_size755);
-            uint32_t _i759;
-            for (_i759 = 0; _i759 < _size755; ++_i759)
+            uint32_t _size757;
+            ::apache::thrift::protocol::TType _etype760;
+            xfer += iprot->readListBegin(_etype760, _size757);
+            (*(this->success)).resize(_size757);
+            uint32_t _i761;
+            for (_i761 = 0; _i761 < _size757; ++_i761)
             {
-              xfer += iprot->readString((*(this->success))[_i759]);
+              xfer += iprot->readString((*(this->success))[_i761]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size760;
-            ::apache::thrift::protocol::TType _etype763;
-            xfer += iprot->readListBegin(_etype763, _size760);
-            this->success.resize(_size760);
-            uint32_t _i764;
-            for (_i764 = 0; _i764 < _size760; ++_i764)
+            uint32_t _size762;
+            ::apache::thrift::protocol::TType _etype765;
+            xfer += iprot->readListBegin(_etype765, _size762);
+            this->success.resize(_size762);
+            uint32_t _i766;
+            for (_i766 = 0; _i766 < _size762; ++_i766)
             {
-              xfer += iprot->readString(this->success[_i764]);
+              xfer += iprot->readString(this->success[_i766]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter765;
-      for (_iter765 = this->success.begin(); _iter765 != this->success.end(); ++_iter765)
+      std::vector<std::string> ::const_iterator _iter767;
+      for (_iter767 = this->success.begin(); _iter767 != this->success.end(); ++_iter767)
       {
-        xfer += oprot->writeString((*_iter765));
+        xfer += oprot->writeString((*_iter767));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size766;
-            ::apache::thrift::protocol::TType _etype769;
-            xfer += iprot->readListBegin(_etype769, _size766);
-            (*(this->success)).resize(_size766);
-            uint32_t _i770;
-            for (_i770 = 0; _i770 < _size766; ++_i770)
+            uint32_t _size768;
+            ::apache::thrift::protocol::TType _etype771;
+            xfer += iprot->readListBegin(_etype771, _size768);
+            (*(this->success)).resize(_size768);
+            uint32_t _i772;
+            for (_i772 = 0; _i772 < _size768; ++_i772)
             {
-              xfer += iprot->readString((*(this->success))[_i770]);
+              xfer += iprot->readString((*(this->success))[_i772]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size771;
-            ::apache::thrift::protocol::TType _ktype772;
-            ::apache::thrift::protocol::TType _vtype773;
-            xfer += iprot->readMapBegin(_ktype772, _vtype773, _size771);
-            uint32_t _i775;
-            for (_i775 = 0; _i775 < _size771; ++_i775)
+            uint32_t _size773;
+            ::apache::thrift::protocol::TType _ktype774;
+            ::apache::thrift::protocol::TType _vtype775;
+            xfer += iprot->readMapBegin(_ktype774, _vtype775, _size773);
+            uint32_t _i777;
+            for (_i777 = 0; _i777 < _size773; ++_i777)
             {
-              std::string _key776;
-              xfer += iprot->readString(_key776);
-              Type& _val777 = this->success[_key776];
-              xfer += _val777.read(iprot);
+              std::string _key778;
+              xfer += iprot->readString(_key778);
+              Type& _val779 = this->success[_key778];
+              xfer += _val779.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter778;
-      for (_iter778 = this->success.begin(); _iter778 != this->success.end(); ++_iter778)
+      std::map<std::string, Type> ::const_iterator _iter780;
+      for (_iter780 = this->success.begin(); _iter780 != this->success.end(); ++_iter780)
       {
-        xfer += oprot->writeString(_iter778->first);
-        xfer += _iter778->second.write(oprot);
+        xfer += oprot->writeString(_iter780->first);
+        xfer += _iter780->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size779;
-            ::apache::thrift::protocol::TType _ktype780;
-            ::apache::thrift::protocol::TType _vtype781;
-            xfer += iprot->readMapBegin(_ktype780, _vtype781, _size779);
-            uint32_t _i783;
-            for (_i783 = 0; _i783 < _size779; ++_i783)
+            uint32_t _size781;
+            ::apache::thrift::protocol::TType _ktype782;
+            ::apache::thrift::protocol::TType _vtype783;
+            xfer += iprot->readMapBegin(_ktype782, _vtype783, _size781);
+            uint32_t _i785;
+            for (_i785 = 0; _i785 < _size781; ++_i785)
             {
-              std::string _key784;
-              xfer += iprot->readString(_key784);
-              Type& _val785 = (*(this->success))[_key784];
-              xfer += _val785.read(iprot);
+              std::string _key786;
+              xfer += iprot->readString(_key786);
+              Type& _val787 = (*(this->success))[_key786];
+              xfer += _val787.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size786;
-            ::apache::thrift::protocol::TType _etype789;
-            xfer += iprot->readListBegin(_etype789, _size786);
-            this->success.resize(_size786);
-            uint32_t _i790;
-            for (_i790 = 0; _i790 < _size786; ++_i790)
+            uint32_t _size788;
+            ::apache::thrift::protocol::TType _etype791;
+            xfer += iprot->readListBegin(_etype791, _size788);
+            this->success.resize(_size788);
+            uint32_t _i792;
+            for (_i792 = 0; _i792 < _size788; ++_i792)
             {
-              xfer += this->success[_i790].read(iprot);
+              xfer += this->success[_i792].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter791;
-      for (_iter791 = this->success.begin(); _iter791 != this->success.end(); ++_iter791)
+      std::vector<FieldSchema> ::const_iterator _iter793;
+      for (_iter793 = this->success.begin(); _iter793 != this->success.end(); ++_iter793)
       {
-        xfer += (*_iter791).write(oprot);
+        xfer += (*_iter793).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size792;
-            ::apache::thrift::protocol::TType _etype795;
-            xfer += iprot->readListBegin(_etype795, _size792);
-            (*(this->success)).resize(_size792);
-            uint32_t _i796;
-            for (_i796 = 0; _i796 < _size792; ++_i796)
+            uint32_t _size794;
+            ::apache::thrift::protocol::TType _etype797;
+            xfer += iprot->readListBegin(_etype797, _size794);
+            (*(this->success)).resize(_size794);
+            uint32_t _i798;
+            for (_i798 = 0; _i798 < _size794; ++_i798)
             {
-              xfer += (*(this->success))[_i796].read(iprot);
+              xfer += (*(this->success))[_i798].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size797;
-            ::apache::thrift::protocol::TType _etype800;
-            xfer += iprot->readListBegin(_etype800, _size797);
-            this->success.resize(_size797);
-            uint32_t _i801;
-            for (_i801 = 0; _i801 < _size797; ++_i801)
+            uint32_t _size799;
+            ::apache::thrift::protocol::TType _etype802;
+            xfer += iprot->readListBegin(_etype802, _size799);
+            this->success.resize(_size799);
+            uint32_t _i803;
+            for (_i803 = 0; _i803 < _size799; ++_i803)
             {
-              xfer += this->success[_i801].read(iprot);
+              xfer += this->success[_i803].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter802;
-      for (_iter802 = this->success.begin(); _iter802 != this->success.end(); ++_iter802)
+      std::vector<FieldSchema> ::const_iterator _iter804;
+      for (_iter804 = this->success.begin(); _iter804 != this->success.end(); ++_iter804)
       {
-        xfer += (*_iter802).write(oprot);
+        xfer += (*_iter804).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size803;
-            ::apache::thrift::protocol::TType _etype806;
-            xfer += iprot->readListBegin(_etype806, _size803);
-            (*(this->success)).resize(_size803);
-            uint32_t _i807;
-            for (_i807 = 0; _i807 < _size803; ++_i807)
+            uint32_t _size805;
+            ::apache::thrift::protocol::TType _etype808;
+            xfer += iprot->readListBegin(_etype808, _size805);
+            (*(this->success)).resize(_size805);
+            uint32_t _i809;
+            for (_i809 = 0; _i809 < _size805; ++_i809)
             {
-              xfer += (*(this->success))[_i807].read(iprot);
+              xfer += (*(this->success))[_i809].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size808;
-            ::apache::thrift::protocol::TType _etype811;
-            xfer += iprot->readListBegin(_etype811, _size808);
-            this->success.resize(_size808);
-            uint32_t _i812;
-            for (_i812 = 0; _i812 < _size808; ++_i812)
+            uint32_t _size810;
+            ::apache::thrift::protocol::TType _etype813;
+            xfer += iprot->readListBegin(_etype813, _size810);
+            this->success.resize(_size810);
+            uint32_t _i814;
+            for (_i814 = 0; _i814 < _size810; ++_i814)
             {
-              xfer += this->success[_i812].read(iprot);
+              xfer += this->success[_i814].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter813;
-      for (_iter813 = this->success.begin(); _iter813 != this->success.end(); ++_iter813)
+      std::vector<FieldSchema> ::const_iterator _iter815;
+      for (_iter815 = this->success.begin(); _iter815 != this->success.end(); ++_iter815)
       {
-        xfer += (*_iter813).write(oprot);
+        xfer += (*_iter815).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size814;
-            ::apache::thrift::protocol::TType _etype817;
-            xfer += iprot->readListBegin(_etype817, _size814);
-            (*(this->success)).resize(_size814);
-            uint32_t _i818;
-            for (_i818 = 0; _i818 < _size814; ++_i818)
+            uint32_t _size816;
+            ::apache::thrift::protocol::TType _etype819;
+            xfer += iprot->readListBegin(_etype819, _size816);
+            (*(this->success)).resize(_size816);
+            uint32_t _i820;
+            for (_i820 = 0; _i820 < _size816; ++_i820)
             {
-              xfer += (*(this->success))[_i818].read(iprot);
+              xfer += (*(this->success))[_i820].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size819;
-            ::apache::thrift::protocol::TType _etype822;
-            xfer += iprot->readListBegin(_etype822, _size819);
-            this->success.resize(_size819);
-            uint32_t _i823;
-            for (_i823 = 0; _i823 < _size819; ++_i823)
+            uint32_t _size821;
+            ::apache::thrift::protocol::TType _etype824;
+            xfer += iprot->readListBegin(_etype824, _size821);
+            this->success.resize(_size821);
+            uint32_t _i825;
+            for (_i825 = 0; _i825 < _size821; ++_i825)
             {
-              xfer += this->success[_i823].read(iprot);
+              xfer += this->success[_i825].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter824;
-      for (_iter824 = this->success.begin(); _iter824 != this->success.end(); ++_iter824)
+      std::vector<FieldSchema> ::const_iterator _iter826;
+      for (_iter826 = this->success.begin(); _iter826 != this->success.end(); ++_iter826)
       {
-        xfer += (*_iter824).write(oprot);
+        xfer += (*_iter826).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size825;
-            ::apache::thrift::protocol::TType _etype828;
-            xfer += iprot->readListBegin(_etype828, _size825);
-            (*(this->success)).resize(_size825);
-            uint32_t _i829;
-            for (_i829 = 0; _i829 < _size825; ++_i829)
+            uint32_t _size827;
+            ::apache::thrift::protocol::TType _etype830;
+            xfer += iprot->readListBegin(_etype830, _size827);
+            (*(this->success)).resize(_size827);
+            uint32_t _i831;
+            for (_i831 = 0; _i831 < _size827; ++_i831)
             {
-              xfer += (*(this->success))[_i829].read(iprot);
+              xfer += (*(this->success))[_i831].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4518,14 +4518,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size830;
-            ::apache::thrift::protocol::TType _etype833;
-            xfer += iprot->readListBegin(_etype833, _size830);
-            this->primaryKeys.resize(_size830);
-            uint32_t _i834;
-            for (_i834 = 0; _i834 < _size830; ++_i834)
+            uint32_t _size832;
+            ::apache::thrift::protocol::TType _etype835;
+            xfer += iprot->readListBegin(_etype835, _size832);
+            this->primaryKeys.resize(_size832);
+            uint32_t _i836;
+            for (_i836 = 0; _i836 < _size832; ++_i836)
             {
-              xfer += this->primaryKeys[_i834].read(iprot);
+              xfer += this->primaryKeys[_i836].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4538,14 +4538,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size835;
-            ::apache::thrift::protocol::TType _etype838;
-            xfer += iprot->readListBegin(_etype838, _size835);
-            this->foreignKeys.resize(_size835);
-            uint32_t _i839;
-            for (_i839 = 0; _i839 < _size835; ++_i839)
+            uint32_t _size837;
+            ::apache::thrift::protocol::TType _etype840;
+            xfer += iprot->readListBegin(_etype840, _size837);
+            this->foreignKeys.resize(_size837);
+            uint32_t _i841;
+            for (_i841 = 0; _i841 < _size837; ++_i841)
             {
-              xfer += this->foreignKeys[_i839].read(iprot);
+              xfer += this->foreignKeys[_i841].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4578,10 +4578,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter840;
-    for (_iter840 = this->primaryKeys.begin(); _iter840 != this->primaryKeys.end(); ++_iter840)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter842;
+    for (_iter842 = this->primaryKeys.begin(); _iter842 != this->primaryKeys.end(); ++_iter842)
     {
-      xfer += (*_iter840).write(oprot);
+      xfer += (*_iter842).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4590,10 +4590,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter841;
-    for (_iter841 = this->foreignKeys.begin(); _iter841 != this->foreignKeys.end(); ++_iter841)
+    std::vector<SQLForeignKey> ::const_iterator _iter843;
+    for (_iter843 = this->foreignKeys.begin(); _iter843 != this->foreignKeys.end(); ++_iter843)
     {
-      xfer += (*_iter841).write(oprot);
+      xfer += (*_iter843).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4621,10 +4621,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter842;
-    for (_iter842 = (*(this->primaryKeys)).begin(); _iter842 != (*(this->primaryKeys)).end(); ++_iter842)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter844;
+    for (_iter844 = (*(this->primaryKeys)).begin(); _iter844 != (*(this->primaryKeys)).end(); ++_iter844)
     {
-      xfer += (*_iter842).write(oprot);
+      xfer += (*_iter844).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4633,10 +4633,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter843;
-    for (_iter843 = (*(this->foreignKeys)).begin(); _iter843 != (*(this->foreignKeys)).end(); ++_iter843)
+    std::vector<SQLForeignKey> ::const_iterator _iter845;
+    for (_iter845 = (*(this->foreignKeys)).begin(); _iter845 != (*(this->foreignKeys)).end(); ++_iter845)
     {
-      xfer += (*_iter843).write(oprot);
+      xfer += (*_iter845).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4816,6 +4816,213 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_presult::read(::apach
 }
 
 
+ThriftHiveMetastore_drop_constraint_args::~ThriftHiveMetastore_drop_constraint_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_drop_constraint_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->req.read(iprot);
+          this->__isset.req = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_drop_constraint_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_constraint_args");
+
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->req.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_drop_constraint_pargs::~ThriftHiveMetastore_drop_constraint_pargs() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_drop_constraint_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_constraint_pargs");
+
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->req)).write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_drop_constraint_result::~ThriftHiveMetastore_drop_constraint_result() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_drop_constraint_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o3.read(iprot);
+          this->__isset.o3 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_drop_constraint_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_constraint_result");
+
+  if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o3) {
+    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2);
+    xfer += this->o3.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_drop_constraint_presult::~ThriftHiveMetastore_drop_constraint_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_drop_constraint_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o3.read(iprot);
+          this->__isset.o3 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
 ThriftHiveMetastore_drop_table_args::~ThriftHiveMetastore_drop_table_args() throw() {
 }
 
@@ -5434,14 +5641,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size844;
-            ::apache::thrift::protocol::TType _etype847;
-            xfer += iprot->readListBegin(_etype847, _size844);
-            this->success.resize(_size844);
-            uint32_t _i848;
-            for (_i848 = 0; _i848 < _size844; ++_i848)
+            uint32_t _size846;
+            ::apache::thrift::protocol::TType _etype849;
+            xfer += iprot->readListBegin(_etype849, _size846);
+            this->success.resize(_size846);
+            uint32_t _i850;
+            for (_i850 = 0; _i850 < _size846; ++_i850)
             {
-              xfer += iprot->readString(this->success[_i848]);
+              xfer += iprot->readString(this->success[_i850]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5480,10 +5687,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter849;
-      for (_iter849 = this->success.begin(); _iter849 != this->success.end(); ++_iter849)
+      std::vector<std::string> ::const_iterator _iter851;
+      for (_iter851 = this->success.begin(); _iter851 != this->success.end(); ++_iter851)
       {
-        xfer += oprot->writeString((*_iter849));
+        xfer += oprot->writeString((*_iter851));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5528,14 +5735,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size850;
-            ::apache::thrift::protocol::TType _etype853;
-            xfer += iprot->readListBegin(_etype853, _size850);
-            (*(this->success)).resize(_size850);
-            uint32_t _i854;
-            for (_i854 = 0; _i854 < _size850; ++_i854)
+            uint32_t _size852;
+            ::apache::thrift::protocol::TType _etype855;
+            xfer += iprot->readListBegin(_etype855, _size852);
+            (*(this->success)).resize(_size852);
+            uint32_t _i856;
+            for (_i856 = 0; _i856 < _size852; ++_i856)
             {
-              xfer += iprot->readString((*(this->success))[_i854]);
+              xfer += iprot->readString((*(this->success))[_i856]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5610,14 +5817,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size855;
-            ::apache::thrift::protocol::TType _etype858;
-            xfer += iprot->readListBegin(_etype858, _size855);
-            this->tbl_types.resize(_size855);
-            uint32_t _i859;
-            for (_i859 = 0; _i859 < _size855; ++_i859)
+            uint32_t _size857;
+            ::apache::thrift::protocol::TType _etype860;
+            xfer += iprot->readListBegin(_etype860, _size857);
+            this->tbl_types.resize(_size857);
+            uint32_t _i861;
+            for (_i861 = 0; _i861 < _size857; ++_i861)
             {
-              xfer += iprot->readString(this->tbl_types[_i859]);
+              xfer += iprot->readString(this->tbl_types[_i861]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5654,10 +5861,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-    std::vector<std::string> ::const_iterator _iter860;
-    for (_iter860 = this->tbl_types.begin(); _iter860 != this->tbl_types.end(); ++_iter860)
+    std::vector<std::string> ::const_iterator _iter862;
+    for (_iter862 = this->tbl_types.begin(); _iter862 != this->tbl_types.end(); ++_iter862)
     {
-      xfer += oprot->writeString((*_iter860));
+      xfer += oprot->writeString((*_iter862));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5689,10 +5896,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-    std::vector<std::string> ::const_iterator _iter861;
-    for (_iter861 = (*(this->tbl_types)).begin(); _iter861 != (*(this->tbl_types)).end(); ++_iter861)
+    std::vector<std::string> ::const_iterator _iter863;
+    for (_iter863 = (*(this->tbl_types)).begin(); _iter863 != (*(this->tbl_types)).end(); ++_iter863)
     {
-      xfer += oprot->writeString((*_iter861));
+      xfer += oprot->writeString((*_iter863));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5733,14 +5940,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size862;
-            ::apache::thrift::protocol::TType _etype865;
-            xfer += iprot->readListBegin(_etype865, _size862);
-            this->success.resize(_size862);
-            uint32_t _i866;
-            for (_i866 = 0; _i866 < _size862; ++_i866)
+            uint32_t _size864;
+            ::apache::thrift::protocol::TType _etype867;
+            xfer += iprot->readListBegin(_etype867, _size864);
+            this->success.resize(_size864);
+            uint32_t _i868;
+            for (_i868 = 0; _i868 < _size864; ++_i868)
             {
-              xfer += this->success[_i866].read(iprot);
+              xfer += this->success[_i868].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5779,10 +5986,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<TableMeta> ::const_iterator _iter867;
-      for (_iter867 = this->success.begin(); _iter867 != this->success.end(); ++_iter867)
+      std::vector<TableMeta> ::const_iterator _iter869;
+      for (_iter869 = this->success.begin(); _iter869 != this->success.end(); ++_iter869)
       {
-        xfer += (*_iter867).write(oprot);
+        xfer += (*_iter869).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5827,14 +6034,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size868;
-            ::apache::thrift::protocol::TType _etype871;
-            xfer += iprot->readListBegin(_etype871, _size868);
-            (*(this->success)).resize(_size868);
-            uint32_t _i872;
-            for (_i872 = 0; _i872 < _size868; ++_i872)
+            uint32_t _size870;
+            ::apache::thrift::protocol::TType _etype873;
+            xfer += iprot->readListBegin(_etype873, _size870);
+            (*(this->success)).resize(_size870);
+            uint32_t _i874;
+            for (_i874 = 0; _i874 < _size870; ++_i874)
             {
-              xfer += (*(this->success))[_i872].read(iprot);
+              xfer += (*(this->success))[_i874].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5972,14 +6179,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size873;
-            ::apache::thrift::protocol::TType _etype876;
-            xfer += iprot->readListBegin(_etype876, _size873);
-            this->success.resize(_size873);
-            uint32_t _i877;
-            for (_i877 = 0; _i877 < _size873; ++_i877)
+            uint32_t _size875;
+            ::apache::thrift::protocol::TType _etype878;
+            xfer += iprot->readListBegin(_etype878, _size875);
+            this->success.resize(_size875);
+            uint32_t _i879;
+            for (_i879 = 0; _i879 < _size875; ++_i879)
             {
-              xfer += iprot->readString(this->success[_i877]);
+              xfer += iprot->readString(this->success[_i879]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6018,10 +6225,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter878;
-      for (_iter878 = this->success.begin(); _iter878 != this->success.end(); ++_iter878)
+      std::vector<std::string> ::const_iterator _iter880;
+      for (_iter880 = this->success.begin(); _iter880 != this->success.end(); ++_iter880)
       {
-        xfer += oprot->writeString((*_iter878));
+        xfer += oprot->writeString((*_iter880));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6066,14 +6273,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size879;
-            ::apache::thrift::protocol::TType _etype882;
-            xfer += iprot->readListBegin(_etype882, _size879);
-            (*(this->success)).resize(_size879);
-            uint32_t _i883;
-            for (_i883 = 0; _i883 < _size879; ++_i883)
+            uint32_t _size881;
+            ::apache::thrift::protocol::TType _etype884;
+            xfer += iprot->readListBegin(_etype884, _size881);
+            (*(this->success)).resize(_size881);
+            uint32_t _i885;
+            for (_i885 = 0; _i885 < _size881; ++_i885)
             {
-              xfer += iprot->readString((*(this->success))[_i883]);
+              xfer += iprot->readString((*(this->success))[_i885]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6383,14 +6590,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size884;
-            ::apache::thrift::protocol::TType _etype887;
-            xfer += iprot->readListBegin(_etype887, _size884);
-            this->tbl_names.resize(_size884);
-            uint32_t _i888;
-            for (_i888 = 0; _i888 < _size884; ++_i888)
+            uint32_t _size886;
+            ::apache::thrift::protocol::TType _etype889;
+            xfer += iprot->readListBegin(_etype889, _size886);
+            this->tbl_names.resize(_size886);
+            uint32_t _i890;
+            for (_i890 = 0; _i890 < _size886; ++_i890)
             {
-              xfer += iprot->readString(this->tbl_names[_i888]);
+              xfer += iprot->readString(this->tbl_names[_i890]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6423,10 +6630,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter889;
-    for (_iter889 = this->tbl_names.begin(); _iter889 != this->tbl_names.end(); ++_iter889)
+    std::vector<std::string> ::const_iterator _iter891;
+    for (_iter891 = this->tbl_names.begin(); _iter891 != this->tbl_names.end(); ++_iter891)
     {
-      xfer += oprot->writeString((*_iter889));
+      xfer += oprot->writeString((*_iter891));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6454,10 +6661,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter890;
-    for (_iter890 = (*(this->tbl_names)).begin(); _iter890 != (*(this->tbl_names)).end(); ++_iter890)
+    std::vector<std::string> ::const_iterator _iter892;
+    for (_iter892 = (*(this->tbl_names)).begin(); _iter892 != (*(this->tbl_names)).end(); ++_iter892)
     {
-      xfer += oprot->writeString((*_iter890));
+      xfer += oprot->writeString((*_iter892));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6498,14 +6705,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size891;
-            ::apache::thrift::protocol::TType _etype894;
-            xfer += iprot->readListBegin(_etype894, _size891);
-            this->success.resize(_size891);
-            uint32_t _i895;
-            for (_i895 = 0; _i895 < _size891; ++_i895)
+            uint32_t _size893;
+            ::apache::thrift::protocol::TType _etype896;
+            xfer += iprot->readListBegin(_etype896, _size893);
+            this->success.resize(_size893);
+            uint32_t _i897;
+            for (_i897 = 0; _i897 < _size893; ++_i897)
             {
-              xfer += this->success[_i895].read(iprot);
+              xfer += this->success[_i897].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6560,10 +6767,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter896;
-      for (_iter896 = this->success.begin(); _iter896 != this->success.end(); ++_iter896)
+      std::vector<Table> ::const_iterator _iter898;
+      for (_iter898 = this->success.begin(); _iter898 != this->success.end(); ++_iter898)
       {
-        xfer += (*_iter896).write(oprot);
+        xfer += (*_iter898).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6616,14 +6823,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size897;
-            ::apache::thrift::protocol::TType _etype900;
-            xfer += iprot->readListBegin(_etype900, _size897);
-            (*(this->success)).resize(_size897);
-            uint32_t _i901;
-            for (_i901 = 0; _i901 < _size897; ++_i901)
+            uint32_t _size899;
+            ::apache::thrift::protocol::TType _etype902;
+            xfer += iprot->readListBegin(_etype902, _size899);
+            (*(this->success)).resize(_size899);
+            uint32_t _i903;
+            for (_i903 = 0; _i903 < _size899; ++_i903)
             {
-              xfer += (*(this->success))[_i901].read(iprot);
+              xfer += (*(this->success))[_i903].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6809,14 +7016,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size902;
-            ::apache::thrift::protocol::TType _etype905;
-            xfer += iprot->readListBegin(_etype905, _size902);
-            this->success.resize(_size902);
-            uint32_t _i906;
-            for (_i906 = 0; _i906 < _size902; ++_i906)
+            uint32_t _size904;
+            ::apache::thrift::protocol::TType _etype907;
+            xfer += iprot->readListBegin(_etype907, _size904);
+            this->success.resize(_size904);
+            uint32_t _i908;
+            for (_i908 = 0; _i908 < _size904; ++_i908)
             {
-              xfer += iprot->readString(this->success[_i906]);
+              xfer += iprot->readString(this->success[_i908]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6871,10 +7078,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter907;
-      for (_iter907 = this->success.begin(); _iter907 != this->success.end(); ++_iter907)
+      std::vector<std::string> ::const_iterator _iter909;
+      for (_iter909 = this->success.begin(); _iter909 != this->success.end(); ++_iter909)
       {
-        xfer += oprot->writeString((*_iter907));
+        xfer += oprot->writeString((*_iter909));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6927,14 +7134,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size908;
-            ::apache::thrift::protocol::TType _etype911;
-            xfer += iprot->readListBegin(_etype911, _size908);
-            (*(this->success)).resize(_size908);
-            uint32_t _i912;
-            for (_i912 = 0; _i912 < _size908; ++_i912)
+            uint32_t _size910;
+            ::apache::thrift::protocol::TType _etype913;
+            xfer += iprot->readListBegin(_etype913, _size910);
+            (*(this->success)).resize(_size910);
+            uint32_t _i914;
+            for (_i914 = 0; _i914 < _size910; ++_i914)
             {
-              xfer += iprot->readString((*(this->success))[_i912]);
+              xfer += iprot->readString((*(this->success))[_i914]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8268,14 +8475,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size913;
-            ::apache::thrift::protocol::TType _etype916;
-            xfer += iprot->readListBegin(_etype916, _size913);
-            this->new_parts.resize(_size913);
-            uint32_t _i917;
-            for (_i917 = 0; _i917 < _size913; ++_i917)
+            uint32_t _size915;
+            ::apache::thrift::protocol::TType _etype918;
+            xfer += iprot->readListBegin(_etype918, _size915);
+            this->new_parts.resize(_size915);
+            uint32_t _i919;
+            for (_i919 = 0; _i919 < _size915; ++_i919)
             {
-              xfer += this->new_parts[_i917].read(iprot);
+              xfer += this->new_parts[_i919].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8304,10 +8511,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter918;
-    for (_iter918 = this->new_parts.begin(); _iter918 != this->new_parts.end(); ++_iter918)
+    std::vector<Partition> ::const_iterator _iter920;
+    for (_iter920 = this->new_parts.begin(); _iter920 != this->new_parts.end(); ++_iter920)
     {
-      xfer += (*_iter918).write(oprot);
+      xfer += (*_iter920).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8331,10 +8538,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter919;
-    for (_iter919 = (*(this->new_parts)).begin(); _iter919 != (*(this->new_parts)).end(); ++_iter919)
+    std::vector<Partition> ::const_iterator _iter921;
+    for (_iter921 = (*(this->new_parts)).begin(); _iter921 != (*(this->new_parts)).end(); ++_iter921)
     {
-      xfer += (*_iter919).write(oprot);
+      xfer += (*_iter921).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8543,14 +8750,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size920;
-            ::apache::thrift::protocol::TType _etype923;
-            xfer += iprot->readListBegin(_etype923, _size920);
-            this->new_parts.resize(_size920);
-            uint32_t _i924;
-            for (_i924 = 0; _i924 < _size920; ++_i924)
+            uint32_t _size922;
+            ::apache::thrift::protocol::TType _etype925;
+            xfer += iprot->readListBegin(_etype925, _size922);
+            this->new_parts.resize(_size922);
+            uint32_t _i926;
+            for (_i926 = 0; _i926 < _size922; ++_i926)
             {
-              xfer += this->new_parts[_i924].read(iprot);
+              xfer += this->new_parts[_i926].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8579,10 +8786,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter925;
-    for (_iter925 = this->new_parts.begin(); _iter925 != this->new_parts.end(); ++_iter925)
+    std::vector<PartitionSpec> ::const_iterator _iter927;
+    for (_iter927 = this->new_parts.begin(); _iter927 != this->new_parts.end(); ++_iter927)
     {
-      xfer += (*_iter925).write(oprot);
+      xfer += (*_iter927).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8606,10 +8813,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter926;
-    for (_iter926 = (*(this->new_parts)).begin(); _iter926 != (*(this->new_parts)).end(); ++_iter926)
+    std::vector<PartitionSpec> ::const_iterator _iter928;
+    for (_iter928 = (*(this->new_parts)).begin(); _iter928 != (*(this->new_parts)).end(); ++_iter928)
     {
-      xfer += (*_iter926).write(oprot);
+      xfer += (*_iter928).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8834,14 +9041,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size927;
-            ::apache::thrift::protocol::TType _etype930;
-            xfer += iprot->readListBegin(_etype930, _size927);
-            this->part_vals.resize(_size927);
-            uint32_t _i931;
-            for (_i931 = 0; _i931 < _size927; ++_i931)
+            uint32_t _size929;
+            ::apache::thrift::protocol::TType _etype932;
+            xfer += iprot->readListBegin(_etype932, _size929);
+            this->part_vals.resize(_size929);
+            uint32_t _i933;
+            for (_i933 = 0; _i933 < _size929; ++_i933)
             {
-              xfer += iprot->readString(this->part_vals[_i931]);
+              xfer += iprot->readString(this->part_vals[_i933]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8878,10 +9085,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter932;
-    for (_iter932 = this->part_vals.begin(); _iter932 != this->part_vals.end(); ++_iter932)
+    std::vector<std::string> ::const_iterator _iter934;
+    for (_iter934 = this->part_vals.begin(); _iter934 != this->part_vals.end(); ++_iter934)
     {
-      xfer += oprot->writeString((*_iter932));
+      xfer += oprot->writeString((*_iter934));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8913,10 +9120,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter933;
-    for (_iter933 = (*(this->part_vals)).begin(); _iter933 != (*(this->part_vals)).end(); ++_iter933)
+    std::vector<std::string> ::const_iterator _iter935;
+    for (_iter935 = (*(this->part_vals)).begin(); _iter935 != (*(this->part_vals)).end(); ++_iter935)
     {
-      xfer += oprot->writeString((*_iter933));
+      xfer += oprot->writeString((*_iter935));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9388,14 +9595,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size934;
-            ::apache::thrift::protocol::TType _etype937;
-            xfer += iprot->readListBegin(_etype937, _size934);
-            this->part_vals.resize(_size934);
-            uint32_t _i938;
-            for (_i938 = 0; _i938 < _size934; ++_i938)
+            uint32_t _size936;
+            ::apache::thrift::protocol::TType _etype939;
+            xfer += iprot->readListBegin(_etype939, _size936);
+            this->part_vals.resize(_size936);
+            uint32_t _i940;
+            for (_i940 = 0; _i940 < _size936; ++_i940)
             {
-              xfer += iprot->readString(this->part_vals[_i938]);
+              xfer += iprot->readString(this->part_vals[_i940]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9440,10 +9647,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter939;
-    for (_iter939 = this->part_vals.begin(); _iter939 != this->part_vals.end(); ++_iter939)
+    std::vector<std::string> ::const_iterator _iter941;
+    for (_iter941 = this->part_vals.begin(); _iter941 != this->part_vals.end(); ++_iter941)
     {
-      xfer += oprot->writeString((*_iter939));
+      xfer += oprot->writeString((*_iter941));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9479,10 +9686,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter940;
-    for (_iter940 = (*(this->part_vals)).begin(); _iter940 != (*(this->part_vals)).end(); ++_iter940)
+    std::vector<std::string> ::const_iterator _iter942;
+    for (_iter942 = (*(this->part_vals)).begin(); _iter942 != (*(this->part_vals)).end(); ++_iter942)
     {
-      xfer += oprot->writeString((*_iter940));
+      xfer += oprot->writeString((*_iter942));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10285,14 +10492,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size941;
-            ::apache::thrift::protocol::TType _etype944;
-            xfer += iprot->readListBegin(_etype944, _size941);
-            this->part_vals.resize(_size941);
-            uint32_t _i945;
-            for (_i945 = 0; _i945 < _size941; ++_i945)
+            uint32_t _size943;
+            ::apache::thrift::protocol::TType _etype946;
+            xfer += iprot->readListBegin(_etype946, _size943);
+            this->part_vals.resize(_size943);
+            uint32_t _i947;
+            for (_i947 = 0; _i947 < _size943; ++_i947)
             {
-              xfer += iprot->readString(this->part_vals[_i945]);
+              xfer += iprot->readString(this->part_vals[_i947]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10337,10 +10544,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter946;
-    for (_iter946 = this->part_vals.begin(); _iter946 != this->part_vals.end(); ++_iter946)
+    std::vector<std::string> ::const_iterator _iter948;
+    for (_iter948 = this->part_vals.begin(); _iter948 != this->part_vals.end(); ++_iter948)
     {
-      xfer += oprot->writeString((*_iter946));
+      xfer += oprot->writeString((*_iter948));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10376,10 +10583,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter947;
-    for (_iter947 = (*(this->part_vals)).begin(); _iter947 != (*(this->part_vals)).end(); ++_iter947)
+    std::vector<std::string> ::const_iterator _iter949;
+    for (_iter949 = (*(this->part_vals)).begin(); _iter949 != (*(this->part_vals)).end(); ++_iter949)
     {
-      xfer += oprot->writeString((*_iter947));
+      xfer += oprot->writeString((*_iter949));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10588,14 +10795,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size948;
-            ::apache::thrift::protocol::TType _etype951;
-            xfer += iprot->readListBegin(_etype951, _size948);
-            this->part_vals.resize(_size948);
-            uint32_t _i952;
-            for (_i952 = 0; _i952 < _size948; ++_i952)
+            uint32_t _size950;
+            ::apache::thrift::protocol::TType _etype953;
+            xfer += iprot->readListBegin(_etype953, _size950);
+            this->part_vals.resize(_size950);
+            uint32_t _i954;
+            for (_i954 = 0; _i954 < _size950; ++_i954)
             {
-              xfer += iprot->readString(this->part_vals[_i952]);
+              xfer += iprot->readString(this->part_vals[_i954]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10648,10 +10855,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter953;
-    for (_iter953 = this->part_vals.begin(); _iter953 != this->part_vals.end(); ++_iter953)
+    std::vector<std::string> ::const_iterator _iter955;
+    for (_iter955 = this->part_vals.begin(); _iter955 != this->part_vals.end(); ++_iter955)
     {
-      xfer += oprot->writeString((*_iter953));
+      xfer += oprot->writeString((*_iter955));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10691,10 +10898,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter954;
-    for (_iter954 = (*(this->part_vals)).begin(); _iter954 != (*(this->part_vals)).end(); ++_iter954)
+    std::vector<std::string> ::const_iterator _iter956;
+    for (_iter956 = (*(this->part_vals)).begin(); _iter956 != (*(this->part_vals)).end(); ++_iter956)
     {
-      xfer += oprot->writeString((*_iter954));
+      xfer += oprot->writeString((*_iter956));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11700,14 +11907,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size955;
-            ::apache::thrift::protocol::TType _etype958;
-            xfer += iprot->readListBegin(_etype958, _size955);
-            this->part_vals.resize(_size955);
-            uint32_t _i959;
-            for (_i959 = 0; _i959 < _size955; ++_i959)
+            uint32_t _size957;
+            ::apache::thrift::protocol::TType _etype960;
+            xfer += iprot->readListBegin(_etype960, _size957);
+            this->part_vals.resize(_size957);
+            uint32_t _i961;
+            for (_i961 = 0; _i961 < _size957; ++_i961)
             {
-              xfer += iprot->readString(this->part_vals[_i959]);
+              xfer += iprot->readString(this->part_vals[_i961]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11744,10 +11951,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter960;
-    for (_iter960 = this->part_vals.begin(); _iter960 != this->part_vals.end(); ++_iter960)
+    std::vector<std::string> ::const_iterator _iter962;
+    for (_iter962 = this->part_vals.begin(); _iter962 != this->part_vals.end(); ++_iter962)
     {
-      xfer += oprot->writeString((*_iter960));
+      xfer += oprot->writeString((*_iter962));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11779,10 +11986,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter961;
-    for (_iter961 = (*(this->part_vals)).begin(); _iter961 != (*(this->part_vals)).end(); ++_iter961)
+    std::vector<std::string> ::const_iterator _iter963;
+    for (_iter963 = (*(this->part_vals)).begin(); _iter963 != (*(this->part_vals)).end(); ++_iter963)
     {
-      xfer += oprot->writeString((*_iter961));
+      xfer += oprot->writeString((*_iter963));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11971,17 +12178,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size962;
-            ::apache::thrift::protocol::TType _ktype963;
-            ::apache::thrift::protocol::TType _vtype964;
-            xfer += iprot->readMapBegin(_ktype963, _vtype964, _size962);
-            uint32_t _i966;
-            for (_i966 = 0; _i966 < _size962; ++_i966)
+            uint32_t _size964;
+            ::apache::thrift::protocol::TType _ktype965;
+            ::apache::thrift::protocol::TType _vtype966;
+            xfer += iprot->readMapBegin(_ktype965, _vtype966, _size964);
+            uint32_t _i968;
+            for (_i968 = 0; _i968 < _size964; ++_i968)
             {
-              std::string _key967;
-              xfer += iprot->readString(_key967);
-              std::string& _val968 = this->partitionSpecs[_key967];
-              xfer += iprot->readString(_val968);
+              std::string _key969;
+              xfer += iprot->readString(_key969);
+              std::string& _val970 = this->partitionSpecs[_key969];
+              xfer += iprot->readString(_val970);
             }
             xfer += iprot->readMapEnd();
           }
@@ -12042,11 +12249,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter969;
-    for (_iter969 = this->partitionSpecs.begin(); _iter969 != this->partitionSpecs.end(); ++_iter969)
+    std::map<std::string, std::string> ::const_iterator _iter971;
+    for (_iter971 = this->partitionSpecs.begin(); _iter971 != this->partitionSpecs.end(); ++_iter971)
     {
-      xfer += oprot->writeString(_iter969->first);
-      xfer += oprot->writeString(_iter969->second);
+      xfer += oprot->writeString(_iter971->first);
+      xfer += oprot->writeString(_iter971->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12086,11 +12293,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter970;
-    for (_iter970 = (*(this->partitionSpecs)).begin(); _iter970 != (*(this->partitionSpecs)).end(); ++_iter970)
+    std::map<std::string, std::string> ::const_iterator _iter972;
+    for (_iter972 = (*(this->partitionSpecs)).begin(); _iter972 != (*(this->partitionSpecs)).end(); ++_iter972)
     {
-      xfer += oprot->writeString(_iter970->first);
-      xfer += oprot->writeString(_iter970->second);
+      xfer += oprot->writeString(_iter972->first);
+      xfer += oprot->writeString(_iter972->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12335,17 +12542,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size971;
-            ::apache::thrift::protocol::TType _ktype972;
-            ::apache::thrift::protocol::TType _vtype973;
-            xfer += iprot->readMapBegin(_ktype972, _vtype973, _size971);
-            uint32_t _i975;
-            for (_i975 = 0; _i975 < _size971; ++_i975)
+            uint32_t _size973;
+            ::apache::thrift::protocol::TType _ktype974;
+            ::apache::thrift::protocol::TType _vtype975;
+            xfer += iprot->readMapBegin(_ktype974, _vtype975, _size973);
+            uint32_t _i977;
+            for (_i977 = 0; _i977 < _size973; ++_i977)
             {
-              std::string _key976;
-              xfer += iprot->readString(_key976);
-              std::string& _val977 = this->partitionSpecs[_key976];
-              xfer += iprot->readString(_val977);
+              std::string _key978;
+              xfer += iprot->readString(_key978);
+              std::string& _val979 = this->partitionSpecs[_key978];
+              xfer += iprot->readString(_val979);
             }
             xfer += iprot->readMapEnd();
           }
@@ -12406,11 +12613,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter978;
-    for (_iter978 = this->partitionSpecs.begin(); _iter978 != this->partitionSpecs.end(); ++_iter978)
+    std::map<std::string, std::string> ::const_iterator _iter980;
+    for (_iter980 = this->partitionSpecs.begin(); _iter980 != this->partitionSpecs.end(); ++_iter980)
     {
-      xfer += oprot->writeString(_iter978->first);
-      xfer += oprot->writeString(_iter978->second);
+      xfer += oprot->writeString(_iter980->first);
+      xfer += oprot->writeString(_iter980->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12450,11 +12657,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter979;
-    for (_iter979 = (*(this->partitionSpecs)).begin(); _iter979 != (*(this->partitionSpecs)).end(); ++_iter979)
+    std::map<std::string, std::string> ::const_iterator _iter981;
+    for (_iter981 = (*(this->partitionSpecs)).begin(); _iter981 != (*(this->partitionSpecs)).end(); ++_iter981)
     {
-      xfer += oprot->writeString(_iter979->first);
-      xfer += oprot->writeString(_iter979->second);
+      xfer += oprot->writeString(_iter981->first);
+      xfer += oprot->writeString(_iter981->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12511,14 +12718,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size980;
-            ::apache::thrift::protocol::TType _etype983;
-            xfer += iprot->readListBegin(_etype983, _size980);
-            this->success.resize(_size980);
-            uint32_t _i984;
-            for (_i984 = 0; _i984 < _size980; ++_i984)
+            uint32_t _size982;
+            ::apache::thrift::protocol::TType _etype985;
+            xfer += iprot->readListBegin(_etype985, _size982);
+            this->success.resize(_size982);
+            uint32_t _i986;
+            for (_i986 = 0; _i986 < _size982; ++_i986)
             {
-              xfer += this->success[_i984].read(iprot);
+              xfer += this->success[_i986].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12581,10 +12788,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter985;
-      for (_iter985 = this->success.begin(); _iter985 != this->success.end(); ++_iter985)
+      std::vector<Partition> ::const_iterator _iter987;
+      for (_iter987 = this->success.begin(); _iter987 != this->success.end(); ++_iter987)
       {
-        xfer += (*_iter985).write(oprot);
+        xfer += (*_iter987).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12641,14 +12848,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size986;
-            ::apache::thrift::protocol::TType _etype989;
-            xfer += iprot->readListBegin(_etype989, _size986);
-            (*(this->success)).resize(_size986);
-            uint32_t _i990;
-            for (_i990 = 0; _i990 < _size986; ++_i990)
+            uint32_t _size988;
+            ::apache::thrift::protocol::TType _etype991;
+            xfer += iprot->readListBegin(_etype991, _size988);
+            (*(this->success)).resize(_size988);
+            uint32_t _i992;
+            for (_i992 = 0; _i992 < _size988; ++_i992)
             {
-              xfer += (*(this->success))[_i990].read(iprot);
+              xfer += (*(this->success))[_i992].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12747,14 +12954,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size991;
-            ::apache::thrift::protocol::TType _etype994;
-            xfer += iprot->readListBegin(_etype994, _size991);
-            this->part_vals.resize(_size991);
-            uint32_t _i995;
-            for (_i995 = 0; _i995 < _size991; ++_i995)
+            uint32_t _size993;
+            ::apache::thrift::protocol::TType _etype996;
+            xfer += iprot->readListBegin(_etype996, _size993);
+            this->part_vals.resize(_size993);
+            uint32_t _i997;
+            for (_i997 = 0; _i997 < _size993; ++_i997)
             {
-              xfer += iprot->readString(this->part_vals[_i995]);
+              xfer += iprot->readString(this->part_vals[_i997]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12775,14 +12982,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size996;
-            ::apache::thrift::protocol::TType _etype999;
-            xfer += iprot->readListBegin(_etype999, _size996);
-            this->group_names.resize(_size996);
-            uint32_t _i1000;
-            for (_i1000 = 0; _i1000 < _size996; ++_i1000)
+            uint32_t _size998;
+            ::apache::thrift::protocol::TType _etype1001;
+            xfer += iprot->readListBegin(_etype1001, _size998);
+            this->group_names.resize(_size998);
+            uint32_t _i1002;
+            for (_i1002 = 0; _i1002 < _size998; ++_i1002)
             {
-              xfer += iprot->readString(this->group_names[_i1000]);
+              xfer += iprot->readString(this->group_names[_i1002]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12819,10 +13026,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1001;
-    for (_iter1001 = this->part_vals.begin(); _iter1001 != this->part_vals.end(); ++_iter1001)
+    std::vector<std::string> ::const_iterator _iter1003;
+    for (_iter1003 = this->part_vals.begin(); _iter1003 != this->part_vals.end(); ++_iter1003)
     {
-      xfer += oprot->writeString((*_iter1001));
+      xfer += oprot->writeString((*_iter1003));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12835,10 +13042,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter1002;
-    for (_iter1002 = this->group_names.begin(); _iter1002 != this->group_names.end(); ++_iter1002)
+    std::vector<std::string> ::const_iterator _iter1004;
+    for (_iter1004 = this->group_names.begin(); _iter1004 != this->group_names.end(); ++_iter1004)
     {
-      xfer += oprot->writeString((*_iter1002));
+      xfer += oprot->writeString((*_iter1004));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12870,10 +13077,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1003;
-    for (_iter1003 = (*(this->part_vals)).begin(); _iter1003 != (*(this->part_vals)).end(); ++_iter1003)
+    std::vector<std::string> ::const_iterator _iter1005;
+    for (_iter1005 = (*(this->part_vals)).begin(); _iter1005 != (*(this->part_vals)).end(); ++_iter1005)
     {
-      xfer += oprot->writeString((*_iter1003));
+      xfer += oprot->writeString((*_iter1005));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12886,10 +13093,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1004;
-    for (_iter1004 = (*(this->group_names)).begin(); _iter1004 != (*(this->group_names)).end(); ++_iter1004)
+    std::vector<std::string> ::const_iterator _iter1006;
+    for (_iter1006 = (*(this->group_names)).begin(); _iter1006 != (*(this->group_names)).end(); ++_iter1006)
     {
-      xfer += oprot->writeString((*_iter1004));
+      xfer += oprot->writeString((*_iter1006));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13448,14 +13655,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1005;
-            ::apache::thrift::protocol::TType _etype1008;
-            xfer += iprot->readListBegin(_etype1008, _size1005);
-            this->success.resize(_size1005);
-            uint32_t _i1009;
-            for (_i1009 = 0; _i1009 < _size1005; ++_i1009)
+            uint32_t _size1007;
+            ::apache::thrift::protocol::TType _etype1010;
+            xfer += iprot->readListBegin(_etype1010, _size1007);
+            this->success.resize(_size1007);
+            uint32_t _i1011;
+            for (_i1011 = 0; _i1011 < _size1007; ++_i1011)
             {
-              xfer += this->success[_i1009].read(iprot);
+              xfer += this->success[_i1011].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13502,10 +13709,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter1010;
-      for (_iter1010 = this->success.begin(); _iter1010 != this->success.end(); ++_iter1010)
+      std::vector<Partition> ::const_iterator _iter1012;
+      for (_iter1012 = this->success.begin(); _iter1012 != this->success.end(); ++_iter1012)
       {
-        xfer += (*_iter1010).write(oprot);
+        xfer += (*_iter1012).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13554,14 +13761,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1011;
-            ::apache::thrift::protocol::TType _etype1014;
-            xfer += iprot->readListBegin(_etype1014, _size1011);
-            (*(this->success)).resize(_size1011);
-            uint32_t _i1015;
-            for (_i1015 = 0; _i1015 < _size1011; ++_i1015)
+            uint32_t _size1013;
+            ::apache::thrift::protocol::TType _etype1016;
+            xfer += iprot->readListBegin(_etype1016, _size1013);
+            (*(this->success)).resize(_size1013);
+            uint32_t _i1017;
+            for (_i1017 = 0; _i1017 < _size1013; ++_i1017)
             {
-              xfer += (*(this->success))[_i1015].read(iprot);
+              xfer += (*(this->success))[_i1017].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13660,14 +13867,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size1016;
-            ::apache::thrift::protocol::TType _etype1019;
-            xfer += iprot->readListBegin(_etype1019, _size1016);
-            this->group_names.resize(_size1016);
-            uint32_t _i1020;
-            for (_i1020 = 0; _i1020 < _size1016; ++_i1020)
+            uint32_t _size1018;
+            ::apache::thrift::protocol::TType _etype1021;
+            xfer += iprot->readListBegin(_etype1021, _size1018);
+            this->group_names.resize(_size1018);
+            uint32_t _i1022;
+            for (_i1022 = 0; _i1022 < _size1018; ++_i1022)
             {
-              xfer += iprot->readString(this->group_names[_i1020]);
+              xfer += iprot->readString(this->group_names[_i1022]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13712,10 +13919,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter1021;
-    for (_iter1021 = this->group_names.begin(); _iter1021 != this->group_names.end(); ++_iter1021)
+    std::vector<std::string> ::const_iterator _iter1023;
+    for (_iter1023 = this->group_names.begin(); _iter1023 != this->group_names.end(); ++_iter1023)
     {
-      xfer += oprot->writeString((*_iter1021));
+      xfer += oprot->writeString((*_iter1023));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13755,10 +13962,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1022;
-    for (_iter1022 = (*(this->group_names)).begin(); _iter1022 != (*(this->group_names)).end(); ++_iter1022)
+    std::vector<std::string> ::const_iterator _iter1024;
+    for (_iter1024 = (*(this->group_names)).begin(); _iter1024 != (*(this->group_names)).end(); ++_iter1024)
     {
-      xfer += oprot->writeString((*_iter1022));
+      xfer += oprot->writeString((*_iter1024));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13799,14 +14006,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1023;
-            ::apache::thrift::protocol::TType _etype1026;
-            xfer += iprot->readListBegin(_etype1026, _size1023);
-            this->success.resize(_size1023);
-            uint32_t _i1027;
-            for (_i1027 = 0; _i1027 < _size1023; ++_i1027)
+            uint32_t _size1025;
+            ::apache::thrift::protocol::TType _etype1028;
+            xfer += iprot->readListBegin(_etype1028, _size1025);
+            this->success.resize(_size1025);
+            uint32_t _i1029;
+            for (_i1029 = 0; _i1029 < _size1025; ++_i1029)
             {
-              xfer += this->success[_i1027].read(iprot);
+       

<TRUNCATED>

[45/50] [abbrv] hive git commit: HIVE-13679: Pass diagnostic message to failure hooks (Jimmy Xiang, reviewed by Aihua Xu)

Posted by sp...@apache.org.
HIVE-13679: Pass diagnostic message to failure hooks (Jimmy Xiang, reviewed by Aihua Xu)


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

Branch: refs/heads/java8
Commit: 2b1e273e44fe367c12167409e8552efa2770ae7e
Parents: b870d52
Author: Jimmy Xiang <jx...@apache.org>
Authored: Tue May 3 14:48:09 2016 -0700
Committer: Jimmy Xiang <jx...@apache.org>
Committed: Fri May 6 07:41:43 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/Driver.java  | 11 ++++++----
 .../org/apache/hadoop/hive/ql/exec/Task.java    | 21 ++++++++++++++++----
 .../apache/hadoop/hive/ql/exec/TaskResult.java  |  7 +++++--
 .../apache/hadoop/hive/ql/exec/TaskRunner.java  |  5 ++++-
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    |  1 +
 .../hadoop/hive/ql/exec/mr/JobDebugger.java     | 18 +++++++++++------
 6 files changed, 46 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2b1e273e/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 6a610cb..3fecc5c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -118,6 +118,7 @@ import org.apache.hive.common.util.ShutdownHookManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
 
@@ -1598,7 +1599,8 @@ public class Driver implements CommandProcessor {
 
           } else {
             setErrorMsgAndDetail(exitVal, result.getTaskError(), tsk);
-            invokeFailureHooks(perfLogger, hookContext, result.getTaskError());
+            invokeFailureHooks(perfLogger, hookContext,
+              errorMessage + Strings.nullToEmpty(tsk.getDiagnosticsMessage()), result.getTaskError());
             SQLState = "08S01";
             console.printError(errorMessage);
             driverCxt.shutdown();
@@ -1634,7 +1636,7 @@ public class Driver implements CommandProcessor {
       if (driverCxt.isShutdown()) {
         SQLState = "HY008";
         errorMessage = "FAILED: Operation cancelled";
-        invokeFailureHooks(perfLogger, hookContext, null);
+        invokeFailureHooks(perfLogger, hookContext, errorMessage, null);
         console.printError(errorMessage);
         return 1000;
       }
@@ -1691,7 +1693,7 @@ public class Driver implements CommandProcessor {
       errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e);
       if (hookContext != null) {
         try {
-          invokeFailureHooks(perfLogger, hookContext, e);
+          invokeFailureHooks(perfLogger, hookContext, errorMessage, e);
         } catch (Exception t) {
           LOG.warn("Failed to invoke failure hook", t);
         }
@@ -1790,7 +1792,8 @@ public class Driver implements CommandProcessor {
     }
   }
 
-  private void invokeFailureHooks(PerfLogger perfLogger, HookContext hookContext, Throwable exception) throws Exception {
+  private void invokeFailureHooks(PerfLogger perfLogger,
+      HookContext hookContext, String errorMessage, Throwable exception) throws Exception {
     hookContext.setHookType(HookContext.HookType.ON_FAILURE_HOOK);
     hookContext.setErrorMessage(errorMessage);
     hookContext.setException(exception);

http://git-wip-us.apache.org/repos/asf/hive/blob/2b1e273e/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
index 34bdafd..eeaa543 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
@@ -27,10 +27,12 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.*;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.DriverContext;
+import org.apache.hadoop.hive.ql.QueryDisplay;
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.QueryState;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -40,6 +42,8 @@ import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Task implementation.
@@ -84,8 +88,17 @@ public abstract class Task<T extends Serializable> implements Serializable, Node
   protected T work;
   private TaskState taskState = TaskState.CREATED;
   private String statusMessage;
+  private String diagnosticMesg;
   private transient boolean fetchSource;
 
+  public void setDiagnosticMessage(String diagnosticMesg) {
+    this.diagnosticMesg = diagnosticMesg;
+  }
+
+  public String getDiagnosticsMessage() {
+    return diagnosticMesg;
+  }
+
   public void setStatusMessage(String statusMessage) {
     this.statusMessage = statusMessage;
     updateStatusInQueryDisplay();
@@ -321,7 +334,7 @@ public abstract class Task<T extends Serializable> implements Serializable, Node
     return ret;
   }
 
-  @SuppressWarnings("unchecked")
+  @SuppressWarnings({"unchecked", "rawtypes"})
   public static List<Task<? extends Serializable>>
       findLeafs(List<Task<? extends Serializable>> rootTasks) {
     final List<Task<? extends Serializable>> leafTasks = new ArrayList<Task<?>>();

http://git-wip-us.apache.org/repos/asf/hive/blob/2b1e273e/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskResult.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskResult.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskResult.java
index def9389..3c4ee17 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskResult.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskResult.java
@@ -37,10 +37,13 @@ public class TaskResult {
     this.exitVal = exitVal;
     setRunning(false);
   }
-  public void setExitVal(int exitVal, Throwable taskError) {
-    this.setExitVal(exitVal);
+  public void setTaskError(Throwable taskError) {
     this.taskError = taskError;
   }
+  public void setExitVal(int exitVal, Throwable taskError) {
+    setExitVal(exitVal);
+    setTaskError(taskError);
+  }
 
   public int getExitVal() {
     return exitVal;

http://git-wip-us.apache.org/repos/asf/hive/blob/2b1e273e/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java
index 81f6db0..a596e92 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java
@@ -104,7 +104,10 @@ public class TaskRunner extends Thread {
       }
       LOG.error("Error in executeTask", t);
     }
-    result.setExitVal(exitVal, tsk.getException());
+    result.setExitVal(exitVal);
+    if (tsk.getException() != null) {
+      result.setTaskError(tsk.getException());
+    }
   }
 
   public static long getTaskRunnerID () {

http://git-wip-us.apache.org/repos/asf/hive/blob/2b1e273e/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
index 11f5cfd..c15316bb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
@@ -574,6 +574,7 @@ public class HadoopJobExecHelper {
           Thread t = new Thread(jd);
           t.start();
           t.join(HiveConf.getIntVar(job, HiveConf.ConfVars.JOB_DEBUG_TIMEOUT));
+          task.setDiagnosticMessage(jd.getDiagnosticMesg());
           int ec = jd.getErrorCode();
           if (ec > 0) {
             returnVal = ec;

http://git-wip-us.apache.org/repos/asf/hive/blob/2b1e273e/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobDebugger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobDebugger.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobDebugger.java
index 6e4e3bf..d320536 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobDebugger.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobDebugger.java
@@ -53,6 +53,7 @@ public class JobDebugger implements Runnable {
   private final Map<String, Integer> failures = new HashMap<String, Integer>();
   private final Set<String> successes = new HashSet<String>(); // Successful task ID's
   private final Map<String, TaskInfo> taskIdToInfo = new HashMap<String, TaskInfo>();
+  private String diagnosticMesg;
   private int maxFailures = 0;
 
   // Used for showJobFailDebugInfo
@@ -115,7 +116,7 @@ public class JobDebugger implements Runnable {
 
   public void run() {
     try {
-      showJobFailDebugInfo();
+      diagnosticMesg = showJobFailDebugInfo();
     } catch (IOException e) {
       console.printError(e.getMessage());
     }
@@ -216,8 +217,7 @@ public class JobDebugger implements Runnable {
     }
   }
 
-  @SuppressWarnings("deprecation")
-  private void showJobFailDebugInfo() throws IOException {
+  private String showJobFailDebugInfo() throws IOException {
     console.printError("Error during job, obtaining debugging information...");
     if (!conf.get("mapred.job.tracker", "local").equals("local")) {
       // Show Tracking URL for remotely running jobs.
@@ -241,7 +241,7 @@ public class JobDebugger implements Runnable {
     }
 
     if (failures.keySet().size() == 0) {
-      return;
+      return null;
     }
     // Find the highest failure count
     computeMaxFailures() ;
@@ -255,6 +255,7 @@ public class JobDebugger implements Runnable {
           + e.getMessage());
     }
 
+    String msg = null;
     for (String task : failures.keySet()) {
       if (failures.get(task).intValue() == maxFailures) {
         TaskInfo ti = taskIdToInfo.get(task);
@@ -303,14 +304,19 @@ public class JobDebugger implements Runnable {
           for (String mesg : diagMesgs) {
             sb.append(mesg + "\n");
           }
-          console.printError(sb.toString());
+          msg = sb.toString();
+          console.printError(msg);
         }
 
         // Only print out one task because that's good enough for debugging.
         break;
       }
     }
-    return;
+    return msg;
+  }
+
+  public String getDiagnosticMesg() {
+    return diagnosticMesg;
   }
 
   public int getErrorCode() {


[27/50] [abbrv] hive git commit: HIVE-13632: Hive failing on insert empty array into parquet table. (Yongzhi Chen, reviewed by Sergio Pena)

Posted by sp...@apache.org.
HIVE-13632: Hive failing on insert empty array into parquet table. (Yongzhi Chen, reviewed by Sergio Pena)


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

Branch: refs/heads/java8
Commit: 96f2dc723270bb4c38e5ab842371929c2c1c849a
Parents: cbebb4d
Author: Yongzhi Chen <yc...@apache.org>
Authored: Thu Apr 28 14:52:16 2016 -0400
Committer: Yongzhi Chen <yc...@apache.org>
Committed: Thu May 5 09:58:39 2016 -0400

----------------------------------------------------------------------
 .../serde/AbstractParquetMapInspector.java      |  4 +-
 .../serde/ParquetHiveArrayInspector.java        |  4 +-
 .../ql/io/parquet/write/DataWritableWriter.java | 67 ++++++++-------
 .../ql/io/parquet/TestDataWritableWriter.java   | 29 +++++++
 .../serde/TestAbstractParquetMapInspector.java  |  4 +-
 .../serde/TestParquetHiveArrayInspector.java    |  4 +-
 .../parquet_array_map_emptynullvals.q           | 20 +++++
 .../parquet_array_map_emptynullvals.q.out       | 87 ++++++++++++++++++++
 8 files changed, 180 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/96f2dc72/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/AbstractParquetMapInspector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/AbstractParquetMapInspector.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/AbstractParquetMapInspector.java
index 49bf1c5..e80206e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/AbstractParquetMapInspector.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/AbstractParquetMapInspector.java
@@ -60,7 +60,7 @@ public abstract class AbstractParquetMapInspector implements SettableMapObjectIn
 
     if (data instanceof ArrayWritable) {
       final Writable[] mapArray = ((ArrayWritable) data).get();
-      if (mapArray == null || mapArray.length == 0) {
+      if (mapArray == null) {
         return null;
       }
 
@@ -90,7 +90,7 @@ public abstract class AbstractParquetMapInspector implements SettableMapObjectIn
     if (data instanceof ArrayWritable) {
       final Writable[] mapArray = ((ArrayWritable) data).get();
 
-      if (mapArray == null || mapArray.length == 0) {
+      if (mapArray == null) {
         return -1;
       } else {
         return mapArray.length;

http://git-wip-us.apache.org/repos/asf/hive/blob/96f2dc72/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveArrayInspector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveArrayInspector.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveArrayInspector.java
index 05e92b5..55614a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveArrayInspector.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveArrayInspector.java
@@ -83,7 +83,7 @@ public class ParquetHiveArrayInspector implements SettableListObjectInspector {
 
     if (data instanceof ArrayWritable) {
       final Writable[] array = ((ArrayWritable) data).get();
-      if (array == null || array.length == 0) {
+      if (array == null) {
         return -1;
       }
 
@@ -105,7 +105,7 @@ public class ParquetHiveArrayInspector implements SettableListObjectInspector {
 
     if (data instanceof ArrayWritable) {
       final Writable[] array = ((ArrayWritable) data).get();
-      if (array == null || array.length == 0) {
+      if (array == null) {
         return null;
       }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/96f2dc72/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
index 69272dc..1e26c19 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
@@ -259,21 +259,24 @@ public class DataWritableWriter {
     @Override
     public void write(Object value) {
       recordConsumer.startGroup();
-      recordConsumer.startField(repeatedGroupName, 0);
-
       int listLength = inspector.getListLength(value);
-      for (int i = 0; i < listLength; i++) {
-        Object element = inspector.getListElement(value, i);
-        recordConsumer.startGroup();
-        if (element != null) {
-          recordConsumer.startField(elementName, 0);
-          elementWriter.write(element);
-          recordConsumer.endField(elementName, 0);
+
+      if (listLength > 0) {
+        recordConsumer.startField(repeatedGroupName, 0);
+
+        for (int i = 0; i < listLength; i++) {
+          Object element = inspector.getListElement(value, i);
+          recordConsumer.startGroup();
+          if (element != null) {
+            recordConsumer.startField(elementName, 0);
+            elementWriter.write(element);
+            recordConsumer.endField(elementName, 0);
+          }
+          recordConsumer.endGroup();
         }
-        recordConsumer.endGroup();
-      }
 
-      recordConsumer.endField(repeatedGroupName, 0);
+        recordConsumer.endField(repeatedGroupName, 0);
+      }
       recordConsumer.endGroup();
     }
   }
@@ -307,30 +310,32 @@ public class DataWritableWriter {
     @Override
     public void write(Object value) {
       recordConsumer.startGroup();
-      recordConsumer.startField(repeatedGroupName, 0);
 
       Map<?, ?> mapValues = inspector.getMap(value);
-      for (Map.Entry<?, ?> keyValue : mapValues.entrySet()) {
-        recordConsumer.startGroup();
-        if (keyValue != null) {
-          // write key element
-          Object keyElement = keyValue.getKey();
-          recordConsumer.startField(keyName, 0);
-          keyWriter.write(keyElement);
-          recordConsumer.endField(keyName, 0);
-
-          // write value element
-          Object valueElement = keyValue.getValue();
-          if (valueElement != null) {
-            recordConsumer.startField(valueName, 1);
-            valueWriter.write(valueElement);
-            recordConsumer.endField(valueName, 1);
+      if (mapValues != null && mapValues.size() > 0) {
+        recordConsumer.startField(repeatedGroupName, 0);
+        for (Map.Entry<?, ?> keyValue : mapValues.entrySet()) {
+          recordConsumer.startGroup();
+          if (keyValue != null) {
+            // write key element
+            Object keyElement = keyValue.getKey();
+            recordConsumer.startField(keyName, 0);
+            keyWriter.write(keyElement);
+            recordConsumer.endField(keyName, 0);
+
+            // write value element
+            Object valueElement = keyValue.getValue();
+            if (valueElement != null) {
+              recordConsumer.startField(valueName, 1);
+              valueWriter.write(valueElement);
+              recordConsumer.endField(valueName, 1);
+            }
           }
+          recordConsumer.endGroup();
         }
-        recordConsumer.endGroup();
-      }
 
-      recordConsumer.endField(repeatedGroupName, 0);
+        recordConsumer.endField(repeatedGroupName, 0);
+      }
       recordConsumer.endGroup();
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/96f2dc72/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestDataWritableWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestDataWritableWriter.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestDataWritableWriter.java
index 7049139..934ae9f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestDataWritableWriter.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestDataWritableWriter.java
@@ -411,6 +411,35 @@ public class TestDataWritableWriter {
   }
 
   @Test
+  public void testEmptyArrays() throws Exception {
+    String columnNames = "arrayCol";
+    String columnTypes = "array<int>";
+
+    String fileSchema = "message hive_schema {\n"
+        + "  optional group arrayCol (LIST) {\n"
+        + "    repeated group array {\n"
+        + "      optional int32 array_element;\n"
+        + "    }\n"
+        + "  }\n"
+        + "}\n";
+
+    ArrayWritable hiveRecord = createGroup(
+       new ArrayWritable(Writable.class) // Empty array
+    );
+
+   // Write record to Parquet format
+    writeParquetRecord(fileSchema, getParquetWritable(columnNames, columnTypes, hiveRecord));
+
+    // Verify record was written correctly to Parquet
+    startMessage();
+      startField("arrayCol", 0);
+        startGroup();
+        endGroup();
+      endField("arrayCol", 0);
+    endMessage();
+  }
+
+  @Test
   public void testArrayOfArrays() throws Exception {
     String columnNames = "array_of_arrays";
     String columnTypes = "array<array<int>>";

http://git-wip-us.apache.org/repos/asf/hive/blob/96f2dc72/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestAbstractParquetMapInspector.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestAbstractParquetMapInspector.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestAbstractParquetMapInspector.java
index f5d9cb4..6af8c53 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestAbstractParquetMapInspector.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestAbstractParquetMapInspector.java
@@ -62,8 +62,8 @@ public class TestAbstractParquetMapInspector extends TestCase {
   @Test
   public void testEmptyContainer() {
     final ArrayWritable map = new ArrayWritable(ArrayWritable.class, new ArrayWritable[0]);
-    assertEquals("Wrong size", -1, inspector.getMapSize(map));
-    assertNull("Should be null", inspector.getMap(map));
+    assertEquals("Wrong size", 0, inspector.getMapSize(map));
+    assertNotNull("Should not be null", inspector.getMap(map));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/96f2dc72/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetHiveArrayInspector.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetHiveArrayInspector.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetHiveArrayInspector.java
index 0ce654d..9e0c1ff 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetHiveArrayInspector.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetHiveArrayInspector.java
@@ -51,8 +51,8 @@ public class TestParquetHiveArrayInspector extends TestCase {
   @Test
   public void testEmptyContainer() {
     final ArrayWritable list = new ArrayWritable(ArrayWritable.class, new ArrayWritable[0]);
-    assertEquals("Wrong size", -1, inspector.getListLength(list));
-    assertNull("Should be null", inspector.getList(list));
+    assertEquals("Wrong size", 0, inspector.getListLength(list));
+    assertNotNull("Should not be null", inspector.getList(list));
     assertNull("Should be null", inspector.getListElement(list, 0));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/96f2dc72/ql/src/test/queries/clientpositive/parquet_array_map_emptynullvals.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_array_map_emptynullvals.q b/ql/src/test/queries/clientpositive/parquet_array_map_emptynullvals.q
new file mode 100644
index 0000000..eeae5cf
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/parquet_array_map_emptynullvals.q
@@ -0,0 +1,20 @@
+drop table if exists testSets;
+drop table if exists testSets2;
+create table testSets (
+key string,
+arrayValues array<string>,
+mapValues map<string,string>)
+stored as parquet;
+
+insert into table testSets select 'abcd', array(), map() from src limit 1;
+
+create table testSets2 (
+key string,
+arrayValues array<string>,
+mapValues map<string,string>)
+stored as parquet;
+insert into table testSets2 select * from testSets;
+select * from testSets2;
+drop table testSets;
+drop table testSets2;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/96f2dc72/ql/src/test/results/clientpositive/parquet_array_map_emptynullvals.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_array_map_emptynullvals.q.out b/ql/src/test/results/clientpositive/parquet_array_map_emptynullvals.q.out
new file mode 100644
index 0000000..4608607
--- /dev/null
+++ b/ql/src/test/results/clientpositive/parquet_array_map_emptynullvals.q.out
@@ -0,0 +1,87 @@
+PREHOOK: query: drop table if exists testSets
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists testSets
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists testSets2
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists testSets2
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table testSets (
+key string,
+arrayValues array<string>,
+mapValues map<string,string>)
+stored as parquet
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@testSets
+POSTHOOK: query: create table testSets (
+key string,
+arrayValues array<string>,
+mapValues map<string,string>)
+stored as parquet
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@testSets
+PREHOOK: query: insert into table testSets select 'abcd', array(), map() from src limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@testsets
+POSTHOOK: query: insert into table testSets select 'abcd', array(), map() from src limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@testsets
+POSTHOOK: Lineage: testsets.arrayvalues EXPRESSION []
+POSTHOOK: Lineage: testsets.key SIMPLE []
+POSTHOOK: Lineage: testsets.mapvalues EXPRESSION []
+PREHOOK: query: create table testSets2 (
+key string,
+arrayValues array<string>,
+mapValues map<string,string>)
+stored as parquet
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@testSets2
+POSTHOOK: query: create table testSets2 (
+key string,
+arrayValues array<string>,
+mapValues map<string,string>)
+stored as parquet
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@testSets2
+PREHOOK: query: insert into table testSets2 select * from testSets
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testsets
+PREHOOK: Output: default@testsets2
+POSTHOOK: query: insert into table testSets2 select * from testSets
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testsets
+POSTHOOK: Output: default@testsets2
+POSTHOOK: Lineage: testsets2.arrayvalues SIMPLE [(testsets)testsets.FieldSchema(name:arrayvalues, type:array<string>, comment:null), ]
+POSTHOOK: Lineage: testsets2.key SIMPLE [(testsets)testsets.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: testsets2.mapvalues SIMPLE [(testsets)testsets.FieldSchema(name:mapvalues, type:map<string,string>, comment:null), ]
+PREHOOK: query: select * from testSets2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testsets2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from testSets2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testsets2
+#### A masked pattern was here ####
+abcd	[]	{}
+PREHOOK: query: drop table testSets
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@testsets
+PREHOOK: Output: default@testsets
+POSTHOOK: query: drop table testSets
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@testsets
+POSTHOOK: Output: default@testsets
+PREHOOK: query: drop table testSets2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@testsets2
+PREHOOK: Output: default@testsets2
+POSTHOOK: query: drop table testSets2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@testsets2
+POSTHOOK: Output: default@testsets2


[15/50] [abbrv] hive git commit: HIVE-13351: Support drop Primary Key/Foreign Key constraints (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 4f0c8fd..0e7b745 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -167,6 +167,12 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys);
   /**
+   * @param \metastore\DropConstraintRequest $req
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function drop_constraint(\metastore\DropConstraintRequest $req);
+  /**
    * @param string $dbname
    * @param string $name
    * @param bool $deleteData
@@ -2250,6 +2256,60 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
+  public function drop_constraint(\metastore\DropConstraintRequest $req)
+  {
+    $this->send_drop_constraint($req);
+    $this->recv_drop_constraint();
+  }
+
+  public function send_drop_constraint(\metastore\DropConstraintRequest $req)
+  {
+    $args = new \metastore\ThriftHiveMetastore_drop_constraint_args();
+    $args->req = $req;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'drop_constraint', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('drop_constraint', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_drop_constraint()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_drop_constraint_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_drop_constraint_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o3 !== null) {
+      throw $result->o3;
+    }
+    return;
+  }
+
   public function drop_table($dbname, $name, $deleteData)
   {
     $this->send_drop_table($dbname, $name, $deleteData);
@@ -13889,6 +13949,188 @@ class ThriftHiveMetastore_create_table_with_constraints_result {
 
 }
 
+class ThriftHiveMetastore_drop_constraint_args {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\DropConstraintRequest
+   */
+  public $req = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'req',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\DropConstraintRequest',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['req'])) {
+        $this->req = $vals['req'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_constraint_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->req = new \metastore\DropConstraintRequest();
+            $xfer += $this->req->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_constraint_args');
+    if ($this->req !== null) {
+      if (!is_object($this->req)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('req', TType::STRUCT, 1);
+      $xfer += $this->req->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_constraint_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o3 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
+        2 => array(
+          'var' => 'o3',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o3'])) {
+        $this->o3 = $vals['o3'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_constraint_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o3 = new \metastore\MetaException();
+            $xfer += $this->o3->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_constraint_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o3 !== null) {
+      $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2);
+      $xfer += $this->o3->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ThriftHiveMetastore_drop_table_args {
   static $_TSPEC;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index e2fa963..a8a7db9 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -9153,6 +9153,127 @@ class ForeignKeysResponse {
 
 }
 
+class DropConstraintRequest {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbname = null;
+  /**
+   * @var string
+   */
+  public $tablename = null;
+  /**
+   * @var string
+   */
+  public $constraintname = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbname',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tablename',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'constraintname',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbname'])) {
+        $this->dbname = $vals['dbname'];
+      }
+      if (isset($vals['tablename'])) {
+        $this->tablename = $vals['tablename'];
+      }
+      if (isset($vals['constraintname'])) {
+        $this->constraintname = $vals['constraintname'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'DropConstraintRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbname);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tablename);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->constraintname);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('DropConstraintRequest');
+    if ($this->dbname !== null) {
+      $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbname);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tablename !== null) {
+      $xfer += $output->writeFieldBegin('tablename', TType::STRING, 2);
+      $xfer += $output->writeString($this->tablename);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->constraintname !== null) {
+      $xfer += $output->writeFieldBegin('constraintname', TType::STRING, 3);
+      $xfer += $output->writeString($this->constraintname);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class PartitionsByExprResult {
   static $_TSPEC;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 3ec46f1..5323d9f 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -43,6 +43,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void create_table(Table tbl)')
   print('  void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context)')
   print('  void create_table_with_constraints(Table tbl,  primaryKeys,  foreignKeys)')
+  print('  void drop_constraint(DropConstraintRequest req)')
   print('  void drop_table(string dbname, string name, bool deleteData)')
   print('  void drop_table_with_environment_context(string dbname, string name, bool deleteData, EnvironmentContext environment_context)')
   print('   get_tables(string db_name, string pattern)')
@@ -353,6 +354,12 @@ elif cmd == 'create_table_with_constraints':
     sys.exit(1)
   pp.pprint(client.create_table_with_constraints(eval(args[0]),eval(args[1]),eval(args[2]),))
 
+elif cmd == 'drop_constraint':
+  if len(args) != 1:
+    print('drop_constraint requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.drop_constraint(eval(args[0]),))
+
 elif cmd == 'drop_table':
   if len(args) != 3:
     print('drop_table requires 3 args')

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 119a5f1..bf8d383 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -165,6 +165,13 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def drop_constraint(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
   def drop_table(self, dbname, name, deleteData):
     """
     Parameters:
@@ -1875,6 +1882,39 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o4
     return
 
+  def drop_constraint(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_drop_constraint(req)
+    self.recv_drop_constraint()
+
+  def send_drop_constraint(self, req):
+    self._oprot.writeMessageBegin('drop_constraint', TMessageType.CALL, self._seqid)
+    args = drop_constraint_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_drop_constraint(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = drop_constraint_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o3 is not None:
+      raise result.o3
+    return
+
   def drop_table(self, dbname, name, deleteData):
     """
     Parameters:
@@ -6499,6 +6539,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["create_table"] = Processor.process_create_table
     self._processMap["create_table_with_environment_context"] = Processor.process_create_table_with_environment_context
     self._processMap["create_table_with_constraints"] = Processor.process_create_table_with_constraints
+    self._processMap["drop_constraint"] = Processor.process_drop_constraint
     self._processMap["drop_table"] = Processor.process_drop_table
     self._processMap["drop_table_with_environment_context"] = Processor.process_drop_table_with_environment_context
     self._processMap["get_tables"] = Processor.process_get_tables
@@ -7141,6 +7182,31 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_drop_constraint(self, seqid, iprot, oprot):
+    args = drop_constraint_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = drop_constraint_result()
+    try:
+      self._handler.drop_constraint(args.req)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o3:
+      msg_type = TMessageType.REPLY
+      result.o3 = o3
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("drop_constraint", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_drop_table(self, seqid, iprot, oprot):
     args = drop_table_args()
     args.read(iprot)
@@ -13467,6 +13533,152 @@ class create_table_with_constraints_result:
   def __ne__(self, other):
     return not (self == other)
 
+class drop_constraint_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (DropConstraintRequest, DropConstraintRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, req=None,):
+    self.req = req
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.req = DropConstraintRequest()
+          self.req.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('drop_constraint_args')
+    if self.req is not None:
+      oprot.writeFieldBegin('req', TType.STRUCT, 1)
+      self.req.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.req)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class drop_constraint_result:
+  """
+  Attributes:
+   - o1
+   - o3
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, o1=None, o3=None,):
+    self.o1 = o1
+    self.o3 = o3
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = NoSuchObjectException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.o3 = MetaException()
+          self.o3.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('drop_constraint_result')
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o3 is not None:
+      oprot.writeFieldBegin('o3', TType.STRUCT, 2)
+      self.o3.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.o1)
+    value = (value * 31) ^ hash(self.o3)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class drop_table_args:
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index f008788..8e0cb71 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -6264,6 +6264,103 @@ class ForeignKeysResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class DropConstraintRequest:
+  """
+  Attributes:
+   - dbname
+   - tablename
+   - constraintname
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'dbname', None, None, ), # 1
+    (2, TType.STRING, 'tablename', None, None, ), # 2
+    (3, TType.STRING, 'constraintname', None, None, ), # 3
+  )
+
+  def __init__(self, dbname=None, tablename=None, constraintname=None,):
+    self.dbname = dbname
+    self.tablename = tablename
+    self.constraintname = constraintname
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.dbname = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.tablename = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.constraintname = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('DropConstraintRequest')
+    if self.dbname is not None:
+      oprot.writeFieldBegin('dbname', TType.STRING, 1)
+      oprot.writeString(self.dbname)
+      oprot.writeFieldEnd()
+    if self.tablename is not None:
+      oprot.writeFieldBegin('tablename', TType.STRING, 2)
+      oprot.writeString(self.tablename)
+      oprot.writeFieldEnd()
+    if self.constraintname is not None:
+      oprot.writeFieldBegin('constraintname', TType.STRING, 3)
+      oprot.writeString(self.constraintname)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.dbname is None:
+      raise TProtocol.TProtocolException(message='Required field dbname is unset!')
+    if self.tablename is None:
+      raise TProtocol.TProtocolException(message='Required field tablename is unset!')
+    if self.constraintname is None:
+      raise TProtocol.TProtocolException(message='Required field constraintname is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.dbname)
+    value = (value * 31) ^ hash(self.tablename)
+    value = (value * 31) ^ hash(self.constraintname)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class PartitionsByExprResult:
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 4a24a19..4d3e49d 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -1390,6 +1390,29 @@ class ForeignKeysResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class DropConstraintRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DBNAME = 1
+  TABLENAME = 2
+  CONSTRAINTNAME = 3
+
+  FIELDS = {
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'},
+    TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tablename'},
+    CONSTRAINTNAME => {:type => ::Thrift::Types::STRING, :name => 'constraintname'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbname is unset!') unless @dbname
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tablename is unset!') unless @tablename
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field constraintname is unset!') unless @constraintname
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class PartitionsByExprResult
   include ::Thrift::Struct, ::Thrift::Struct_Union
   PARTITIONS = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 99a764e..61d1832 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -336,6 +336,22 @@ module ThriftHiveMetastore
       return
     end
 
+    def drop_constraint(req)
+      send_drop_constraint(req)
+      recv_drop_constraint()
+    end
+
+    def send_drop_constraint(req)
+      send_message('drop_constraint', Drop_constraint_args, :req => req)
+    end
+
+    def recv_drop_constraint()
+      result = receive_message(Drop_constraint_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o3 unless result.o3.nil?
+      return
+    end
+
     def drop_table(dbname, name, deleteData)
       send_drop_table(dbname, name, deleteData)
       recv_drop_table()
@@ -2704,6 +2720,19 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'create_table_with_constraints', seqid)
     end
 
+    def process_drop_constraint(seqid, iprot, oprot)
+      args = read_args(iprot, Drop_constraint_args)
+      result = Drop_constraint_result.new()
+      begin
+        @handler.drop_constraint(args.req)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o3
+        result.o3 = o3
+      end
+      write_result(result, oprot, 'drop_constraint', seqid)
+    end
+
     def process_drop_table(seqid, iprot, oprot)
       args = read_args(iprot, Drop_table_args)
       result = Drop_table_result.new()
@@ -4954,6 +4983,40 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Drop_constraint_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::DropConstraintRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Drop_constraint_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O3 = 2
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Drop_table_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     DBNAME = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 4ada9c1..9a09e7a 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -1483,6 +1483,35 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         endFunction("create_table", success, ex, tbl.getTableName());
       }
     }
+
+    @Override
+    public void drop_constraint(DropConstraintRequest req)
+        throws MetaException, InvalidObjectException {
+      String dbName = req.getDbname();
+      String tableName = req.getTablename();
+      String constraintName = req.getConstraintname();
+      startFunction("drop_constraint", ": " + constraintName.toString());
+      boolean success = false;
+      Exception ex = null;
+      try {
+        getMS().dropConstraint(dbName, tableName, constraintName);
+        success = true;
+      } catch (NoSuchObjectException e) {
+        ex = e;
+        throw new InvalidObjectException(e.getMessage());
+      } catch (Exception e) {
+        ex = e;
+        if (e instanceof MetaException) {
+          throw (MetaException) e;
+        } else if (e instanceof InvalidObjectException) {
+          throw (InvalidObjectException) e;
+        } else {
+          throw newMetaException(e);
+        }
+      } finally {
+        endFunction("drop_constraint", success, ex, constraintName);
+      }
+    }
     private boolean is_table_exists(RawStore ms, String dbname, String name)
         throws MetaException {
       return (ms.getTable(dbname, name) != null);

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 7d37d07..75fea5b 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DropConstraintRequest;
 import org.apache.hadoop.hive.metastore.api.DropPartitionsExpr;
 import org.apache.hadoop.hive.metastore.api.DropPartitionsRequest;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
@@ -765,6 +766,11 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     }
   }
 
+  @Override
+  public void dropConstraint(String dbName, String tableName, String constraintName) throws
+    NoSuchObjectException, MetaException, TException {
+    client.drop_constraint(new DropConstraintRequest(dbName, tableName, constraintName));
+  }
 
 /**
    * @param type

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index c900a2d..3965475 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -1570,4 +1570,7 @@ public interface IMetaStoreClient {
     List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
     throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, TException;
 
+  void dropConstraint(String dbName, String tableName, String constraintName) throws 
+    MetaException, NoSuchObjectException, TException;
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index f651a13..5c49be9 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -1025,7 +1025,8 @@ public class ObjectStore implements RawStore, Configurable {
           " table " + tableName + " record to delete");
         }
 
-        List<MConstraint> tabConstraints = listAllTableConstraints(dbName, tableName);
+        List<MConstraint> tabConstraints = listAllTableConstraintsWithOptionalConstraintName(
+                                           dbName, tableName, null);
         if (tabConstraints != null && tabConstraints.size() > 0) {
           pm.deletePersistentAll(tabConstraints);
         }
@@ -1043,19 +1044,27 @@ public class ObjectStore implements RawStore, Configurable {
     return success;
   }
 
-  private List<MConstraint> listAllTableConstraints(String dbName, String tableName) {
+  private List<MConstraint> listAllTableConstraintsWithOptionalConstraintName
+    (String dbName, String tableName, String constraintname) {
     List<MConstraint> mConstraints = null;
     List<String> constraintNames = new ArrayList<String>();
     Query query = null;
 
     try {
       query = pm.newQuery("select constraintName from org.apache.hadoop.hive.metastore.model.MConstraint  where "
-        + "(parentTable.tableName == ptblname && parentTable.database.name == pdbname) || "
-        + "(childTable != null && childTable.tableName == ctblname && childTable.database.name == cdbname)");
+        + "((parentTable.tableName == ptblname && parentTable.database.name == pdbname) || "
+        + "(childTable != null && childTable.tableName == ctblname && "
+        + "childTable.database.name == cdbname)) " + (constraintname != null ?
+        " && constraintName == constraintname" : ""));
       query.declareParameters("java.lang.String ptblname, java.lang.String pdbname,"
-      + "java.lang.String ctblname, java.lang.String cdbname");
-      Collection<?> constraintNamesColl = (Collection<?>) query.
-        executeWithArray(tableName, dbName, tableName, dbName);
+      + "java.lang.String ctblname, java.lang.String cdbname" +
+        (constraintname != null ? ", java.lang.String constraintname" : ""));
+      Collection<?> constraintNamesColl =
+        constraintname != null ?
+          ((Collection<?>) query.
+            executeWithArray(tableName, dbName, tableName, dbName, constraintname)):
+          ((Collection<?>) query.
+            executeWithArray(tableName, dbName, tableName, dbName));
       for (Iterator<?> i = constraintNamesColl.iterator(); i.hasNext();) {
         String currName = (String) i.next();
         constraintNames.add(currName);
@@ -8389,4 +8398,27 @@ public class ObjectStore implements RawStore, Configurable {
     return foreignKeys;
   }
 
+  @Override
+  public void dropConstraint(String dbName, String tableName,
+    String constraintName) throws NoSuchObjectException {
+    boolean success = false;
+    try {
+      openTransaction();
+
+      List<MConstraint> tabConstraints = listAllTableConstraintsWithOptionalConstraintName(
+                                         dbName, tableName, constraintName);
+      if (tabConstraints != null && tabConstraints.size() > 0) {
+        pm.deletePersistentAll(tabConstraints);
+      } else {
+        throw new NoSuchObjectException("The constraint: " + constraintName +
+          " does not exist for the associated table: " + dbName + "." + tableName);
+      }
+      success = commitTransaction();
+    } finally {
+      if (!success) {
+        rollbackTransaction();
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
index 100c396..06b8135 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -675,4 +675,6 @@ public interface RawStore extends Configurable {
 
   void createTableWithConstraints(Table tbl, List<SQLPrimaryKey> primaryKeys,
     List<SQLForeignKey> foreignKeys) throws InvalidObjectException, MetaException;
+
+  void dropConstraint(String dbName, String tableName, String constraintName) throws NoSuchObjectException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index d4e5da4..ec5b92c 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -2615,4 +2615,10 @@ public class HBaseStore implements RawStore {
     throws InvalidObjectException, MetaException {
     // TODO Auto-generated method stub
   }
+
+  @Override
+  public void dropConstraint(String dbName, String tableName,
+    String constraintName) throws NoSuchObjectException {
+    // TODO Auto-generated method stub 
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 86e7bea..63fcb28 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -844,4 +844,10 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
     throws InvalidObjectException, MetaException {
     // TODO Auto-generated method stub
   }
+
+  @Override
+  public void dropConstraint(String dbName, String tableName,
+   String constraintName) throws NoSuchObjectException {
+   // TODO Auto-generated method stub
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 5b32f00..386c70a 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -860,6 +860,12 @@ public class DummyRawStoreForJdoConnection implements RawStore {
     throws InvalidObjectException, MetaException {
     // TODO Auto-generated method stub
   }
+
+  @Override
+  public void dropConstraint(String dbName, String tableName,
+  String constraintName) throws NoSuchObjectException {
+    // TODO Auto-generated method stub
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 9887d77..c4d3bfb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -131,6 +131,7 @@ import org.apache.hadoop.hive.ql.parse.AlterTablePartMergeFilesDesc;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
 import org.apache.hadoop.hive.ql.plan.AlterDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.AlterIndexDesc;
@@ -356,7 +357,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       AlterTableDesc alterTbl = work.getAlterTblDesc();
       if (alterTbl != null) {
-        return alterTable(db, alterTbl);
+        if (alterTbl.getOp() == AlterTableTypes.DROPCONSTRAINT ) {
+          return dropConstraint(db, alterTbl);
+        } else {
+          return alterTable(db, alterTbl);
+        }
       }
 
       CreateViewDesc crtView = work.getCreateViewDesc();
@@ -3596,7 +3601,19 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
-  /**
+   private int dropConstraint(Hive db, AlterTableDesc alterTbl)
+    throws SemanticException, HiveException {
+     try {
+      db.dropConstraint(Utilities.getDatabaseName(alterTbl.getOldName()),
+        Utilities.getTableName(alterTbl.getOldName()),
+          alterTbl.getConstraintName());
+      } catch (NoSuchObjectException e) {
+        throw new HiveException(e);
+      }
+     return 0;
+   }
+
+   /**
    * Drop a given table or some partitions. DropTableDesc is currently used for both.
    *
    * @param db

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java
index 515f8b2..2194a6d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java
@@ -203,7 +203,8 @@ public class WriteEntity extends Entity implements Serializable {
       case ADDCOLS:
       case RENAME:
       case TRUNCATE:
-      case MERGEFILES: return WriteType.DDL_EXCLUSIVE;
+      case MERGEFILES:
+      case DROPCONSTRAINT: return WriteType.DDL_EXCLUSIVE;
 
       case ADDPARTITION:
       case ADDSERDEPROPS:

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index 26c458c..6862f70 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -3593,4 +3593,13 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
+  public void dropConstraint(String dbName, String tableName, String constraintName)
+    throws HiveException, NoSuchObjectException {
+    try {
+      getMSC().dropConstraint(dbName, tableName, constraintName);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
 };

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index 04e2a41..4a6617f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -26,7 +26,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -88,7 +87,6 @@ import org.apache.hadoop.hive.ql.plan.ColumnStatsDesc;
 import org.apache.hadoop.hive.ql.plan.ColumnStatsUpdateWork;
 import org.apache.hadoop.hive.ql.plan.CreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.CreateIndexDesc;
-import org.apache.hadoop.hive.ql.plan.DDLDesc;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.DescDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.DescFunctionDesc;
@@ -321,6 +319,8 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         analyzeAlterTableCompact(ast, tableName, partSpec);
       } else if(ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_UPDATECOLSTATS){
         analyzeAlterTableUpdateStats(ast, tableName, partSpec);
+      }  else if(ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_DROPCONSTRAINT) {
+        analyzeAlterTableDropConstraint(ast, tableName);
       }
       break;
     }
@@ -1740,6 +1740,15 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
   }
 
+  private void analyzeAlterTableDropConstraint(ASTNode ast, String tableName)
+    throws SemanticException {
+    String dropConstraintName = unescapeIdentifier(ast.getChild(0).getText());
+    AlterTableDesc alterTblDesc = new AlterTableDesc(tableName, dropConstraintName);
+
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
+        alterTblDesc), conf));
+  }
+
   static HashMap<String, String> getProps(ASTNode prop) {
     // Must be deterministic order map for consistent q-test output across Java versions
     HashMap<String, String> mapProp = new LinkedHashMap<String, String>();

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 6531b03..2c66396 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -179,6 +179,7 @@ TOK_ALTERTABLE_SKEWED_LOCATION;
 TOK_ALTERTABLE_BUCKETS;
 TOK_ALTERTABLE_CLUSTER_SORT;
 TOK_ALTERTABLE_COMPACT;
+TOK_ALTERTABLE_DROPCONSTRAINT;
 TOK_ALTERINDEX_REBUILD;
 TOK_ALTERINDEX_PROPERTIES;
 TOK_MSCK;
@@ -1040,6 +1041,7 @@ alterTableStatementSuffix
     | alterStatementSuffixSkewedby
     | alterStatementSuffixExchangePartition
     | alterStatementPartitionKeyType
+    | alterStatementSuffixDropConstraint
     | partitionSpec? alterTblPartitionStatementSuffix -> alterTblPartitionStatementSuffix partitionSpec?
     ;
 
@@ -1129,6 +1131,13 @@ alterStatementSuffixAddCol
     ->                 ^(TOK_ALTERTABLE_REPLACECOLS columnNameTypeList restrictOrCascade?)
     ;
 
+alterStatementSuffixDropConstraint
+@init { pushMsg("drop constraint statement", state); }
+@after { popMsg(state); }
+   : KW_DROP KW_CONSTRAINT cName=identifier
+   ->^(TOK_ALTERTABLE_DROPCONSTRAINT $cName)
+   ;
+
 alterStatementSuffixRenameCol
 @init { pushMsg("rename column name", state); }
 @after { popMsg(state); }

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index fb8a33c..7b83381 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -62,6 +62,7 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_ALTERTABLE_PROPERTIES, HiveOperation.ALTERTABLE_PROPERTIES);
     commandType.put(HiveParser.TOK_ALTERTABLE_DROPPROPERTIES, HiveOperation.ALTERTABLE_PROPERTIES);
     commandType.put(HiveParser.TOK_ALTERTABLE_EXCHANGEPARTITION, HiveOperation.ALTERTABLE_EXCHANGEPARTITION);
+    commandType.put(HiveParser.TOK_ALTERTABLE_DROPCONSTRAINT, HiveOperation.ALTERTABLE_DROPCONSTRAINT);
     commandType.put(HiveParser.TOK_SHOWDATABASES, HiveOperation.SHOWDATABASES);
     commandType.put(HiveParser.TOK_SHOWTABLES, HiveOperation.SHOWTABLES);
     commandType.put(HiveParser.TOK_SHOWCOLUMNS, HiveOperation.SHOWCOLUMNS);
@@ -195,6 +196,7 @@ public final class SemanticAnalyzerFactory {
           case HiveParser.TOK_ALTERTABLE_DROPPROPERTIES:
           case HiveParser.TOK_ALTERTABLE_EXCHANGEPARTITION:
           case HiveParser.TOK_ALTERTABLE_SKEWED:
+          case HiveParser.TOK_ALTERTABLE_DROPCONSTRAINT:
           queryState.setCommandType(commandType.get(child.getType()));
           return new DDLSemanticAnalyzer(queryState);
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
index 4ba51ec..38d8d5a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
@@ -56,7 +56,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
     DROPPARTITION("drop partition"), RENAMEPARTITION("rename partition"), ADDSKEWEDBY("add skew column"),
     ALTERSKEWEDLOCATION("alter skew location"), ALTERBUCKETNUM("alter bucket number"),
     ALTERPARTITION("alter partition"), COMPACT("compact"),
-    TRUNCATE("truncate"), MERGEFILES("merge files");
+    TRUNCATE("truncate"), MERGEFILES("merge files"), DROPCONSTRAINT("drop constraint");
     ;
 
     private final String name;
@@ -116,6 +116,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
   boolean isTurnOffSorting = false;
   boolean isCascade = false;
   EnvironmentContext environmentContext;
+  String dropConstraintName;
 
   public AlterTableDesc() {
   }
@@ -263,6 +264,12 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
     this.numberBuckets = numBuckets;
   }
 
+  public AlterTableDesc(String tableName, String dropConstraintName) {
+    this.oldName = tableName;
+    this.dropConstraintName = dropConstraintName;
+    op = AlterTableTypes.DROPCONSTRAINT;
+  }
+
   @Explain(displayName = "new columns", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
   public List<String> getNewColsString() {
     return Utilities.getFieldSchemaString(getNewCols());
@@ -408,6 +415,22 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
   }
 
   /**
+   * @return the drop constraint name of the table
+   */
+  @Explain(displayName = "drop constraint name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public String getConstraintName() {
+    return dropConstraintName;
+  }
+
+  /**
+   * @param constraintName
+   *          the dropConstraintName to set
+   */
+  public void setDropConstraintName(String constraintName) {
+    this.dropConstraintName = constraintName;
+  }
+
+  /**
    * @param storageHandler
    *          the storage handler to set
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
index 188cd6f..e651016 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
@@ -115,6 +115,8 @@ public enum HiveOperation {
       new Privilege[] {Privilege.ALTER_DATA}, null),
   ALTERTABLE_PARTCOLTYPE("ALTERTABLE_PARTCOLTYPE", new Privilege[] { Privilege.SELECT }, new Privilege[] { Privilege.ALTER_DATA }),
   ALTERTABLE_EXCHANGEPARTITION("ALTERTABLE_EXCHANGEPARTITION", null, null),
+  ALTERTABLE_DROPCONSTRAINT("ALTERTABLE_DROPCONSTRAINT",
+      new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERVIEW_RENAME("ALTERVIEW_RENAME", new Privilege[] {Privilege.ALTER_METADATA}, null),
   ALTERVIEW_AS("ALTERVIEW_AS", new Privilege[] {Privilege.ALTER_METADATA}, null),
   ALTERTABLE_COMPACT("ALTERTABLE_COMPACT", new Privilege[]{Privilege.SELECT}, new Privilege[]{Privilege.ALTER_DATA}),

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/test/queries/clientnegative/drop_invalid_constraint1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/drop_invalid_constraint1.q b/ql/src/test/queries/clientnegative/drop_invalid_constraint1.q
new file mode 100644
index 0000000..2055f9e
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/drop_invalid_constraint1.q
@@ -0,0 +1,3 @@
+CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate);
+ALTER TABLE table1 DROP CONSTRAINT pk1;
+ALTER TABLE table1 DROP CONSTRAINT pk1;

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/test/queries/clientnegative/drop_invalid_constraint2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/drop_invalid_constraint2.q b/ql/src/test/queries/clientnegative/drop_invalid_constraint2.q
new file mode 100644
index 0000000..d253617
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/drop_invalid_constraint2.q
@@ -0,0 +1,2 @@
+CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate);
+ALTER TABLE table1 DROP CONSTRAINT pk1;

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/test/queries/clientnegative/drop_invalid_constraint3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/drop_invalid_constraint3.q b/ql/src/test/queries/clientnegative/drop_invalid_constraint3.q
new file mode 100644
index 0000000..04eb1fb
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/drop_invalid_constraint3.q
@@ -0,0 +1,2 @@
+CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate);
+ALTER TABLE table2 DROP CONSTRAINT pk2;

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/test/queries/clientnegative/drop_invalid_constraint4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/drop_invalid_constraint4.q b/ql/src/test/queries/clientnegative/drop_invalid_constraint4.q
new file mode 100644
index 0000000..3cf2d2a
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/drop_invalid_constraint4.q
@@ -0,0 +1,3 @@
+CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate);
+CREATE TABLE table2 (a STRING, b STRING, constraint pk2 primary key (a) disable novalidate);
+ALTER TABLE table1 DROP CONSTRAINT pk2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/test/queries/clientpositive/create_with_constraints.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/create_with_constraints.q b/ql/src/test/queries/clientpositive/create_with_constraints.q
index eef0c64..0bb92e4 100644
--- a/ql/src/test/queries/clientpositive/create_with_constraints.q
+++ b/ql/src/test/queries/clientpositive/create_with_constraints.q
@@ -8,5 +8,17 @@ CREATE TABLE table6 (x string, y string, PRIMARY KEY (x) disable novalidate, FOR
 CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a)  DISABLE NOVALIDATE);
 CREATE TABLE table7 (a STRING, b STRING, primary key (a) disable novalidate rely);
 CREATE TABLE table8 (a STRING, b STRING, constraint pk8 primary key (a) disable novalidate norely);
+CREATE TABLE table9 (a STRING, b STRING, primary key (a, b) disable novalidate rely);
+CREATE TABLE table10 (a STRING, b STRING, constraint pk10 primary key (a) disable novalidate norely, foreign key (a, b) references table9(a, b) disable novalidate);
+CREATE TABLE table11 (a STRING, b STRING, c STRING, constraint pk11 primary key (a) disable novalidate rely, foreign key (a, b) references table9(a, b) disable novalidate,
+foreign key (c) references table4(x) disable novalidate);
 
+ALTER TABLE table2 DROP CONSTRAINT pk1;
+ALTER TABLE table3 DROP CONSTRAINT fk1;
+ALTER TABLE table6 DROP CONSTRAINT fk4;
 
+CREATE DATABASE dbconstraint;
+USE dbconstraint;
+CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate);
+USE default;
+ALTER TABLE dbconstraint.table2 DROP CONSTRAINT pk1;

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/test/results/clientnegative/drop_invalid_constraint1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/drop_invalid_constraint1.q.out b/ql/src/test/results/clientnegative/drop_invalid_constraint1.q.out
new file mode 100644
index 0000000..4568ccb
--- /dev/null
+++ b/ql/src/test/results/clientnegative/drop_invalid_constraint1.q.out
@@ -0,0 +1,15 @@
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table1
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table1
+PREHOOK: query: ALTER TABLE table1 DROP CONSTRAINT pk1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table1 DROP CONSTRAINT pk1
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: ALTER TABLE table1 DROP CONSTRAINT pk1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidObjectException(message:The constraint: pk1 does not exist for the associated table: default.table1)

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/test/results/clientnegative/drop_invalid_constraint2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/drop_invalid_constraint2.q.out b/ql/src/test/results/clientnegative/drop_invalid_constraint2.q.out
new file mode 100644
index 0000000..0051131
--- /dev/null
+++ b/ql/src/test/results/clientnegative/drop_invalid_constraint2.q.out
@@ -0,0 +1,11 @@
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table2
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table2
+PREHOOK: query: ALTER TABLE table1 DROP CONSTRAINT pk1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidObjectException(message:The constraint: pk1 does not exist for the associated table: default.table1)

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/test/results/clientnegative/drop_invalid_constraint3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/drop_invalid_constraint3.q.out b/ql/src/test/results/clientnegative/drop_invalid_constraint3.q.out
new file mode 100644
index 0000000..9c60e94
--- /dev/null
+++ b/ql/src/test/results/clientnegative/drop_invalid_constraint3.q.out
@@ -0,0 +1,11 @@
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table2
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table2
+PREHOOK: query: ALTER TABLE table2 DROP CONSTRAINT pk2
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidObjectException(message:The constraint: pk2 does not exist for the associated table: default.table2)

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/test/results/clientnegative/drop_invalid_constraint4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/drop_invalid_constraint4.q.out b/ql/src/test/results/clientnegative/drop_invalid_constraint4.q.out
new file mode 100644
index 0000000..1d93c42
--- /dev/null
+++ b/ql/src/test/results/clientnegative/drop_invalid_constraint4.q.out
@@ -0,0 +1,19 @@
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table1
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table1
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk2 primary key (a) disable novalidate)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table2
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk2 primary key (a) disable novalidate)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table2
+PREHOOK: query: ALTER TABLE table1 DROP CONSTRAINT pk2
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidObjectException(message:The constraint: pk2 does not exist for the associated table: default.table1)

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/ql/src/test/results/clientpositive/create_with_constraints.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_with_constraints.q.out b/ql/src/test/results/clientpositive/create_with_constraints.q.out
index 5cf8d83..7a7a50a 100644
--- a/ql/src/test/results/clientpositive/create_with_constraints.q.out
+++ b/ql/src/test/results/clientpositive/create_with_constraints.q.out
@@ -66,3 +66,71 @@ POSTHOOK: query: CREATE TABLE table8 (a STRING, b STRING, constraint pk8 primary
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table8
+PREHOOK: query: CREATE TABLE table9 (a STRING, b STRING, primary key (a, b) disable novalidate rely)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table9
+POSTHOOK: query: CREATE TABLE table9 (a STRING, b STRING, primary key (a, b) disable novalidate rely)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table9
+PREHOOK: query: CREATE TABLE table10 (a STRING, b STRING, constraint pk10 primary key (a) disable novalidate norely, foreign key (a, b) references table9(a, b) disable novalidate)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table10
+POSTHOOK: query: CREATE TABLE table10 (a STRING, b STRING, constraint pk10 primary key (a) disable novalidate norely, foreign key (a, b) references table9(a, b) disable novalidate)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table10
+PREHOOK: query: CREATE TABLE table11 (a STRING, b STRING, c STRING, constraint pk11 primary key (a) disable novalidate rely, foreign key (a, b) references table9(a, b) disable novalidate,
+foreign key (c) references table4(x) disable novalidate)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table11
+POSTHOOK: query: CREATE TABLE table11 (a STRING, b STRING, c STRING, constraint pk11 primary key (a) disable novalidate rely, foreign key (a, b) references table9(a, b) disable novalidate,
+foreign key (c) references table4(x) disable novalidate)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table11
+PREHOOK: query: ALTER TABLE table2 DROP CONSTRAINT pk1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table2 DROP CONSTRAINT pk1
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: ALTER TABLE table3 DROP CONSTRAINT fk1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table3 DROP CONSTRAINT fk1
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: ALTER TABLE table6 DROP CONSTRAINT fk4
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table6 DROP CONSTRAINT fk4
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: CREATE DATABASE dbconstraint
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:dbconstraint
+POSTHOOK: query: CREATE DATABASE dbconstraint
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:dbconstraint
+PREHOOK: query: USE dbconstraint
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:dbconstraint
+POSTHOOK: query: USE dbconstraint
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:dbconstraint
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:dbconstraint
+PREHOOK: Output: dbconstraint@table2
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:dbconstraint
+POSTHOOK: Output: dbconstraint@table2
+PREHOOK: query: USE default
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:default
+POSTHOOK: query: USE default
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:default
+PREHOOK: query: ALTER TABLE dbconstraint.table2 DROP CONSTRAINT pk1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE dbconstraint.table2 DROP CONSTRAINT pk1
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT

http://git-wip-us.apache.org/repos/asf/hive/blob/212077b8/service/src/gen/thrift/gen-py/__init__.py
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-py/__init__.py b/service/src/gen/thrift/gen-py/__init__.py
deleted file mode 100644
index e69de29..0000000


[48/50] [abbrv] hive git commit: Merge branch 'master' into llap

Posted by sp...@apache.org.
Merge branch 'master' into llap


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

Branch: refs/heads/java8
Commit: f089f2e64241592ecf8144d044bec8a0659ff422
Parents: 89ec219 3f07bfc
Author: Jason Dere <jd...@hortonworks.com>
Authored: Fri May 6 10:14:21 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Fri May 6 10:14:21 2016 -0700

----------------------------------------------------------------------
 beeline/src/main/resources/BeeLine.properties   |   9 +-
 cli/pom.xml                                     |   6 +
 common/pom.xml                                  |   6 +
 .../org/apache/hadoop/hive/common/LogUtils.java |  35 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   8 +
 .../src/main/resources/hive-log4j2.properties   |   2 +-
 .../hadoop/hive/conf/TestHiveAsyncLogging.java  |  49 ++
 data/conf/hive-log4j2.properties                |   2 +-
 hcatalog/core/pom.xml                           |   6 +
 .../hive/metastore/TestHiveMetaStoreTxns.java   |   2 +-
 llap-server/bin/llapDaemon.sh                   |   2 +-
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |  21 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |  10 +-
 ...doop-metrics2-llapdaemon.properties.template |  50 ++
 ...trics2-llaptaskscheduler.properties.template |  50 ++
 .../hadoop-metrics2.properties.template         |  50 --
 .../main/resources/llap-cli-log4j2.properties   |   2 +-
 .../resources/llap-daemon-log4j2.properties     |   4 +-
 llap-server/src/main/resources/package.py       |   6 +-
 .../resources/llap-daemon-log4j2.properties     |   4 +-
 .../tezplugins/LlapTaskSchedulerService.java    |   2 +-
 .../metrics/LlapTaskSchedulerMetrics.java       |   6 +-
 metastore/pom.xml                               |   6 +
 .../upgrade/derby/035-HIVE-13395.derby.sql      |  11 +
 .../upgrade/derby/hive-schema-2.1.0.derby.sql   |   2 +-
 .../derby/hive-txn-schema-1.3.0.derby.sql       |  11 +-
 .../derby/hive-txn-schema-2.1.0.derby.sql       | 130 ++++
 .../derby/upgrade-1.2.0-to-1.3.0.derby.sql      |   1 +
 .../derby/upgrade-2.0.0-to-2.1.0.derby.sql      |   1 +
 .../upgrade/mssql/020-HIVE-13395.mssql.sql      |   9 +
 .../upgrade/mssql/hive-schema-1.3.0.mssql.sql   |  12 +-
 .../upgrade/mssql/hive-schema-2.1.0.mssql.sql   |  12 +-
 .../mssql/upgrade-1.2.0-to-1.3.0.mssql.sql      |   1 +
 .../mssql/upgrade-2.0.0-to-2.1.0.mssql.sql      |   1 +
 .../upgrade/mysql/035-HIVE-13395.mysql.sql      |  10 +
 .../upgrade/mysql/hive-schema-2.1.0.mysql.sql   |   2 +-
 .../mysql/hive-txn-schema-1.3.0.mysql.sql       |  10 +
 .../mysql/hive-txn-schema-2.1.0.mysql.sql       | 131 ++++
 .../mysql/upgrade-1.2.0-to-1.3.0.mysql.sql      |   1 +
 .../mysql/upgrade-2.0.0-to-2.1.0.mysql.sql      |   1 +
 .../upgrade/oracle/035-HIVE-13395.oracle.sql    |  10 +
 .../upgrade/oracle/hive-schema-2.1.0.oracle.sql |   2 +-
 .../oracle/hive-txn-schema-1.3.0.oracle.sql     |  12 +-
 .../oracle/hive-txn-schema-2.1.0.oracle.sql     | 129 ++++
 .../oracle/upgrade-1.2.0-to-1.3.0.oracle.sql    |   1 +
 .../oracle/upgrade-2.0.0-to-2.1.0.oracle.sql    |   1 +
 .../postgres/034-HIVE-13395.postgres.sql        |  10 +
 .../postgres/hive-schema-2.1.0.postgres.sql     |   2 +-
 .../postgres/hive-txn-schema-1.3.0.postgres.sql |  11 +-
 .../postgres/hive-txn-schema-2.1.0.postgres.sql | 129 ++++
 .../upgrade-1.2.0-to-1.3.0.postgres.sql         |   1 +
 .../upgrade-2.0.0-to-2.1.0.postgres.sql         |   1 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   1 +
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    | 130 ++--
 .../hadoop/hive/metastore/txn/TxnHandler.java   | 466 +++++++++++---
 .../hadoop/hive/metastore/txn/TxnStore.java     |   8 +-
 .../hadoop/hive/metastore/txn/TxnUtils.java     |   2 +
 .../metastore/txn/TestCompactionTxnHandler.java |   6 +-
 .../hive/metastore/txn/TestTxnHandler.java      |  29 +-
 pom.xml                                         |   2 +
 ql/pom.xml                                      |   6 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  11 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |   2 +-
 .../hadoop/hive/ql/exec/OperatorUtils.java      |   2 +-
 .../org/apache/hadoop/hive/ql/exec/Task.java    |  21 +-
 .../apache/hadoop/hive/ql/exec/TaskResult.java  |   7 +-
 .../apache/hadoop/hive/ql/exec/TaskRunner.java  |   5 +-
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    |   1 +
 .../hadoop/hive/ql/exec/mr/JobDebugger.java     |  18 +-
 .../hadoop/hive/ql/lockmgr/DbLockManager.java   |   5 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |  27 +-
 .../calcite/translator/JoinTypeCheckCtx.java    |   2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  17 +-
 .../hadoop/hive/ql/parse/TypeCheckCtx.java      |  19 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |  26 +
 .../hadoop/hive/ql/txn/AcidWriteSetService.java |  61 ++
 .../txn/compactor/HouseKeeperServiceBase.java   |   2 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |   2 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java    |   2 +-
 .../main/resources/hive-exec-log4j2.properties  |   2 +-
 .../resources/tez-container-log4j2.properties   |   2 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |   2 +-
 .../apache/hadoop/hive/ql/io/TestAcidUtils.java |  20 +
 .../hive/ql/lockmgr/TestDbTxnManager.java       |   7 +
 .../hive/ql/lockmgr/TestDbTxnManager2.java      | 610 ++++++++++++++++++-
 .../hive/ql/txn/compactor/TestCleaner.java      |   4 +
 .../queries/clientpositive/constantPropWhen.q   |   2 +
 .../hive/ptest/execution/ExecutionPhase.java    |   2 +
 .../hive/ptest/execution/HostExecutor.java      |  48 +-
 .../hive/ptest/execution/LocalCommand.java      |  31 +-
 .../apache/hive/ptest/execution/PrepPhase.java  |   1 +
 .../apache/hive/ptest/execution/conf/Host.java  |   3 +
 92 files changed, 2294 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f089f2e6/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f089f2e6/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f089f2e6/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f089f2e6/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f089f2e6/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------


[30/50] [abbrv] hive git commit: HIVE-13639: CBO rule to pull up constants through Union (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/perf/query75.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query75.q.out b/ql/src/test/results/clientpositive/perf/query75.q.out
index 15c46c2..731ff62 100644
--- a/ql/src/test/results/clientpositive/perf/query75.q.out
+++ b/ql/src/test/results/clientpositive/perf/query75.q.out
@@ -41,363 +41,367 @@ Stage-0
           <-Reducer 7 [SIMPLE_EDGE]
             SHUFFLE [RS_153]
               Select Operator [SEL_152] (rows=169103 width=1436)
-                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
+                Output:["_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
                 Filter Operator [FIL_151] (rows=169103 width=1436)
                   predicate:(UDFToDouble((CAST( _col5 AS decimal(17,2)) / CAST( _col12 AS decimal(17,2)))) < 0.9)
                   Merge Join Operator [MERGEJOIN_259] (rows=507310 width=1436)
-                    Conds:RS_148._col1, _col2, _col3, _col4=RS_149._col1, _col2, _col3, _col4(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col12","_col13"]
+                    Conds:RS_148._col1, _col2, _col3, _col4=RS_149._col1, _col2, _col3, _col4(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col12","_col13"]
                   <-Reducer 31 [SIMPLE_EDGE]
                     SHUFFLE [RS_149]
                       PartitionCols:_col1, _col2, _col3, _col4
-                      Group By Operator [GBY_146] (rows=461191 width=1436)
-                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
-                      <-Union 30 [SIMPLE_EDGE]
-                        <-Reducer 29 [CONTAINS]
-                          Reduce Output Operator [RS_145]
-                            PartitionCols:_col0, _col1, _col2, _col3, _col4
-                            Group By Operator [GBY_144] (rows=922383 width=1436)
-                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4
-                              Select Operator [SEL_142] (rows=922383 width=1436)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                Select Operator [SEL_95] (rows=307461 width=1436)
+                      Select Operator [SEL_147] (rows=461191 width=1436)
+                        Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
+                        Group By Operator [GBY_146] (rows=461191 width=1436)
+                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:2001, KEY._col1, KEY._col2, KEY._col3, KEY._col4
+                        <-Union 30 [SIMPLE_EDGE]
+                          <-Reducer 29 [CONTAINS]
+                            Reduce Output Operator [RS_145]
+                              PartitionCols:2001, _col1, _col2, _col3, _col4
+                              Group By Operator [GBY_144] (rows=922383 width=1436)
+                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:2001, _col1, _col2, _col3, _col4
+                                Select Operator [SEL_142] (rows=922383 width=1436)
                                   Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
-                                  Merge Join Operator [MERGEJOIN_252] (rows=307461 width=1436)
-                                    Conds:RS_92._col2, _col1=RS_93._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
-                                  <-Map 34 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_93]
-                                      PartitionCols:_col1, _col0
-                                      Select Operator [SEL_85] (rows=1 width=0)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_232] (rows=1 width=0)
-                                          predicate:cr_item_sk is not null
-                                          TableScan [TS_83] (rows=1 width=0)
-                                            default@catalog_returns,catalog_returns,Tbl:PARTIAL,Col:NONE,Output:["cr_item_sk","cr_order_number","cr_return_quantity","cr_return_amount"]
-                                  <-Reducer 28 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_92]
-                                      PartitionCols:_col2, _col1
-                                      Merge Join Operator [MERGEJOIN_251] (rows=279510 width=1436)
-                                        Conds:RS_89._col0=RS_90._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                      <-Map 33 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_90]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_82] (rows=36524 width=1119)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_231] (rows=36524 width=1119)
-                                              predicate:((d_year = 2001) and d_date_sk is not null)
-                                              TableScan [TS_80] (rows=73049 width=1119)
-                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
-                                      <-Reducer 27 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_89]
-                                          PartitionCols:_col0
-                                          Merge Join Operator [MERGEJOIN_250] (rows=254100 width=1436)
-                                            Conds:RS_86._col1=RS_87._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                          <-Map 26 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_86]
-                                              PartitionCols:_col1
-                                              Select Operator [SEL_76] (rows=1 width=0)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_229] (rows=1 width=0)
-                                                  predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
-                                                  TableScan [TS_74] (rows=1 width=0)
-                                                    default@catalog_sales,catalog_sales,Tbl:PARTIAL,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_order_number","cs_quantity","cs_ext_sales_price"]
-                                          <-Map 32 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_87]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_79] (rows=231000 width=1436)
-                                                Output:["_col0","_col1","_col2","_col3","_col5"]
-                                                Filter Operator [FIL_230] (rows=231000 width=1436)
-                                                  predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                                  TableScan [TS_77] (rows=462000 width=1436)
-                                                    default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
-                        <-Reducer 38 [CONTAINS]
-                          Reduce Output Operator [RS_145]
-                            PartitionCols:_col0, _col1, _col2, _col3, _col4
-                            Group By Operator [GBY_144] (rows=922383 width=1436)
-                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4
-                              Select Operator [SEL_142] (rows=922383 width=1436)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                Select Operator [SEL_117] (rows=307461 width=1436)
+                                  Select Operator [SEL_95] (rows=307461 width=1436)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                    Merge Join Operator [MERGEJOIN_252] (rows=307461 width=1436)
+                                      Conds:RS_92._col2, _col1=RS_93._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
+                                    <-Map 34 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_93]
+                                        PartitionCols:_col1, _col0
+                                        Select Operator [SEL_85] (rows=1 width=0)
+                                          Output:["_col0","_col1","_col2","_col3"]
+                                          Filter Operator [FIL_232] (rows=1 width=0)
+                                            predicate:cr_item_sk is not null
+                                            TableScan [TS_83] (rows=1 width=0)
+                                              default@catalog_returns,catalog_returns,Tbl:PARTIAL,Col:NONE,Output:["cr_item_sk","cr_order_number","cr_return_quantity","cr_return_amount"]
+                                    <-Reducer 28 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_92]
+                                        PartitionCols:_col2, _col1
+                                        Merge Join Operator [MERGEJOIN_251] (rows=279510 width=1436)
+                                          Conds:RS_89._col0=RS_90._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                        <-Map 33 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_90]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_82] (rows=36524 width=1119)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_231] (rows=36524 width=1119)
+                                                predicate:((d_year = 2001) and d_date_sk is not null)
+                                                TableScan [TS_80] (rows=73049 width=1119)
+                                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
+                                        <-Reducer 27 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_89]
+                                            PartitionCols:_col0
+                                            Merge Join Operator [MERGEJOIN_250] (rows=254100 width=1436)
+                                              Conds:RS_86._col1=RS_87._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                            <-Map 26 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_86]
+                                                PartitionCols:_col1
+                                                Select Operator [SEL_76] (rows=1 width=0)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                  Filter Operator [FIL_229] (rows=1 width=0)
+                                                    predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
+                                                    TableScan [TS_74] (rows=1 width=0)
+                                                      default@catalog_sales,catalog_sales,Tbl:PARTIAL,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_order_number","cs_quantity","cs_ext_sales_price"]
+                                            <-Map 32 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_87]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_79] (rows=231000 width=1436)
+                                                  Output:["_col0","_col1","_col2","_col3","_col5"]
+                                                  Filter Operator [FIL_230] (rows=231000 width=1436)
+                                                    predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                    TableScan [TS_77] (rows=462000 width=1436)
+                                                      default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
+                          <-Reducer 38 [CONTAINS]
+                            Reduce Output Operator [RS_145]
+                              PartitionCols:2001, _col1, _col2, _col3, _col4
+                              Group By Operator [GBY_144] (rows=922383 width=1436)
+                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:2001, _col1, _col2, _col3, _col4
+                                Select Operator [SEL_142] (rows=922383 width=1436)
                                   Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
-                                  Merge Join Operator [MERGEJOIN_255] (rows=307461 width=1436)
-                                    Conds:RS_114._col2, _col1=RS_115._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
-                                  <-Map 41 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_115]
-                                      PartitionCols:_col1, _col0
-                                      Select Operator [SEL_107] (rows=1 width=0)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_236] (rows=1 width=0)
-                                          predicate:sr_item_sk is not null
-                                          TableScan [TS_105] (rows=1 width=0)
-                                            default@store_returns,store_returns,Tbl:PARTIAL,Col:NONE,Output:["sr_item_sk","sr_ticket_number","sr_return_quantity","sr_return_amt"]
-                                  <-Reducer 37 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_114]
-                                      PartitionCols:_col2, _col1
-                                      Merge Join Operator [MERGEJOIN_254] (rows=279510 width=1436)
-                                        Conds:RS_111._col0=RS_112._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                      <-Map 40 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_112]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_104] (rows=36524 width=1119)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_235] (rows=36524 width=1119)
-                                              predicate:((d_year = 2001) and d_date_sk is not null)
-                                              TableScan [TS_102] (rows=73049 width=1119)
-                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
-                                      <-Reducer 36 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_111]
-                                          PartitionCols:_col0
-                                          Merge Join Operator [MERGEJOIN_253] (rows=254100 width=1436)
-                                            Conds:RS_108._col1=RS_109._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                          <-Map 35 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_108]
-                                              PartitionCols:_col1
-                                              Select Operator [SEL_98] (rows=1 width=0)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_233] (rows=1 width=0)
-                                                  predicate:(ss_item_sk is not null and ss_sold_date_sk is not null)
-                                                  TableScan [TS_96] (rows=1 width=0)
-                                                    default@store_sales,store_sales,Tbl:PARTIAL,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ticket_number","ss_quantity","ss_ext_sales_price"]
-                                          <-Map 39 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_109]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_101] (rows=231000 width=1436)
-                                                Output:["_col0","_col1","_col2","_col3","_col5"]
-                                                Filter Operator [FIL_234] (rows=231000 width=1436)
-                                                  predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                                  TableScan [TS_99] (rows=462000 width=1436)
-                                                    default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
-                        <-Reducer 45 [CONTAINS]
-                          Reduce Output Operator [RS_145]
-                            PartitionCols:_col0, _col1, _col2, _col3, _col4
-                            Group By Operator [GBY_144] (rows=922383 width=1436)
-                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4
-                              Select Operator [SEL_142] (rows=922383 width=1436)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                Select Operator [SEL_141] (rows=307461 width=1436)
+                                  Select Operator [SEL_117] (rows=307461 width=1436)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                    Merge Join Operator [MERGEJOIN_255] (rows=307461 width=1436)
+                                      Conds:RS_114._col2, _col1=RS_115._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
+                                    <-Map 41 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_115]
+                                        PartitionCols:_col1, _col0
+                                        Select Operator [SEL_107] (rows=1 width=0)
+                                          Output:["_col0","_col1","_col2","_col3"]
+                                          Filter Operator [FIL_236] (rows=1 width=0)
+                                            predicate:sr_item_sk is not null
+                                            TableScan [TS_105] (rows=1 width=0)
+                                              default@store_returns,store_returns,Tbl:PARTIAL,Col:NONE,Output:["sr_item_sk","sr_ticket_number","sr_return_quantity","sr_return_amt"]
+                                    <-Reducer 37 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_114]
+                                        PartitionCols:_col2, _col1
+                                        Merge Join Operator [MERGEJOIN_254] (rows=279510 width=1436)
+                                          Conds:RS_111._col0=RS_112._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                        <-Map 40 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_112]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_104] (rows=36524 width=1119)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_235] (rows=36524 width=1119)
+                                                predicate:((d_year = 2001) and d_date_sk is not null)
+                                                TableScan [TS_102] (rows=73049 width=1119)
+                                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
+                                        <-Reducer 36 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_111]
+                                            PartitionCols:_col0
+                                            Merge Join Operator [MERGEJOIN_253] (rows=254100 width=1436)
+                                              Conds:RS_108._col1=RS_109._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                            <-Map 35 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_108]
+                                                PartitionCols:_col1
+                                                Select Operator [SEL_98] (rows=1 width=0)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                  Filter Operator [FIL_233] (rows=1 width=0)
+                                                    predicate:(ss_item_sk is not null and ss_sold_date_sk is not null)
+                                                    TableScan [TS_96] (rows=1 width=0)
+                                                      default@store_sales,store_sales,Tbl:PARTIAL,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ticket_number","ss_quantity","ss_ext_sales_price"]
+                                            <-Map 39 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_109]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_101] (rows=231000 width=1436)
+                                                  Output:["_col0","_col1","_col2","_col3","_col5"]
+                                                  Filter Operator [FIL_234] (rows=231000 width=1436)
+                                                    predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                    TableScan [TS_99] (rows=462000 width=1436)
+                                                      default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
+                          <-Reducer 45 [CONTAINS]
+                            Reduce Output Operator [RS_145]
+                              PartitionCols:2001, _col1, _col2, _col3, _col4
+                              Group By Operator [GBY_144] (rows=922383 width=1436)
+                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:2001, _col1, _col2, _col3, _col4
+                                Select Operator [SEL_142] (rows=922383 width=1436)
                                   Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
-                                  Merge Join Operator [MERGEJOIN_258] (rows=307461 width=1436)
-                                    Conds:RS_138._col2, _col1=RS_139._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
-                                  <-Map 48 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_139]
-                                      PartitionCols:_col1, _col0
-                                      Select Operator [SEL_131] (rows=1 width=0)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_240] (rows=1 width=0)
-                                          predicate:wr_item_sk is not null
-                                          TableScan [TS_129] (rows=1 width=0)
-                                            default@web_returns,web_returns,Tbl:PARTIAL,Col:NONE,Output:["wr_item_sk","wr_order_number","wr_return_quantity","wr_return_amt"]
-                                  <-Reducer 44 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_138]
-                                      PartitionCols:_col2, _col1
-                                      Merge Join Operator [MERGEJOIN_257] (rows=279510 width=1436)
-                                        Conds:RS_135._col0=RS_136._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                      <-Map 47 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_136]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_128] (rows=36524 width=1119)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_239] (rows=36524 width=1119)
-                                              predicate:((d_year = 2001) and d_date_sk is not null)
-                                              TableScan [TS_126] (rows=73049 width=1119)
-                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
-                                      <-Reducer 43 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_135]
-                                          PartitionCols:_col0
-                                          Merge Join Operator [MERGEJOIN_256] (rows=254100 width=1436)
-                                            Conds:RS_132._col1=RS_133._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                          <-Map 42 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_132]
-                                              PartitionCols:_col1
-                                              Select Operator [SEL_122] (rows=1 width=0)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_237] (rows=1 width=0)
-                                                  predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
-                                                  TableScan [TS_120] (rows=1 width=0)
-                                                    default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_order_number","ws_quantity","ws_ext_sales_price"]
-                                          <-Map 46 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_133]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_125] (rows=231000 width=1436)
-                                                Output:["_col0","_col1","_col2","_col3","_col5"]
-                                                Filter Operator [FIL_238] (rows=231000 width=1436)
-                                                  predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                                  TableScan [TS_123] (rows=462000 width=1436)
-                                                    default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
+                                  Select Operator [SEL_141] (rows=307461 width=1436)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                    Merge Join Operator [MERGEJOIN_258] (rows=307461 width=1436)
+                                      Conds:RS_138._col2, _col1=RS_139._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
+                                    <-Map 48 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_139]
+                                        PartitionCols:_col1, _col0
+                                        Select Operator [SEL_131] (rows=1 width=0)
+                                          Output:["_col0","_col1","_col2","_col3"]
+                                          Filter Operator [FIL_240] (rows=1 width=0)
+                                            predicate:wr_item_sk is not null
+                                            TableScan [TS_129] (rows=1 width=0)
+                                              default@web_returns,web_returns,Tbl:PARTIAL,Col:NONE,Output:["wr_item_sk","wr_order_number","wr_return_quantity","wr_return_amt"]
+                                    <-Reducer 44 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_138]
+                                        PartitionCols:_col2, _col1
+                                        Merge Join Operator [MERGEJOIN_257] (rows=279510 width=1436)
+                                          Conds:RS_135._col0=RS_136._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                        <-Map 47 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_136]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_128] (rows=36524 width=1119)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_239] (rows=36524 width=1119)
+                                                predicate:((d_year = 2001) and d_date_sk is not null)
+                                                TableScan [TS_126] (rows=73049 width=1119)
+                                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
+                                        <-Reducer 43 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_135]
+                                            PartitionCols:_col0
+                                            Merge Join Operator [MERGEJOIN_256] (rows=254100 width=1436)
+                                              Conds:RS_132._col1=RS_133._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                            <-Map 42 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_132]
+                                                PartitionCols:_col1
+                                                Select Operator [SEL_122] (rows=1 width=0)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                  Filter Operator [FIL_237] (rows=1 width=0)
+                                                    predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
+                                                    TableScan [TS_120] (rows=1 width=0)
+                                                      default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_order_number","ws_quantity","ws_ext_sales_price"]
+                                            <-Map 46 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_133]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_125] (rows=231000 width=1436)
+                                                  Output:["_col0","_col1","_col2","_col3","_col5"]
+                                                  Filter Operator [FIL_238] (rows=231000 width=1436)
+                                                    predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                    TableScan [TS_123] (rows=462000 width=1436)
+                                                      default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
                   <-Reducer 6 [SIMPLE_EDGE]
                     SHUFFLE [RS_148]
                       PartitionCols:_col1, _col2, _col3, _col4
-                      Group By Operator [GBY_72] (rows=461191 width=1436)
-                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
-                      <-Union 5 [SIMPLE_EDGE]
-                        <-Reducer 15 [CONTAINS]
-                          Reduce Output Operator [RS_71]
-                            PartitionCols:_col0, _col1, _col2, _col3, _col4
-                            Group By Operator [GBY_70] (rows=922383 width=1436)
-                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4
-                              Select Operator [SEL_68] (rows=922383 width=1436)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                Select Operator [SEL_43] (rows=307461 width=1436)
+                      Select Operator [SEL_73] (rows=461191 width=1436)
+                        Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
+                        Group By Operator [GBY_72] (rows=461191 width=1436)
+                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:2002, KEY._col1, KEY._col2, KEY._col3, KEY._col4
+                        <-Union 5 [SIMPLE_EDGE]
+                          <-Reducer 15 [CONTAINS]
+                            Reduce Output Operator [RS_71]
+                              PartitionCols:2002, _col1, _col2, _col3, _col4
+                              Group By Operator [GBY_70] (rows=922383 width=1436)
+                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:2002, _col1, _col2, _col3, _col4
+                                Select Operator [SEL_68] (rows=922383 width=1436)
                                   Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
-                                  Merge Join Operator [MERGEJOIN_246] (rows=307461 width=1436)
-                                    Conds:RS_40._col2, _col1=RS_41._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
-                                  <-Map 18 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_41]
-                                      PartitionCols:_col1, _col0
-                                      Select Operator [SEL_33] (rows=1 width=0)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_224] (rows=1 width=0)
-                                          predicate:sr_item_sk is not null
-                                          TableScan [TS_31] (rows=1 width=0)
-                                            default@store_returns,store_returns,Tbl:PARTIAL,Col:NONE,Output:["sr_item_sk","sr_ticket_number","sr_return_quantity","sr_return_amt"]
-                                  <-Reducer 14 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_40]
-                                      PartitionCols:_col2, _col1
-                                      Merge Join Operator [MERGEJOIN_245] (rows=279510 width=1436)
-                                        Conds:RS_37._col0=RS_38._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                      <-Map 17 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_38]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_30] (rows=36524 width=1119)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_223] (rows=36524 width=1119)
-                                              predicate:((d_year = 2002) and d_date_sk is not null)
-                                              TableScan [TS_28] (rows=73049 width=1119)
-                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
-                                      <-Reducer 13 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_37]
-                                          PartitionCols:_col0
-                                          Merge Join Operator [MERGEJOIN_244] (rows=254100 width=1436)
-                                            Conds:RS_34._col1=RS_35._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                          <-Map 12 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_34]
-                                              PartitionCols:_col1
-                                              Select Operator [SEL_24] (rows=1 width=0)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_221] (rows=1 width=0)
-                                                  predicate:(ss_item_sk is not null and ss_sold_date_sk is not null)
-                                                  TableScan [TS_22] (rows=1 width=0)
-                                                    default@store_sales,store_sales,Tbl:PARTIAL,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ticket_number","ss_quantity","ss_ext_sales_price"]
-                                          <-Map 16 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_35]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_27] (rows=231000 width=1436)
-                                                Output:["_col0","_col1","_col2","_col3","_col5"]
-                                                Filter Operator [FIL_222] (rows=231000 width=1436)
-                                                  predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                                  TableScan [TS_25] (rows=462000 width=1436)
-                                                    default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
-                        <-Reducer 22 [CONTAINS]
-                          Reduce Output Operator [RS_71]
-                            PartitionCols:_col0, _col1, _col2, _col3, _col4
-                            Group By Operator [GBY_70] (rows=922383 width=1436)
-                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4
-                              Select Operator [SEL_68] (rows=922383 width=1436)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                Select Operator [SEL_67] (rows=307461 width=1436)
+                                  Select Operator [SEL_43] (rows=307461 width=1436)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                    Merge Join Operator [MERGEJOIN_246] (rows=307461 width=1436)
+                                      Conds:RS_40._col2, _col1=RS_41._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
+                                    <-Map 18 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_41]
+                                        PartitionCols:_col1, _col0
+                                        Select Operator [SEL_33] (rows=1 width=0)
+                                          Output:["_col0","_col1","_col2","_col3"]
+                                          Filter Operator [FIL_224] (rows=1 width=0)
+                                            predicate:sr_item_sk is not null
+                                            TableScan [TS_31] (rows=1 width=0)
+                                              default@store_returns,store_returns,Tbl:PARTIAL,Col:NONE,Output:["sr_item_sk","sr_ticket_number","sr_return_quantity","sr_return_amt"]
+                                    <-Reducer 14 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_40]
+                                        PartitionCols:_col2, _col1
+                                        Merge Join Operator [MERGEJOIN_245] (rows=279510 width=1436)
+                                          Conds:RS_37._col0=RS_38._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                        <-Map 17 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_38]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_30] (rows=36524 width=1119)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_223] (rows=36524 width=1119)
+                                                predicate:((d_year = 2002) and d_date_sk is not null)
+                                                TableScan [TS_28] (rows=73049 width=1119)
+                                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
+                                        <-Reducer 13 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_37]
+                                            PartitionCols:_col0
+                                            Merge Join Operator [MERGEJOIN_244] (rows=254100 width=1436)
+                                              Conds:RS_34._col1=RS_35._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                            <-Map 12 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_34]
+                                                PartitionCols:_col1
+                                                Select Operator [SEL_24] (rows=1 width=0)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                  Filter Operator [FIL_221] (rows=1 width=0)
+                                                    predicate:(ss_item_sk is not null and ss_sold_date_sk is not null)
+                                                    TableScan [TS_22] (rows=1 width=0)
+                                                      default@store_sales,store_sales,Tbl:PARTIAL,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ticket_number","ss_quantity","ss_ext_sales_price"]
+                                            <-Map 16 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_35]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_27] (rows=231000 width=1436)
+                                                  Output:["_col0","_col1","_col2","_col3","_col5"]
+                                                  Filter Operator [FIL_222] (rows=231000 width=1436)
+                                                    predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                    TableScan [TS_25] (rows=462000 width=1436)
+                                                      default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
+                          <-Reducer 22 [CONTAINS]
+                            Reduce Output Operator [RS_71]
+                              PartitionCols:2002, _col1, _col2, _col3, _col4
+                              Group By Operator [GBY_70] (rows=922383 width=1436)
+                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:2002, _col1, _col2, _col3, _col4
+                                Select Operator [SEL_68] (rows=922383 width=1436)
                                   Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
-                                  Merge Join Operator [MERGEJOIN_249] (rows=307461 width=1436)
-                                    Conds:RS_64._col2, _col1=RS_65._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
-                                  <-Map 25 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_65]
-                                      PartitionCols:_col1, _col0
-                                      Select Operator [SEL_57] (rows=1 width=0)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_228] (rows=1 width=0)
-                                          predicate:wr_item_sk is not null
-                                          TableScan [TS_55] (rows=1 width=0)
-                                            default@web_returns,web_returns,Tbl:PARTIAL,Col:NONE,Output:["wr_item_sk","wr_order_number","wr_return_quantity","wr_return_amt"]
-                                  <-Reducer 21 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_64]
-                                      PartitionCols:_col2, _col1
-                                      Merge Join Operator [MERGEJOIN_248] (rows=279510 width=1436)
-                                        Conds:RS_61._col0=RS_62._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                      <-Map 24 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_62]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_54] (rows=36524 width=1119)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_227] (rows=36524 width=1119)
-                                              predicate:((d_year = 2002) and d_date_sk is not null)
-                                              TableScan [TS_52] (rows=73049 width=1119)
-                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
-                                      <-Reducer 20 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_61]
-                                          PartitionCols:_col0
-                                          Merge Join Operator [MERGEJOIN_247] (rows=254100 width=1436)
-                                            Conds:RS_58._col1=RS_59._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                          <-Map 19 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_58]
-                                              PartitionCols:_col1
-                                              Select Operator [SEL_48] (rows=1 width=0)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_225] (rows=1 width=0)
-                                                  predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
-                                                  TableScan [TS_46] (rows=1 width=0)
-                                                    default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_order_number","ws_quantity","ws_ext_sales_price"]
-                                          <-Map 23 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_59]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_51] (rows=231000 width=1436)
-                                                Output:["_col0","_col1","_col2","_col3","_col5"]
-                                                Filter Operator [FIL_226] (rows=231000 width=1436)
-                                                  predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                                  TableScan [TS_49] (rows=462000 width=1436)
-                                                    default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
-                        <-Reducer 4 [CONTAINS]
-                          Reduce Output Operator [RS_71]
-                            PartitionCols:_col0, _col1, _col2, _col3, _col4
-                            Group By Operator [GBY_70] (rows=922383 width=1436)
-                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4
-                              Select Operator [SEL_68] (rows=922383 width=1436)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                Select Operator [SEL_21] (rows=307461 width=1436)
+                                  Select Operator [SEL_67] (rows=307461 width=1436)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                    Merge Join Operator [MERGEJOIN_249] (rows=307461 width=1436)
+                                      Conds:RS_64._col2, _col1=RS_65._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
+                                    <-Map 25 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_65]
+                                        PartitionCols:_col1, _col0
+                                        Select Operator [SEL_57] (rows=1 width=0)
+                                          Output:["_col0","_col1","_col2","_col3"]
+                                          Filter Operator [FIL_228] (rows=1 width=0)
+                                            predicate:wr_item_sk is not null
+                                            TableScan [TS_55] (rows=1 width=0)
+                                              default@web_returns,web_returns,Tbl:PARTIAL,Col:NONE,Output:["wr_item_sk","wr_order_number","wr_return_quantity","wr_return_amt"]
+                                    <-Reducer 21 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_64]
+                                        PartitionCols:_col2, _col1
+                                        Merge Join Operator [MERGEJOIN_248] (rows=279510 width=1436)
+                                          Conds:RS_61._col0=RS_62._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                        <-Map 24 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_62]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_54] (rows=36524 width=1119)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_227] (rows=36524 width=1119)
+                                                predicate:((d_year = 2002) and d_date_sk is not null)
+                                                TableScan [TS_52] (rows=73049 width=1119)
+                                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
+                                        <-Reducer 20 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_61]
+                                            PartitionCols:_col0
+                                            Merge Join Operator [MERGEJOIN_247] (rows=254100 width=1436)
+                                              Conds:RS_58._col1=RS_59._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                            <-Map 19 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_58]
+                                                PartitionCols:_col1
+                                                Select Operator [SEL_48] (rows=1 width=0)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                  Filter Operator [FIL_225] (rows=1 width=0)
+                                                    predicate:(ws_item_sk is not null and ws_sold_date_sk is not null)
+                                                    TableScan [TS_46] (rows=1 width=0)
+                                                      default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_item_sk","ws_order_number","ws_quantity","ws_ext_sales_price"]
+                                            <-Map 23 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_59]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_51] (rows=231000 width=1436)
+                                                  Output:["_col0","_col1","_col2","_col3","_col5"]
+                                                  Filter Operator [FIL_226] (rows=231000 width=1436)
+                                                    predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                    TableScan [TS_49] (rows=462000 width=1436)
+                                                      default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
+                          <-Reducer 4 [CONTAINS]
+                            Reduce Output Operator [RS_71]
+                              PartitionCols:2002, _col1, _col2, _col3, _col4
+                              Group By Operator [GBY_70] (rows=922383 width=1436)
+                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:2002, _col1, _col2, _col3, _col4
+                                Select Operator [SEL_68] (rows=922383 width=1436)
                                   Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
-                                  Merge Join Operator [MERGEJOIN_243] (rows=307461 width=1436)
-                                    Conds:RS_18._col2, _col1=RS_19._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
-                                  <-Map 11 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_19]
-                                      PartitionCols:_col1, _col0
-                                      Select Operator [SEL_11] (rows=1 width=0)
-                                        Output:["_col0","_col1","_col2","_col3"]
-                                        Filter Operator [FIL_220] (rows=1 width=0)
-                                          predicate:cr_item_sk is not null
-                                          TableScan [TS_9] (rows=1 width=0)
-                                            default@catalog_returns,catalog_returns,Tbl:PARTIAL,Col:NONE,Output:["cr_item_sk","cr_order_number","cr_return_quantity","cr_return_amount"]
-                                  <-Reducer 3 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_18]
-                                      PartitionCols:_col2, _col1
-                                      Merge Join Operator [MERGEJOIN_242] (rows=279510 width=1436)
-                                        Conds:RS_15._col0=RS_16._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                      <-Map 10 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_16]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_8] (rows=36524 width=1119)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_219] (rows=36524 width=1119)
-                                              predicate:((d_year = 2002) and d_date_sk is not null)
-                                              TableScan [TS_6] (rows=73049 width=1119)
-                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
-                                      <-Reducer 2 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_15]
-                                          PartitionCols:_col0
-                                          Merge Join Operator [MERGEJOIN_241] (rows=254100 width=1436)
-                                            Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
-                                          <-Map 1 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_12]
-                                              PartitionCols:_col1
-                                              Select Operator [SEL_2] (rows=1 width=0)
-                                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                Filter Operator [FIL_217] (rows=1 width=0)
-                                                  predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
-                                                  TableScan [TS_0] (rows=1 width=0)
-                                                    default@catalog_sales,catalog_sales,Tbl:PARTIAL,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_order_number","cs_quantity","cs_ext_sales_price"]
-                                          <-Map 9 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_13]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_5] (rows=231000 width=1436)
-                                                Output:["_col0","_col1","_col2","_col3","_col5"]
-                                                Filter Operator [FIL_218] (rows=231000 width=1436)
-                                                  predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
-                                                  TableScan [TS_3] (rows=462000 width=1436)
-                                                    default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
+                                  Select Operator [SEL_21] (rows=307461 width=1436)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                    Merge Join Operator [MERGEJOIN_243] (rows=307461 width=1436)
+                                      Conds:RS_18._col2, _col1=RS_19._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"]
+                                    <-Map 11 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_19]
+                                        PartitionCols:_col1, _col0
+                                        Select Operator [SEL_11] (rows=1 width=0)
+                                          Output:["_col0","_col1","_col2","_col3"]
+                                          Filter Operator [FIL_220] (rows=1 width=0)
+                                            predicate:cr_item_sk is not null
+                                            TableScan [TS_9] (rows=1 width=0)
+                                              default@catalog_returns,catalog_returns,Tbl:PARTIAL,Col:NONE,Output:["cr_item_sk","cr_order_number","cr_return_quantity","cr_return_amount"]
+                                    <-Reducer 3 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_18]
+                                        PartitionCols:_col2, _col1
+                                        Merge Join Operator [MERGEJOIN_242] (rows=279510 width=1436)
+                                          Conds:RS_15._col0=RS_16._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                        <-Map 10 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_16]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_8] (rows=36524 width=1119)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_219] (rows=36524 width=1119)
+                                                predicate:((d_year = 2002) and d_date_sk is not null)
+                                                TableScan [TS_6] (rows=73049 width=1119)
+                                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year"]
+                                        <-Reducer 2 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_15]
+                                            PartitionCols:_col0
+                                            Merge Join Operator [MERGEJOIN_241] (rows=254100 width=1436)
+                                              Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col10"]
+                                            <-Map 1 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_12]
+                                                PartitionCols:_col1
+                                                Select Operator [SEL_2] (rows=1 width=0)
+                                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                  Filter Operator [FIL_217] (rows=1 width=0)
+                                                    predicate:(cs_item_sk is not null and cs_sold_date_sk is not null)
+                                                    TableScan [TS_0] (rows=1 width=0)
+                                                      default@catalog_sales,catalog_sales,Tbl:PARTIAL,Col:NONE,Output:["cs_sold_date_sk","cs_item_sk","cs_order_number","cs_quantity","cs_ext_sales_price"]
+                                            <-Map 9 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_13]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_5] (rows=231000 width=1436)
+                                                  Output:["_col0","_col1","_col2","_col3","_col5"]
+                                                  Filter Operator [FIL_218] (rows=231000 width=1436)
+                                                    predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
+                                                    TableScan [TS_3] (rows=462000 width=1436)
+                                                      default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_25.q.out b/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
index 190bea5..eb95cad 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
@@ -458,21 +458,17 @@ STAGE PLANS:
                   Number of rows: 1000
                   Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
-                    outputColumnNames: _col0, _col1, _col3
-                    Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col3 (type: string)
-                      outputColumnNames: _col0, _col1, _col2, _col3
+                    expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col2 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
                       Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
-                      File Output Operator
-                        compressed: false
-                        Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
-                        table:
-                            input format: org.apache.hadoop.mapred.TextInputFormat
-                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                            name: default.outputtbl3
+                      table:
+                          input format: org.apache.hadoop.mapred.TextInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                          name: default.outputtbl3
         Reducer 4 
             Reduce Operator Tree:
               Select Operator
@@ -483,21 +479,17 @@ STAGE PLANS:
                   Number of rows: 1000
                   Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
-                    outputColumnNames: _col0, _col1, _col3
-                    Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col3 (type: string)
-                      outputColumnNames: _col0, _col1, _col2, _col3
+                    expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col2 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
                       Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
-                      File Output Operator
-                        compressed: false
-                        Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE
-                        table:
-                            input format: org.apache.hadoop.mapred.TextInputFormat
-                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                            name: default.outputtbl3
+                      table:
+                          input format: org.apache.hadoop.mapred.TextInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                          name: default.outputtbl3
 
   Stage: Stage-0
     Move Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/spark/union_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_view.q.out b/ql/src/test/results/clientpositive/spark/union_view.q.out
index 402d9fd..3372afb 100644
--- a/ql/src/test/results/clientpositive/spark/union_view.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_view.q.out
@@ -272,10 +272,10 @@ STAGE PLANS:
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '1' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -296,10 +296,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '1' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -320,10 +320,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '1' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -360,10 +360,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '2' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '2' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -384,10 +384,10 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '2' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '2' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -408,10 +408,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '2' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '2' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -448,10 +448,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '3' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '3' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -472,10 +472,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '3' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '3' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -496,10 +496,10 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '3' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '3' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -538,10 +538,10 @@ STAGE PLANS:
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: value (type: string), ds (type: string)
-                      outputColumnNames: _col1, _col2
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                       Select Operator
-                        expressions: _col1 (type: string), _col2 (type: string)
+                        expressions: _col0 (type: string), _col1 (type: string)
                         outputColumnNames: _col1, _col2
                         Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
@@ -560,10 +560,10 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: value (type: string), ds (type: string)
-                      outputColumnNames: _col1, _col2
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Select Operator
-                        expressions: _col1 (type: string), _col2 (type: string)
+                        expressions: _col0 (type: string), _col1 (type: string)
                         outputColumnNames: _col1, _col2
                         Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
@@ -582,10 +582,10 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: value (type: string), ds (type: string)
-                      outputColumnNames: _col1, _col2
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Select Operator
-                        expressions: _col1 (type: string), _col2 (type: string)
+                        expressions: _col0 (type: string), _col1 (type: string)
                         outputColumnNames: _col1, _col2
                         Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
@@ -931,10 +931,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '4' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '4' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -955,10 +955,10 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '4' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '4' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
@@ -979,10 +979,10 @@ STAGE PLANS:
                     Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: value (type: string)
-                      outputColumnNames: _col1
+                      outputColumnNames: _col0
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                       Select Operator
-                        expressions: 86 (type: int), _col1 (type: string), '4' (type: string)
+                        expressions: 86 (type: int), _col0 (type: string), '4' (type: string)
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator


[44/50] [abbrv] hive git commit: HIVE-13027: Configuration changes to improve logging performance (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Posted by sp...@apache.org.
HIVE-13027: Configuration changes to improve logging performance (Prasanth Jayachandran reviewed by Sergey Shelukhin)


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

Branch: refs/heads/java8
Commit: b870d526edbac1831d66f2529cf1a854b57bddb2
Parents: 0cc4045
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Fri May 6 03:08:28 2016 -0500
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Fri May 6 03:09:40 2016 -0500

----------------------------------------------------------------------
 cli/pom.xml                                     |  6 +++
 common/pom.xml                                  |  6 +++
 .../org/apache/hadoop/hive/common/LogUtils.java | 35 ++++++++++++--
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  6 +++
 .../src/main/resources/hive-log4j2.properties   |  2 +-
 .../hadoop/hive/conf/TestHiveAsyncLogging.java  | 49 ++++++++++++++++++++
 data/conf/hive-log4j2.properties                |  2 +-
 hcatalog/core/pom.xml                           |  6 +++
 llap-server/bin/llapDaemon.sh                   |  2 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       | 10 ++--
 .../main/resources/llap-cli-log4j2.properties   |  2 +-
 .../resources/llap-daemon-log4j2.properties     |  4 +-
 .../resources/llap-daemon-log4j2.properties     |  4 +-
 metastore/pom.xml                               |  6 +++
 pom.xml                                         |  2 +
 ql/pom.xml                                      |  6 +++
 .../main/resources/hive-exec-log4j2.properties  |  2 +-
 .../resources/tez-container-log4j2.properties   |  2 +-
 18 files changed, 136 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/cli/pom.xml
----------------------------------------------------------------------
diff --git a/cli/pom.xml b/cli/pom.xml
index 76f6d11..6f2e664 100644
--- a/cli/pom.xml
+++ b/cli/pom.xml
@@ -82,6 +82,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>jline</groupId>
       <artifactId>jline</artifactId>
       <version>${jline.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 67aab7c..9933072 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -164,6 +164,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.json</groupId>
       <artifactId>json</artifactId>
       <version>${json.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/LogUtils.java b/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
index adcf805..599e798 100644
--- a/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
@@ -21,12 +21,18 @@ package org.apache.hadoop.hive.common;
 import java.io.File;
 import java.net.URL;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.LoggerContext;
 import org.apache.logging.log4j.core.config.Configurator;
+import org.apache.logging.log4j.core.impl.Log4jContextFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Utilities common to logging operations.
  */
@@ -66,8 +72,14 @@ public class LogUtils {
   }
 
   private static String initHiveLog4jCommon(ConfVars confVarName)
-    throws LogInitializationException {
+      throws LogInitializationException {
     HiveConf conf = new HiveConf();
+    return initHiveLog4jCommon(conf, confVarName);
+  }
+
+  @VisibleForTesting
+  public static String initHiveLog4jCommon(HiveConf conf, ConfVars confVarName)
+    throws LogInitializationException {
     if (HiveConf.getVar(conf, confVarName).equals("")) {
       // if log4j configuration file not set, or could not found, use default setting
       return initHiveLog4jDefault(conf, "", confVarName);
@@ -91,13 +103,28 @@ public class LogUtils {
           }
           System.setProperty(HiveConf.ConfVars.HIVEQUERYID.toString(), queryId);
         }
+        final boolean async = checkAndSetAsyncLogging(conf);
         Configurator.initialize(null, log4jFileName);
         logConfigLocation(conf);
-        return ("Logging initialized using configuration in " + log4jConfigFile);
+        return "Logging initialized using configuration in " + log4jConfigFile + " Async: " + async;
       }
     }
   }
 
+  public static boolean checkAndSetAsyncLogging(final Configuration conf) {
+    final boolean asyncLogging = HiveConf.getBoolVar(conf, ConfVars.HIVE_ASYNC_LOG_ENABLED);
+    if (asyncLogging) {
+      System.setProperty("Log4jContextSelector",
+          "org.apache.logging.log4j.core.async.AsyncLoggerContextSelector");
+      // default is ClassLoaderContextSelector which is created during automatic logging
+      // initialization in a static initialization block.
+      // Changing ContextSelector at runtime requires creating new context factory which will
+      // internally create new context selector based on system property.
+      LogManager.setFactory(new Log4jContextFactory());
+    }
+    return asyncLogging;
+  }
+
   private static String initHiveLog4jDefault(
     HiveConf conf, String logMessage, ConfVars confVarName)
     throws LogInitializationException {
@@ -118,9 +145,11 @@ public class LogUtils {
         break;
     }
     if (hive_l4j != null) {
+      final boolean async = checkAndSetAsyncLogging(conf);
       Configurator.initialize(null, hive_l4j.toString());
       logConfigLocation(conf);
-      return (logMessage + "\n" + "Logging initialized using configuration in " + hive_l4j);
+      return (logMessage + "\n" + "Logging initialized using configuration in " + hive_l4j +
+          " Async: " + async);
     } else {
       throw new LogInitializationException(
         logMessage + "Unable to initialize logging using "

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index bb74d99..07dff08 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1955,6 +1955,12 @@ public class HiveConf extends Configuration {
         "If the property is not set, then logging will be initialized using hive-exec-log4j2.properties found on the classpath.\n" +
         "If the property is set, the value must be a valid URI (java.net.URI, e.g. \"file:///tmp/my-logging.xml\"), \n" +
         "which you can then extract a URL from and pass to PropertyConfigurator.configure(URL)."),
+    HIVE_ASYNC_LOG_ENABLED("hive.async.log.enabled", true,
+        "Whether to enable Log4j2's asynchronous logging. Asynchronous logging can give\n" +
+        " significant performance improvement as logging will be handled in separate thread\n" +
+        " that uses LMAX disruptor queue for buffering log messages.\n" +
+        " Refer https://logging.apache.org/log4j/2.x/manual/async.html for benefits and\n" +
+        " drawbacks."),
 
     HIVE_LOG_EXPLAIN_OUTPUT("hive.log.explain.output", false,
         "Whether to log explain output for every query.\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/common/src/main/resources/hive-log4j2.properties
----------------------------------------------------------------------
diff --git a/common/src/main/resources/hive-log4j2.properties b/common/src/main/resources/hive-log4j2.properties
index cf0369a..2f67be8 100644
--- a/common/src/main/resources/hive-log4j2.properties
+++ b/common/src/main/resources/hive-log4j2.properties
@@ -36,7 +36,7 @@ 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.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

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java
new file mode 100644
index 0000000..e2631cf
--- /dev/null
+++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.conf;
+
+import org.apache.hadoop.hive.common.LogUtils;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.async.AsyncLoggerContextSelector;
+import org.apache.logging.log4j.core.impl.Log4jContextFactory;
+import org.apache.logging.log4j.core.selector.ClassLoaderContextSelector;
+import org.apache.logging.log4j.core.selector.ContextSelector;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestHiveAsyncLogging extends TestCase {
+
+  // this test requires disruptor jar in classpath
+  @Test
+  public void testAsyncLoggingInitialization() throws Exception {
+    HiveConf conf = new HiveConf();
+    conf.setBoolVar(ConfVars.HIVE_ASYNC_LOG_ENABLED, false);
+    LogUtils.initHiveLog4jCommon(conf, ConfVars.HIVE_LOG4J_FILE);
+    Log4jContextFactory log4jContextFactory = (Log4jContextFactory) LogManager.getFactory();
+    ContextSelector contextSelector = log4jContextFactory.getSelector();
+    assertTrue(contextSelector instanceof ClassLoaderContextSelector);
+
+    conf.setBoolVar(ConfVars.HIVE_ASYNC_LOG_ENABLED, true);
+    LogUtils.initHiveLog4jCommon(conf, ConfVars.HIVE_LOG4J_FILE);
+    log4jContextFactory = (Log4jContextFactory) LogManager.getFactory();
+    contextSelector = log4jContextFactory.getSelector();
+    assertTrue(contextSelector instanceof AsyncLoggerContextSelector);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/data/conf/hive-log4j2.properties
----------------------------------------------------------------------
diff --git a/data/conf/hive-log4j2.properties b/data/conf/hive-log4j2.properties
index 6bace1f..f60d5be 100644
--- a/data/conf/hive-log4j2.properties
+++ b/data/conf/hive-log4j2.properties
@@ -35,7 +35,7 @@ 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.type = RollingRandomAccessFile
 appender.DRFA.name = DRFA
 appender.DRFA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
 appender.DRFA.filePattern = ${sys:hive.log.dir}/${sys:hive.log.file}.%d{yyyy-MM-dd}

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/hcatalog/core/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/core/pom.xml b/hcatalog/core/pom.xml
index 1e970bf..c9a6c01 100644
--- a/hcatalog/core/pom.xml
+++ b/hcatalog/core/pom.xml
@@ -131,6 +131,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/llap-server/bin/llapDaemon.sh
----------------------------------------------------------------------
diff --git a/llap-server/bin/llapDaemon.sh b/llap-server/bin/llapDaemon.sh
index 6f57998..566bbc8 100755
--- a/llap-server/bin/llapDaemon.sh
+++ b/llap-server/bin/llapDaemon.sh
@@ -113,7 +113,7 @@ case $startStop in
     #rotate_log $logOut
     echo starting llapdaemon, logging to $logLog and $logOut
     export LLAP_DAEMON_LOGFILE=${LLAP_DAEMON_LOG_BASE}.log
-    nohup nice -n $LLAP_DAEMON_NICENESS "$LLAP_DAEMON_BIN_HOME"/runLlapDaemon.sh run  > "$logOut" 2>&1 < /dev/null &
+    nohup nice -n $LLAP_DAEMON_NICENESS "$LLAP_DAEMON_BIN_HOME"/runLlapDaemon.sh run  >> "$logOut" 2>&1 < /dev/null &
     echo $! > $pid
     ;;
           

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index e662de9..2e07a8c 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -31,6 +31,7 @@ import java.util.regex.Pattern;
 import javax.management.ObjectName;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.LogUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.DaemonId;
@@ -119,7 +120,8 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       int mngPort, int shufflePort, int webPort, String appName) {
     super("LlapDaemon");
 
-    initializeLogging();
+    initializeLogging(daemonConf);
+
     printAsciiArt();
 
     Preconditions.checkArgument(numExecutors > 0);
@@ -264,13 +266,15 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     addIfService(amReporter);
   }
 
-  private void initializeLogging() {
+  private void initializeLogging(final Configuration conf) {
     long start = System.currentTimeMillis();
     URL llap_l4j2 = LlapDaemon.class.getClassLoader().getResource(LOG4j2_PROPERTIES_FILE);
     if (llap_l4j2 != null) {
+      final boolean async = LogUtils.checkAndSetAsyncLogging(conf);
       Configurator.initialize("LlapDaemonLog4j2", llap_l4j2.toString());
       long end = System.currentTimeMillis();
-      LOG.warn("LLAP daemon logging initialized from {} in {} ms", llap_l4j2, (end - start));
+      LOG.warn("LLAP daemon logging initialized from {} in {} ms. Async: {}",
+          llap_l4j2, (end - start), async);
     } else {
       throw new RuntimeException("Log initialization failed." +
           " Unable to locate " + LOG4j2_PROPERTIES_FILE + " file in classpath");

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/llap-server/src/main/resources/llap-cli-log4j2.properties
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/llap-cli-log4j2.properties b/llap-server/src/main/resources/llap-cli-log4j2.properties
index 2f27b5e..c6b8f20 100644
--- a/llap-server/src/main/resources/llap-cli-log4j2.properties
+++ b/llap-server/src/main/resources/llap-cli-log4j2.properties
@@ -36,7 +36,7 @@ appender.console.layout.type = PatternLayout
 appender.console.layout.pattern = %p %c{2}: %m%n
 
 # daily rolling file appender
-appender.DRFA.type = RollingFile
+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

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/llap-server/src/main/resources/llap-daemon-log4j2.properties
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/llap-daemon-log4j2.properties b/llap-server/src/main/resources/llap-daemon-log4j2.properties
index 268eb59..c5166e3 100644
--- a/llap-server/src/main/resources/llap-daemon-log4j2.properties
+++ b/llap-server/src/main/resources/llap-daemon-log4j2.properties
@@ -38,7 +38,7 @@ appender.console.layout.type = PatternLayout
 appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t%x] %p %c{2} : %m%n
 
 # rolling file appender
-appender.RFA.type = RollingFile
+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}_%i
@@ -51,7 +51,7 @@ appender.RFA.strategy.type = DefaultRolloverStrategy
 appender.RFA.strategy.max = ${sys:llap.daemon.log.maxbackupindex}
 
 # history file appender
-appender.HISTORYAPPENDER.type = RollingFile
+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}_%i

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/llap-server/src/test/resources/llap-daemon-log4j2.properties
----------------------------------------------------------------------
diff --git a/llap-server/src/test/resources/llap-daemon-log4j2.properties b/llap-server/src/test/resources/llap-daemon-log4j2.properties
index 7b5f4ed..2714dbd 100644
--- a/llap-server/src/test/resources/llap-daemon-log4j2.properties
+++ b/llap-server/src/test/resources/llap-daemon-log4j2.properties
@@ -38,7 +38,7 @@ appender.console.layout.type = PatternLayout
 appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t%x] %p %c{2} : %m%n
 
 # rolling file appender
-appender.RFA.type = RollingFile
+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}_%i
@@ -51,7 +51,7 @@ appender.RFA.strategy.type = DefaultRolloverStrategy
 appender.RFA.strategy.max = ${sys:llap.daemon.log.maxbackupindex}
 
 # history file appender
-appender.HISTORYAPPENDER.type = RollingFile
+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}_%i

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/metastore/pom.xml
----------------------------------------------------------------------
diff --git a/metastore/pom.xml b/metastore/pom.xml
index 8816829..3827a51 100644
--- a/metastore/pom.xml
+++ b/metastore/pom.xml
@@ -228,6 +228,12 @@
       <version>${mockito-all.version}</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index dff2a72..bde6857 100644
--- a/pom.xml
+++ b/pom.xml
@@ -134,6 +134,8 @@
     <hadoop.version>2.6.0</hadoop.version>
     <hadoop.bin.path>${basedir}/${hive.path.to.root}/testutils/hadoop</hadoop.bin.path>
     <hbase.version>1.1.1</hbase.version>
+    <!-- required for logging test to avoid including hbase which pulls disruptor transitively -->
+    <disruptor.version>3.3.0</disruptor.version>
     <!-- httpcomponents are not always in version sync -->
     <httpcomponents.client.version>4.4</httpcomponents.client.version>
     <httpcomponents.core.version>4.4</httpcomponents.core.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index aaa3271..8b2d0e6 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -325,6 +325,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.codehaus.groovy</groupId>
       <artifactId>groovy-all</artifactId>
       <version>${groovy.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/ql/src/main/resources/hive-exec-log4j2.properties
----------------------------------------------------------------------
diff --git a/ql/src/main/resources/hive-exec-log4j2.properties b/ql/src/main/resources/hive-exec-log4j2.properties
index 4fba04c..21e24fd 100644
--- a/ql/src/main/resources/hive-exec-log4j2.properties
+++ b/ql/src/main/resources/hive-exec-log4j2.properties
@@ -36,7 +36,7 @@ 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.type = RandomAccessFile
 appender.FA.name = FA
 appender.FA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
 appender.FA.layout.type = PatternLayout

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/ql/src/main/resources/tez-container-log4j2.properties
----------------------------------------------------------------------
diff --git a/ql/src/main/resources/tez-container-log4j2.properties b/ql/src/main/resources/tez-container-log4j2.properties
index 5d2b138..a048b17 100644
--- a/ql/src/main/resources/tez-container-log4j2.properties
+++ b/ql/src/main/resources/tez-container-log4j2.properties
@@ -28,7 +28,7 @@ property.tez.container.log.file = syslog
 appenders = CLA
 
 # daily rolling file appender
-appender.CLA.type = RollingFile
+appender.CLA.type = RollingRandomAccessFile
 appender.CLA.name = CLA
 appender.CLA.fileName = ${sys:tez.container.log.dir}/${sys:tez.container.log.file}
 appender.CLA.filePattern = ${sys:tez.container.log.dir}/${sys:tez.container.log.file}.%d{yyyy-MM-dd}


[31/50] [abbrv] hive git commit: HIVE-13639: CBO rule to pull up constants through Union (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by sp...@apache.org.
HIVE-13639: CBO rule to pull up constants through Union (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/java8
Commit: 092718720a4abc77ce74c2efcf42cfef0243e9d4
Parents: f41d693
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed May 4 22:01:52 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu May 5 20:21:50 2016 +0100

----------------------------------------------------------------------
 .../rules/HiveUnionPullUpConstantsRule.java     | 133 ++++
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   2 +
 .../queries/clientpositive/cbo_union_view.q     |  19 +
 .../results/clientpositive/cbo_input26.q.out    |  64 +-
 .../results/clientpositive/cbo_union_view.q.out | 228 ++++++
 .../results/clientpositive/groupby_ppd.q.out    |  28 +-
 .../results/clientpositive/perf/query66.q.out   | 328 ++++-----
 .../results/clientpositive/perf/query75.q.out   | 692 ++++++++++---------
 .../clientpositive/spark/union_remove_25.q.out  |  48 +-
 .../clientpositive/spark/union_view.q.out       |  60 +-
 .../results/clientpositive/union_view.q.out     |  60 +-
 11 files changed, 1021 insertions(+), 641 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java
new file mode 100644
index 0000000..3155cb1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Planner rule that pulls up constants through a Union operator.
+ */
+public class HiveUnionPullUpConstantsRule extends RelOptRule {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(HiveUnionPullUpConstantsRule.class);
+
+
+  public static final HiveUnionPullUpConstantsRule INSTANCE =
+          new HiveUnionPullUpConstantsRule(HiveUnion.class,
+                  HiveRelFactories.HIVE_BUILDER);
+
+  private HiveUnionPullUpConstantsRule(
+      Class<? extends Union> unionClass,
+      RelBuilderFactory relBuilderFactory) {
+    super(operand(unionClass, any()),
+            relBuilderFactory, null);
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final Union union = call.rel(0);
+
+    final int count = union.getRowType().getFieldCount();
+    if (count == 1) {
+      // No room for optimization since we cannot create an empty
+      // Project operator.
+      return;
+    }
+
+    final RexBuilder rexBuilder = union.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    final RelOptPredicateList predicates = mq.getPulledUpPredicates(union);
+    if (predicates == null) {
+      return;
+    }
+
+    Map<RexNode, RexNode> constants = HiveReduceExpressionsRule.predicateConstants(
+            RexNode.class, rexBuilder, predicates);
+
+    // None of the expressions are constant. Nothing to do.
+    if (constants.isEmpty()) {
+      return;
+    }
+
+    if (count == constants.size()) {
+      // At least a single item in project is required.
+      final Map<RexNode, RexNode> map = new HashMap<>(constants);
+      map.remove(map.keySet().iterator().next());
+      constants = map;
+    }
+
+    // Create expressions for Project operators before and after the Union
+    List<RelDataTypeField> fields = union.getRowType().getFieldList();
+    List<Pair<RexNode, String>> newChildExprs = new ArrayList<>();
+    List<RexNode> topChildExprs = new ArrayList<>();
+    List<String> topChildExprsFields = new ArrayList<>();
+    for (int i = 0; i < count ; i++) {
+      RexNode expr = rexBuilder.makeInputRef(union, i);
+      RelDataTypeField field = fields.get(i);
+      if (constants.containsKey(expr)) {
+        topChildExprs.add(constants.get(expr));
+        topChildExprsFields.add(field.getName());
+      } else {
+        newChildExprs.add(Pair.<RexNode,String>of(expr, field.getName()));
+        topChildExprs.add(expr);
+        topChildExprsFields.add(field.getName());
+      }
+    }
+
+    // Update top Project positions
+    final Mappings.TargetMapping mapping =
+            RelOptUtil.permutation(Pair.left(newChildExprs), union.getInput(0).getRowType()).inverse();
+    topChildExprs = ImmutableList.copyOf(RexUtil.apply(mapping, topChildExprs));
+
+    // Create new Project-Union-Project sequences
+    final RelBuilder relBuilder = call.builder();
+    for (int i = 0; i < union.getInputs().size() ; i++) {
+      relBuilder.push(union.getInput(i));
+      relBuilder.project(Pair.left(newChildExprs), Pair.right(newChildExprs));
+    }
+    relBuilder.union(union.all, union.getInputs().size());
+    relBuilder.project(topChildExprs, topChildExprsFields);
+
+    call.transformTo(relBuilder.build());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 377573b..de6a053 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -154,6 +154,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinProjectTranspos
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinPushTransitivePredicatesRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinToMultiJoinRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortLimitPullUpConstantsRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveUnionPullUpConstantsRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePartitionPruneRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePointLookupOptimizerRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePreFilteringRule;
@@ -1166,6 +1167,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_SEMIJOIN);
       rules.add(HiveSortMergeRule.INSTANCE);
       rules.add(HiveSortLimitPullUpConstantsRule.INSTANCE);
+      rules.add(HiveUnionPullUpConstantsRule.INSTANCE);
       perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
       basePlan = hepPlan(basePlan, true, mdProvider, executorProvider, HepMatchOrder.BOTTOM_UP,
               rules.toArray(new RelOptRule[rules.size()]));

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/queries/clientpositive/cbo_union_view.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_union_view.q b/ql/src/test/queries/clientpositive/cbo_union_view.q
new file mode 100644
index 0000000..d889b1d
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/cbo_union_view.q
@@ -0,0 +1,19 @@
+set hive.mapred.mode=nonstrict;
+set hive.optimize.constant.propagation=false;
+
+CREATE TABLE src_union_1 (key int, value string) PARTITIONED BY (ds string);
+CREATE TABLE src_union_2 (key int, value string) PARTITIONED BY (ds string, part_1 string);
+CREATE TABLE src_union_3 (key int, value string) PARTITIONED BY (ds string, part_1 string, part_2 string);
+
+CREATE VIEW src_union_view PARTITIONED ON (ds) as
+SELECT key, value, ds FROM (
+SELECT key, value, ds FROM src_union_1
+UNION ALL
+SELECT key, value, ds FROM src_union_2
+UNION ALL
+SELECT key, value, ds FROM src_union_3
+) subq;
+
+EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86;
+
+EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='1';

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/cbo_input26.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_input26.q.out b/ql/src/test/results/clientpositive/cbo_input26.q.out
index 5c4c771..c9ed67a 100644
--- a/ql/src/test/results/clientpositive/cbo_input26.q.out
+++ b/ql/src/test/results/clientpositive/cbo_input26.q.out
@@ -490,16 +490,12 @@ STAGE PLANS:
           Limit
             Number of rows: 5
             Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: _col0 (type: string), '2008-04-08' (type: string), _col1 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
-                table:
-                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-2
     Map Reduce
@@ -507,23 +503,31 @@ STAGE PLANS:
           TableScan
             Union
               Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
+              Select Operator
+                expressions: _col0 (type: string), '2008-04-08' (type: string), _col1 (type: string)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-                table:
-                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             Union
               Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
+              Select Operator
+                expressions: _col0 (type: string), '2008-04-08' (type: string), _col1 (type: string)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
-                table:
-                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-3
     Map Reduce
@@ -554,16 +558,12 @@ STAGE PLANS:
           Limit
             Number of rows: 5
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-            Select Operator
-              expressions: _col0 (type: string), '2008-04-08' (type: string), _col1 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-              File Output Operator
-                compressed: false
-                table:
-                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/cbo_union_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_union_view.q.out b/ql/src/test/results/clientpositive/cbo_union_view.q.out
new file mode 100644
index 0000000..ed6bba9
--- /dev/null
+++ b/ql/src/test/results/clientpositive/cbo_union_view.q.out
@@ -0,0 +1,228 @@
+PREHOOK: query: CREATE TABLE src_union_1 (key int, value string) PARTITIONED BY (ds string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_union_1
+POSTHOOK: query: CREATE TABLE src_union_1 (key int, value string) PARTITIONED BY (ds string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_union_1
+PREHOOK: query: CREATE TABLE src_union_2 (key int, value string) PARTITIONED BY (ds string, part_1 string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_union_2
+POSTHOOK: query: CREATE TABLE src_union_2 (key int, value string) PARTITIONED BY (ds string, part_1 string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_union_2
+PREHOOK: query: CREATE TABLE src_union_3 (key int, value string) PARTITIONED BY (ds string, part_1 string, part_2 string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_union_3
+POSTHOOK: query: CREATE TABLE src_union_3 (key int, value string) PARTITIONED BY (ds string, part_1 string, part_2 string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_union_3
+PREHOOK: query: CREATE VIEW src_union_view PARTITIONED ON (ds) as
+SELECT key, value, ds FROM (
+SELECT key, value, ds FROM src_union_1
+UNION ALL
+SELECT key, value, ds FROM src_union_2
+UNION ALL
+SELECT key, value, ds FROM src_union_3
+) subq
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: default@src_union_1
+PREHOOK: Input: default@src_union_2
+PREHOOK: Input: default@src_union_3
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_union_view
+POSTHOOK: query: CREATE VIEW src_union_view PARTITIONED ON (ds) as
+SELECT key, value, ds FROM (
+SELECT key, value, ds FROM src_union_1
+UNION ALL
+SELECT key, value, ds FROM src_union_2
+UNION ALL
+SELECT key, value, ds FROM src_union_3
+) subq
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: default@src_union_1
+POSTHOOK: Input: default@src_union_2
+POSTHOOK: Input: default@src_union_3
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_union_view
+PREHOOK: query: EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src_union_1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (key = 86) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), _col1 (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          TableScan
+            alias: src_union_2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (key = 86) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), _col1 (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          TableScan
+            alias: src_union_3
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (key = 86) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), _col1 (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='1'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='1'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src_union_1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((key = 86) and (ds = '1')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          TableScan
+            alias: src_union_2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((key = 86) and (ds = '1')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          TableScan
+            alias: src_union_3
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((key = 86) and (ds = '1')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: value (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: 86 (type: int), _col0 (type: string), '1' (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 3 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/groupby_ppd.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_ppd.q.out b/ql/src/test/results/clientpositive/groupby_ppd.q.out
index c63acd3..515f62e 100644
--- a/ql/src/test/results/clientpositive/groupby_ppd.q.out
+++ b/ql/src/test/results/clientpositive/groupby_ppd.q.out
@@ -28,23 +28,23 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: foo (type: int)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
-                    expressions: 1 (type: int), _col1 (type: int)
-                    outputColumnNames: _col0, _col1
+                    expressions: _col0 (type: int)
+                    outputColumnNames: _col1
                     Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int), _col1 (type: int)
+                      keys: 1 (type: int), _col1 (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int), _col1 (type: int)
+                        key expressions: 1 (type: int), _col1 (type: int)
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                        Map-reduce partition columns: 1 (type: int), _col1 (type: int)
                         Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           TableScan
             alias: c
@@ -54,32 +54,32 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: foo (type: int)
-                outputColumnNames: _col1
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Select Operator
-                    expressions: 1 (type: int), _col1 (type: int)
-                    outputColumnNames: _col0, _col1
+                    expressions: _col0 (type: int)
+                    outputColumnNames: _col1
                     Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      keys: _col0 (type: int), _col1 (type: int)
+                      keys: 1 (type: int), _col1 (type: int)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int), _col1 (type: int)
+                        key expressions: 1 (type: int), _col1 (type: int)
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                        Map-reduce partition columns: 1 (type: int), _col1 (type: int)
                         Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
       Reduce Operator Tree:
         Group By Operator
-          keys: KEY._col0 (type: int), KEY._col1 (type: int)
+          keys: 1 (type: int), KEY._col1 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
-            expressions: _col1 (type: int), _col0 (type: int)
+            expressions: _col1 (type: int), 1 (type: int)
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/09271872/ql/src/test/results/clientpositive/perf/query66.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query66.q.out b/ql/src/test/results/clientpositive/perf/query66.q.out
index d698602..a606946 100644
--- a/ql/src/test/results/clientpositive/perf/query66.q.out
+++ b/ql/src/test/results/clientpositive/perf/query66.q.out
@@ -464,167 +464,169 @@ Stage-0
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"]
           <-Reducer 8 [SIMPLE_EDGE]
             SHUFFLE [RS_73]
-              Group By Operator [GBY_71] (rows=26136 width=471)
-                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)","sum(VALUE._col24)","sum(VALUE._col25)","sum(VALUE._col26)","sum(VALUE._col27)","sum(VALUE._col28)","s
 um(VALUE._col29)","sum(VALUE._col30)","sum(VALUE._col31)","sum(VALUE._col32)","sum(VALUE._col33)","sum(VALUE._col34)","sum(VALUE._col35)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7
-              <-Union 7 [SIMPLE_EDGE]
-                <-Reducer 19 [CONTAINS]
-                  Reduce Output Operator [RS_70]
-                    PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                    Group By Operator [GBY_69] (rows=52272 width=471)
-                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)","sum(_col42)","sum(_col43)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                      Select Operator [SEL_67] (rows=52272 width=471)
-                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"]
-                        Select Operator [SEL_65] (rows=26136 width=471)
-                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31"]
-                          Group By Operator [GBY_64] (rows=26136 width=471)
-                            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 2002
-                          <-Reducer 18 [SIMPLE_EDGE]
-                            SHUFFLE [RS_63]
-                              PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 2002
-                              Group By Operator [GBY_62] (rows=52272 width=471)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 2002
-                                Select Operator [SEL_60] (rows=52272 width=471)
-                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"]
-                                  Merge Join Operator [MERGEJOIN_122] (rows=52272 width=471)
-                                    Conds:RS_57._col2=RS_58._col0(Inner),Output:["_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                  <-Map 23 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_58]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_47] (rows=1 width=0)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_114] (rows=1 width=0)
-                                          predicate:((sm_carrier) IN ('DIAMOND', 'AIRBORNE') and sm_ship_mode_sk is not null)
-                                          TableScan [TS_45] (rows=1 width=0)
-                                            default@ship_mode,ship_mode,Tbl:PARTIAL,Col:NONE,Output:["sm_ship_mode_sk","sm_carrier"]
-                                  <-Reducer 17 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_57]
-                                      PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_121] (rows=47520 width=471)
-                                        Conds:RS_54._col1=RS_55._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                      <-Map 22 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_55]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_44] (rows=43200 width=471)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_113] (rows=43200 width=471)
-                                              predicate:(t_time BETWEEN 49530 AND 78330 and t_time_sk is not null)
-                                              TableScan [TS_42] (rows=86400 width=471)
-                                                default@time_dim,time_dim,Tbl:COMPLETE,Col:NONE,Output:["t_time_sk","t_time"]
-                                      <-Reducer 16 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_54]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_120] (rows=40176 width=1119)
-                                            Conds:RS_51._col0=RS_52._col0(Inner),Output:["_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                          <-Map 21 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_52]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_41] (rows=36524 width=1119)
-                                                Output:["_col0","_col2"]
-                                                Filter Operator [FIL_112] (rows=36524 width=1119)
-                                                  predicate:((d_year = 2002) and d_date_sk is not null)
-                                                  TableScan [TS_39] (rows=73049 width=1119)
-                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
-                                          <-Reducer 15 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_51]
-                                              PartitionCols:_col0
-                                              Merge Join Operator [MERGEJOIN_119] (rows=29 width=1054)
-                                                Conds:RS_48._col3=RS_49._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13"]
-                                              <-Map 14 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_48]
-                                                  PartitionCols:_col3
-                                                  Select Operator [SEL_35] (rows=1 width=0)
-                                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                    Filter Operator [FIL_110] (rows=1 width=0)
-                                                      predicate:(cs_warehouse_sk is not null and cs_sold_date_sk is not null and cs_sold_time_sk is not null and cs_ship_mode_sk is not null)
-                                                      TableScan [TS_33] (rows=1 width=0)
-                                                        default@catalog_sales,catalog_sales,Tbl:PARTIAL,Col:NONE,Output:["cs_sold_date_sk","cs_sold_time_sk","cs_ship_mode_sk","cs_warehouse_sk","cs_quantity","cs_ext_sales_price","cs_net_paid_inc_ship_tax"]
-                                              <-Map 20 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_49]
-                                                  PartitionCols:_col0
-                                                  Select Operator [SEL_38] (rows=27 width=1029)
-                                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                    Filter Operator [FIL_111] (rows=27 width=1029)
-                                                      predicate:w_warehouse_sk is not null
-                                                      TableScan [TS_36] (rows=27 width=1029)
-                                                        default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name","w_warehouse_sq_ft","w_city","w_county","w_state","w_country"]
-                <-Reducer 6 [CONTAINS]
-                  Reduce Output Operator [RS_70]
-                    PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                    Group By Operator [GBY_69] (rows=52272 width=471)
-                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)","sum(_col42)","sum(_col43)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                      Select Operator [SEL_67] (rows=52272 width=471)
-                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"]
-                        Select Operator [SEL_32] (rows=26136 width=471)
-                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31"]
-                          Group By Operator [GBY_31] (rows=26136 width=471)
-                            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 2002
-                          <-Reducer 5 [SIMPLE_EDGE]
-                            SHUFFLE [RS_30]
-                              PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 2002
-                              Group By Operator [GBY_29] (rows=52272 width=471)
-                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 2002
-                                Select Operator [SEL_27] (rows=52272 width=471)
-                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"]
-                                  Merge Join Operator [MERGEJOIN_118] (rows=52272 width=471)
-                                    Conds:RS_24._col2=RS_25._col0(Inner),Output:["_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                  <-Map 13 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_25]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_14] (rows=1 width=0)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_109] (rows=1 width=0)
-                                          predicate:((sm_carrier) IN ('DIAMOND', 'AIRBORNE') and sm_ship_mode_sk is not null)
-                                          TableScan [TS_12] (rows=1 width=0)
-                                            default@ship_mode,ship_mode,Tbl:PARTIAL,Col:NONE,Output:["sm_ship_mode_sk","sm_carrier"]
-                                  <-Reducer 4 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_24]
-                                      PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_117] (rows=47520 width=471)
-                                        Conds:RS_21._col1=RS_22._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                      <-Map 12 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_22]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_11] (rows=43200 width=471)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_108] (rows=43200 width=471)
-                                              predicate:(t_time BETWEEN 49530 AND 78330 and t_time_sk is not null)
-                                              TableScan [TS_9] (rows=86400 width=471)
-                                                default@time_dim,time_dim,Tbl:COMPLETE,Col:NONE,Output:["t_time_sk","t_time"]
-                                      <-Reducer 3 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_21]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_116] (rows=40176 width=1119)
-                                            Conds:RS_18._col0=RS_19._col0(Inner),Output:["_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
-                                          <-Map 11 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_19]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_8] (rows=36524 width=1119)
-                                                Output:["_col0","_col2"]
-                                                Filter Operator [FIL_107] (rows=36524 width=1119)
-                                                  predicate:((d_year = 2002) and d_date_sk is not null)
-                                                  TableScan [TS_6] (rows=73049 width=1119)
-                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
-                                          <-Reducer 2 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_18]
-                                              PartitionCols:_col0
-                                              Merge Join Operator [MERGEJOIN_115] (rows=29 width=1054)
-                                                Conds:RS_15._col3=RS_16._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13"]
-                                              <-Map 1 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_15]
-                                                  PartitionCols:_col3
-                                                  Select Operator [SEL_2] (rows=1 width=0)
-                                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                    Filter Operator [FIL_105] (rows=1 width=0)
-                                                      predicate:(ws_warehouse_sk is not null and ws_sold_date_sk is not null and ws_sold_time_sk is not null and ws_ship_mode_sk is not null)
-                                                      TableScan [TS_0] (rows=1 width=0)
-                                                        default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_sold_time_sk","ws_ship_mode_sk","ws_warehouse_sk","ws_quantity","ws_sales_price","ws_net_paid_inc_tax"]
-                                              <-Map 10 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_16]
-                                                  PartitionCols:_col0
-                                                  Select Operator [SEL_5] (rows=27 width=1029)
-                                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                    Filter Operator [FIL_106] (rows=27 width=1029)
-                                                      predicate:w_warehouse_sk is not null
-                                                      TableScan [TS_3] (rows=27 width=1029)
-                                                        default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name","w_warehouse_sq_ft","w_city","w_county","w_state","w_country"]
+              Select Operator [SEL_72] (rows=26136 width=471)
+                Output:["_col0","_col1","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col2","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col3","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col4","_col40","_col41","_col42","_col43","_col5","_col8","_col9"]
+                Group By Operator [GBY_71] (rows=26136 width=471)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)","sum(VALUE._col24)","sum(VALUE._col25)","sum(VALUE._col26)","sum(VALUE._col27)","sum(VALUE._col28)",
 "sum(VALUE._col29)","sum(VALUE._col30)","sum(VALUE._col31)","sum(VALUE._col32)","sum(VALUE._col33)","sum(VALUE._col34)","sum(VALUE._col35)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 'DIAMOND,AIRBORNE', 2002
+                <-Union 7 [SIMPLE_EDGE]
+                  <-Reducer 19 [CONTAINS]
+                    Reduce Output Operator [RS_70]
+                      PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 'DIAMOND,AIRBORNE', 2002
+                      Group By Operator [GBY_69] (rows=52272 width=471)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)","sum(_col42)","sum(_col43)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 'DIAMOND,AIRBO
 RNE', 2002
+                        Select Operator [SEL_67] (rows=52272 width=471)
+                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"]
+                          Select Operator [SEL_65] (rows=26136 width=471)
+                            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"]
+                            Group By Operator [GBY_64] (rows=26136 width=471)
+                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 2002
+                            <-Reducer 18 [SIMPLE_EDGE]
+                              SHUFFLE [RS_63]
+                                PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 2002
+                                Group By Operator [GBY_62] (rows=52272 width=471)
+                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 2002
+                                  Select Operator [SEL_60] (rows=52272 width=471)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"]
+                                    Merge Join Operator [MERGEJOIN_122] (rows=52272 width=471)
+                                      Conds:RS_57._col2=RS_58._col0(Inner),Output:["_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                    <-Map 23 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_58]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_47] (rows=1 width=0)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_114] (rows=1 width=0)
+                                            predicate:((sm_carrier) IN ('DIAMOND', 'AIRBORNE') and sm_ship_mode_sk is not null)
+                                            TableScan [TS_45] (rows=1 width=0)
+                                              default@ship_mode,ship_mode,Tbl:PARTIAL,Col:NONE,Output:["sm_ship_mode_sk","sm_carrier"]
+                                    <-Reducer 17 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_57]
+                                        PartitionCols:_col2
+                                        Merge Join Operator [MERGEJOIN_121] (rows=47520 width=471)
+                                          Conds:RS_54._col1=RS_55._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                        <-Map 22 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_55]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_44] (rows=43200 width=471)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_113] (rows=43200 width=471)
+                                                predicate:(t_time BETWEEN 49530 AND 78330 and t_time_sk is not null)
+                                                TableScan [TS_42] (rows=86400 width=471)
+                                                  default@time_dim,time_dim,Tbl:COMPLETE,Col:NONE,Output:["t_time_sk","t_time"]
+                                        <-Reducer 16 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_54]
+                                            PartitionCols:_col1
+                                            Merge Join Operator [MERGEJOIN_120] (rows=40176 width=1119)
+                                              Conds:RS_51._col0=RS_52._col0(Inner),Output:["_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                            <-Map 21 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_52]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_41] (rows=36524 width=1119)
+                                                  Output:["_col0","_col2"]
+                                                  Filter Operator [FIL_112] (rows=36524 width=1119)
+                                                    predicate:((d_year = 2002) and d_date_sk is not null)
+                                                    TableScan [TS_39] (rows=73049 width=1119)
+                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
+                                            <-Reducer 15 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_51]
+                                                PartitionCols:_col0
+                                                Merge Join Operator [MERGEJOIN_119] (rows=29 width=1054)
+                                                  Conds:RS_48._col3=RS_49._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13"]
+                                                <-Map 14 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_48]
+                                                    PartitionCols:_col3
+                                                    Select Operator [SEL_35] (rows=1 width=0)
+                                                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                      Filter Operator [FIL_110] (rows=1 width=0)
+                                                        predicate:(cs_warehouse_sk is not null and cs_sold_date_sk is not null and cs_sold_time_sk is not null and cs_ship_mode_sk is not null)
+                                                        TableScan [TS_33] (rows=1 width=0)
+                                                          default@catalog_sales,catalog_sales,Tbl:PARTIAL,Col:NONE,Output:["cs_sold_date_sk","cs_sold_time_sk","cs_ship_mode_sk","cs_warehouse_sk","cs_quantity","cs_ext_sales_price","cs_net_paid_inc_ship_tax"]
+                                                <-Map 20 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_49]
+                                                    PartitionCols:_col0
+                                                    Select Operator [SEL_38] (rows=27 width=1029)
+                                                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                      Filter Operator [FIL_111] (rows=27 width=1029)
+                                                        predicate:w_warehouse_sk is not null
+                                                        TableScan [TS_36] (rows=27 width=1029)
+                                                          default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name","w_warehouse_sq_ft","w_city","w_county","w_state","w_country"]
+                  <-Reducer 6 [CONTAINS]
+                    Reduce Output Operator [RS_70]
+                      PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 'DIAMOND,AIRBORNE', 2002
+                      Group By Operator [GBY_69] (rows=52272 width=471)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)","sum(_col42)","sum(_col43)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 'DIAMOND,AIRBO
 RNE', 2002
+                        Select Operator [SEL_67] (rows=52272 width=471)
+                          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"]
+                          Select Operator [SEL_32] (rows=26136 width=471)
+                            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"]
+                            Group By Operator [GBY_31] (rows=26136 width=471)
+                              Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 2002
+                            <-Reducer 5 [SIMPLE_EDGE]
+                              SHUFFLE [RS_30]
+                                PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 2002
+                                Group By Operator [GBY_29] (rows=52272 width=471)
+                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 2002
+                                  Select Operator [SEL_27] (rows=52272 width=471)
+                                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"]
+                                    Merge Join Operator [MERGEJOIN_118] (rows=52272 width=471)
+                                      Conds:RS_24._col2=RS_25._col0(Inner),Output:["_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                    <-Map 13 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_25]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_14] (rows=1 width=0)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_109] (rows=1 width=0)
+                                            predicate:((sm_carrier) IN ('DIAMOND', 'AIRBORNE') and sm_ship_mode_sk is not null)
+                                            TableScan [TS_12] (rows=1 width=0)
+                                              default@ship_mode,ship_mode,Tbl:PARTIAL,Col:NONE,Output:["sm_ship_mode_sk","sm_carrier"]
+                                    <-Reducer 4 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_24]
+                                        PartitionCols:_col2
+                                        Merge Join Operator [MERGEJOIN_117] (rows=47520 width=471)
+                                          Conds:RS_21._col1=RS_22._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                        <-Map 12 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_22]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_11] (rows=43200 width=471)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_108] (rows=43200 width=471)
+                                                predicate:(t_time BETWEEN 49530 AND 78330 and t_time_sk is not null)
+                                                TableScan [TS_9] (rows=86400 width=471)
+                                                  default@time_dim,time_dim,Tbl:COMPLETE,Col:NONE,Output:["t_time_sk","t_time"]
+                                        <-Reducer 3 [SIMPLE_EDGE]
+                                          SHUFFLE [RS_21]
+                                            PartitionCols:_col1
+                                            Merge Join Operator [MERGEJOIN_116] (rows=40176 width=1119)
+                                              Conds:RS_18._col0=RS_19._col0(Inner),Output:["_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"]
+                                            <-Map 11 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_19]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_8] (rows=36524 width=1119)
+                                                  Output:["_col0","_col2"]
+                                                  Filter Operator [FIL_107] (rows=36524 width=1119)
+                                                    predicate:((d_year = 2002) and d_date_sk is not null)
+                                                    TableScan [TS_6] (rows=73049 width=1119)
+                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
+                                            <-Reducer 2 [SIMPLE_EDGE]
+                                              SHUFFLE [RS_18]
+                                                PartitionCols:_col0
+                                                Merge Join Operator [MERGEJOIN_115] (rows=29 width=1054)
+                                                  Conds:RS_15._col3=RS_16._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13"]
+                                                <-Map 1 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_15]
+                                                    PartitionCols:_col3
+                                                    Select Operator [SEL_2] (rows=1 width=0)
+                                                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                      Filter Operator [FIL_105] (rows=1 width=0)
+                                                        predicate:(ws_warehouse_sk is not null and ws_sold_date_sk is not null and ws_sold_time_sk is not null and ws_ship_mode_sk is not null)
+                                                        TableScan [TS_0] (rows=1 width=0)
+                                                          default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_sold_time_sk","ws_ship_mode_sk","ws_warehouse_sk","ws_quantity","ws_sales_price","ws_net_paid_inc_tax"]
+                                                <-Map 10 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_16]
+                                                    PartitionCols:_col0
+                                                    Select Operator [SEL_5] (rows=27 width=1029)
+                                                      Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                      Filter Operator [FIL_106] (rows=27 width=1029)
+                                                        predicate:w_warehouse_sk is not null
+                                                        TableScan [TS_3] (rows=27 width=1029)
+                                                          default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name","w_warehouse_sq_ft","w_city","w_county","w_state","w_country"]
 


[35/50] [abbrv] hive git commit: HIVE-13395 Lost Update problem in ACID (Eugene Koifman, reviewed by Alan Gates)

Posted by sp...@apache.org.
HIVE-13395 Lost Update problem in ACID (Eugene Koifman, reviewed by Alan Gates)


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

Branch: refs/heads/java8
Commit: 10d05491379bb6f8e607a030811e8d4e530604de
Parents: 0927187
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Thu May 5 12:45:44 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Thu May 5 12:45:44 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   2 +
 .../hive/metastore/TestHiveMetaStoreTxns.java   |   2 +-
 .../upgrade/derby/035-HIVE-13395.derby.sql      |  11 +
 .../upgrade/derby/hive-schema-2.1.0.derby.sql   |   2 +-
 .../derby/hive-txn-schema-1.3.0.derby.sql       |  11 +-
 .../derby/hive-txn-schema-2.1.0.derby.sql       | 130 ++++
 .../derby/upgrade-1.2.0-to-1.3.0.derby.sql      |   1 +
 .../derby/upgrade-2.0.0-to-2.1.0.derby.sql      |   1 +
 .../upgrade/mssql/020-HIVE-13395.mssql.sql      |   9 +
 .../upgrade/mssql/hive-schema-1.3.0.mssql.sql   |  12 +-
 .../upgrade/mssql/hive-schema-2.1.0.mssql.sql   |  12 +-
 .../mssql/upgrade-1.2.0-to-1.3.0.mssql.sql      |   1 +
 .../mssql/upgrade-2.0.0-to-2.1.0.mssql.sql      |   1 +
 .../upgrade/mysql/035-HIVE-13395.mysql.sql      |  10 +
 .../upgrade/mysql/hive-schema-2.1.0.mysql.sql   |   2 +-
 .../mysql/hive-txn-schema-2.1.0.mysql.sql       | 131 ++++
 .../mysql/upgrade-1.2.0-to-1.3.0.mysql.sql      |   1 +
 .../mysql/upgrade-2.0.0-to-2.1.0.mysql.sql      |   1 +
 .../upgrade/oracle/035-HIVE-13395.oracle.sql    |  10 +
 .../upgrade/oracle/hive-schema-2.1.0.oracle.sql |   2 +-
 .../oracle/hive-txn-schema-1.3.0.oracle.sql     |  12 +-
 .../oracle/hive-txn-schema-2.1.0.oracle.sql     | 129 ++++
 .../oracle/upgrade-1.2.0-to-1.3.0.oracle.sql    |   1 +
 .../oracle/upgrade-2.0.0-to-2.1.0.oracle.sql    |   1 +
 .../postgres/034-HIVE-13395.postgres.sql        |  10 +
 .../postgres/hive-schema-2.1.0.postgres.sql     |   2 +-
 .../postgres/hive-txn-schema-1.3.0.postgres.sql |  11 +-
 .../postgres/hive-txn-schema-2.1.0.postgres.sql | 129 ++++
 .../upgrade-1.2.0-to-1.3.0.postgres.sql         |   1 +
 .../upgrade-2.0.0-to-2.1.0.postgres.sql         |   1 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   1 +
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    | 130 ++--
 .../hadoop/hive/metastore/txn/TxnHandler.java   | 466 +++++++++++---
 .../hadoop/hive/metastore/txn/TxnStore.java     |   8 +-
 .../hadoop/hive/metastore/txn/TxnUtils.java     |   2 +
 .../metastore/txn/TestCompactionTxnHandler.java |   6 +-
 .../hive/metastore/txn/TestTxnHandler.java      |  29 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |   2 +-
 .../hadoop/hive/ql/lockmgr/DbLockManager.java   |   5 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |  27 +-
 .../hadoop/hive/ql/txn/AcidWriteSetService.java |  61 ++
 .../txn/compactor/HouseKeeperServiceBase.java   |   2 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |   2 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java    |   2 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |   2 +-
 .../apache/hadoop/hive/ql/io/TestAcidUtils.java |  20 +
 .../hive/ql/lockmgr/TestDbTxnManager.java       |   7 +
 .../hive/ql/lockmgr/TestDbTxnManager2.java      | 610 ++++++++++++++++++-
 .../hive/ql/txn/compactor/TestCleaner.java      |   4 +
 49 files changed, 1843 insertions(+), 192 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 06a6906..bb74d99 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1769,6 +1769,8 @@ public class HiveConf extends Configuration {
       new TimeValidator(TimeUnit.MILLISECONDS), "Time delay of 1st reaper run after metastore start"),
     HIVE_TIMEDOUT_TXN_REAPER_INTERVAL("hive.timedout.txn.reaper.interval", "180s",
       new TimeValidator(TimeUnit.MILLISECONDS), "Time interval describing how often the reaper runs"),
+    WRITE_SET_REAPER_INTERVAL("hive.writeset.reaper.interval", "60s",
+      new TimeValidator(TimeUnit.MILLISECONDS), "Frequency of WriteSet reaper runs"),
 
     // For HBase storage handler
     HIVE_HBASE_WAL_ENABLED("hive.hbase.wal.enabled", true,

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
index e9ce789..22354ab 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
@@ -187,7 +187,7 @@ public class TestHiveMetaStoreTxns {
         .setDbName("mydb")
         .setTableName("mytable")
         .setPartitionName("mypartition")
-        .setExclusive()
+        .setSemiShared()
         .build())
       .addLockComponent(new LockComponentBuilder()
         .setDbName("mydb")

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/derby/035-HIVE-13395.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/035-HIVE-13395.derby.sql b/metastore/scripts/upgrade/derby/035-HIVE-13395.derby.sql
new file mode 100644
index 0000000..df33b95
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/035-HIVE-13395.derby.sql
@@ -0,0 +1,11 @@
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar(128) NOT NULL,
+  WS_TABLE varchar(128) NOT NULL,
+  WS_PARTITION varchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);
+ALTER TABLE TXN_COMPONENTS ADD TC_OPERATION_TYPE char(1);
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql b/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
index 1d00499..dc27afc 100644
--- a/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
@@ -338,7 +338,7 @@ ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SQL110318025505550" CHECK (IS_COMPRESSED
 -- ----------------------------
 -- Transaction and Lock Tables
 -- ----------------------------
-RUN 'hive-txn-schema-2.0.0.derby.sql';
+RUN 'hive-txn-schema-2.1.0.derby.sql';
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/derby/hive-txn-schema-1.3.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-txn-schema-1.3.0.derby.sql b/metastore/scripts/upgrade/derby/hive-txn-schema-1.3.0.derby.sql
index 13f3340..480c19e 100644
--- a/metastore/scripts/upgrade/derby/hive-txn-schema-1.3.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-txn-schema-1.3.0.derby.sql
@@ -32,7 +32,8 @@ 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)
+  TC_PARTITION varchar(767),
+  TC_OPERATION_TYPE char(1) NOT NULL
 );
 
 CREATE TABLE COMPLETED_TXN_COMPONENTS (
@@ -117,3 +118,11 @@ CREATE TABLE AUX_TABLE (
   PRIMARY KEY(MT_KEY1, MT_KEY2)
 );
 
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar(128) NOT NULL,
+  WS_TABLE varchar(128) NOT NULL,
+  WS_PARTITION varchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/derby/hive-txn-schema-2.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-txn-schema-2.1.0.derby.sql b/metastore/scripts/upgrade/derby/hive-txn-schema-2.1.0.derby.sql
new file mode 100644
index 0000000..11d86ca
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/hive-txn-schema-2.1.0.derby.sql
@@ -0,0 +1,130 @@
+-- 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 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,
+  TXN_AGENT_INFO varchar(128),
+  TXN_META_INFO varchar(128),
+  TXN_HEARTBEAT_COUNT integer
+);
+
+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),
+  TC_OPERATION_TYPE char(1) NOT 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),
+  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,
+  HL_HEARTBEAT_COUNT integer,
+  HL_AGENT_INFO varchar(128),
+  HL_BLOCKEDBY_EXT_ID bigint,
+  HL_BLOCKEDBY_INT_ID bigint,
+  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 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),
+  CQ_HIGHEST_TXN_ID bigint,
+  CQ_META_INFO varchar(2048) for bit data,
+  CQ_HADOOP_JOB_ID varchar(32)
+);
+
+CREATE TABLE NEXT_COMPACTION_QUEUE_ID (
+  NCQ_NEXT bigint NOT NULL
+);
+INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1);
+
+CREATE TABLE COMPLETED_COMPACTIONS (
+  CC_ID bigint PRIMARY KEY,
+  CC_DATABASE varchar(128) NOT NULL,
+  CC_TABLE varchar(128) NOT NULL,
+  CC_PARTITION varchar(767),
+  CC_STATE char(1) NOT NULL,
+  CC_TYPE char(1) NOT NULL,
+  CC_WORKER_ID varchar(128),
+  CC_START bigint,
+  CC_END bigint,
+  CC_RUN_AS varchar(128),
+  CC_HIGHEST_TXN_ID bigint,
+  CC_META_INFO varchar(2048) for bit data,
+  CC_HADOOP_JOB_ID varchar(32)
+);
+
+CREATE TABLE AUX_TABLE (
+  MT_KEY1 varchar(128) NOT NULL,
+  MT_KEY2 bigint NOT NULL,
+  MT_COMMENT varchar(255),
+  PRIMARY KEY(MT_KEY1, MT_KEY2)
+);
+
+--1st 4 cols make up a PK but since WS_PARTITION is nullable we can't declare such PK
+--This is a good candidate for Index orgainzed table
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar(128) NOT NULL,
+  WS_TABLE varchar(128) NOT NULL,
+  WS_PARTITION varchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/derby/upgrade-1.2.0-to-1.3.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-1.2.0-to-1.3.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-1.2.0-to-1.3.0.derby.sql
index 74ecac2..6b90b73 100644
--- a/metastore/scripts/upgrade/derby/upgrade-1.2.0-to-1.3.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-1.2.0-to-1.3.0.derby.sql
@@ -10,5 +10,6 @@ RUN '029-HIVE-12822.derby.sql';
 RUN '030-HIVE-12823.derby.sql';
 RUN '031-HIVE-12831.derby.sql';
 RUN '032-HIVE-12832.derby.sql';
+RUN '035-HIVE-13395.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='1.3.0', VERSION_COMMENT='Hive release version 1.3.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
index dde8c45..94c686b 100644
--- a/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
@@ -1,5 +1,6 @@
 -- Upgrade MetaStore schema from 2.0.0 to 2.1.0
 RUN '033-HIVE-12892.derby.sql';
 RUN '034-HIVE-13076.derby.sql';
+RUN '035-HIVE-13395.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/mssql/020-HIVE-13395.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/020-HIVE-13395.mssql.sql b/metastore/scripts/upgrade/mssql/020-HIVE-13395.mssql.sql
new file mode 100644
index 0000000..281014c
--- /dev/null
+++ b/metastore/scripts/upgrade/mssql/020-HIVE-13395.mssql.sql
@@ -0,0 +1,9 @@
+CREATE TABLE WRITE_SET (
+  WS_DATABASE nvarchar(128) NOT NULL,
+  WS_TABLE nvarchar(128) NOT NULL,
+  WS_PARTITION nvarchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);
+ALTER TABLE TXN_COMPONENTS ADD TC_OPERATION_TYPE char(1) NULL;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/mssql/hive-schema-1.3.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-1.3.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-1.3.0.mssql.sql
index 57d2343..a184f24 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-1.3.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-1.3.0.mssql.sql
@@ -964,7 +964,8 @@ CREATE TABLE TXN_COMPONENTS(
 	TC_TXNID bigint NULL,
 	TC_DATABASE nvarchar(128) NOT NULL,
 	TC_TABLE nvarchar(128) NULL,
-	TC_PARTITION nvarchar(767) NULL
+	TC_PARTITION nvarchar(767) NULL,
+	TC_OPERATION_TYPE char(1) NOT NULL
 );
 
 ALTER TABLE TXN_COMPONENTS  WITH CHECK ADD FOREIGN KEY(TC_TXNID) REFERENCES TXNS (TXN_ID);
@@ -980,6 +981,15 @@ CREATE TABLE AUX_TABLE (
 )
 );
 
+CREATE TABLE WRITE_SET (
+  WS_DATABASE nvarchar(128) NOT NULL,
+  WS_TABLE nvarchar(128) NOT NULL,
+  WS_PARTITION nvarchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);
+
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
index 2d9cf76..d9194ff 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
@@ -977,7 +977,8 @@ CREATE TABLE TXN_COMPONENTS(
 	TC_TXNID bigint NULL,
 	TC_DATABASE nvarchar(128) NOT NULL,
 	TC_TABLE nvarchar(128) NULL,
-	TC_PARTITION nvarchar(767) NULL
+	TC_PARTITION nvarchar(767) NULL,
+	TC_OPERATION_TYPE char(1) NOT NULL
 );
 
 ALTER TABLE TXN_COMPONENTS  WITH CHECK ADD FOREIGN KEY(TC_TXNID) REFERENCES TXNS (TXN_ID);
@@ -1011,6 +1012,15 @@ ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY (CONSTRAIN
 
 CREATE INDEX CONSTRAINTS_PARENT_TBL_ID__INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);
 
+CREATE TABLE WRITE_SET (
+  WS_DATABASE nvarchar(128) NOT NULL,
+  WS_TABLE nvarchar(128) NOT NULL,
+  WS_PARTITION nvarchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);
+
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/mssql/upgrade-1.2.0-to-1.3.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/upgrade-1.2.0-to-1.3.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-1.2.0-to-1.3.0.mssql.sql
index b0f28bb..251e621 100644
--- a/metastore/scripts/upgrade/mssql/upgrade-1.2.0-to-1.3.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-1.2.0-to-1.3.0.mssql.sql
@@ -11,6 +11,7 @@ SELECT 'Upgrading MetaStore schema from 1.2.0 to 1.3.0' AS MESSAGE;
 :r 015-HIVE-12823.mssql.sql;
 :r 016-HIVE-12831.mssql.sql;
 :r 017-HIVE-12832.mssql.sql;
+:r 020-HIVE-13395.mssql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='1.3.0', VERSION_COMMENT='Hive release version 1.3.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 1.2.0 to 1.3.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
index 3e5cb30..c796126 100644
--- a/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
@@ -2,6 +2,7 @@ SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS MESSAGE;
 
 :r 018-HIVE-12892.mssql.sql;
 :r 019-HIVE-13076.mssql.sql;
+:r 020-HIVE-13395.mssql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/mysql/035-HIVE-13395.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/035-HIVE-13395.mysql.sql b/metastore/scripts/upgrade/mysql/035-HIVE-13395.mysql.sql
new file mode 100644
index 0000000..586caef
--- /dev/null
+++ b/metastore/scripts/upgrade/mysql/035-HIVE-13395.mysql.sql
@@ -0,0 +1,10 @@
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar(128) NOT NULL,
+  WS_TABLE varchar(128) NOT NULL,
+  WS_PARTITION varchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+ALTER TABLE TXN_COMPONENTS ADD TC_OPERATION_TYPE char(1);

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
index 466e950..a6b783c 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
@@ -839,7 +839,7 @@ CREATE INDEX `CONSTRAINTS_PARENT_TABLE_ID_INDEX` ON KEY_CONSTRAINTS (`PARENT_TBL
 -- ----------------------------
 -- Transaction and Lock Tables
 -- ----------------------------
-SOURCE hive-txn-schema-2.0.0.mysql.sql;
+SOURCE hive-txn-schema-2.1.0.mysql.sql;
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/mysql/hive-txn-schema-2.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-txn-schema-2.1.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-txn-schema-2.1.0.mysql.sql
new file mode 100644
index 0000000..369d6bb
--- /dev/null
+++ b/metastore/scripts/upgrade/mysql/hive-txn-schema-2.1.0.mysql.sql
@@ -0,0 +1,131 @@
+-- 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 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,
+  TXN_AGENT_INFO varchar(128),
+  TXN_META_INFO varchar(128),
+  TXN_HEARTBEAT_COUNT int
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE TABLE TXN_COMPONENTS (
+  TC_TXNID bigint NOT NULL,
+  TC_DATABASE varchar(128) NOT NULL,
+  TC_TABLE varchar(128) NOT NULL,
+  TC_PARTITION varchar(767),
+  TC_OPERATION_TYPE char(1) NOT NULL,
+  FOREIGN KEY (TC_TXNID) REFERENCES TXNS (TXN_ID)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE TABLE COMPLETED_TXN_COMPONENTS (
+  CTC_TXNID bigint NOT NULL,
+  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,
+  HL_HEARTBEAT_COUNT int,
+  HL_AGENT_INFO varchar(128),
+  HL_BLOCKEDBY_EXT_ID bigint,
+  HL_BLOCKEDBY_INT_ID bigint,
+  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),
+  CQ_HIGHEST_TXN_ID bigint,
+  CQ_META_INFO varbinary(2048),
+  CQ_HADOOP_JOB_ID varchar(32)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE TABLE COMPLETED_COMPACTIONS (
+  CC_ID bigint PRIMARY KEY,
+  CC_DATABASE varchar(128) NOT NULL,
+  CC_TABLE varchar(128) NOT NULL,
+  CC_PARTITION varchar(767),
+  CC_STATE char(1) NOT NULL,
+  CC_TYPE char(1) NOT NULL,
+  CC_WORKER_ID varchar(128),
+  CC_START bigint,
+  CC_END bigint,
+  CC_RUN_AS varchar(128),
+  CC_HIGHEST_TXN_ID bigint,
+  CC_META_INFO varbinary(2048),
+  CC_HADOOP_JOB_ID varchar(32)
+) 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);
+
+CREATE TABLE AUX_TABLE (
+  MT_KEY1 varchar(128) NOT NULL,
+  MT_KEY2 bigint NOT NULL,
+  MT_COMMENT varchar(255),
+  PRIMARY KEY(MT_KEY1, MT_KEY2)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar(128) NOT NULL,
+  WS_TABLE varchar(128) NOT NULL,
+  WS_PARTITION varchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/mysql/upgrade-1.2.0-to-1.3.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/upgrade-1.2.0-to-1.3.0.mysql.sql b/metastore/scripts/upgrade/mysql/upgrade-1.2.0-to-1.3.0.mysql.sql
index 477c10b..b65aee5 100644
--- a/metastore/scripts/upgrade/mysql/upgrade-1.2.0-to-1.3.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-1.2.0-to-1.3.0.mysql.sql
@@ -11,6 +11,7 @@ SOURCE 029-HIVE-12822.mysql.sql;
 SOURCE 030-HIVE-12823.mysql.sql;
 SOURCE 031-HIVE-12831.mysql.sql;
 SOURCE 032-HIVE-12832.mysql.sql;
+SOURCE 035-HIVE-13395.mysql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='1.3.0', VERSION_COMMENT='Hive release version 1.3.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 1.2.0 to 1.3.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql b/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
index eb21f73..c3f83b3 100644
--- a/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
@@ -2,6 +2,7 @@ SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS ' ';
 
 SOURCE 033-HIVE-12892.mysql.sql;
 SOURCE 034-HIVE-13076.mysql.sql;
+SOURCE 035-HIVE-12295.mysql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/oracle/035-HIVE-13395.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/035-HIVE-13395.oracle.sql b/metastore/scripts/upgrade/oracle/035-HIVE-13395.oracle.sql
new file mode 100644
index 0000000..ad1bbd9
--- /dev/null
+++ b/metastore/scripts/upgrade/oracle/035-HIVE-13395.oracle.sql
@@ -0,0 +1,10 @@
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar2(128) NOT NULL,
+  WS_TABLE varchar2(128) NOT NULL,
+  WS_PARTITION varchar2(767),
+  WS_TXNID number(19) NOT NULL,
+  WS_COMMIT_ID number(19) NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);
+
+ALTER TABLE TXN_COMPONENTS ADD TC_OPERATION_TYPE char(1);

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
index f57e588..d003a16 100644
--- a/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
@@ -808,7 +808,7 @@ CREATE INDEX CONSTRAINTS_PARENT_TBL_ID_INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);
 ------------------------------
 -- Transaction and lock tables
 ------------------------------
-@hive-txn-schema-2.0.0.oracle.sql;
+@hive-txn-schema-2.1.0.oracle.sql;
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/oracle/hive-txn-schema-1.3.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-txn-schema-1.3.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-txn-schema-1.3.0.oracle.sql
index 788741a..199ff4c 100644
--- a/metastore/scripts/upgrade/oracle/hive-txn-schema-1.3.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-txn-schema-1.3.0.oracle.sql
@@ -33,7 +33,8 @@ 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
+  TC_PARTITION VARCHAR2(767) NULL,
+  TC_OPERATION_TYPE char(1) NOT NULL
 ) ROWDEPENDENCIES;
 
 CREATE TABLE COMPLETED_TXN_COMPONENTS (
@@ -118,3 +119,12 @@ CREATE TABLE AUX_TABLE (
   PRIMARY KEY(MT_KEY1, MT_KEY2)
 );
 
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar2(128) NOT NULL,
+  WS_TABLE varchar2(128) NOT NULL,
+  WS_PARTITION varchar2(767),
+  WS_TXNID number(19) NOT NULL,
+  WS_COMMIT_ID number(19) NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);
+

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/oracle/hive-txn-schema-2.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-txn-schema-2.1.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-txn-schema-2.1.0.oracle.sql
new file mode 100644
index 0000000..d39baab
--- /dev/null
+++ b/metastore/scripts/upgrade/oracle/hive-txn-schema-2.1.0.oracle.sql
@@ -0,0 +1,129 @@
+-- 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,
+  TXN_AGENT_INFO varchar2(128),
+  TXN_META_INFO varchar2(128),
+  TXN_HEARTBEAT_COUNT number(10)
+) ROWDEPENDENCIES;
+
+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,
+  TC_OPERATION_TYPE char(1) NOT NULL
+) ROWDEPENDENCIES;
+
+CREATE TABLE COMPLETED_TXN_COMPONENTS (
+  CTC_TXNID NUMBER(19),
+  CTC_DATABASE varchar(128) NOT NULL,
+  CTC_TABLE varchar(128),
+  CTC_PARTITION varchar(767)
+) ROWDEPENDENCIES;
+
+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,
+  HL_HEARTBEAT_COUNT number(10),
+  HL_AGENT_INFO varchar2(128),
+  HL_BLOCKEDBY_EXT_ID number(19),
+  HL_BLOCKEDBY_INT_ID number(19),
+  PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID)
+) ROWDEPENDENCIES;
+
+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),
+  CQ_HIGHEST_TXN_ID NUMBER(19),
+  CQ_META_INFO BLOB,
+  CQ_HADOOP_JOB_ID varchar2(32)
+) ROWDEPENDENCIES;
+
+CREATE TABLE NEXT_COMPACTION_QUEUE_ID (
+  NCQ_NEXT NUMBER(19) NOT NULL
+);
+INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1);
+
+CREATE TABLE COMPLETED_COMPACTIONS (
+  CC_ID NUMBER(19) PRIMARY KEY,
+  CC_DATABASE varchar(128) NOT NULL,
+  CC_TABLE varchar(128) NOT NULL,
+  CC_PARTITION varchar(767),
+  CC_STATE char(1) NOT NULL,
+  CC_TYPE char(1) NOT NULL,
+  CC_WORKER_ID varchar(128),
+  CC_START NUMBER(19),
+  CC_END NUMBER(19),
+  CC_RUN_AS varchar(128),
+  CC_HIGHEST_TXN_ID NUMBER(19),
+  CC_META_INFO BLOB,
+  CC_HADOOP_JOB_ID varchar2(32)
+) ROWDEPENDENCIES;
+
+CREATE TABLE AUX_TABLE (
+  MT_KEY1 varchar2(128) NOT NULL,
+  MT_KEY2 number(19) NOT NULL,
+  MT_COMMENT varchar2(255),
+  PRIMARY KEY(MT_KEY1, MT_KEY2)
+);
+
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar2(128) NOT NULL,
+  WS_TABLE varchar2(128) NOT NULL,
+  WS_PARTITION varchar2(767),
+  WS_TXNID number(19) NOT NULL,
+  WS_COMMIT_ID number(19) NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/oracle/upgrade-1.2.0-to-1.3.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/upgrade-1.2.0-to-1.3.0.oracle.sql b/metastore/scripts/upgrade/oracle/upgrade-1.2.0-to-1.3.0.oracle.sql
index 94ee2c4..5939b34 100644
--- a/metastore/scripts/upgrade/oracle/upgrade-1.2.0-to-1.3.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-1.2.0-to-1.3.0.oracle.sql
@@ -11,6 +11,7 @@ SELECT 'Upgrading MetaStore schema from 1.2.0 to 1.3.0' AS Status from dual;
 @030-HIVE-12823.oracle.sql;
 @031-HIVE-12381.oracle.sql;
 @032-HIVE-12832.oracle.sql;
+@035-HIVE-13395.oracle.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='1.3.0', VERSION_COMMENT='Hive release version 1.3.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 1.2.0 to 1.3.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql b/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
index 8c065a1..a226d9a 100644
--- a/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
@@ -2,6 +2,7 @@ SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS Status from dual;
 
 @033-HIVE-12892.oracle.sql;
 @034-HIVE-13076.oracle.sql;
+@035-HIVE-13395.oracle.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/postgres/034-HIVE-13395.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/034-HIVE-13395.postgres.sql b/metastore/scripts/upgrade/postgres/034-HIVE-13395.postgres.sql
new file mode 100644
index 0000000..4dda283
--- /dev/null
+++ b/metastore/scripts/upgrade/postgres/034-HIVE-13395.postgres.sql
@@ -0,0 +1,10 @@
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar(128) NOT NULL,
+  WS_TABLE varchar(128) NOT NULL,
+  WS_PARTITION varchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);
+
+ALTER TABLE TXN_COMPONENTS ADD TC_OPERATION_TYPE char(1);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
index e209489..43e984c 100644
--- a/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
@@ -1480,7 +1480,7 @@ GRANT ALL ON SCHEMA public TO PUBLIC;
 ------------------------------
 -- Transaction and lock tables
 ------------------------------
-\i hive-txn-schema-2.0.0.postgres.sql;
+\i hive-txn-schema-2.1.0.postgres.sql;
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/postgres/hive-txn-schema-1.3.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-txn-schema-1.3.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-txn-schema-1.3.0.postgres.sql
index b2fc1a8..b606f81 100644
--- a/metastore/scripts/upgrade/postgres/hive-txn-schema-1.3.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-txn-schema-1.3.0.postgres.sql
@@ -33,7 +33,8 @@ 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
+  TC_PARTITION varchar(767) DEFAULT NULL,
+  TC_OPERATION_TYPE char(1) NOT NULL
 );
 
 CREATE TABLE COMPLETED_TXN_COMPONENTS (
@@ -118,4 +119,12 @@ CREATE TABLE AUX_TABLE (
   PRIMARY KEY(MT_KEY1, MT_KEY2)
 );
 
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar(128) NOT NULL,
+  WS_TABLE varchar(128) NOT NULL,
+  WS_PARTITION varchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/postgres/hive-txn-schema-2.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-txn-schema-2.1.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-txn-schema-2.1.0.postgres.sql
new file mode 100644
index 0000000..262b93e
--- /dev/null
+++ b/metastore/scripts/upgrade/postgres/hive-txn-schema-2.1.0.postgres.sql
@@ -0,0 +1,129 @@
+-- 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 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,
+  TXN_AGENT_INFO varchar(128),
+  TXN_META_INFO varchar(128),
+  TXN_HEARTBEAT_COUNT integer
+);
+
+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,
+  TC_OPERATION_TYPE char(1) NOT 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,
+  HL_HEARTBEAT_COUNT integer,
+  HL_AGENT_INFO varchar(128),
+  HL_BLOCKEDBY_EXT_ID bigint,
+  HL_BLOCKEDBY_INT_ID bigint,
+  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),
+  CQ_HIGHEST_TXN_ID bigint,
+  CQ_META_INFO bytea,
+  CQ_HADOOP_JOB_ID varchar(32)
+);
+
+CREATE TABLE NEXT_COMPACTION_QUEUE_ID (
+  NCQ_NEXT bigint NOT NULL
+);
+INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1);
+
+CREATE TABLE COMPLETED_COMPACTIONS (
+  CC_ID bigint PRIMARY KEY,
+  CC_DATABASE varchar(128) NOT NULL,
+  CC_TABLE varchar(128) NOT NULL,
+  CC_PARTITION varchar(767),
+  CC_STATE char(1) NOT NULL,
+  CC_TYPE char(1) NOT NULL,
+  CC_WORKER_ID varchar(128),
+  CC_START bigint,
+  CC_END bigint,
+  CC_RUN_AS varchar(128),
+  CC_HIGHEST_TXN_ID bigint,
+  CC_META_INFO bytea,
+  CC_HADOOP_JOB_ID varchar(32)
+);
+
+CREATE TABLE AUX_TABLE (
+  MT_KEY1 varchar(128) NOT NULL,
+  MT_KEY2 bigint NOT NULL,
+  MT_COMMENT varchar(255),
+  PRIMARY KEY(MT_KEY1, MT_KEY2)
+);
+
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar(128) NOT NULL,
+  WS_TABLE varchar(128) NOT NULL,
+  WS_PARTITION varchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+);

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/postgres/upgrade-1.2.0-to-1.3.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/upgrade-1.2.0-to-1.3.0.postgres.sql b/metastore/scripts/upgrade/postgres/upgrade-1.2.0-to-1.3.0.postgres.sql
index 6eb5620..b1bcac0 100644
--- a/metastore/scripts/upgrade/postgres/upgrade-1.2.0-to-1.3.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-1.2.0-to-1.3.0.postgres.sql
@@ -11,6 +11,7 @@ SELECT 'Upgrading MetaStore schema from 1.2.0 to 1.3.0';
 \i 029-HIVE-12823.postgres.sql;
 \i 030-HIVE-12831.postgres.sql;
 \i 031-HIVE-12832.postgres.sql;
+\i 034-HIVE-13395.postgres.sql;
 
 UPDATE "VERSION" SET "SCHEMA_VERSION"='1.3.0', "VERSION_COMMENT"='Hive release version 1.3.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 1.2.0 to 1.3.0';

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql b/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
index e96a6ec..7fc603f 100644
--- a/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
@@ -2,6 +2,7 @@ SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0';
 
 \i 032-HIVE-12892.postgres.sql;
 \i 033-HIVE-13076.postgres.sql;
+\i 034-HIVE-13395.postgres.sql;
 
 UPDATE "VERSION" SET "SCHEMA_VERSION"='2.1.0', "VERSION_COMMENT"='Hive release version 2.1.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0';

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 9a09e7a..044b960 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -6748,6 +6748,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
     startHouseKeeperService(conf, Class.forName("org.apache.hadoop.hive.ql.txn.AcidHouseKeeperService"));
     startHouseKeeperService(conf, Class.forName("org.apache.hadoop.hive.ql.txn.AcidCompactionHistoryService"));
+    startHouseKeeperService(conf, Class.forName("org.apache.hadoop.hive.ql.txn.AcidWriteSetService"));
   }
   private static void startHouseKeeperService(HiveConf conf, Class c) throws Exception {
     //todo: when metastore adds orderly-shutdown logic, houseKeeper.stop()

http://git-wip-us.apache.org/repos/asf/hive/blob/10d05491/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
index c82d23a..facce54 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
@@ -21,11 +21,13 @@ import java.sql.Connection;
 import java.sql.Driver;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.SQLTransactionRollbackException;
 import java.sql.Statement;
 import java.util.Properties;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -82,7 +84,8 @@ public final class TxnDbUtil {
           "  TC_TXNID bigint REFERENCES TXNS (TXN_ID)," +
           "  TC_DATABASE varchar(128) NOT NULL," +
           "  TC_TABLE varchar(128)," +
-          "  TC_PARTITION varchar(767))");
+          "  TC_PARTITION varchar(767)," +
+          "  TC_OPERATION_TYPE char(1) NOT NULL)");
       stmt.execute("CREATE TABLE COMPLETED_TXN_COMPONENTS (" +
           "  CTC_TXNID bigint," +
           "  CTC_DATABASE varchar(128) NOT NULL," +
@@ -146,18 +149,24 @@ public final class TxnDbUtil {
         " CC_HADOOP_JOB_ID varchar(32))");
       
       stmt.execute("CREATE TABLE AUX_TABLE (" +
-        "  MT_KEY1 varchar(128) NOT NULL," +
-        "  MT_KEY2 bigint NOT NULL," +
-        "  MT_COMMENT varchar(255)," +
-        "  PRIMARY KEY(MT_KEY1, MT_KEY2)" +
-        ")");
-
-      conn.commit();
+        " MT_KEY1 varchar(128) NOT NULL," +
+        " MT_KEY2 bigint NOT NULL," +
+        " MT_COMMENT varchar(255)," +
+        " PRIMARY KEY(MT_KEY1, MT_KEY2))");
+      
+      stmt.execute("CREATE TABLE WRITE_SET (" +
+        " WS_DATABASE varchar(128) NOT NULL," +
+        " WS_TABLE varchar(128) NOT NULL," +
+        " WS_PARTITION varchar(767)," +
+        " WS_TXNID bigint NOT NULL," +
+        " WS_COMMIT_ID bigint NOT NULL," +
+        " WS_OPERATION_TYPE char(1) NOT NULL)"
+      );
     } catch (SQLException e) {
       try {
         conn.rollback();
       } catch (SQLException re) {
-        System.err.println("Error rolling back: " + re.getMessage());
+        LOG.error("Error rolling back: " + re.getMessage());
       }
 
       // This might be a deadlock, if so, let's retry
@@ -174,41 +183,60 @@ public final class TxnDbUtil {
   }
 
   public static void cleanDb() throws Exception {
-    Connection conn = null;
-    Statement stmt = null;
-    try {
-      conn = getConnection();
-      stmt = conn.createStatement();
-
-      // We want to try these, whether they succeed or fail.
+    int retryCount = 0;
+    while(++retryCount <= 3) {
+      boolean success = true;
+      Connection conn = null;
+      Statement stmt = null;
       try {
-        stmt.execute("DROP INDEX HL_TXNID_INDEX");
-      } catch (Exception e) {
-        System.err.println("Unable to drop index HL_TXNID_INDEX " + e.getMessage());
-      }
+        conn = getConnection();
+        stmt = conn.createStatement();
 
-      dropTable(stmt, "TXN_COMPONENTS");
-      dropTable(stmt, "COMPLETED_TXN_COMPONENTS");
-      dropTable(stmt, "TXNS");
-      dropTable(stmt, "NEXT_TXN_ID");
-      dropTable(stmt, "HIVE_LOCKS");
-      dropTable(stmt, "NEXT_LOCK_ID");
-      dropTable(stmt, "COMPACTION_QUEUE");
-      dropTable(stmt, "NEXT_COMPACTION_QUEUE_ID");
-      dropTable(stmt, "COMPLETED_COMPACTIONS");
-      dropTable(stmt, "AUX_TABLE");
-      conn.commit();
-    } finally {
-      closeResources(conn, stmt, null);
+        // We want to try these, whether they succeed or fail.
+        try {
+          stmt.execute("DROP INDEX HL_TXNID_INDEX");
+        } catch (SQLException e) {
+          if(!("42X65".equals(e.getSQLState()) && 30000 == e.getErrorCode())) {
+            //42X65/3000 means index doesn't exist
+            LOG.error("Unable to drop index HL_TXNID_INDEX " + e.getMessage() +
+              "State=" + e.getSQLState() + " code=" + e.getErrorCode() + " retryCount=" + retryCount);
+            success = false;
+          }
+        }
+
+        success &= dropTable(stmt, "TXN_COMPONENTS", retryCount);
+        success &= dropTable(stmt, "COMPLETED_TXN_COMPONENTS", retryCount);
+        success &= dropTable(stmt, "TXNS", retryCount);
+        success &= dropTable(stmt, "NEXT_TXN_ID", retryCount);
+        success &= dropTable(stmt, "HIVE_LOCKS", retryCount);
+        success &= dropTable(stmt, "NEXT_LOCK_ID", retryCount);
+        success &= dropTable(stmt, "COMPACTION_QUEUE", retryCount);
+        success &= dropTable(stmt, "NEXT_COMPACTION_QUEUE_ID", retryCount);
+        success &= dropTable(stmt, "COMPLETED_COMPACTIONS", retryCount);
+        success &= dropTable(stmt, "AUX_TABLE", retryCount);
+        success &= dropTable(stmt, "WRITE_SET", retryCount);
+      } finally {
+        closeResources(conn, stmt, null);
+      }
+      if(success) {
+        return;
+      }
     }
   }
 
-  private static void dropTable(Statement stmt, String name) {
+  private static boolean dropTable(Statement stmt, String name, int retryCount) throws SQLException {
     try {
       stmt.execute("DROP TABLE " + name);
-    } catch (Exception e) {
-      System.err.println("Unable to drop table " + name + ": " + e.getMessage());
+      return true;
+    } catch (SQLException e) {
+      if("42Y55".equals(e.getSQLState()) && 30000 == e.getErrorCode()) {
+        //failed because object doesn't exist
+        return true;
+      }
+      LOG.error("Unable to drop table " + name + ": " + e.getMessage() +
+        " State=" + e.getSQLState() + " code=" + e.getErrorCode() + " retryCount=" + retryCount);
     }
+    return false;
   }
 
   /**
@@ -259,6 +287,32 @@ public final class TxnDbUtil {
       closeResources(conn, stmt, rs);
     }
   }
+  @VisibleForTesting
+  public static String queryToString(String query) throws Exception {
+    Connection conn = null;
+    Statement stmt = null;
+    ResultSet rs = null;
+    StringBuilder sb = new StringBuilder();
+    try {
+      conn = getConnection();
+      stmt = conn.createStatement();
+      rs = stmt.executeQuery(query);
+      ResultSetMetaData rsmd = rs.getMetaData();
+      for(int colPos = 1; colPos <= rsmd.getColumnCount(); colPos++) {
+        sb.append(rsmd.getColumnName(colPos)).append("   ");
+      }
+      sb.append('\n');
+      while(rs.next()) {
+        for (int colPos = 1; colPos <= rsmd.getColumnCount(); colPos++) {
+          sb.append(rs.getObject(colPos)).append("   ");
+        }
+        sb.append('\n');
+      }
+    } finally {
+      closeResources(conn, stmt, rs);
+    }
+    return sb.toString();
+  }
 
   static Connection getConnection() throws Exception {
     HiveConf conf = new HiveConf();
@@ -272,7 +326,7 @@ public final class TxnDbUtil {
     prop.setProperty("user", user);
     prop.setProperty("password", passwd);
     Connection conn = driver.connect(driverUrl, prop);
-    conn.setAutoCommit(false);
+    conn.setAutoCommit(true);
     return conn;
   }
 
@@ -281,7 +335,7 @@ public final class TxnDbUtil {
       try {
         rs.close();
       } catch (SQLException e) {
-        System.err.println("Error closing ResultSet: " + e.getMessage());
+        LOG.error("Error closing ResultSet: " + e.getMessage());
       }
     }
 


[41/50] [abbrv] hive git commit: HIVE-13395 (addednum) Lost Update problem in ACID (Eugene Koifman, reviewed by Alan Gates)

Posted by sp...@apache.org.
HIVE-13395 (addednum) Lost Update problem in ACID (Eugene Koifman, reviewed by Alan Gates)


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

Branch: refs/heads/java8
Commit: eb2c54b3f80d958c36c22dfb0ee962806e673830
Parents: 794f161
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Thu May 5 15:29:00 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Thu May 5 15:29:00 2016 -0700

----------------------------------------------------------------------
 .../scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/eb2c54b3/metastore/scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql
index ea42757..d873012 100644
--- a/metastore/scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql
@@ -34,6 +34,7 @@ CREATE TABLE TXN_COMPONENTS (
   TC_DATABASE varchar(128) NOT NULL,
   TC_TABLE varchar(128),
   TC_PARTITION varchar(767),
+  TC_OPERATION_TYPE char(1) NOT NULL,
   FOREIGN KEY (TC_TXNID) REFERENCES TXNS (TXN_ID)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
@@ -120,3 +121,12 @@ CREATE TABLE AUX_TABLE (
   PRIMARY KEY(MT_KEY1, MT_KEY2)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
+CREATE TABLE WRITE_SET (
+  WS_DATABASE varchar(128) NOT NULL,
+  WS_TABLE varchar(128) NOT NULL,
+  WS_PARTITION varchar(767),
+  WS_TXNID bigint NOT NULL,
+  WS_COMMIT_ID bigint NOT NULL,
+  WS_OPERATION_TYPE char(1) NOT NULL
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+


[23/50] [abbrv] hive git commit: HIVE-13660: Vectorizing IN expression with list of columns throws java.lang.ClassCastException ExprNodeColumnDesc cannot be cast to ExprNodeConstantDesc (Matt McCline, reviewed by Prasanth Jayachandran)

Posted by sp...@apache.org.
HIVE-13660: Vectorizing IN expression with list of columns throws java.lang.ClassCastException ExprNodeColumnDesc cannot be cast to ExprNodeConstantDesc (Matt McCline, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/java8
Commit: e68783c8e5cdb0cc00db6d725f15392bd5a6fe06
Parents: 652f88a
Author: Matt McCline <mm...@hortonworks.com>
Authored: Wed May 4 14:59:00 2016 -0700
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Wed May 4 14:59:30 2016 -0700

----------------------------------------------------------------------
 .../ql/exec/vector/VectorizationContext.java    |  7 ++++
 .../vector_non_constant_in_expr.q               |  4 +++
 .../vector_non_constant_in_expr.q.out           | 36 ++++++++++++++++++++
 3 files changed, 47 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e68783c8/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 5454ba3..9558d31 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -1519,6 +1519,13 @@ public class VectorizationContext {
 
     VectorExpression expr = null;
 
+    // Validate the IN items are only constants.
+    for (ExprNodeDesc inListChild : childrenForInList) {
+      if (!(inListChild instanceof ExprNodeConstantDesc)) {
+        throw new HiveException("Vectorizing IN expression only supported for constant values");
+      }
+    }
+
     // determine class
     Class<?> cl = null;
     if (isIntFamily(colType)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/e68783c8/ql/src/test/queries/clientpositive/vector_non_constant_in_expr.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_non_constant_in_expr.q b/ql/src/test/queries/clientpositive/vector_non_constant_in_expr.q
new file mode 100644
index 0000000..69142bf
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_non_constant_in_expr.q
@@ -0,0 +1,4 @@
+SET hive.vectorized.execution.enabled=true;
+set hive.fetch.task.conversion=none;
+
+explain SELECT * FROM alltypesorc WHERE cint in (ctinyint, cbigint);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/e68783c8/ql/src/test/results/clientpositive/vector_non_constant_in_expr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_non_constant_in_expr.q.out b/ql/src/test/results/clientpositive/vector_non_constant_in_expr.q.out
new file mode 100644
index 0000000..8845cb2
--- /dev/null
+++ b/ql/src/test/results/clientpositive/vector_non_constant_in_expr.q.out
@@ -0,0 +1,36 @@
+PREHOOK: query: explain SELECT * FROM alltypesorc WHERE cint in (ctinyint, cbigint)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain SELECT * FROM alltypesorc WHERE cint in (ctinyint, cbigint)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: alltypesorc
+            Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (cint) IN (ctinyint, cbigint) (type: boolean)
+              Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+


[32/50] [abbrv] hive git commit: HIVE-13620: Merge llap branch work to master (committing changes from review feedback)

Posted by sp...@apache.org.
HIVE-13620: Merge llap branch work to master (committing changes from review feedback)


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

Branch: refs/heads/java8
Commit: 2a03f1f4648c683414c0b23be0aebbfd614d105c
Parents: e057909
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu May 5 12:29:14 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu May 5 12:29:14 2016 -0700

----------------------------------------------------------------------
 .../hive/llap/ext/TestLlapInputSplit.java       |  18 ++
 .../apache/hive/jdbc/TestJdbcWithMiniLlap.java  |  37 +----
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   3 +-
 .../hadoop/hive/llap/LlapBaseRecordReader.java  |  38 +++--
 .../hadoop/hive/llap/LlapRowRecordReader.java   |  48 ++++--
 .../apache/hadoop/hive/llap/SubmitWorkInfo.java |  16 ++
 .../ext/LlapTaskUmbilicalExternalClient.java    |  46 +++---
 .../helpers/LlapTaskUmbilicalServer.java        |  16 ++
 .../hadoop/hive/llap/LlapBaseInputFormat.java   |  20 +--
 .../org/apache/hadoop/hive/llap/LlapDump.java   |  11 +-
 .../hadoop/hive/llap/LlapRowInputFormat.java    |  18 ++
 .../hive/llap/daemon/impl/LlapDaemon.java       |   2 +-
 .../llap/daemon/impl/TaskRunnerCallable.java    |   6 +-
 .../daemon/impl/TaskExecutorTestHelpers.java    |   2 +-
 .../hadoop/hive/llap/LlapDataOutputBuffer.java  | 165 -------------------
 .../hive/llap/LlapOutputFormatService.java      |  27 +--
 .../hive/ql/exec/tez/HiveSplitGenerator.java    |   4 +-
 .../hive/ql/io/HivePassThroughRecordWriter.java |   4 -
 .../hive/ql/parse/TypeCheckProcFactory.java     |   9 +-
 .../ql/udf/generic/GenericUDTFGetSplits.java    |   1 -
 .../org/apache/tez/dag/api/TaskSpecBuilder.java |  17 +-
 .../hadoop/hive/llap/TestLlapOutputFormat.java  |   2 +-
 .../results/clientpositive/show_functions.q.out |   1 +
 23 files changed, 209 insertions(+), 302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
index 8264190..1de8aa6 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.hive.llap.ext;
 
 import java.io.ByteArrayInputStream;

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
index 5b4ba49..48b9493 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
@@ -161,39 +161,7 @@ public class TestJdbcWithMiniLlap {
     stmt.close();
   }
 
-  private static boolean timedOut = false;
-
-  private static class TestTimerTask extends TimerTask {
-    private boolean timedOut = false;
-    private Thread threadToInterrupt;
-
-    public TestTimerTask(Thread threadToInterrupt) {
-      this.threadToInterrupt = threadToInterrupt;
-    }
-
-    @Override
-    public void run() {
-      System.out.println("Test timed out!");
-      timedOut = true;
-      threadToInterrupt.interrupt();
-    }
-
-    public boolean isTimedOut() {
-      return timedOut;
-    }
-
-    public void setTimedOut(boolean timedOut) {
-      this.timedOut = timedOut;
-    }
-
-  }
-
   private int getLlapIFRowCount(String query, int numSplits) throws Exception {
-    // Add a timer task to stop this test if it has not finished in a reasonable amount of time.
-    Timer timer = new Timer();
-    long delay = 30000;
-    TestTimerTask timerTask = new TestTimerTask(Thread.currentThread());
-    timer.schedule(timerTask, delay);
 
     // Setup LlapInputFormat
     String url = miniHS2.getJdbcURL();
@@ -245,13 +213,10 @@ public class TestJdbcWithMiniLlap {
       }
     }
 
-    timer.cancel();
-    assertFalse("Test timed out", timerTask.isTimedOut());
-
     return rowCount;
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testLlapInputFormatEndToEnd() throws Exception {
     createTestTable("testtab1");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 3f8c3c5..a6e8efa 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -1468,8 +1468,7 @@ public class QTestUtil {
       ".*Input:.*/data/files/.*",
       ".*Output:.*/data/files/.*",
       ".*total number of created files now is.*",
-      ".*.hive-staging.*",
-      "table_.*"
+      ".*.hive-staging.*"
   });
 
   private final Pattern[] partialReservedPlanMask = toPattern(new String[] {

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
index 0cd9672..3c858a8 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
@@ -23,8 +23,10 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.DataInputStream;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.llap.Schema;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -37,22 +39,27 @@ import org.apache.hadoop.mapred.JobConf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * Base LLAP RecordReader to handle receiving of the data from the LLAP daemon.
+ */
 public class LlapBaseRecordReader<V extends WritableComparable> implements RecordReader<NullWritable, V> {
   private static final Logger LOG = LoggerFactory.getLogger(LlapBaseRecordReader.class);
 
-  DataInputStream din;
-  Schema schema;
-  Class<V> clazz;
-
+  protected final DataInputStream din;
+  protected final Schema schema;
+  protected final Class<V> clazz;
 
   protected Thread readerThread = null;
-  protected LinkedBlockingQueue<ReaderEvent> readerEvents = new LinkedBlockingQueue<ReaderEvent>();
+  protected final LinkedBlockingQueue<ReaderEvent> readerEvents = new LinkedBlockingQueue<ReaderEvent>();
+  protected final long timeout;
 
-  public LlapBaseRecordReader(InputStream in, Schema schema, Class<V> clazz) {
+  public LlapBaseRecordReader(InputStream in, Schema schema, Class<V> clazz, JobConf job) {
     din = new DataInputStream(in);
     this.schema = schema;
     this.clazz = clazz;
     this.readerThread = Thread.currentThread();
+    this.timeout = 3 * HiveConf.getTimeVar(job,
+        HiveConf.ConfVars.LLAP_DAEMON_AM_LIVENESS_CONNECTION_TIMEOUT_MS, TimeUnit.MILLISECONDS);
   }
 
   public Schema getSchema() {
@@ -65,10 +72,16 @@ public class LlapBaseRecordReader<V extends WritableComparable> implements Recor
   }
 
   @Override
-  public long getPos() { return 0; }
+  public long getPos() {
+    // dummy impl
+    return 0;
+  }
 
   @Override
-  public float getProgress() { return 0f; }
+  public float getProgress() {
+    // dummy impl
+    return 0f;
+  }
 
   @Override
   public NullWritable createKey() {
@@ -106,7 +119,7 @@ public class LlapBaseRecordReader<V extends WritableComparable> implements Recor
     } catch (IOException io) {
       if (Thread.interrupted()) {
         // Either we were interrupted by one of:
-        // 1. handleEvent(), in which case there is a reader event waiting for us in the queue
+        // 1. handleEvent(), in which case there is a reader (error) event waiting for us in the queue
         // 2. Some other unrelated cause which interrupted us, in which case there may not be a reader event coming.
         // Either way we should not try to block trying to read the reader events queue.
         if (readerEvents.isEmpty()) {
@@ -186,9 +199,12 @@ public class LlapBaseRecordReader<V extends WritableComparable> implements Recor
     }
   }
 
-  protected ReaderEvent getReaderEvent() {
+  protected ReaderEvent getReaderEvent() throws IOException {
     try {
-      ReaderEvent event = readerEvents.take();
+      ReaderEvent event = readerEvents.poll(timeout, TimeUnit.MILLISECONDS);
+      if (event == null) {
+        throw new IOException("Timed out getting readerEvents");
+      }
       return event;
     } catch (InterruptedException ie) {
       throw new RuntimeException("Interrupted while getting readerEvents, not expected: " + ie.getMessage(), ie);

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
index 4e000ff..084da0a 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.hive.llap;
 
 import com.google.common.base.Preconditions;
@@ -32,21 +50,29 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
+/**
+ * Row-based record reader for LLAP.
+ */
 public class LlapRowRecordReader implements RecordReader<NullWritable, Row> {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapRowRecordReader.class);
 
-  Configuration conf;
-  RecordReader<NullWritable, Text> reader;
-  Schema schema;
-  SerDe serde;
-  final Text textData = new Text();
+  protected final Configuration conf;
+  protected final RecordReader<NullWritable, Text> reader;
+  protected final Schema schema;
+  protected final SerDe serde;
+  protected final Text textData = new Text();
 
-  public LlapRowRecordReader(Configuration conf, Schema schema, RecordReader<NullWritable, Text> reader) {
+  public LlapRowRecordReader(Configuration conf, Schema schema, RecordReader<NullWritable, Text> reader) throws IOException {
     this.conf = conf;
     this.schema = schema;
     this.reader = reader;
+
+    try {
+      serde = initSerDe(conf);
+    } catch (SerDeException err) {
+      throw new IOException(err);
+    }
   }
 
   @Override
@@ -78,14 +104,6 @@ public class LlapRowRecordReader implements RecordReader<NullWritable, Row> {
   public boolean next(NullWritable key, Row value) throws IOException {
     Preconditions.checkArgument(value != null);
 
-    if (serde == null) {
-      try {
-        serde = initSerDe(conf);
-      } catch (SerDeException err) {
-        throw new IOException(err);
-      }
-    }
-
     boolean hasNext = reader.next(key,  textData);
     if (hasNext) {
       // Deserialize Text to column values, and populate the row record

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java b/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
index 83149ab..6704294 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hive.llap;
 
 import java.io.DataInput;

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java b/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
index 6e2c85d..0edb1cd 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hive.llap.ext;
 
 import java.io.IOException;
@@ -83,10 +99,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
     }
   }
 
-  // TODO KKK Work out the details of the tokenIdentifier, and the session token.
-  // It may just be possible to create one here - since Shuffle is not involved, and this is only used
-  // for communication from LLAP-Daemons to the server. It will need to be sent in as part
-  // of the job submission request.
   public LlapTaskUmbilicalExternalClient(Configuration conf, String tokenIdentifier,
       Token<JobTokenIdentifier> sessionToken, LlapTaskUmbilicalExternalResponder responder) {
     super(LlapTaskUmbilicalExternalClient.class.getName());
@@ -96,9 +108,9 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
     this.sessionToken = sessionToken;
     this.responder = responder;
     this.timer = new ScheduledThreadPoolExecutor(1);
-    this.connectionTimeout = HiveConf.getTimeVar(conf,
+    this.connectionTimeout = 3 * HiveConf.getTimeVar(conf,
         HiveConf.ConfVars.LLAP_DAEMON_AM_LIVENESS_CONNECTION_TIMEOUT_MS, TimeUnit.MILLISECONDS);
-    // TODO. No support for the LLAP token yet. Add support for configurable threads, however 1 should always be enough.
+    // No support for the LLAP token yet. Add support for configurable threads, however 1 should always be enough.
     this.communicator = new LlapProtocolClientProxy(1, conf, null);
     this.communicator.init(conf);
   }
@@ -173,24 +185,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
             responder.submissionFailed(fragmentId, err);
           }
         });
-
-
-
-
-
-//    // TODO Also send out information saying that the fragment is finishable - if that is not already included in the main fragment.
-//    // This entire call is only required if we're doing more than scans. MRInput has no dependencies and is always finishable
-//    QueryIdentifierProto queryIdentifier = QueryIdentifierProto
-//        .newBuilder()
-//        .setAppIdentifier(submitWorkRequestProto.getApplicationIdString()).setDagIdentifier(submitWorkRequestProto.getFragmentSpec().getDagId())
-//        .build();
-//    LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto sourceStateUpdatedRequest =
-//        LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto.newBuilder().setQueryIdentifier(queryIdentifier).setState(
-//            LlapDaemonProtocolProtos.SourceStateProto.S_SUCCEEDED).
-//            setSrcName(TODO)
-//    communicator.sendSourceStateUpdate(LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto.newBuilder().setQueryIdentifier(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString()).set);
-
-
   }
 
   private void updateHeartbeatInfo(String taskAttemptId) {
@@ -261,7 +255,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
         LOG.info("Pending taskAttemptId " + timedOutTask + " timed out");
         responder.heartbeatTimeout(timedOutTask);
         pendingEvents.remove(timedOutTask);
-        // TODO: Do we need to tell the LLAP daemon we are no longer interested in this task?
       }
 
       timedOutTasks.clear();
@@ -277,7 +270,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
         LOG.info("Running taskAttemptId " + timedOutTask + " timed out");
         responder.heartbeatTimeout(timedOutTask);
         registeredTasks.remove(timedOutTask);
-        // TODO: Do we need to tell the LLAP daemon we are no longer interested in this task?
       }
     }
   }
@@ -291,7 +283,7 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
 
 
 
-  // TODO Ideally, the server should be shared across all client sessions running on the same node.
+  // Ideally, the server should be shared across all client sessions running on the same node.
   private class LlapTaskUmbilicalExternalImpl implements  LlapTaskUmbilicalProtocol {
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java b/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
index dbd591a..79800da 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hive.llap.tezplugins.helpers;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
index 988002f..4306c22 100644
--- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
@@ -97,6 +97,9 @@ import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 
 
+/**
+ * Base LLAP input format to handle requesting of splits and communication with LLAP daemon.
+ */
 public class LlapBaseInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapBaseInputFormat.class);
@@ -178,7 +181,7 @@ public class LlapBaseInputFormat<V extends WritableComparable> implements InputF
 
     LOG.info("Registered id: " + id);
 
-    LlapBaseRecordReader recordReader = new LlapBaseRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
+    LlapBaseRecordReader recordReader = new LlapBaseRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class, job);
     umbilicalResponder.setRecordReader(recordReader);
     return recordReader;
   }
@@ -312,21 +315,6 @@ public class LlapBaseInputFormat<V extends WritableComparable> implements InputF
 
     Credentials taskCredentials = new Credentials();
     // Credentials can change across DAGs. Ideally construct only once per DAG.
-    // TODO Figure out where credentials will come from. Normally Hive sets up
-    // URLs on the tez dag, for which Tez acquires credentials.
-
-    //    taskCredentials.addAll(getContext().getCredentials());
-
-    //    Preconditions.checkState(currentQueryIdentifierProto.getDagIdentifier() ==
-    //        taskSpec.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getId());
-    //    ByteBuffer credentialsBinary = credentialMap.get(currentQueryIdentifierProto);
-    //    if (credentialsBinary == null) {
-    //      credentialsBinary = serializeCredentials(getContext().getCredentials());
-    //      credentialMap.putIfAbsent(currentQueryIdentifierProto, credentialsBinary.duplicate());
-    //    } else {
-    //      credentialsBinary = credentialsBinary.duplicate();
-    //    }
-    //    builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
     Credentials credentials = new Credentials();
     TokenCache.setSessionToken(token, credentials);
     ByteBuffer credentialsBinary = serializeCredentials(credentials);

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
index d485bfa..08ad1f5 100644
--- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
@@ -57,6 +57,11 @@ import org.apache.hadoop.hive.llap.LlapRowRecordReader;
 import org.apache.hadoop.hive.llap.Row;
 import org.apache.hadoop.hive.llap.Schema;
 
+/**
+ * Utility to test query and data retrieval via the LLAP input format.
+ * llapdump --hiveconf hive.zookeeper.quorum=localhost --hiveconf hive.zookeeper.client.port=2181 --hiveconf hive.llap.daemon.service.hosts=@llap_MiniLlapCluster 'select * from employee where employee_id < 10'
+ *
+ */
 public class LlapDump {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapDump.class);
@@ -64,7 +69,7 @@ public class LlapDump {
   private static String url = "jdbc:hive2://localhost:10000/default";
   private static String user = "hive";
   private static String pwd = "";
-  private static String query = "select * from test";
+  private static String query = null;
   private static String numSplits = "1";
 
   public static void main(String[] args) throws Exception {
@@ -99,6 +104,10 @@ public class LlapDump {
       query = cli.getArgs()[0];
     }
 
+    if (query == null) {
+      throw new IllegalArgumentException("No query string specified");
+    }
+
     System.out.println("url: "+url);
     System.out.println("user: "+user);
     System.out.println("query: "+query);

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
index 56ad555..7efc711 100644
--- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.hive.llap;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index 8b2b978..223c390 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -343,7 +343,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     if (webServices != null) {
       getConfig().setInt(ConfVars.LLAP_DAEMON_WEB_PORT.varname, webServices.getPort());
     }
-    LlapOutputFormatService.get();
+    getConfig().setInt(ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT.varname, LlapOutputFormatService.get().getPort());
 
     this.registry.init(getConfig());
     this.registry.start();

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
index 8594ee1..74359fa 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
@@ -141,8 +141,10 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
         vertex, request.getFragmentNumber(), request.getAttemptNumber(), attemptId);
     this.amReporter = amReporter;
     // Register with the AMReporter when the callable is setup. Unregister once it starts running.
-    this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
-        vertex.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
+    if (amReporter != null && jobToken != null) {
+      this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
+          vertex.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
+    }
     this.metrics = metrics;
     this.requestId = taskSpec.getTaskAttemptID().toString();
     // TODO Change this to the queryId/Name when that's available.

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
index d699f20..279baf1 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
@@ -147,7 +147,7 @@ public class TaskExecutorTestHelpers {
     public MockRequest(SubmitWorkRequestProto requestProto, QueryFragmentInfo fragmentInfo,
                        boolean canFinish, long workTime) {
       super(requestProto, fragmentInfo, new Configuration(),
-          new ExecutionContextImpl("localhost"), null, new Credentials(), 0, null, null, mock(
+          new ExecutionContextImpl("localhost"), null, new Credentials(), 0, mock(AMReporter.class), null, mock(
               LlapDaemonExecutorMetrics.class),
           mock(KilledTaskHandler.class), mock(
               FragmentCompletionHandler.class), new DefaultHadoopShim(), null);

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java
deleted file mode 100644
index aad8968..0000000
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.llap;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-/**
- * A thread-not-safe version of Hadoop's DataOutputBuffer, which removes all
- * synchronized modifiers.
- */
-public class LlapDataOutputBuffer implements DataOutput {
-
-  int readOffset;
-  int writeOffset;
-  byte[] buffer;
-
-  /** Constructs a new empty buffer. */
-  public LlapDataOutputBuffer(int length) {
-    buffer = new byte[length];
-    reset();
-  }
-
-  /**
-   * Returns the current contents of the buffer. Data is only valid to
-   * {@link #getLength()}.
-   */
-  public byte[] getData() {
-    return buffer;
-  }
-
-  /** Returns the length of the valid data currently in the buffer. */
-  public int getLength() {
-    return (writeOffset - readOffset) % buffer.length;
-  }
-
-  /** Resets the buffer to empty. */
-  public LlapDataOutputBuffer reset() {
-    readOffset = 0;
-    writeOffset = 0;
-    return this;
-  }
-
-  /** Writes bytes from a DataInput directly into the buffer. */
-  public void write(DataInput in, int length) throws IOException {
-    //
-  }
-
-  @Override
-  public synchronized void write(int b) throws IOException {
-    while (readOffset == writeOffset) {
-      try {
-	wait();
-      } catch(InterruptedException e) {
-      }
-    }
-    buffer[writeOffset] = (byte)b;
-    writeOffset = (writeOffset + 1) % buffer.length;
-    notify();
-  }
-
-  public synchronized int read() throws IOException {
-    while (readOffset == writeOffset) {
-      try {
-	wait();
-      } catch(InterruptedException e) {
-      }
-    }
-    int b = buffer[readOffset];
-    readOffset = (readOffset + 1) % buffer.length;
-    notify();
-    return b;
-  }
-
-  @Override
-  public void write(byte b[], int off, int len) throws IOException {
-    while(len-- != 0) {
-      write(b[off++]);
-    }
-  }
-
-  @Override
-  public void write(byte b[]) throws IOException {
-    write(b, 0, b.length);
-  }
-
-
-  @Override
-  public void writeBoolean(boolean v) throws IOException {
-    write(v?1:0);
-  }
-
-  @Override
-  public void writeByte(int v) throws IOException  {
-    write(v);
-  }
-
-  @Override
-  public void writeChar(int v) throws IOException  {
-    write(v);
-  }
-
-  @Override
-  public void writeBytes(String v) throws IOException  {
-    write(v.getBytes(), 0, v.length());
-  }
-
-  @Override
-  public void writeChars(String v) throws IOException  {
-    write(v.getBytes(), 0, v.length());
-  }
-
-  @Override
-  public void writeDouble(double v) throws IOException  {
-    write(ByteBuffer.allocate(8).putDouble(v).array(),0,8);
-  }
-
-  @Override
-  public void writeFloat(float v) throws IOException  {
-    write(ByteBuffer.allocate(4).putFloat(v).array(),0,4);
-  }
-
-  @Override
-  public void writeInt(int v) throws IOException  {
-    write(v);
-    write(v>>>8);
-    write(v>>>16);
-    write(v>>>24);
-  }
-
-  @Override
-  public void writeLong(long v) throws IOException  {
-    int v1 = (int)v;
-    int v2 = (int)v>>>32;
-    write(v1);
-    write(v2);
-  }
-
-  @Override
-  public void writeShort(int v) throws IOException  {
-    write(v);
-    write(v>>>8);
-  }
-
-  @Override
-  public void writeUTF(String v) throws IOException  {
-    write(v.getBytes(), 0, v.length());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
index b39f085..6adbf7c 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
@@ -20,6 +20,7 @@ import java.util.Map;
 import java.util.HashMap;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.net.InetSocketAddress;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -60,7 +61,7 @@ import io.netty.util.concurrent.Future;
 
 
 /**
- *
+ * Responsible for sending back result set data to the connections made by external clients via the LLAP input format.
  */
 public class LlapOutputFormatService {
 
@@ -75,6 +76,7 @@ public class LlapOutputFormatService {
   private EventLoopGroup eventLoopGroup;
   private ServerBootstrap serverBootstrap;
   private ChannelFuture listeningChannelFuture;
+  private int port;
 
   private LlapOutputFormatService() throws IOException {
     writers = new HashMap<String, RecordWriter>();
@@ -92,17 +94,18 @@ public class LlapOutputFormatService {
   public void start() throws IOException {
     LOG.info("Starting LlapOutputFormatService");
 
-    int port = conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
+    int portFromConf = conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
     eventLoopGroup = new NioEventLoopGroup(1);
     serverBootstrap = new ServerBootstrap();
     serverBootstrap.group(eventLoopGroup);
     serverBootstrap.channel(NioServerSocketChannel.class);
     serverBootstrap.childHandler(new LlapOutputFormatServiceChannelHandler());
     try {
-      LOG.info("LlapOutputFormatService: Binding to port " + port);
-      listeningChannelFuture = serverBootstrap.bind(port).sync();
+      listeningChannelFuture = serverBootstrap.bind(portFromConf).sync();
+      this.port = ((InetSocketAddress) listeningChannelFuture.channel().localAddress()).getPort();
+      LOG.info("LlapOutputFormatService: Binding to port " + this.port);
     } catch (InterruptedException err) {
-      throw new IOException("LlapOutputFormatService: Error binding to port " + port, err);
+      throw new IOException("LlapOutputFormatService: Error binding to port " + portFromConf, err);
     }
   }
 
@@ -132,6 +135,10 @@ public class LlapOutputFormatService {
     return writer;
   }
 
+  public int getPort() {
+    return port;
+  }
+
   protected class LlapOutputFormatServiceHandler extends SimpleChannelInboundHandler<String> {
     @Override
     public void channelRead0(ChannelHandlerContext ctx, String msg) {
@@ -179,11 +186,11 @@ public class LlapOutputFormatService {
   protected class LlapOutputFormatServiceChannelHandler extends ChannelInitializer<SocketChannel> {
     @Override
     public void initChannel(SocketChannel ch) throws Exception {
-        ch.pipeline().addLast(
-            new DelimiterBasedFrameDecoder(MAX_QUERY_ID_LENGTH, Delimiters.nulDelimiter()),
-            new StringDecoder(),
-            new StringEncoder(),
-            new LlapOutputFormatServiceHandler());
+      ch.pipeline().addLast(
+          new DelimiterBasedFrameDecoder(MAX_QUERY_ID_LENGTH, Delimiters.nulDelimiter()),
+          new StringDecoder(),
+          new StringEncoder(),
+          new LlapOutputFormatServiceHandler());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
index 4e6272f..54fc3af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
@@ -93,7 +93,7 @@ public class HiveSplitGenerator extends InputInitializer {
     this.work = work;
     this.jobConf = new JobConf(conf);
 
-    // TODO RSHACK - assuming grouping enabled always.
+    // Assuming grouping enabled always.
     userPayloadProto = MRInputUserPayloadProto.newBuilder().setGroupingEnabled(true).build();
 
     this.splitLocationProvider = Utils.getSplitLocationProvider(conf, LOG);
@@ -106,7 +106,7 @@ public class HiveSplitGenerator extends InputInitializer {
     // must be setup and initialized here so that it sets up it's structures to start accepting events.
     // Setting it up in initialize leads to a window where events may come in before the pruner is
     // initialized, which may cause it to drop events.
-    // TODO RSHACK - No dynamic partition pruning
+    // No dynamic partition pruning
     pruner = null;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
index 6d00a0a..454c321 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
@@ -23,14 +23,11 @@ import java.io.IOException;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 public class HivePassThroughRecordWriter <K extends WritableComparable<?>, V extends Writable>
 implements RecordWriter {
 
-  public static final Logger LOG = LoggerFactory.getLogger(HivePassThroughRecordWriter.class);
   private final org.apache.hadoop.mapred.RecordWriter<K, V> mWriter;
 
   public HivePassThroughRecordWriter(org.apache.hadoop.mapred.RecordWriter<K, V> writer) {
@@ -45,7 +42,6 @@ implements RecordWriter {
 
   @Override
   public void close(boolean abort) throws IOException {
-    LOG.info("Closing the pass through writer.");
     //close with null reporter
     mWriter.close(null);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 81320a2..19e80f7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -1277,17 +1277,14 @@ public class TypeCheckProcFactory {
       try {
         return getXpathOrFuncExprNodeDesc(expr, isFunction, children, ctx);
       } catch (UDFArgumentTypeException e) {
-        LOG.error("UDFArgumentTypeException: ", e);
         throw new SemanticException(ErrorMsg.INVALID_ARGUMENT_TYPE.getMsg(expr
-            .getChild(childrenBegin + e.getArgumentId()), e.getMessage()));
+            .getChild(childrenBegin + e.getArgumentId()), e.getMessage()), e);
       } catch (UDFArgumentLengthException e) {
-        LOG.error("UDFArgumentLengthException: ", e);
         throw new SemanticException(ErrorMsg.INVALID_ARGUMENT_LENGTH.getMsg(
-            expr, e.getMessage()));
+            expr, e.getMessage()), e);
       } catch (UDFArgumentException e) {
-        LOG.error("UDFArgumentException: ", e);
         throw new SemanticException(ErrorMsg.INVALID_ARGUMENT.getMsg(expr, e
-            .getMessage()));
+            .getMessage()), e);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
index 50cdadb..ce69ee6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
@@ -277,7 +277,6 @@ public class GenericUDTFGetSplits extends GenericUDTF {
 
     DAG dag = DAG.create(work.getName());
     dag.setCredentials(job.getCredentials());
-    // TODO: set access control? TezTask.setAccessControlsForCurrentUser(dag);
 
     DagUtils utils = DagUtils.getInstance();
     Context ctx = new Context(job);

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
index 5db8c48..6d31802 100644
--- a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
+++ b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.tez.dag.api;
 
 import java.util.ArrayList;
@@ -25,7 +41,6 @@ public class TaskSpecBuilder {
     List<RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>> outputs =
         vertex.getOutputs();
 
-    // TODO RSHACK - for now these must be of size 1.
     Preconditions.checkState(inputs.size() == 1);
     Preconditions.checkState(outputs.size() == 1);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
index 37e21b8..907d5b0 100644
--- a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
@@ -113,7 +113,7 @@ public class TestLlapOutputFormat {
       writer.close(null);
 
       InputStream in = socket.getInputStream();
-      LlapBaseRecordReader reader = new LlapBaseRecordReader(in, null, Text.class);
+      LlapBaseRecordReader reader = new LlapBaseRecordReader(in, null, Text.class, job);
 
       LOG.debug("Have record reader");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/test/results/clientpositive/show_functions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_functions.q.out b/ql/src/test/results/clientpositive/show_functions.q.out
index 5c8b982..a811747 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -91,6 +91,7 @@ format_number
 from_unixtime
 from_utc_timestamp
 get_json_object
+get_splits
 greatest
 hash
 hex


[07/50] [abbrv] hive git commit: HIVE-13442 : LLAP: refactor submit API to be amenable to signing (Sergey Shelukhin, reviewed by Siddharth Seth)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
----------------------------------------------------------------------
diff --git a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
index 820f6be..16531e3 100644
--- a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
+++ b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
@@ -3212,202 +3212,73 @@ public final class LlapDaemonProtocolProtos {
     // @@protoc_insertion_point(class_scope:GroupInputSpecProto)
   }
 
-  public interface FragmentSpecProtoOrBuilder
+  public interface VertexIdentifierOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional string fragment_identifier_string = 1;
+    // optional string application_id_string = 1;
     /**
-     * <code>optional string fragment_identifier_string = 1;</code>
+     * <code>optional string application_id_string = 1;</code>
      */
-    boolean hasFragmentIdentifierString();
+    boolean hasApplicationIdString();
     /**
-     * <code>optional string fragment_identifier_string = 1;</code>
+     * <code>optional string application_id_string = 1;</code>
      */
-    java.lang.String getFragmentIdentifierString();
+    java.lang.String getApplicationIdString();
     /**
-     * <code>optional string fragment_identifier_string = 1;</code>
+     * <code>optional string application_id_string = 1;</code>
      */
     com.google.protobuf.ByteString
-        getFragmentIdentifierStringBytes();
+        getApplicationIdStringBytes();
 
-    // optional string dag_name = 2;
-    /**
-     * <code>optional string dag_name = 2;</code>
-     */
-    boolean hasDagName();
+    // optional int32 app_attempt_number = 2;
     /**
-     * <code>optional string dag_name = 2;</code>
+     * <code>optional int32 app_attempt_number = 2;</code>
      */
-    java.lang.String getDagName();
+    boolean hasAppAttemptNumber();
     /**
-     * <code>optional string dag_name = 2;</code>
+     * <code>optional int32 app_attempt_number = 2;</code>
      */
-    com.google.protobuf.ByteString
-        getDagNameBytes();
+    int getAppAttemptNumber();
 
-    // optional int32 dag_id = 11;
+    // optional int32 dag_id = 3;
     /**
-     * <code>optional int32 dag_id = 11;</code>
+     * <code>optional int32 dag_id = 3;</code>
      */
     boolean hasDagId();
     /**
-     * <code>optional int32 dag_id = 11;</code>
+     * <code>optional int32 dag_id = 3;</code>
      */
     int getDagId();
 
-    // optional string vertex_name = 3;
-    /**
-     * <code>optional string vertex_name = 3;</code>
-     */
-    boolean hasVertexName();
-    /**
-     * <code>optional string vertex_name = 3;</code>
-     */
-    java.lang.String getVertexName();
-    /**
-     * <code>optional string vertex_name = 3;</code>
-     */
-    com.google.protobuf.ByteString
-        getVertexNameBytes();
-
-    // optional .EntityDescriptorProto processor_descriptor = 4;
-    /**
-     * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-     */
-    boolean hasProcessorDescriptor();
-    /**
-     * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-     */
-    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto getProcessorDescriptor();
-    /**
-     * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-     */
-    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder getProcessorDescriptorOrBuilder();
-
-    // repeated .IOSpecProto input_specs = 5;
-    /**
-     * <code>repeated .IOSpecProto input_specs = 5;</code>
-     */
-    java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> 
-        getInputSpecsList();
-    /**
-     * <code>repeated .IOSpecProto input_specs = 5;</code>
-     */
-    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getInputSpecs(int index);
-    /**
-     * <code>repeated .IOSpecProto input_specs = 5;</code>
-     */
-    int getInputSpecsCount();
-    /**
-     * <code>repeated .IOSpecProto input_specs = 5;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder> 
-        getInputSpecsOrBuilderList();
-    /**
-     * <code>repeated .IOSpecProto input_specs = 5;</code>
-     */
-    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getInputSpecsOrBuilder(
-        int index);
-
-    // repeated .IOSpecProto output_specs = 6;
-    /**
-     * <code>repeated .IOSpecProto output_specs = 6;</code>
-     */
-    java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> 
-        getOutputSpecsList();
-    /**
-     * <code>repeated .IOSpecProto output_specs = 6;</code>
-     */
-    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getOutputSpecs(int index);
-    /**
-     * <code>repeated .IOSpecProto output_specs = 6;</code>
-     */
-    int getOutputSpecsCount();
-    /**
-     * <code>repeated .IOSpecProto output_specs = 6;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder> 
-        getOutputSpecsOrBuilderList();
-    /**
-     * <code>repeated .IOSpecProto output_specs = 6;</code>
-     */
-    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getOutputSpecsOrBuilder(
-        int index);
-
-    // repeated .GroupInputSpecProto grouped_input_specs = 7;
-    /**
-     * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
-     */
-    java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto> 
-        getGroupedInputSpecsList();
-    /**
-     * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
-     */
-    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto getGroupedInputSpecs(int index);
-    /**
-     * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
-     */
-    int getGroupedInputSpecsCount();
-    /**
-     * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder> 
-        getGroupedInputSpecsOrBuilderList();
-    /**
-     * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
-     */
-    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder getGroupedInputSpecsOrBuilder(
-        int index);
-
-    // optional int32 vertex_parallelism = 8;
-    /**
-     * <code>optional int32 vertex_parallelism = 8;</code>
-     */
-    boolean hasVertexParallelism();
-    /**
-     * <code>optional int32 vertex_parallelism = 8;</code>
-     */
-    int getVertexParallelism();
-
-    // optional int32 fragment_number = 9;
-    /**
-     * <code>optional int32 fragment_number = 9;</code>
-     */
-    boolean hasFragmentNumber();
-    /**
-     * <code>optional int32 fragment_number = 9;</code>
-     */
-    int getFragmentNumber();
-
-    // optional int32 attempt_number = 10;
+    // optional int32 vertex_id = 4;
     /**
-     * <code>optional int32 attempt_number = 10;</code>
+     * <code>optional int32 vertex_id = 4;</code>
      */
-    boolean hasAttemptNumber();
+    boolean hasVertexId();
     /**
-     * <code>optional int32 attempt_number = 10;</code>
+     * <code>optional int32 vertex_id = 4;</code>
      */
-    int getAttemptNumber();
+    int getVertexId();
   }
   /**
-   * Protobuf type {@code FragmentSpecProto}
+   * Protobuf type {@code VertexIdentifier}
    */
-  public static final class FragmentSpecProto extends
+  public static final class VertexIdentifier extends
       com.google.protobuf.GeneratedMessage
-      implements FragmentSpecProtoOrBuilder {
-    // Use FragmentSpecProto.newBuilder() to construct.
-    private FragmentSpecProto(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements VertexIdentifierOrBuilder {
+    // Use VertexIdentifier.newBuilder() to construct.
+    private VertexIdentifier(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private FragmentSpecProto(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private VertexIdentifier(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final FragmentSpecProto defaultInstance;
-    public static FragmentSpecProto getDefaultInstance() {
+    private static final VertexIdentifier defaultInstance;
+    public static VertexIdentifier getDefaultInstance() {
       return defaultInstance;
     }
 
-    public FragmentSpecProto getDefaultInstanceForType() {
+    public VertexIdentifier getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -3417,7 +3288,7 @@ public final class LlapDaemonProtocolProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private FragmentSpecProto(
+    private VertexIdentifier(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -3442,76 +3313,24 @@ public final class LlapDaemonProtocolProtos {
             }
             case 10: {
               bitField0_ |= 0x00000001;
-              fragmentIdentifierString_ = input.readBytes();
+              applicationIdString_ = input.readBytes();
               break;
             }
-            case 18: {
+            case 16: {
               bitField0_ |= 0x00000002;
-              dagName_ = input.readBytes();
-              break;
-            }
-            case 26: {
-              bitField0_ |= 0x00000008;
-              vertexName_ = input.readBytes();
-              break;
-            }
-            case 34: {
-              org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000010) == 0x00000010)) {
-                subBuilder = processorDescriptor_.toBuilder();
-              }
-              processorDescriptor_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(processorDescriptor_);
-                processorDescriptor_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000010;
-              break;
-            }
-            case 42: {
-              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-                inputSpecs_ = new java.util.ArrayList<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto>();
-                mutable_bitField0_ |= 0x00000020;
-              }
-              inputSpecs_.add(input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto.PARSER, extensionRegistry));
-              break;
-            }
-            case 50: {
-              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
-                outputSpecs_ = new java.util.ArrayList<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto>();
-                mutable_bitField0_ |= 0x00000040;
-              }
-              outputSpecs_.add(input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto.PARSER, extensionRegistry));
-              break;
-            }
-            case 58: {
-              if (!((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
-                groupedInputSpecs_ = new java.util.ArrayList<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto>();
-                mutable_bitField0_ |= 0x00000080;
-              }
-              groupedInputSpecs_.add(input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto.PARSER, extensionRegistry));
-              break;
-            }
-            case 64: {
-              bitField0_ |= 0x00000020;
-              vertexParallelism_ = input.readInt32();
-              break;
-            }
-            case 72: {
-              bitField0_ |= 0x00000040;
-              fragmentNumber_ = input.readInt32();
-              break;
-            }
-            case 80: {
-              bitField0_ |= 0x00000080;
-              attemptNumber_ = input.readInt32();
+              appAttemptNumber_ = input.readInt32();
               break;
             }
-            case 88: {
+            case 24: {
               bitField0_ |= 0x00000004;
               dagId_ = input.readInt32();
               break;
             }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              vertexId_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -3520,61 +3339,52 @@ public final class LlapDaemonProtocolProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-          inputSpecs_ = java.util.Collections.unmodifiableList(inputSpecs_);
-        }
-        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
-          outputSpecs_ = java.util.Collections.unmodifiableList(outputSpecs_);
-        }
-        if (((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
-          groupedInputSpecs_ = java.util.Collections.unmodifiableList(groupedInputSpecs_);
-        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_FragmentSpecProto_descriptor;
+      return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexIdentifier_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_FragmentSpecProto_fieldAccessorTable
+      return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexIdentifier_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.class, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.Builder.class);
+              org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.class, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<FragmentSpecProto> PARSER =
-        new com.google.protobuf.AbstractParser<FragmentSpecProto>() {
-      public FragmentSpecProto parsePartialFrom(
+    public static com.google.protobuf.Parser<VertexIdentifier> PARSER =
+        new com.google.protobuf.AbstractParser<VertexIdentifier>() {
+      public VertexIdentifier parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new FragmentSpecProto(input, extensionRegistry);
+        return new VertexIdentifier(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<FragmentSpecProto> getParserForType() {
+    public com.google.protobuf.Parser<VertexIdentifier> getParserForType() {
       return PARSER;
     }
 
     private int bitField0_;
-    // optional string fragment_identifier_string = 1;
-    public static final int FRAGMENT_IDENTIFIER_STRING_FIELD_NUMBER = 1;
-    private java.lang.Object fragmentIdentifierString_;
+    // optional string application_id_string = 1;
+    public static final int APPLICATION_ID_STRING_FIELD_NUMBER = 1;
+    private java.lang.Object applicationIdString_;
     /**
-     * <code>optional string fragment_identifier_string = 1;</code>
+     * <code>optional string application_id_string = 1;</code>
      */
-    public boolean hasFragmentIdentifierString() {
+    public boolean hasApplicationIdString() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>optional string fragment_identifier_string = 1;</code>
+     * <code>optional string application_id_string = 1;</code>
      */
-    public java.lang.String getFragmentIdentifierString() {
-      java.lang.Object ref = fragmentIdentifierString_;
+    public java.lang.String getApplicationIdString() {
+      java.lang.Object ref = applicationIdString_;
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
@@ -3582,320 +3392,81 @@ public final class LlapDaemonProtocolProtos {
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
-          fragmentIdentifierString_ = s;
+          applicationIdString_ = s;
         }
         return s;
       }
     }
     /**
-     * <code>optional string fragment_identifier_string = 1;</code>
+     * <code>optional string application_id_string = 1;</code>
      */
     public com.google.protobuf.ByteString
-        getFragmentIdentifierStringBytes() {
-      java.lang.Object ref = fragmentIdentifierString_;
+        getApplicationIdStringBytes() {
+      java.lang.Object ref = applicationIdString_;
       if (ref instanceof java.lang.String) {
         com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
-        fragmentIdentifierString_ = b;
+        applicationIdString_ = b;
         return b;
       } else {
         return (com.google.protobuf.ByteString) ref;
       }
     }
 
-    // optional string dag_name = 2;
-    public static final int DAG_NAME_FIELD_NUMBER = 2;
-    private java.lang.Object dagName_;
+    // optional int32 app_attempt_number = 2;
+    public static final int APP_ATTEMPT_NUMBER_FIELD_NUMBER = 2;
+    private int appAttemptNumber_;
     /**
-     * <code>optional string dag_name = 2;</code>
+     * <code>optional int32 app_attempt_number = 2;</code>
      */
-    public boolean hasDagName() {
+    public boolean hasAppAttemptNumber() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional string dag_name = 2;</code>
-     */
-    public java.lang.String getDagName() {
-      java.lang.Object ref = dagName_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          dagName_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string dag_name = 2;</code>
+     * <code>optional int32 app_attempt_number = 2;</code>
      */
-    public com.google.protobuf.ByteString
-        getDagNameBytes() {
-      java.lang.Object ref = dagName_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        dagName_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
+    public int getAppAttemptNumber() {
+      return appAttemptNumber_;
     }
 
-    // optional int32 dag_id = 11;
-    public static final int DAG_ID_FIELD_NUMBER = 11;
+    // optional int32 dag_id = 3;
+    public static final int DAG_ID_FIELD_NUMBER = 3;
     private int dagId_;
     /**
-     * <code>optional int32 dag_id = 11;</code>
+     * <code>optional int32 dag_id = 3;</code>
      */
     public boolean hasDagId() {
       return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>optional int32 dag_id = 11;</code>
+     * <code>optional int32 dag_id = 3;</code>
      */
     public int getDagId() {
       return dagId_;
     }
 
-    // optional string vertex_name = 3;
-    public static final int VERTEX_NAME_FIELD_NUMBER = 3;
-    private java.lang.Object vertexName_;
+    // optional int32 vertex_id = 4;
+    public static final int VERTEX_ID_FIELD_NUMBER = 4;
+    private int vertexId_;
     /**
-     * <code>optional string vertex_name = 3;</code>
+     * <code>optional int32 vertex_id = 4;</code>
      */
-    public boolean hasVertexName() {
+    public boolean hasVertexId() {
       return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>optional string vertex_name = 3;</code>
-     */
-    public java.lang.String getVertexName() {
-      java.lang.Object ref = vertexName_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          vertexName_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string vertex_name = 3;</code>
+     * <code>optional int32 vertex_id = 4;</code>
      */
-    public com.google.protobuf.ByteString
-        getVertexNameBytes() {
-      java.lang.Object ref = vertexName_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        vertexName_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional .EntityDescriptorProto processor_descriptor = 4;
-    public static final int PROCESSOR_DESCRIPTOR_FIELD_NUMBER = 4;
-    private org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto processorDescriptor_;
-    /**
-     * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-     */
-    public boolean hasProcessorDescriptor() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-     */
-    public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto getProcessorDescriptor() {
-      return processorDescriptor_;
-    }
-    /**
-     * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-     */
-    public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder getProcessorDescriptorOrBuilder() {
-      return processorDescriptor_;
-    }
-
-    // repeated .IOSpecProto input_specs = 5;
-    public static final int INPUT_SPECS_FIELD_NUMBER = 5;
-    private java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> inputSpecs_;
-    /**
-     * <code>repeated .IOSpecProto input_specs = 5;</code>
-     */
-    public java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> getInputSpecsList() {
-      return inputSpecs_;
-    }
-    /**
-     * <code>repeated .IOSpecProto input_specs = 5;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder> 
-        getInputSpecsOrBuilderList() {
-      return inputSpecs_;
-    }
-    /**
-     * <code>repeated .IOSpecProto input_specs = 5;</code>
-     */
-    public int getInputSpecsCount() {
-      return inputSpecs_.size();
-    }
-    /**
-     * <code>repeated .IOSpecProto input_specs = 5;</code>
-     */
-    public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getInputSpecs(int index) {
-      return inputSpecs_.get(index);
-    }
-    /**
-     * <code>repeated .IOSpecProto input_specs = 5;</code>
-     */
-    public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getInputSpecsOrBuilder(
-        int index) {
-      return inputSpecs_.get(index);
-    }
-
-    // repeated .IOSpecProto output_specs = 6;
-    public static final int OUTPUT_SPECS_FIELD_NUMBER = 6;
-    private java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> outputSpecs_;
-    /**
-     * <code>repeated .IOSpecProto output_specs = 6;</code>
-     */
-    public java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> getOutputSpecsList() {
-      return outputSpecs_;
-    }
-    /**
-     * <code>repeated .IOSpecProto output_specs = 6;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder> 
-        getOutputSpecsOrBuilderList() {
-      return outputSpecs_;
-    }
-    /**
-     * <code>repeated .IOSpecProto output_specs = 6;</code>
-     */
-    public int getOutputSpecsCount() {
-      return outputSpecs_.size();
-    }
-    /**
-     * <code>repeated .IOSpecProto output_specs = 6;</code>
-     */
-    public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getOutputSpecs(int index) {
-      return outputSpecs_.get(index);
-    }
-    /**
-     * <code>repeated .IOSpecProto output_specs = 6;</code>
-     */
-    public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getOutputSpecsOrBuilder(
-        int index) {
-      return outputSpecs_.get(index);
-    }
-
-    // repeated .GroupInputSpecProto grouped_input_specs = 7;
-    public static final int GROUPED_INPUT_SPECS_FIELD_NUMBER = 7;
-    private java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto> groupedInputSpecs_;
-    /**
-     * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
-     */
-    public java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto> getGroupedInputSpecsList() {
-      return groupedInputSpecs_;
-    }
-    /**
-     * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder> 
-        getGroupedInputSpecsOrBuilderList() {
-      return groupedInputSpecs_;
-    }
-    /**
-     * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
-     */
-    public int getGroupedInputSpecsCount() {
-      return groupedInputSpecs_.size();
-    }
-    /**
-     * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
-     */
-    public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto getGroupedInputSpecs(int index) {
-      return groupedInputSpecs_.get(index);
-    }
-    /**
-     * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
-     */
-    public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder getGroupedInputSpecsOrBuilder(
-        int index) {
-      return groupedInputSpecs_.get(index);
-    }
-
-    // optional int32 vertex_parallelism = 8;
-    public static final int VERTEX_PARALLELISM_FIELD_NUMBER = 8;
-    private int vertexParallelism_;
-    /**
-     * <code>optional int32 vertex_parallelism = 8;</code>
-     */
-    public boolean hasVertexParallelism() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
-    }
-    /**
-     * <code>optional int32 vertex_parallelism = 8;</code>
-     */
-    public int getVertexParallelism() {
-      return vertexParallelism_;
-    }
-
-    // optional int32 fragment_number = 9;
-    public static final int FRAGMENT_NUMBER_FIELD_NUMBER = 9;
-    private int fragmentNumber_;
-    /**
-     * <code>optional int32 fragment_number = 9;</code>
-     */
-    public boolean hasFragmentNumber() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
-    }
-    /**
-     * <code>optional int32 fragment_number = 9;</code>
-     */
-    public int getFragmentNumber() {
-      return fragmentNumber_;
-    }
-
-    // optional int32 attempt_number = 10;
-    public static final int ATTEMPT_NUMBER_FIELD_NUMBER = 10;
-    private int attemptNumber_;
-    /**
-     * <code>optional int32 attempt_number = 10;</code>
-     */
-    public boolean hasAttemptNumber() {
-      return ((bitField0_ & 0x00000080) == 0x00000080);
-    }
-    /**
-     * <code>optional int32 attempt_number = 10;</code>
-     */
-    public int getAttemptNumber() {
-      return attemptNumber_;
+    public int getVertexId() {
+      return vertexId_;
     }
 
     private void initFields() {
-      fragmentIdentifierString_ = "";
-      dagName_ = "";
+      applicationIdString_ = "";
+      appAttemptNumber_ = 0;
       dagId_ = 0;
-      vertexName_ = "";
-      processorDescriptor_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.getDefaultInstance();
-      inputSpecs_ = java.util.Collections.emptyList();
-      outputSpecs_ = java.util.Collections.emptyList();
-      groupedInputSpecs_ = java.util.Collections.emptyList();
-      vertexParallelism_ = 0;
-      fragmentNumber_ = 0;
-      attemptNumber_ = 0;
+      vertexId_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -3910,37 +3481,16 @@ public final class LlapDaemonProtocolProtos {
                         throws java.io.IOException {
       getSerializedSize();
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, getFragmentIdentifierStringBytes());
+        output.writeBytes(1, getApplicationIdStringBytes());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, getDagNameBytes());
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeBytes(3, getVertexNameBytes());
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeMessage(4, processorDescriptor_);
-      }
-      for (int i = 0; i < inputSpecs_.size(); i++) {
-        output.writeMessage(5, inputSpecs_.get(i));
-      }
-      for (int i = 0; i < outputSpecs_.size(); i++) {
-        output.writeMessage(6, outputSpecs_.get(i));
-      }
-      for (int i = 0; i < groupedInputSpecs_.size(); i++) {
-        output.writeMessage(7, groupedInputSpecs_.get(i));
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeInt32(8, vertexParallelism_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeInt32(9, fragmentNumber_);
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        output.writeInt32(10, attemptNumber_);
+        output.writeInt32(2, appAttemptNumber_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt32(11, dagId_);
+        output.writeInt32(3, dagId_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeInt32(4, vertexId_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -3953,47 +3503,19 @@ public final class LlapDaemonProtocolProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, getFragmentIdentifierStringBytes());
+          .computeBytesSize(1, getApplicationIdStringBytes());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, getDagNameBytes());
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(3, getVertexNameBytes());
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(4, processorDescriptor_);
+          .computeInt32Size(2, appAttemptNumber_);
       }
-      for (int i = 0; i < inputSpecs_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(5, inputSpecs_.get(i));
-      }
-      for (int i = 0; i < outputSpecs_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(6, outputSpecs_.get(i));
-      }
-      for (int i = 0; i < groupedInputSpecs_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(7, groupedInputSpecs_.get(i));
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(8, vertexParallelism_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(9, fragmentNumber_);
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(10, attemptNumber_);
+          .computeInt32Size(3, dagId_);
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(11, dagId_);
+          .computeInt32Size(4, vertexId_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -4012,57 +3534,31 @@ public final class LlapDaemonProtocolProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto)) {
+      if (!(obj instanceof org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto) obj;
+      org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier) obj;
 
       boolean result = true;
-      result = result && (hasFragmentIdentifierString() == other.hasFragmentIdentifierString());
-      if (hasFragmentIdentifierString()) {
-        result = result && getFragmentIdentifierString()
-            .equals(other.getFragmentIdentifierString());
+      result = result && (hasApplicationIdString() == other.hasApplicationIdString());
+      if (hasApplicationIdString()) {
+        result = result && getApplicationIdString()
+            .equals(other.getApplicationIdString());
       }
-      result = result && (hasDagName() == other.hasDagName());
-      if (hasDagName()) {
-        result = result && getDagName()
-            .equals(other.getDagName());
+      result = result && (hasAppAttemptNumber() == other.hasAppAttemptNumber());
+      if (hasAppAttemptNumber()) {
+        result = result && (getAppAttemptNumber()
+            == other.getAppAttemptNumber());
       }
       result = result && (hasDagId() == other.hasDagId());
       if (hasDagId()) {
         result = result && (getDagId()
             == other.getDagId());
       }
-      result = result && (hasVertexName() == other.hasVertexName());
-      if (hasVertexName()) {
-        result = result && getVertexName()
-            .equals(other.getVertexName());
-      }
-      result = result && (hasProcessorDescriptor() == other.hasProcessorDescriptor());
-      if (hasProcessorDescriptor()) {
-        result = result && getProcessorDescriptor()
-            .equals(other.getProcessorDescriptor());
-      }
-      result = result && getInputSpecsList()
-          .equals(other.getInputSpecsList());
-      result = result && getOutputSpecsList()
-          .equals(other.getOutputSpecsList());
-      result = result && getGroupedInputSpecsList()
-          .equals(other.getGroupedInputSpecsList());
-      result = result && (hasVertexParallelism() == other.hasVertexParallelism());
-      if (hasVertexParallelism()) {
-        result = result && (getVertexParallelism()
-            == other.getVertexParallelism());
-      }
-      result = result && (hasFragmentNumber() == other.hasFragmentNumber());
-      if (hasFragmentNumber()) {
-        result = result && (getFragmentNumber()
-            == other.getFragmentNumber());
-      }
-      result = result && (hasAttemptNumber() == other.hasAttemptNumber());
-      if (hasAttemptNumber()) {
-        result = result && (getAttemptNumber()
-            == other.getAttemptNumber());
+      result = result && (hasVertexId() == other.hasVertexId());
+      if (hasVertexId()) {
+        result = result && (getVertexId()
+            == other.getVertexId());
       }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
@@ -4077,102 +3573,74 @@ public final class LlapDaemonProtocolProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasFragmentIdentifierString()) {
-        hash = (37 * hash) + FRAGMENT_IDENTIFIER_STRING_FIELD_NUMBER;
-        hash = (53 * hash) + getFragmentIdentifierString().hashCode();
+      if (hasApplicationIdString()) {
+        hash = (37 * hash) + APPLICATION_ID_STRING_FIELD_NUMBER;
+        hash = (53 * hash) + getApplicationIdString().hashCode();
       }
-      if (hasDagName()) {
-        hash = (37 * hash) + DAG_NAME_FIELD_NUMBER;
-        hash = (53 * hash) + getDagName().hashCode();
+      if (hasAppAttemptNumber()) {
+        hash = (37 * hash) + APP_ATTEMPT_NUMBER_FIELD_NUMBER;
+        hash = (53 * hash) + getAppAttemptNumber();
       }
       if (hasDagId()) {
         hash = (37 * hash) + DAG_ID_FIELD_NUMBER;
         hash = (53 * hash) + getDagId();
       }
-      if (hasVertexName()) {
-        hash = (37 * hash) + VERTEX_NAME_FIELD_NUMBER;
-        hash = (53 * hash) + getVertexName().hashCode();
-      }
-      if (hasProcessorDescriptor()) {
-        hash = (37 * hash) + PROCESSOR_DESCRIPTOR_FIELD_NUMBER;
-        hash = (53 * hash) + getProcessorDescriptor().hashCode();
-      }
-      if (getInputSpecsCount() > 0) {
-        hash = (37 * hash) + INPUT_SPECS_FIELD_NUMBER;
-        hash = (53 * hash) + getInputSpecsList().hashCode();
-      }
-      if (getOutputSpecsCount() > 0) {
-        hash = (37 * hash) + OUTPUT_SPECS_FIELD_NUMBER;
-        hash = (53 * hash) + getOutputSpecsList().hashCode();
-      }
-      if (getGroupedInputSpecsCount() > 0) {
-        hash = (37 * hash) + GROUPED_INPUT_SPECS_FIELD_NUMBER;
-        hash = (53 * hash) + getGroupedInputSpecsList().hashCode();
-      }
-      if (hasVertexParallelism()) {
-        hash = (37 * hash) + VERTEX_PARALLELISM_FIELD_NUMBER;
-        hash = (53 * hash) + getVertexParallelism();
-      }
-      if (hasFragmentNumber()) {
-        hash = (37 * hash) + FRAGMENT_NUMBER_FIELD_NUMBER;
-        hash = (53 * hash) + getFragmentNumber();
-      }
-      if (hasAttemptNumber()) {
-        hash = (37 * hash) + ATTEMPT_NUMBER_FIELD_NUMBER;
-        hash = (53 * hash) + getAttemptNumber();
+      if (hasVertexId()) {
+        hash = (37 * hash) + VERTEX_ID_FIELD_NUMBER;
+        hash = (53 * hash) + getVertexId();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(
+    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(
+    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(byte[] data)
+    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(
+    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(
+    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseDelimitedFrom(
+    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(
+    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(
+    public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -4181,7 +3649,7 @@ public final class LlapDaemonProtocolProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -4193,24 +3661,24 @@ public final class LlapDaemonProtocolProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code FragmentSpecProto}
+     * Protobuf type {@code VertexIdentifier}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProtoOrBuilder {
+       implements org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifierOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_FragmentSpecProto_descriptor;
+        return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexIdentifier_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_FragmentSpecProto_fieldAccessorTable
+        return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexIdentifier_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.class, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.Builder.class);
+                org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.class, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.newBuilder()
+      // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -4222,10 +3690,6 @@ public final class LlapDaemonProtocolProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getProcessorDescriptorFieldBuilder();
-          getInputSpecsFieldBuilder();
-          getOutputSpecsFieldBuilder();
-          getGroupedInputSpecsFieldBuilder();
         }
       }
       private static Builder create() {
@@ -4234,44 +3698,14 @@ public final class LlapDaemonProtocolProtos {
 
       public Builder clear() {
         super.clear();
-        fragmentIdentifierString_ = "";
+        applicationIdString_ = "";
         bitField0_ = (bitField0_ & ~0x00000001);
-        dagName_ = "";
+        appAttemptNumber_ = 0;
         bitField0_ = (bitField0_ & ~0x00000002);
         dagId_ = 0;
         bitField0_ = (bitField0_ & ~0x00000004);
-        vertexName_ = "";
+        vertexId_ = 0;
         bitField0_ = (bitField0_ & ~0x00000008);
-        if (processorDescriptorBuilder_ == null) {
-          processorDescriptor_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.getDefaultInstance();
-        } else {
-          processorDescriptorBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000010);
-        if (inputSpecsBuilder_ == null) {
-          inputSpecs_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000020);
-        } else {
-          inputSpecsBuilder_.clear();
-        }
-        if (outputSpecsBuilder_ == null) {
-          outputSpecs_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000040);
-        } else {
-          outputSpecsBuilder_.clear();
-        }
-        if (groupedInputSpecsBuilder_ == null) {
-          groupedInputSpecs_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000080);
-        } else {
-          groupedInputSpecsBuilder_.clear();
-        }
-        vertexParallelism_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000100);
-        fragmentNumber_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000200);
-        attemptNumber_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000400);
         return this;
       }
 
@@ -4281,33 +3715,33 @@ public final class LlapDaemonProtocolProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_FragmentSpecProto_descriptor;
+        return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexIdentifier_descriptor;
       }
 
-      public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto getDefaultInstanceForType() {
-        return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.getDefaultInstance();
+      public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier getDefaultInstanceForType() {
+        return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto build() {
-        org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto result = buildPartial();
+      public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier build() {
+        org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto buildPartial() {
-        org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto result = new org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto(this);
+      public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier buildPartial() {
+        org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier result = new org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.fragmentIdentifierString_ = fragmentIdentifierString_;
+        result.applicationIdString_ = applicationIdString_;
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.dagName_ = dagName_;
+        result.appAttemptNumber_ = appAttemptNumber_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
@@ -4315,177 +3749,36 @@ public final class LlapDaemonProtocolProtos {
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000008;
         }
-        result.vertexName_ = vertexName_;
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        if (processorDescriptorBuilder_ == null) {
-          result.processorDescriptor_ = processorDescriptor_;
-        } else {
-          result.processorDescriptor_ = processorDescriptorBuilder_.build();
-        }
-        if (inputSpecsBuilder_ == null) {
-          if (((bitField0_ & 0x00000020) == 0x00000020)) {
-            inputSpecs_ = java.util.Collections.unmodifiableList(inputSpecs_);
-            bitField0_ = (bitField0_ & ~0x00000020);
-          }
-          result.inputSpecs_ = inputSpecs_;
-        } else {
-          result.inputSpecs_ = inputSpecsBuilder_.build();
-        }
-        if (outputSpecsBuilder_ == null) {
-          if (((bitField0_ & 0x00000040) == 0x00000040)) {
-            outputSpecs_ = java.util.Collections.unmodifiableList(outputSpecs_);
-            bitField0_ = (bitField0_ & ~0x00000040);
-          }
-          result.outputSpecs_ = outputSpecs_;
-        } else {
-          result.outputSpecs_ = outputSpecsBuilder_.build();
-        }
-        if (groupedInputSpecsBuilder_ == null) {
-          if (((bitField0_ & 0x00000080) == 0x00000080)) {
-            groupedInputSpecs_ = java.util.Collections.unmodifiableList(groupedInputSpecs_);
-            bitField0_ = (bitField0_ & ~0x00000080);
-          }
-          result.groupedInputSpecs_ = groupedInputSpecs_;
-        } else {
-          result.groupedInputSpecs_ = groupedInputSpecsBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
-          to_bitField0_ |= 0x00000020;
-        }
-        result.vertexParallelism_ = vertexParallelism_;
-        if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
-          to_bitField0_ |= 0x00000040;
-        }
-        result.fragmentNumber_ = fragmentNumber_;
-        if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
-          to_bitField0_ |= 0x00000080;
-        }
-        result.attemptNumber_ = attemptNumber_;
+        result.vertexId_ = vertexId_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto) {
-          return mergeFrom((org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto)other);
+        if (other instanceof org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier) {
+          return mergeFrom((org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto other) {
-        if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.getDefaultInstance()) return this;
-        if (other.hasFragmentIdentifierString()) {
+      public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier other) {
+        if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.getDefaultInstance()) return this;
+        if (other.hasApplicationIdString()) {
           bitField0_ |= 0x00000001;
-          fragmentIdentifierString_ = other.fragmentIdentifierString_;
+          applicationIdString_ = other.applicationIdString_;
           onChanged();
         }
-        if (other.hasDagName()) {
-          bitField0_ |= 0x00000002;
-          dagName_ = other.dagName_;
-          onChanged();
+        if (other.hasAppAttemptNumber()) {
+          setAppAttemptNumber(other.getAppAttemptNumber());
         }
         if (other.hasDagId()) {
           setDagId(other.getDagId());
         }
-        if (other.hasVertexName()) {
-          bitField0_ |= 0x00000008;
-          vertexName_ = other.vertexName_;
-          onChanged();
-        }
-        if (other.hasProcessorDescriptor()) {
-          mergeProcessorDescriptor(other.getProcessorDescriptor());
-        }
-        if (inputSpecsBuilder_ == null) {
-          if (!other.inputSpecs_.isEmpty()) {
-            if (inputSpecs_.isEmpty()) {
-              inputSpecs_ = other.inputSpecs_;
-              bitField0_ = (bitField0_ & ~0x00000020);
-            } else {
-              ensureInputSpecsIsMutable();
-              inputSpecs_.addAll(other.inputSpecs_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.inputSpecs_.isEmpty()) {
-            if (inputSpecsBuilder_.isEmpty()) {
-              inputSpecsBuilder_.dispose();
-              inputSpecsBuilder_ = null;
-              inputSpecs_ = other.inputSpecs_;
-              bitField0_ = (bitField0_ & ~0x00000020);
-              inputSpecsBuilder_ = 
-                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
-                   getInputSpecsFieldBuilder() : null;
-            } else {
-              inputSpecsBuilder_.addAllMessages(other.inputSpecs_);
-            }
-          }
-        }
-        if (outputSpecsBuilder_ == null) {
-          if (!other.outputSpecs_.isEmpty()) {
-            if (outputSpecs_.isEmpty()) {
-              outputSpecs_ = other.outputSpecs_;
-              bitField0_ = (bitField0_ & ~0x00000040);
-            } else {
-              ensureOutputSpecsIsMutable();
-              outputSpecs_.addAll(other.outputSpecs_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.outputSpecs_.isEmpty()) {
-            if (outputSpecsBuilder_.isEmpty()) {
-              outputSpecsBuilder_.dispose();
-              outputSpecsBuilder_ = null;
-              outputSpecs_ = other.outputSpecs_;
-              bitField0_ = (bitField0_ & ~0x00000040);
-              outputSpecsBuilder_ = 
-                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
-                   getOutputSpecsFieldBuilder() : null;
-            } else {
-              outputSpecsBuilder_.addAllMessages(other.outputSpecs_);
-            }
-          }
-        }
-        if (groupedInputSpecsBuilder_ == null) {
-          if (!other.groupedInputSpecs_.isEmpty()) {
-            if (groupedInputSpecs_.isEmpty()) {
-              groupedInputSpecs_ = other.groupedInputSpecs_;
-              bitField0_ = (bitField0_ & ~0x00000080);
-            } else {
-              ensureGroupedInputSpecsIsMutable();
-              groupedInputSpecs_.addAll(other.groupedInputSpecs_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.groupedInputSpecs_.isEmpty()) {
-            if (groupedInputSpecsBuilder_.isEmpty()) {
-              groupedInputSpecsBuilder_.dispose();
-              groupedInputSpecsBuilder_ = null;
-              groupedInputSpecs_ = other.groupedInputSpecs_;
-              bitField0_ = (bitField0_ & ~0x00000080);
-              groupedInputSpecsBuilder_ = 
-                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
-                   getGroupedInputSpecsFieldBuilder() : null;
-            } else {
-              groupedInputSpecsBuilder_.addAllMessages(other.groupedInputSpecs_);
-            }
-          }
-        }
-        if (other.hasVertexParallelism()) {
-          setVertexParallelism(other.getVertexParallelism());
-        }
-        if (other.hasFragmentNumber()) {
-          setFragmentNumber(other.getFragmentNumber());
-        }
-        if (other.hasAttemptNumber()) {
-          setAttemptNumber(other.getAttemptNumber());
+        if (other.hasVertexId()) {
+          setVertexId(other.getVertexId());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
@@ -4499,11 +3792,11 @@ public final class LlapDaemonProtocolProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parsedMessage = null;
+        org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -4514,170 +3807,129 @@ public final class LlapDaemonProtocolProtos {
       }
       private int bitField0_;
 
-      // optional string fragment_identifier_string = 1;
-      private java.lang.Object fragmentIdentifierString_ = "";
+      // optional string application_id_string = 1;
+      private java.lang.Object applicationIdString_ = "";
       /**
-       * <code>optional string fragment_identifier_string = 1;</code>
+       * <code>optional string application_id_string = 1;</code>
        */
-      public boolean hasFragmentIdentifierString() {
+      public boolean hasApplicationIdString() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>optional string fragment_identifier_string = 1;</code>
+       * <code>optional string application_id_string = 1;</code>
        */
-      public java.lang.String getFragmentIdentifierString() {
-        java.lang.Object ref = fragmentIdentifierString_;
+      public java.lang.String getApplicationIdString() {
+        java.lang.Object ref = applicationIdString_;
         if (!(ref instanceof java.lang.String)) {
           java.lang.String s = ((com.google.protobuf.ByteString) ref)
               .toStringUtf8();
-          fragmentIdentifierString_ = s;
+          applicationIdString_ = s;
           return s;
         } else {
           return (java.lang.String) ref;
         }
       }
       /**
-       * <code>optional string fragment_identifier_string = 1;</code>
+       * <code>optional string application_id_string = 1;</code>
        */
       public com.google.protobuf.ByteString
-          getFragmentIdentifierStringBytes() {
-        java.lang.Object ref = fragmentIdentifierString_;
+          getApplicationIdStringBytes() {
+        java.lang.Object ref = applicationIdString_;
         if (ref instanceof String) {
           com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
-          fragmentIdentifierString_ = b;
+          applicationIdString_ = b;
           return b;
         } else {
           return (com.google.protobuf.ByteString) ref;
         }
       }
       /**
-       * <code>optional string fragment_identifier_string = 1;</code>
+       * <code>optional string application_id_string = 1;</code>
        */
-      public Builder setFragmentIdentifierString(
+      public Builder setApplicationIdString(
           java.lang.String value) {
         if (value == null) {
     throw new NullPointerException();
   }
   bitField0_ |= 0x00000001;
-        fragmentIdentifierString_ = value;
+        applicationIdString_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional string fragment_identifier_string = 1;</code>
+       * <code>optional string application_id_string = 1;</code>
        */
-      public Builder clearFragmentIdentifierString() {
+      public Builder clearApplicationIdString() {
         bitField0_ = (bitField0_ & ~0x00000001);
-        fragmentIdentifierString_ = getDefaultInstance().getFragmentIdentifierString();
+        applicationIdString_ = getDefaultInstance().getApplicationIdString();
         onChanged();
         return this;
       }
       /**
-       * <code>optional string fragment_identifier_string = 1;</code>
+       * <code>optional string application_id_string = 1;</code>
        */
-      public Builder setFragmentIdentifierStringBytes(
+      public Builder setApplicationIdStringBytes(
           com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
   bitField0_ |= 0x00000001;
-        fragmentIdentifierString_ = value;
+        applicationIdString_ = value;
         onChanged();
         return this;
       }
 
-      // optional string dag_name = 2;
-      private java.lang.Object dagName_ = "";
+      // optional int32 app_attempt_number = 2;
+      private int appAttemptNumber_ ;
       /**
-       * <code>optional string dag_name = 2;</code>
+       * <code>optional int32 app_attempt_number = 2;</code>
        */
-      public boolean hasDagName() {
+      public boolean hasAppAttemptNumber() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional string dag_name = 2;</code>
+       * <code>optional int32 app_attempt_number = 2;</code>
        */
-      public java.lang.String getDagName() {
-        java.lang.Object ref = dagName_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          dagName_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
+      public int getAppAttemptNumber() {
+        return appAttemptNumber_;
       }
       /**
-       * <code>optional string dag_name = 2;</code>
+       * <code>optional int32 app_attempt_number = 2;</code>
        */
-      public com.google.protobuf.ByteString
-          getDagNameBytes() {
-        java.lang.Object ref = dagName_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          dagName_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string dag_name = 2;</code>
-       */
-      public Builder setDagName(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        dagName_ = value;
+      public Builder setAppAttemptNumber(int value) {
+        bitField0_ |= 0x00000002;
+        appAttemptNumber_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional string dag_name = 2;</code>
+       * <code>optional int32 app_attempt_number = 2;</code>
        */
-      public Builder clearDagName() {
+      public Builder clearAppAttemptNumber() {
         bitField0_ = (bitField0_ & ~0x00000002);
-        dagName_ = getDefaultInstance().getDagName();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string dag_name = 2;</code>
-       */
-      public Builder setDagNameBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        dagName_ = value;
+        appAttemptNumber_ = 0;
         onChanged();
         return this;
       }
 
-      // optional int32 dag_id = 11;
+      // optional int32 dag_id = 3;
       private int dagId_ ;
       /**
-       * <code>optional int32 dag_id = 11;</code>
+       * <code>optional int32 dag_id = 3;</code>
        */
       public boolean hasDagId() {
         return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>optional int32 dag_id = 11;</code>
+       * <code>optional int32 dag_id = 3;</code>
        */
       public int getDagId() {
         return dagId_;
       }
       /**
-       * <code>optional int32 dag_id = 11;</code>
+       * <code>optional int32 dag_id = 3;</code>
        */
       public Builder setDagId(int value) {
         bitField0_ |= 0x00000004;
@@ -4686,7 +3938,7 @@ public final class LlapDaemonProtocolProtos {
         return this;
       }
       /**
-       * <code>optional int32 dag_id = 11;</code>
+       * <code>optional int32 dag_id = 3;</code>
        */
       public Builder clearDagId() {
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -4695,1025 +3947,3552 @@ public final class LlapDaemonProtocolProtos {
         return this;
       }
 
-      // optional string vertex_name = 3;
-      private java.lang.Object vertexName_ = "";
+      // optional int32 vertex_id = 4;
+      private int vertexId_ ;
       /**
-       * <code>optional string vertex_name = 3;</code>
+       * <code>optional int32 vertex_id = 4;</code>
        */
-      public boolean hasVertexName() {
+      public boolean hasVertexId() {
         return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
-       * <code>optional string vertex_name = 3;</code>
-       */
-      public java.lang.String getVertexName() {
-        java.lang.Object ref = vertexName_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          vertexName_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string vertex_name = 3;</code>
+       * <code>optional int32 vertex_id = 4;</code>
        */
-      public com.google.protobuf.ByteString
-          getVertexNameBytes() {
-        java.lang.Object ref = vertexName_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          vertexName_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
+      public int getVertexId() {
+        return vertexId_;
       }
       /**
-       * <code>optional string vertex_name = 3;</code>
+       * <code>optional int32 vertex_id = 4;</code>
        */
-      public Builder setVertexName(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000008;
-        vertexName_ = value;
+      public Builder setVertexId(int value) {
+        bitField0_ |= 0x00000008;
+        vertexId_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional string vertex_name = 3;</code>
+       * <code>optional int32 vertex_id = 4;</code>
        */
-      public Builder clearVertexName() {
+      public Builder clearVertexId() {
         bitField0_ = (bitField0_ & ~0x00000008);
-        vertexName_ = getDefaultInstance().getVertexName();
+        vertexId_ = 0;
         onChanged();
         return this;
       }
-      /**
-       * <code>optional string vertex_name = 3;</code>
-       */
-      public Builder setVertexNameBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
+
+      // @@protoc_insertion_point(builder_scope:VertexIdentifier)
+    }
+
+    static {
+      defaultInstance = new VertexIdentifier(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:VertexIdentifier)
   }
-  bitField0_ |= 0x00000008;
-        vertexName_ = value;
-        onChanged();
-        return this;
-      }
 
-      // optional .EntityDescriptorProto processor_descriptor = 4;
-      private org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto processorDescriptor_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder> processorDescriptorBuilder_;
-      /**
-       * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-       */
-      public boolean hasProcessorDescriptor() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
-      }
-      /**
-       * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-       */
-      public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto getProcessorDescriptor() {
-        if (processorDescriptorBuilder_ == null) {
-          return processorDescriptor_;
-        } else {
-          return processorDescriptorBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-       */
-      public Builder setProcessorDescriptor(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto value) {
-        if (processorDescriptorBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          processorDescriptor_ = value;
-          onChanged();
-        } else {
-          processorDescriptorBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000010;
-        return this;
-      }
-      /**
-       * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-       */
-      public Builder setProcessorDescriptor(
-          org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder builderForValue) {
-        if (processorDescriptorBuilder_ == null) {
-          processorDescriptor_ = builderForValue.build();
-          onChanged();
-        } else {
-          processorDescriptorBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000010;
-        return this;
-      }
-      /**
-       * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-       */
-      public Builder mergeProcessorDescriptor(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto value) {
-        if (processorDescriptorBuilder_ == null) {
-          if (((bitField0_ & 0x00000010) == 0x00000010) &&
-              processorDescriptor_ != org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.getDefaultInstance()) {
-            processorDescriptor_ =
-              org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.newBuilder(processorDescriptor_).mergeFrom(value).buildPartial();
-          } else {
-            processorDescriptor_ = value;
-          }
-          onChanged();
-        } else {
-          processorDescriptorBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000010;
-        return this;
-      }
-      /**
-       * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-       */
-      public Builder clearProcessorDescriptor() {
-        if (processorDescriptorBuilder_ == null) {
-          processorDescriptor_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.getDefaultInstance();
-          onChanged();
-        } else {
-          processorDescriptorBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000010);
-        return this;
-      }
-      /**
-       * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-       */
-      public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder getProcessorDescriptorBuilder() {
-        bitField0_ |= 0x00000010;
-        onChanged();
-        return getProcessorDescriptorFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-       */
-      public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder getProcessorDescriptorOrBuilder() {
-        if (processorDescriptorBuilder_ != null) {
-          return processorDescriptorBuilder_.getMessageOrBuilder();
-        } else {
-          return processorDescriptor_;
-        }
-      }
-      /**
-       * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
-       */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder> 
-          getProcessorDescriptorFieldBuilder() {
-        if (processorDescriptorBuilder_ == null) {
-          processorDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder>(
-                  processorDescriptor_,
-                  getParentForChildren(),
-                  isClean());
-          processorDescriptor_ = null;
-        }
-        return processorDescriptorBuilder_;
-      }
+  public interface SignableVertexSpecOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
 
-      // repeated .IOSpecProto input_specs = 5;
-      private java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> inputSpecs_ =
-        java.util.Collections.emptyList();
-      private void ensureInputSpecsIsMutable() {
-        if (!((bitField0_ & 0x00000020) == 0x00000020)) {
-          inputSpecs_ = new java.util.ArrayList<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto>(inputSpecs_);
-          bitField0_ |= 0x00000020;
-         }
-      }
+    // optional string user = 1;
+    /**
+     * <code>optional string user = 1;</code>
+     */
+    boolean hasUser();
+    /**
+     * <code>optional string user = 1;</code>
+     */
+    java.lang.String getUser();
+    /**
+     * <code>optional string user = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getUserBytes();
 
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto.Builder, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder> inputSpecsBuilder_;
+    // optional int64 signatureKeyId = 2;
+    /**
+     * <code>optional int64 signatureKeyId = 2;</code>
+     */
+    boolean hasSignatureKeyId();
+    /**
+     * <code>optional int64 signatureKeyId = 2;</code>
+     */
+    long getSignatureKeyId();
 
-      /**
-       * <code>repeated .IOSpecProto input_specs = 5;</code>
-       */
-      public java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> getInputSpecsList() {
-        if (inputSpecsBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(inputSpecs_);
-        } else {
-          return inputSpecsBuilder_.getMessageList();
-        }
-      }
-      /**
-       * <code>repeated .IOSpecProto input_specs = 5;</code>
-       */
-      public int getInputSpecsCount() {
-        if (inputSpecsBuilder_ == null) {
-          return inputSpecs_.size();
-        } else {
-          return inputSpecsBuilder_.getCount();
-        }
-      }
-      /**
-       * <code>repeated .IOSpecProto input_specs = 5;</code>
-       */
-      public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getInputSpecs(int index) {
-        if (inputSpecsBuilder_ == null) {
-          return inputSpecs_.get(index);
-        } else {
-          return inputSpecsBuilder_.getMessage(index);
-        }
-      }
-      /**
-       * <code>repeated .IOSpecProto input_specs = 5;</code>
-       */
-      public Builder setInputSpecs(
-          int index, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto value) {
-        if (inputSpecsBuilder_ == null) {
+    // optional .VertexIdentifier vertexIdentifier = 3;
+    /**
+     * <code>optional .VertexIdentifier vertexIdentifier = 3;</code>
+     */
+    boolean hasVertexIdentifier();
+    /**
+     * <code>optional .VertexIdentifier vertexIdentifier = 3;</code>
+     */
+    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier getVertexIdentifier();
+    /**
+     * <code>optional .VertexIdentifier vertexIdentifier = 3;</code>
+     */
+    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifierOrBuilder getVertexIdentifierOrBuilder();
+
+    // optional string dag_name = 4;
+    /**
+     * <code>optional string dag_name = 4;</code>
+     *
+     * <pre>
+     * Display names cannot be modified by the client for now. If needed, they should be sent to HS2 who will put them here.
+     * </pre>
+     */
+    boolean hasDagName();
+    /**
+     * <code>optional string dag_name = 4;</code>
+     *
+     * <pre>
+     * Display names cannot be modified by the client for now. If needed, they should be sent to HS2 who will put them here.
+     * </pre>
+     */
+    java.lang.String getDagName();
+    /**
+     * <code>optional string dag_name = 4;</code>
+     *
+     * <pre>
+     * Display names cannot be modified by the client for now. If needed, they should be sent to HS2 who will put them here.
+     * </pre>
+     */
+    com.google.protobuf.ByteString
+        getDagNameBytes();
+
+    // optional string vertex_name = 5;
+    /**
+     * <code>optional string vertex_name = 5;</code>
+     */
+    boolean hasVertexName();
+    /**
+     * <code>optional string vertex_name = 5;</code>
+     */
+    java.lang.String getVertexName();
+    /**
+     * <code>optional string vertex_name = 5;</code>
+     */
+    com.google.protobuf.ByteString
+        getVertexNameBytes();
+
+    // optional string token_identifier = 6;
+    /**
+     * <code>optional string token_identifier = 6;</code>
+     *
+     * <pre>
+     * The core vertex stuff 
+     * </pre>
+     */
+    boolean hasTokenIdentifier();
+    /**
+     * <code>optional string token_identifier = 6;</code>
+     *
+     * <pre>
+     * The core vertex stuff 
+     * </pre>
+     */
+    java.lang.String getTokenIdentifier();
+    /**
+     * <code>optional string token_identifier = 6;</code>
+     *
+     * <pre>
+     * The core vertex stuff 
+     * </pre>
+     */
+    com.google.protobuf.ByteString
+        getTokenIdentifierBytes();
+
+    // optional .EntityDescriptorProto processor_descriptor = 7;
+    /**
+     * <code>optional .EntityDescriptorProto processor_descriptor = 7;</code>
+     */
+    boolean hasProcessorDescriptor();
+    /**
+     * <code>optional .EntityDescriptorProto processor_descriptor = 7;</code>
+     */
+    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto getProcessorDescriptor();
+    /**
+     * <code>optional .EntityDescriptorProto processor_descriptor = 7;</code>
+     */
+    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder getProcessorDescriptorOrBuilder();
+
+    // repeated .IOSpecProto input_specs = 8;
+    /**
+     * <code>repeated .IOSpecProto input_specs = 8;</code>
+     */
+    java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> 
+        getInputSpecsList();
+    /**
+     * <code>repeated .IOSpecProto input_specs = 8;</code>
+     */
+    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getInputSpecs(int index);
+    /**
+     * <code>repeated .IOSpecProto input_specs = 8;</code>
+     */
+    int getInputSpecsCount();
+    /**
+     * <code>repeated .IOSpecProto input_specs = 8;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder> 
+        getInputSpecsOrBuilderList();
+    /**
+     * <code>repeated .IOSpecProto input_specs = 8;</code>
+     */
+    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getInputSpecsOrBuilder(
+        int index);
+
+    // repeated .IOSpecProto output_specs = 9;
+    /**
+     * <code>repeated .IOSpecProto output_specs = 9;</code>
+     */
+    java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> 
+        getOutputSpecsList();
+    /**
+     * <code>repeated .IOSpecProto output_specs = 9;</code>
+     */
+    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getOutputSpecs(int index);
+    /**
+     * <code>repeated .IOSpecProto output_specs = 9;</code>
+     */
+    int getOutputSpecsCount();
+    /**
+     * <code>repeated .IOSpecProto output_specs = 9;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder> 
+        getOutputSpecsOrBuilderList();
+    /**
+     * <code>repeated .IOSpecProto output_specs = 9;</code>
+     */
+    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getOutputSpecsOrBuilder(
+        int index);
+
+    // repeated .GroupInputSpecProto grouped_input_specs = 10;
+    /**
+     * <code>repeated .GroupInputSpecProto grouped_input_specs = 10;</code>
+     */
+    java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto> 
+        getGroupedInputSpecsList();
+    /**
+     * <code>repeated .GroupInputSpecProto grouped_input_specs = 10;</code>
+     */
+    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto getGroupedInputSpecs(int index);
+    /**
+     * <code>repeated .GroupInputSpecProto grouped_input_specs = 10;</code>
+     */
+    int getGroupedInputSpecsCount();
+    /**
+     * <code>repeated .GroupInputSpecProto grouped_input_specs = 10;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder> 
+        getGroupedInputSpecsOrBuilderList();
+    /**
+     * <code>repeated .GroupInputSpecProto grouped_input_specs = 10;</code>
+     */
+    org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder getGroupedInputSpecsOrBuilder(
+        int index);
+
+    // optional int32 vertex_parallelism = 11;
+    /**
+     * <code>optional int32 vertex_parallelism = 11;</code>
+     *
+     * <pre>
+     * An internal field required for Tez.
+     * </pre>
+     */
+    boolean hasVertexParallelism();
+    /**
+     * <code>optional int32 vertex_parallelism = 11;</code>
+     *
+     * <pre>
+     * An internal field required for Tez.
+     * </pre>
+     */
+    int getVertexParallelism();
+  }
+  /**
+   * Protobuf type {@code SignableVertexSpec}
+   *
+   * <pre>
+   * The part of SubmitWork that can be signed 
+   * </pre>
+   */
+  public static final class SignableVertexSpec extends
+      com.google.protobuf.GeneratedMessage
+      implements SignableVertexSpecOrBuilder {
+    // Use SignableVertexSpec.newBuilder() to construct.
+    private SignableVertexSpec(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private SignableVertexSpec(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final SignableVertexSpec defaultInstance;
+    public static SignableVertexSpec getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public SignableVertexSpec getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SignableVertexSpec(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      c

<TRUNCATED>

[39/50] [abbrv] hive git commit: HIVE-13619: Bucket map join plan is incorrect (Vikram Dixit K, reviewed by Gunther Hagleitner)

Posted by sp...@apache.org.
HIVE-13619: Bucket map join plan is incorrect (Vikram Dixit K, reviewed by Gunther Hagleitner)


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

Branch: refs/heads/java8
Commit: 4eb960305f6cf30aa6e1011ee09388b1ab4c4fd9
Parents: da82819
Author: vikram <vi...@hortonworks.com>
Authored: Thu May 5 14:35:58 2016 -0700
Committer: vikram <vi...@hortonworks.com>
Committed: Thu May 5 14:35:58 2016 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4eb96030/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
index 41507b1..a8ed74c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
@@ -83,7 +83,7 @@ public class OperatorUtils {
 
   public static <T> T findSingleOperatorUpstreamJoinAccounted(Operator<?> start, Class<T> clazz) {
     Set<T> found = findOperatorsUpstreamJoinAccounted(start, clazz, new HashSet<T>());
-    return found.size() == 1 ? found.iterator().next(): null;
+    return found.size() >= 1 ? found.iterator().next(): null;
   }
 
   public static <T> Set<T> findOperatorsUpstream(Collection<Operator<?>> starts, Class<T> clazz) {


[46/50] [abbrv] hive git commit: HIVE-13507: Improved logging for ptest (Siddharth Seth, reviewd by Szehon Ho)

Posted by sp...@apache.org.
HIVE-13507: Improved logging for ptest (Siddharth Seth, reviewd by Szehon Ho)


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

Branch: refs/heads/java8
Commit: 3f07bfcefce775dc77eca13cf623ccde94ff2494
Parents: 2b1e273
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 6 10:06:25 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Fri May 6 10:06:25 2016 -0500

----------------------------------------------------------------------
 .../hive/ptest/execution/ExecutionPhase.java    |  2 +
 .../hive/ptest/execution/HostExecutor.java      | 48 ++++++++++++++++++--
 .../hive/ptest/execution/LocalCommand.java      | 31 +++++++++++--
 .../apache/hive/ptest/execution/PrepPhase.java  |  1 +
 .../apache/hive/ptest/execution/conf/Host.java  |  3 ++
 5 files changed, 76 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3f07bfce/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
index 3026ea0..6063afc 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
@@ -86,6 +86,8 @@ public class ExecutionPhase extends Phase {
         isolatedWorkQueue.add(batch);
       }
     }
+    logger.info("ParallelWorkQueueSize={}, IsolatedWorkQueueSize={}", parallelWorkQueue.size(),
+        isolatedWorkQueue.size());
     try {
       int expectedNumHosts = hostExecutors.size();
       initalizeHosts();

http://git-wip-us.apache.org/repos/asf/hive/blob/3f07bfce/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
index b05d2c2..735b261 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
@@ -29,6 +29,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Stopwatch;
 import org.apache.hive.ptest.execution.conf.Host;
 import org.apache.hive.ptest.execution.conf.TestBatch;
 import org.apache.hive.ptest.execution.ssh.RSyncCommand;
@@ -65,6 +66,8 @@ class HostExecutor {
   private final File mFailedTestLogDir;
   private final long mNumPollSeconds;
   private volatile boolean mShutdown;
+  private int numParallelBatchesProcessed = 0;
+  private int numIsolatedBatchesProcessed = 0;
   
   HostExecutor(Host host, String privateKey, ListeningExecutorService executor,
       SSHCommandExecutor sshCommandExecutor,
@@ -100,7 +103,18 @@ class HostExecutor {
     return mExecutor.submit(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        executeTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults);
+        Stopwatch stopwatch = Stopwatch.createStarted();
+        mLogger.info("Starting SubmitTests on host {}", getHost());
+        try {
+          executeTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults);
+        } finally {
+          stopwatch.stop();
+          mLogger.info("Finishing submitTests on host: {}. ElapsedTime(seconds)={}," +
+              " NumParallelBatchesProcessed={}, NumIsolatedBatchesProcessed={}",
+              new Object[]{getHost().toString(),
+                  stopwatch.elapsed(TimeUnit.SECONDS), numParallelBatchesProcessed,
+                  numIsolatedBatchesProcessed});
+        }
         return null;
       }
 
@@ -143,6 +157,7 @@ class HostExecutor {
         @Override
         public Void call() throws Exception {
           TestBatch batch = null;
+          Stopwatch sw = Stopwatch.createUnstarted();
           try {
             do {
               batch = parallelWorkQueue.poll(mNumPollSeconds, TimeUnit.SECONDS);
@@ -151,8 +166,16 @@ class HostExecutor {
                 return null;
               }
               if(batch != null) {
-                if(!executeTestBatch(drone, batch, failedTestResults)) {
-                  failedTestResults.add(batch);
+                numParallelBatchesProcessed++;
+                sw.reset().start();
+                try {
+                  if (!executeTestBatch(drone, batch, failedTestResults)) {
+                    failedTestResults.add(batch);
+                  }
+                } finally {
+                  sw.stop();
+                  mLogger.info("Finished processing parallel batch [{}] on host {}. ElapsedTime(seconds)={}",
+                      new Object[]{batch.getName(), getHost().toShortString(), sw.elapsed(TimeUnit.SECONDS)});
                 }
               }
             } while(!mShutdown && !parallelWorkQueue.isEmpty());
@@ -176,12 +199,22 @@ class HostExecutor {
     mLogger.info("Starting isolated execution on " + mHost.getName());
     for(Drone drone : ImmutableList.copyOf(mDrones)) {
       TestBatch batch = null;
+      Stopwatch sw = Stopwatch.createUnstarted();
       try {
         do {
+
           batch = isolatedWorkQueue.poll(mNumPollSeconds, TimeUnit.SECONDS);
           if(batch != null) {
-            if(!executeTestBatch(drone, batch, failedTestResults)) {
-              failedTestResults.add(batch);
+            numIsolatedBatchesProcessed++;
+            sw.reset().start();
+            try {
+              if (!executeTestBatch(drone, batch, failedTestResults)) {
+                failedTestResults.add(batch);
+              }
+            } finally {
+              sw.stop();
+              mLogger.info("Finished processing isolated batch [{}] on host {}. ElapsedTime(seconds)={}",
+                  new Object[]{batch.getName(), getHost().toShortString(), sw.elapsed(TimeUnit.SECONDS)});
             }
           }
         } while(!mShutdown && !isolatedWorkQueue.isEmpty());
@@ -215,10 +248,15 @@ class HostExecutor {
     Templates.writeTemplateResult("batch-exec.vm", script, templateVariables);
     copyToDroneFromLocal(drone, script.getAbsolutePath(), "$localDir/$instanceName/scratch/" + scriptName);
     script.delete();
+    Stopwatch sw = Stopwatch.createStarted();
     mLogger.info(drone + " executing " + batch + " with " + command);
     RemoteCommandResult sshResult = new SSHCommand(mSSHCommandExecutor, drone.getPrivateKey(), drone.getUser(),
         drone.getHost(), drone.getInstance(), command, true).
         call();
+    sw.stop();
+    mLogger.info("Completed executing tests for batch [{}] on host {}. ElapsedTime(seconds)={}",
+        new Object[]{batch.getName(),
+            getHost().toShortString(), sw.elapsed(TimeUnit.SECONDS)});
     File batchLogDir = null;
     if(sshResult.getExitCode() == Constants.EXIT_CODE_UNKNOWN) {
       throw new AbortDroneException("Drone " + drone.toString() + " exited with " +

http://git-wip-us.apache.org/repos/asf/hive/blob/3f07bfce/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
index ec99656..de9fe68 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
@@ -22,17 +22,28 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.base.Stopwatch;
 import org.slf4j.Logger;
 
 public class LocalCommand {
 
+  private static final AtomicInteger localCommandCounter = new AtomicInteger(0);
+
+  private final Logger logger;
   private final Process process;
   private final StreamReader streamReader;
   private Integer exitCode;
+  private final int commandId;
+  private final Stopwatch stopwatch = Stopwatch.createUnstarted();
 
   public LocalCommand(Logger logger, OutputPolicy outputPolicy, String command) throws IOException {
-    logger.info("Starting " + command);
+    this.commandId = localCommandCounter.incrementAndGet();
+    this.logger = logger;
+    logger.info("Starting LocalCommandId={}: {}" + commandId, command);
+    stopwatch.start();
     process = new ProcessBuilder().command(new String[] {"bash", "-c", command}).redirectErrorStream(true).start();
     streamReader = new StreamReader(outputPolicy, process.getInputStream());
     streamReader.setName("StreamReader-[" + command + "]");
@@ -42,13 +53,25 @@ public class LocalCommand {
 
   public int getExitCode() throws InterruptedException {
     synchronized (process) {
-      if(exitCode == null) {
+      awaitProcessCompletion();
+      return exitCode;
+    }
+  }
+
+  private void awaitProcessCompletion() throws InterruptedException {
+    synchronized (process) {
+      if (exitCode == null) {
         exitCode = process.waitFor();
+        if (stopwatch.isRunning()) {
+          stopwatch.stop();
+          logger.info("Finished LocalCommandId={}. ElapsedTime(seconds)={}", commandId,
+              stopwatch.elapsed(
+                  TimeUnit.SECONDS));
+        }
       }
-      return exitCode;
     }
   }
-  
+
   public void kill() {
     synchronized (process) {
       process.destroy();

http://git-wip-us.apache.org/repos/asf/hive/blob/3f07bfce/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
index 825f0c0..8fef413 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
@@ -62,6 +62,7 @@ public class PrepPhase extends Phase {
     // source prep
     start = System.currentTimeMillis();
     File sourcePrepScript = new File(mScratchDir, "source-prep.sh");
+    logger.info("Writing {} from template", sourcePrepScript);
     Templates.writeTemplateResult("source-prep.vm", sourcePrepScript, getTemplateDefaults());
     execLocally("bash " + sourcePrepScript.getPath());
     logger.debug("Deleting " + sourcePrepScript + ": " + sourcePrepScript.delete());

http://git-wip-us.apache.org/repos/asf/hive/blob/3f07bfce/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
index c1216c1..a56824c 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/Host.java
@@ -47,6 +47,9 @@ public class Host {
   public String[] getLocalDirectories() {
     return localDirectories;
   }
+  public String toShortString() {
+    return name;
+  }
   @Override
   public String toString() {
     return "Host [name=" + name + ", user=" + user + ", threads=" + threads