You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/09/12 20:24:33 UTC

[01/31] hive git commit: Revert "HIVE-13589: beeline - support prompt for password with '-u' option(Ke Jia, reviewed by Vihang Karajgaonkar and Ferdinand Xu)"

Repository: hive
Updated Branches:
  refs/heads/hive-14535 30fd19f4b -> 3e481b471


Revert "HIVE-13589: beeline - support prompt for password with '-u' option(Ke Jia, reviewed by Vihang Karajgaonkar and Ferdinand Xu)"

This reverts commit e6b5115868ca42f974e525d2309e747be65f5d80.


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

Branch: refs/heads/hive-14535
Commit: 63fdb513f907309b4de4122ea5e37f17da7e0f7b
Parents: 8cec20d
Author: Ferdinand Xu <ch...@intel.com>
Authored: Thu Sep 8 02:50:40 2016 +0800
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Thu Sep 8 02:50:40 2016 +0800

----------------------------------------------------------------------
 .../src/java/org/apache/hive/beeline/BeeLine.java    | 15 ++++++---------
 .../src/java/org/apache/hive/beeline/Commands.java   |  2 +-
 2 files changed, 7 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/63fdb513/beeline/src/java/org/apache/hive/beeline/BeeLine.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
index 8e65e39..ecd60f6 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -881,7 +881,6 @@ public class BeeLine implements Closeable {
     }
 
     try {
-      ConsoleReader reader = getConsoleReader(inputStream);
       if (isBeeLine) {
         int code = initArgs(args);
         if (code != 0) {
@@ -906,7 +905,7 @@ public class BeeLine implements Closeable {
       } catch (Exception e) {
         // ignore
       }
-
+      ConsoleReader reader = getConsoleReader(inputStream);
       return execute(reader, false);
     } finally {
         close();
@@ -1080,16 +1079,14 @@ public class BeeLine implements Closeable {
     Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
         @Override
         public void run() {
-          if(consoleReader != null) {
             History h = consoleReader.getHistory();
             if (h instanceof FileHistory) {
-              try {
-                ((FileHistory) h).flush();
-              } catch (IOException e) {
-                error(e);
-              }
+                try {
+                    ((FileHistory) h).flush();
+                } catch (IOException e) {
+                    error(e);
+                }
             }
-          }
         }
     }));
 

http://git-wip-us.apache.org/repos/asf/hive/blob/63fdb513/beeline/src/java/org/apache/hive/beeline/Commands.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/Commands.java b/beeline/src/java/org/apache/hive/beeline/Commands.java
index 2f3ec13..039e354 100644
--- a/beeline/src/java/org/apache/hive/beeline/Commands.java
+++ b/beeline/src/java/org/apache/hive/beeline/Commands.java
@@ -1511,7 +1511,7 @@ public class Commands {
         username = beeLine.getConsoleReader().readLine("Enter username for " + url + ": ");
       }
       props.setProperty(JdbcConnectionParams.AUTH_USER, username);
-      if (password == null || password.length() == 0) {
+      if (password == null) {
         password = beeLine.getConsoleReader().readLine("Enter password for " + url + ": ",
           new Character('*'));
       }


[14/31] hive git commit: HIVE-14039: HiveServer2: Make the usage of server with JDBC thirft serde enabled, backward compatible for older clients (Ziyang Zhao reviewed by Vaibhav Gumashta)

Posted by se...@apache.org.
HIVE-14039: HiveServer2: Make the usage of server with JDBC thirft serde enabled, backward compatible for older clients (Ziyang Zhao reviewed by Vaibhav Gumashta)


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

Branch: refs/heads/hive-14535
Commit: d2e294cf592aa8f9393537b64562fe11b7355306
Parents: 5edf7c8
Author: Vaibhav Gumashta <vg...@hortonworks.com>
Authored: Fri Sep 9 02:48:38 2016 -0700
Committer: Vaibhav Gumashta <vg...@hortonworks.com>
Committed: Fri Sep 9 02:48:38 2016 -0700

----------------------------------------------------------------------
 .../apache/hive/jdbc/TestJdbcWithMiniHS2.java   | 44 ++++++++++++++++++++
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  3 +-
 .../hadoop/hive/ql/parse/TaskCompiler.java      |  2 +-
 .../hadoop/hive/ql/session/SessionState.java    | 13 ++++++
 .../service/cli/session/HiveSessionImpl.java    |  9 +++-
 5 files changed, 67 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d2e294cf/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
index 0249566..e8e57ef 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
@@ -512,6 +512,11 @@ public class TestJdbcWithMiniHS2 {
     conf.setInt("hive.server2.thrift.resultset.max.fetch.size", 1000);
   }
 
+  private void unsetSerializeInTasksInConf(HiveConf conf) {
+    conf.setBoolean("hive.server2.thrift.resultset.serialize.in.tasks", false);
+    conf.unset("hive.server2.thrift.resultset.max.fetch.size");
+  }
+
   @Test
   public void testMetadataQueriesWithSerializeThriftInTasks() throws Exception {
     //stop HiveServer2
@@ -674,6 +679,45 @@ public class TestJdbcWithMiniHS2 {
     stmt.close();
   }
 
+   @Test
+   public void testEnableThriftSerializeInTasks() throws Exception {
+     //stop HiveServer2
+     if (miniHS2.isStarted()) {
+       miniHS2.stop();
+     }
+
+     HiveConf conf = new HiveConf();
+     String userName;
+     setSerializeInTasksInConf(conf);
+     miniHS2 = new MiniHS2(conf);
+     Map<String, String> confOverlay = new HashMap<String, String>();
+     miniHS2.start(confOverlay);
+
+     userName = System.getProperty("user.name");
+     hs2Conn = getConnection(miniHS2.getJdbcURL(), userName, "password");
+     Statement stmt = hs2Conn.createStatement();
+     stmt.execute("drop table if exists testThriftSerializeShow1");
+     stmt.execute("drop table if exists testThriftSerializeShow2");
+     stmt.execute("create table testThriftSerializeShow1 (a int)");
+     stmt.execute("create table testThriftSerializeShow2 (b int)");
+     stmt.execute("insert into testThriftSerializeShow1 values (1)");
+     stmt.execute("insert into testThriftSerializeShow2 values (2)");
+     ResultSet rs = stmt.executeQuery("select * from testThriftSerializeShow1 inner join testThriftSerializeShow2 where testThriftSerializeShow1.a=testThriftSerializeShow2.b");
+     assertTrue(!rs.next());
+
+     unsetSerializeInTasksInConf(conf);
+     rs = stmt.executeQuery("select * from testThriftSerializeShow1 inner join testThriftSerializeShow2 where testThriftSerializeShow1.a=testThriftSerializeShow2.b");
+     assertTrue(!rs.next());
+
+     setSerializeInTasksInConf(conf);
+     rs = stmt.executeQuery("select * from testThriftSerializeShow1 inner join testThriftSerializeShow2 where testThriftSerializeShow1.a=testThriftSerializeShow2.b");
+     assertTrue(!rs.next());
+
+     stmt.execute("drop table testThriftSerializeShow1");
+     stmt.execute("drop table testThriftSerializeShow2");
+     stmt.close();
+   }
+
   /**
    * Tests the creation of the 3 scratch dirs: hdfs, local, downloaded resources (which is also local).
    * 1. Test with doAs=false: open a new JDBC session and verify the presence of directories/permissions

http://git-wip-us.apache.org/repos/asf/hive/blob/d2e294cf/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 943d9d7..489e70f 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
@@ -6874,8 +6874,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       if (tblDesc == null) {
         if (qb.getIsQuery()) {
           String fileFormat;
-          if (SessionState.get().isHiveServerQuery() &&
-                   conf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS)) {
+          if (SessionState.get().getIsUsingThriftJDBCBinarySerDe()) {
               fileFormat = "SequenceFile";
               HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT, fileFormat);
               table_desc=

http://git-wip-us.apache.org/repos/asf/hive/blob/d2e294cf/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
index fb5ca57..fb2b992 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
@@ -159,7 +159,7 @@ public abstract class TaskCompiler {
       TableDesc resultTab = pCtx.getFetchTableDesc();
       if (resultTab == null) {
         resFileFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT);
-        if (SessionState.get().isHiveServerQuery() && (conf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS))
+        if (SessionState.get().getIsUsingThriftJDBCBinarySerDe()
             && (resFileFormat.equalsIgnoreCase("SequenceFile"))) {
           resultTab =
               PlanUtils.getDefaultQueryOutputTableDesc(cols, colTypes, resFileFormat,

http://git-wip-us.apache.org/repos/asf/hive/blob/d2e294cf/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 408c92e..d23a51f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -139,6 +139,11 @@ public class SessionState {
    */
   private boolean isHiveServerQuery = false;
 
+  /**
+   * The flag to indicate if the session using thrift jdbc binary serde or not.
+   */
+  private boolean isUsingThriftJDBCBinarySerDe = false;
+
   /*
    * HiveHistory Object
    */
@@ -341,6 +346,14 @@ public class SessionState {
     this.isVerbose = isVerbose;
   }
 
+  public void setIsUsingThriftJDBCBinarySerDe(boolean isUsingThriftJDBCBinarySerDe) {
+	this.isUsingThriftJDBCBinarySerDe = isUsingThriftJDBCBinarySerDe;
+  }
+
+  public boolean getIsUsingThriftJDBCBinarySerDe() {
+	return isUsingThriftJDBCBinarySerDe;
+  }
+
   public void setIsHiveServerQuery(boolean isHiveServerQuery) {
     this.isHiveServerQuery = isHiveServerQuery;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/d2e294cf/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 72ad86c..8051f47 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
@@ -166,6 +166,7 @@ public class HiveSessionImpl implements HiveSession {
     sessionState.setUserIpAddress(ipAddress);
     sessionState.setIsHiveServerQuery(true);
     sessionState.setForwardedAddresses(SessionManager.getForwardedAddresses());
+    sessionState.setIsUsingThriftJDBCBinarySerDe(updateIsUsingThriftJDBCBinarySerDe());
     SessionState.start(sessionState);
     try {
       sessionState.loadAuxJars();
@@ -189,7 +190,7 @@ public class HiveSessionImpl implements HiveSession {
     lastIdleTime = lastAccessTime;
   }
 
-  /**
+/**
    * It is used for processing hiverc file from HiveServer2 side.
    */
   private class GlobalHivercFileProcessor extends HiveFileProcessor {
@@ -268,6 +269,11 @@ public class HiveSessionImpl implements HiveSession {
     }
   }
 
+  private boolean updateIsUsingThriftJDBCBinarySerDe() {
+	return (8 <= getProtocolVersion().getValue())
+      && sessionConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS);
+  }
+
   @Override
   public void setOperationLogSessionDir(File operationLogRootDir) {
     if (!operationLogRootDir.exists()) {
@@ -355,6 +361,7 @@ public class HiveSessionImpl implements HiveSession {
     // stored in the thread local for the handler thread.
     SessionState.setCurrentSessionState(sessionState);
     sessionState.setForwardedAddresses(SessionManager.getForwardedAddresses());
+    sessionState.setIsUsingThriftJDBCBinarySerDe(updateIsUsingThriftJDBCBinarySerDe());
     if (userAccess) {
       lastAccessTime = System.currentTimeMillis();
     }


[24/31] hive git commit: HIVE-14644 : use metastore information on the read path appropriately (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 8decc94..53f24b9 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -12667,6 +12667,204 @@ class HeartbeatWriteIdResult:
   def __ne__(self, other):
     return not (self == other)
 
+class GetValidWriteIdsRequest:
+  """
+  Attributes:
+   - dbName
+   - tblName
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'dbName', None, None, ), # 1
+    (2, TType.STRING, 'tblName', None, None, ), # 2
+  )
+
+  def __init__(self, dbName=None, tblName=None,):
+    self.dbName = dbName
+    self.tblName = tblName
+
+  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.tblName = 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('GetValidWriteIdsRequest')
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 1)
+      oprot.writeString(self.dbName)
+      oprot.writeFieldEnd()
+    if self.tblName is not None:
+      oprot.writeFieldBegin('tblName', TType.STRING, 2)
+      oprot.writeString(self.tblName)
+      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.tblName is None:
+      raise TProtocol.TProtocolException(message='Required field tblName is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.tblName)
+    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 GetValidWriteIdsResult:
+  """
+  Attributes:
+   - lowWatermarkId
+   - highWatermarkId
+   - areIdsValid
+   - ids
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I64, 'lowWatermarkId', None, None, ), # 1
+    (2, TType.I64, 'highWatermarkId', None, None, ), # 2
+    (3, TType.BOOL, 'areIdsValid', None, None, ), # 3
+    (4, TType.LIST, 'ids', (TType.I64,None), None, ), # 4
+  )
+
+  def __init__(self, lowWatermarkId=None, highWatermarkId=None, areIdsValid=None, ids=None,):
+    self.lowWatermarkId = lowWatermarkId
+    self.highWatermarkId = highWatermarkId
+    self.areIdsValid = areIdsValid
+    self.ids = ids
+
+  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.I64:
+          self.lowWatermarkId = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I64:
+          self.highWatermarkId = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.BOOL:
+          self.areIdsValid = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.ids = []
+          (_etype562, _size559) = iprot.readListBegin()
+          for _i563 in xrange(_size559):
+            _elem564 = iprot.readI64()
+            self.ids.append(_elem564)
+          iprot.readListEnd()
+        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('GetValidWriteIdsResult')
+    if self.lowWatermarkId is not None:
+      oprot.writeFieldBegin('lowWatermarkId', TType.I64, 1)
+      oprot.writeI64(self.lowWatermarkId)
+      oprot.writeFieldEnd()
+    if self.highWatermarkId is not None:
+      oprot.writeFieldBegin('highWatermarkId', TType.I64, 2)
+      oprot.writeI64(self.highWatermarkId)
+      oprot.writeFieldEnd()
+    if self.areIdsValid is not None:
+      oprot.writeFieldBegin('areIdsValid', TType.BOOL, 3)
+      oprot.writeBool(self.areIdsValid)
+      oprot.writeFieldEnd()
+    if self.ids is not None:
+      oprot.writeFieldBegin('ids', TType.LIST, 4)
+      oprot.writeListBegin(TType.I64, len(self.ids))
+      for iter565 in self.ids:
+        oprot.writeI64(iter565)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.lowWatermarkId is None:
+      raise TProtocol.TProtocolException(message='Required field lowWatermarkId is unset!')
+    if self.highWatermarkId is None:
+      raise TProtocol.TProtocolException(message='Required field highWatermarkId is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.lowWatermarkId)
+    value = (value * 31) ^ hash(self.highWatermarkId)
+    value = (value * 31) ^ hash(self.areIdsValid)
+    value = (value * 31) ^ hash(self.ids)
+    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 GetAllFunctionsResponse:
   """
   Attributes:
@@ -12693,11 +12891,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype562, _size559) = iprot.readListBegin()
-          for _i563 in xrange(_size559):
-            _elem564 = Function()
-            _elem564.read(iprot)
-            self.functions.append(_elem564)
+          (_etype569, _size566) = iprot.readListBegin()
+          for _i570 in xrange(_size566):
+            _elem571 = Function()
+            _elem571.read(iprot)
+            self.functions.append(_elem571)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12714,8 +12912,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter565 in self.functions:
-        iter565.write(oprot)
+      for iter572 in self.functions:
+        iter572.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 95f2075..ca60ba4 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2876,6 +2876,50 @@ class HeartbeatWriteIdResult
   ::Thrift::Struct.generate_accessors self
 end
 
+class GetValidWriteIdsRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DBNAME = 1
+  TBLNAME = 2
+
+  FIELDS = {
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
+    TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'}
+  }
+
+  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 tblName is unset!') unless @tblName
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetValidWriteIdsResult
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  LOWWATERMARKID = 1
+  HIGHWATERMARKID = 2
+  AREIDSVALID = 3
+  IDS = 4
+
+  FIELDS = {
+    LOWWATERMARKID => {:type => ::Thrift::Types::I64, :name => 'lowWatermarkId'},
+    HIGHWATERMARKID => {:type => ::Thrift::Types::I64, :name => 'highWatermarkId'},
+    AREIDSVALID => {:type => ::Thrift::Types::BOOL, :name => 'areIdsValid', :optional => true},
+    IDS => {:type => ::Thrift::Types::LIST, :name => 'ids', :element => {:type => ::Thrift::Types::I64}, :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field lowWatermarkId is unset!') unless @lowWatermarkId
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field highWatermarkId is unset!') unless @highWatermarkId
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class GetAllFunctionsResponse
   include ::Thrift::Struct, ::Thrift::Struct_Union
   FUNCTIONS = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 403e07f..613702f 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -2529,6 +2529,21 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'heartbeat_write_id failed: unknown result')
     end
 
+    def get_valid_write_ids(req)
+      send_get_valid_write_ids(req)
+      return recv_get_valid_write_ids()
+    end
+
+    def send_get_valid_write_ids(req)
+      send_message('get_valid_write_ids', Get_valid_write_ids_args, :req => req)
+    end
+
+    def recv_get_valid_write_ids()
+      result = receive_message(Get_valid_write_ids_result)
+      return result.success unless result.success.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_valid_write_ids failed: unknown result')
+    end
+
   end
 
   class Processor < ::FacebookService::Processor 
@@ -4396,6 +4411,13 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'heartbeat_write_id', seqid)
     end
 
+    def process_get_valid_write_ids(seqid, iprot, oprot)
+      args = read_args(iprot, Get_valid_write_ids_args)
+      result = Get_valid_write_ids_result.new()
+      result.success = @handler.get_valid_write_ids(args.req)
+      write_result(result, oprot, 'get_valid_write_ids', seqid)
+    end
+
   end
 
   # HELPER FUNCTIONS AND STRUCTURES
@@ -10092,5 +10114,37 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_valid_write_ids_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::GetValidWriteIdsRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_valid_write_ids_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::GetValidWriteIdsResult}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
 end
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 f99bcd2..e1d41c4 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -119,6 +119,7 @@ import javax.jdo.JDOException;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.sql.SQLException;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.AbstractMap;
@@ -134,6 +135,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Random;
 import java.util.Set;
 import java.util.Timer;
 import java.util.concurrent.Callable;
@@ -438,19 +440,19 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         updateMetrics();
         LOG.info("Finished metadata count metrics: " + initDatabaseCount + " databases, " + initTableCount +
           " tables, " + initPartCount + " partitions.");
-        metrics.addGauge(MetricsConstant.INIT_TOTAL_DATABASES, new MetricsVariable() {
+        metrics.addGauge(MetricsConstant.INIT_TOTAL_DATABASES, new MetricsVariable<Object>() {
           @Override
           public Object getValue() {
             return initDatabaseCount;
           }
         });
-        metrics.addGauge(MetricsConstant.INIT_TOTAL_TABLES, new MetricsVariable() {
+        metrics.addGauge(MetricsConstant.INIT_TOTAL_TABLES, new MetricsVariable<Object>() {
           @Override
           public Object getValue() {
             return initTableCount;
           }
         });
-        metrics.addGauge(MetricsConstant.INIT_TOTAL_PARTITIONS, new MetricsVariable() {
+        metrics.addGauge(MetricsConstant.INIT_TOTAL_PARTITIONS, new MetricsVariable<Object>() {
           @Override
           public Object getValue() {
             return initPartCount;
@@ -1264,26 +1266,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return (ms.getType(typeName) != null);
     }
 
-    private void drop_type_core(final RawStore ms, String typeName)
-        throws NoSuchObjectException, MetaException {
-      boolean success = false;
-      try {
-        ms.openTransaction();
-        // drop any partitions
-        if (!is_type_exists(ms, typeName)) {
-          throw new NoSuchObjectException(typeName + " doesn't exist");
-        }
-        if (!ms.dropType(typeName)) {
-          throw new MetaException("Unable to drop type " + typeName);
-        }
-        success = ms.commitTransaction();
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-      }
-    }
-
     @Override
     public boolean drop_type(final String name) throws MetaException, NoSuchObjectException {
       startFunction("drop_type", ": " + name);
@@ -1818,7 +1800,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           //No drop part listener events fired for public listeners historically, for drop table case.
           //Limiting to internal listeners for now, to avoid unexpected calls for public listeners.
           if (listener instanceof HMSMetricsListener) {
-            for (Partition part : partsToDelete) {
+            for (@SuppressWarnings("unused") Partition part : partsToDelete) {
               listener.onDropPartition(null);
             }
           }
@@ -2294,7 +2276,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           }
 
 
-          partFutures.add(threadPool.submit(new Callable() {
+          partFutures.add(threadPool.submit(new Callable<Partition>() {
             @Override
             public Partition call() throws Exception {
               boolean madeDir = createLocationForAddedPartition(table, part);
@@ -2456,8 +2438,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
             LOG.info("Not adding partition " + part + " as it already exists");
             continue;
           }
-          partFutures.add(threadPool.submit(new Callable() {
-            @Override public Object call() throws Exception {
+          partFutures.add(threadPool.submit(new Callable<Partition>() {
+            @Override public Partition call() throws Exception {
               boolean madeDir = createLocationForAddedPartition(table, part);
               if (addedPartitions.put(new PartValEqWrapperLite(part), madeDir) != null) {
                 // Technically, for ifNotExists case, we could insert one and discard the other
@@ -2474,7 +2456,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
         try {
           for (Future<Partition> partFuture : partFutures) {
-            Partition part = partFuture.get();
+            partFuture.get();
           }
         } catch (InterruptedException | ExecutionException e) {
           // cancel other tasks
@@ -3777,6 +3759,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               }
             }
 
+            @SuppressWarnings("deprecation")
             Deserializer s = MetaStoreUtils.getDeserializer(curConf, tbl, false);
             ret = MetaStoreUtils.getFieldsFromDeserializer(tableName, s);
           } catch (SerDeException e) {
@@ -5745,7 +5728,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           throw newMetaException(e);
         }
       }
-      endFunction("partition_name_has_valid_characters", true, null);
+      endFunction("partition_name_has_valid_characters", true, ex);
       return ret;
     }
 
@@ -6044,21 +6027,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return new GetRoleGrantsForPrincipalResponse(roleMaps);
     }
 
-    /**
-     * Convert each MRoleMap object into a thrift RolePrincipalGrant object
-     * @param roles
-     * @return
-     */
-    private List<RolePrincipalGrant> getRolePrincipalGrants(List<Role> roles) throws MetaException {
-      List<RolePrincipalGrant> rolePrinGrantList = new ArrayList<RolePrincipalGrant>();
-      if (roles != null) {
-        for (Role role : roles) {
-          rolePrinGrantList.addAll(getMS().listRoleMembers(role.getRoleName()));
-        }
-      }
-      return rolePrinGrantList;
-    }
-
     @Override
     public AggrStats get_aggr_stats_for(PartitionsStatsRequest request)
         throws NoSuchObjectException, MetaException, TException {
@@ -6448,31 +6416,47 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
+    private final Random random = new Random();
     @Override
     public GetNextWriteIdResult get_next_write_id(GetNextWriteIdRequest req) throws TException {
       RawStore ms = getMS();
       String dbName = req.getDbName(), tblName = req.getTblName();
       startFunction("get_next_write_id", " : db=" + dbName + " tbl=" + tblName);
-      Exception ex = null;
+      Exception exception = null;
       long writeId = -1;
-      // TODO# see TXN about how to handle conflicts
       try {
-        boolean ok = false;
-        ms.openTransaction();
-        try {
-          Table tbl = ms.getTable(dbName, tblName);
-          if (tbl == null) {
-            throw new NoSuchObjectException(dbName + "." + tblName);
+        int deadlockTryCount = 10;
+        int deadlockRetryBackoffMs = 200;
+        while (deadlockTryCount > 0) {
+          boolean ok = false;
+          ms.openTransaction();
+          try {
+            Table tbl = ms.getTable(dbName, tblName);
+            if (tbl == null) {
+              throw new NoSuchObjectException(dbName + "." + tblName);
+            }
+            writeId = tbl.isSetMmNextWriteId() ? tbl.getMmNextWriteId() : 0;
+            tbl.setMmNextWriteId(writeId + 1);
+            ms.alterTable(dbName, tblName, tbl);
+            ok = true;
+          } finally {
+            if (!ok) {
+              ms.rollbackTransaction();
+              // Exception should propagate; don't override it by breaking out of the loop.
+            } else {
+              Boolean commitResult = ms.commitTransactionExpectDeadlock();
+              if (commitResult != null) {
+                if (commitResult) break; // Assume no exception; ok to break out of the loop.
+                throw new MetaException("Failed to commit");
+              }
+            }
           }
-          writeId = tbl.isSetMmNextWriteId() ? tbl.getMmNextWriteId() : 0;
-          tbl.setMmNextWriteId(writeId + 1);
-          ms.alterTable(dbName, tblName, tbl);
-          ok = true;
-        } finally {
-          commitOrRollback(ms, ok);
+          LOG.warn("Getting the next write ID failed due to a deadlock; retrying");
+          Thread.sleep(random.nextInt(deadlockRetryBackoffMs));
         }
+
         // Do a separate txn after we have reserved the number. TODO: If we fail, ignore on read.
-        ok = false;
+        boolean ok = false;
         ms.openTransaction();
         try {
           Table tbl = ms.getTable(dbName, tblName);
@@ -6482,10 +6466,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           commitOrRollback(ms, ok);
         }
       } catch (Exception e) {
-        ex = e;
+        exception = e;
         throwMetaException(e);
       } finally {
-        endFunction("get_next_write_id", ex == null, ex, tblName);
+        endFunction("get_next_write_id", exception == null, exception, tblName);
       }
       return new GetNextWriteIdResult(writeId);
     }
@@ -6562,10 +6546,65 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       assert tw.getState().length() == 1;
       char state = tw.getState().charAt(0);
       if (state != MM_WRITE_OPEN) {
-        throw new MetaException("Invalid write state to finalize: " + state);
+        throw new MetaException("Invalid write state: " + state);
       }
       return tw;
     }
+
+    @Override
+    public GetValidWriteIdsResult get_valid_write_ids(
+        GetValidWriteIdsRequest req) throws TException {
+      RawStore ms = getMS();
+      String dbName = req.getDbName(), tblName = req.getTblName();
+      startFunction("get_valid_write_ids", " : db=" + dbName + " tbl=" + tblName);
+      GetValidWriteIdsResult result = new GetValidWriteIdsResult();
+      Exception ex = null;
+      try {
+        boolean ok = false;
+        ms.openTransaction();
+        try {
+          Table tbl = ms.getTable(dbName, tblName);
+          if (tbl == null) {
+            throw new InvalidObjectException(dbName + "." + tblName);
+          }
+          long nextId = tbl.isSetMmNextWriteId() ? tbl.getMmNextWriteId() : 0;
+          long watermarkId = tbl.isSetMmWatermarkWriteId() ? tbl.getMmWatermarkWriteId() : -1;
+          if (nextId > (watermarkId + 1)) {
+            // There may be some intermediate failed or active writes; get the valid ones.
+            List<Long> ids = ms.getWriteIds(
+                dbName, tblName, watermarkId, nextId, MM_WRITE_COMMITTED);
+            // TODO: we could optimize here and send the smaller of the lists, and also use ranges
+            if (ids != null) {
+              Iterator<Long> iter = ids.iterator();
+              long oldWatermarkId = watermarkId;
+              while (iter.hasNext()) {
+                if (iter.next() != watermarkId + 1) break;
+                ++watermarkId;
+              }
+              long removed = watermarkId - oldWatermarkId;
+              if (removed > 0) {
+                ids = ids.subList((int)removed, ids.size());
+              }
+              if (!ids.isEmpty()) {
+                result.setIds(ids);
+                result.setAreIdsValid(true);
+              }
+            }
+          }
+          result.setHighWatermarkId(nextId);
+          result.setLowWatermarkId(watermarkId);
+          ok = true;
+        } finally {
+          commitOrRollback(ms, ok);
+        }
+      } catch (Exception e) {
+        ex = e;
+        throwMetaException(e);
+      } finally {
+        endFunction("get_valid_write_ids", ex == null, ex, tblName);
+      }
+      return result;
+    }
   }
 
 
@@ -7053,7 +7092,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
   }
 
   private static MetaStoreThread instantiateThread(String classname) throws Exception {
-    Class c = Class.forName(classname);
+    Class<?> c = Class.forName(classname);
     Object o = c.newInstance();
     if (MetaStoreThread.class.isAssignableFrom(o.getClass())) {
       return (MetaStoreThread)o;
@@ -7082,7 +7121,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     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 {
+  private static void startHouseKeeperService(HiveConf conf, Class<?> c) throws Exception {
     //todo: when metastore adds orderly-shutdown logic, houseKeeper.stop()
     //should be called form it
     HouseKeeperService houseKeeper = (HouseKeeperService)c.newInstance();

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 6bd6d92..0325854 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -2404,4 +2404,10 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
       String dbName, String tableName, long writeId) throws TException {
     client.heartbeat_write_id(new HeartbeatWriteIdRequest(dbName, tableName, writeId));
   }
+
+  @Override
+  public GetValidWriteIdsResult getValidWriteIds(
+      String dbName, String tableName) throws TException {
+    return client.get_valid_write_ids(new GetValidWriteIdsRequest(dbName, tableName));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 f5d611d..8706312 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
 import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse;
 import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
 import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResult;
 import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
@@ -1626,4 +1627,6 @@ public interface IMetaStoreClient {
 
   void finalizeTableWrite(String dbName, String tableName, long writeId,
       boolean commit) throws TException;
+
+  GetValidWriteIdsResult getValidWriteIds(String dbName, String tableName) throws TException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index 561f3e3..125a3e5 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -113,15 +113,8 @@ class MetaStoreDirectSql {
   private final boolean isAggregateStatsCacheEnabled;
   private AggregateStatsCache aggrStatsCache;
 
-  public MetaStoreDirectSql(PersistenceManager pm, Configuration conf) {
+  public MetaStoreDirectSql(PersistenceManager pm, Configuration conf, DatabaseProduct dbType) {
     this.pm = pm;
-    DatabaseProduct dbType = null;
-    try {
-      dbType = DatabaseProduct.determineDatabaseProduct(getProductName());
-    } catch (SQLException e) {
-      LOG.warn("Cannot determine database product; assuming OTHER", e);
-      dbType = DatabaseProduct.OTHER;
-    }
     this.dbType = dbType;
     int batchSize = HiveConf.getIntVar(conf, ConfVars.METASTORE_DIRECT_SQL_PARTITION_BATCH_SIZE);
     if (batchSize == DETECT_BATCHING) {

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 9dc80b1..fb3b1ad 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -25,6 +25,8 @@ import java.lang.reflect.Field;
 import java.net.InetAddress;
 import java.net.URI;
 import java.nio.ByteBuffer;
+import java.sql.Connection;
+import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -52,6 +54,7 @@ import javax.jdo.PersistenceManagerFactory;
 import javax.jdo.Query;
 import javax.jdo.Transaction;
 import javax.jdo.datastore.DataStoreCache;
+import javax.jdo.datastore.JDOConnection;
 import javax.jdo.identity.IntIdentity;
 
 import com.google.common.collect.Maps;
@@ -220,6 +223,7 @@ public class ObjectStore implements RawStore, Configurable {
   private boolean isInitialized = false;
   private PersistenceManager pm = null;
   private MetaStoreDirectSql directSql = null;
+  private DatabaseProduct dbType = null;
   private PartitionExpressionProxy expressionProxy = null;
   private Configuration hiveConf;
   private volatile int openTrasactionCalls = 0;
@@ -329,15 +333,37 @@ public class ObjectStore implements RawStore, Configurable {
     pm = getPersistenceManager();
     isInitialized = pm != null;
     if (isInitialized) {
+      dbType = determineDatabaseProduct();
       expressionProxy = createExpressionProxy(hiveConf);
       if (HiveConf.getBoolVar(getConf(), ConfVars.METASTORE_TRY_DIRECT_SQL)) {
-        directSql = new MetaStoreDirectSql(pm, hiveConf);
+        directSql = new MetaStoreDirectSql(pm, hiveConf, dbType);
       }
     }
     LOG.debug("RawStore: " + this + ", with PersistenceManager: " + pm +
         " created in the thread with id: " + Thread.currentThread().getId());
   }
 
+  private DatabaseProduct determineDatabaseProduct() {
+    try {
+      return DatabaseProduct.determineDatabaseProduct(getProductName(pm));
+    } catch (SQLException e) {
+      LOG.warn("Cannot determine database product; assuming OTHER", e);
+      return DatabaseProduct.OTHER;
+    }
+  }
+
+  private static String getProductName(PersistenceManager pm) {
+    JDOConnection jdoConn = pm.getDataStoreConnection();
+    try {
+      return ((Connection)jdoConn.getNativeConnection()).getMetaData().getDatabaseProductName();
+    } catch (Throwable t) {
+      LOG.warn("Error retrieving product name", t);
+      return null;
+    } finally {
+      jdoConn.close(); // We must release the connection before we call other pm methods.
+    }
+  }
+
   /**
    * Creates the proxy used to evaluate expressions. This is here to prevent circular
    * dependency - ql -&gt; metastore client &lt;-&gt metastore server -&gt ql. If server and
@@ -511,15 +537,52 @@ public class ObjectStore implements RawStore, Configurable {
     return result;
   }
 
-  /**
-   * if this is the commit of the first open call then an actual commit is
-   * called.
-   *
-   * @return Always returns true
-   */
   @Override
   @SuppressWarnings("nls")
   public boolean commitTransaction() {
+    if (!startCommitTransaction()) return false;
+
+    openTrasactionCalls--;
+    debugLog("Commit transaction: count = " + openTrasactionCalls + ", isactive "+ currentTransaction.isActive());
+    if ((openTrasactionCalls == 0) && currentTransaction.isActive()) {
+      transactionStatus = TXN_STATUS.COMMITED;
+      currentTransaction.commit();
+    }
+
+    return true;
+  }
+
+  @Override
+  @CanNotRetry
+  public Boolean commitTransactionExpectDeadlock() {
+    if (!startCommitTransaction()) return false;
+
+    if (--openTrasactionCalls != 0) {
+      String msg = "commitTransactionExpectDeadlock cannot be called for a nested transaction";
+      LOG.error(msg);
+      throw new AssertionError(msg);
+    }
+
+    transactionStatus = TXN_STATUS.COMMITED;
+    try {
+      currentTransaction.commit();
+    } catch (Exception ex) {
+      Throwable candidate = ex;
+      while (candidate != null && !(candidate instanceof SQLException)) {
+        candidate = candidate.getCause();
+      }
+      if (candidate == null) throw ex;
+      if (DatabaseProduct.isDeadlock(dbType, (SQLException)candidate)) {
+        LOG.info("Deadlock exception during commit: " + candidate.getMessage());
+        return null;
+      }
+      throw ex;
+    }
+
+    return true;
+  }
+
+  private boolean startCommitTransaction() {
     if (TXN_STATUS.ROLLBACK == transactionStatus) {
       debugLog("Commit transaction: rollback");
       return false;
@@ -538,14 +601,6 @@ public class ObjectStore implements RawStore, Configurable {
       LOG.error("Unbalanced calls to open/commit Transaction", e);
       throw e;
     }
-    openTrasactionCalls--;
-    debugLog("Commit transaction: count = " + openTrasactionCalls + ", isactive "+ currentTransaction.isActive());
-
-    if ((openTrasactionCalls == 0) && currentTransaction.isActive()) {
-      transactionStatus = TXN_STATUS.COMMITED;
-      currentTransaction.commit();
-    }
-
     return true;
   }
 
@@ -1487,7 +1542,7 @@ public class ObjectStore implements RawStore, Configurable {
         .getCreateTime(), tbl.getLastAccessTime(), tbl.getRetention(),
         convertToMFieldSchemas(tbl.getPartitionKeys()), tbl.getParameters(),
         tbl.getViewOriginalText(), tbl.getViewExpandedText(),
-        tableType, tbl.isSetMmNextWriteId() ?  tbl.getMmNextWriteId() : -1,
+        tableType, tbl.isSetMmNextWriteId() ?  tbl.getMmNextWriteId() : 0,
             tbl.isSetMmWatermarkWriteId() ?  tbl.getMmWatermarkWriteId() : -1);
   }
 
@@ -2718,7 +2773,8 @@ public class ObjectStore implements RawStore, Configurable {
       boolean isConfigEnabled = HiveConf.getBoolVar(getConf(), ConfVars.METASTORE_TRY_DIRECT_SQL)
           && (HiveConf.getBoolVar(getConf(), ConfVars.METASTORE_TRY_DIRECT_SQL_DDL) || !isInTxn);
       if (isConfigEnabled && directSql == null) {
-        directSql = new MetaStoreDirectSql(pm, getConf());
+        dbType = determineDatabaseProduct();
+        directSql = new MetaStoreDirectSql(pm, getConf(), dbType);
       }
 
       if (!allowJdo && isConfigEnabled && !directSql.isCompatibleDatastore()) {
@@ -8692,16 +8748,10 @@ public class ObjectStore implements RawStore, Configurable {
     Query query = null;
     try {
       openTransaction();
-      dbName = HiveStringUtils.normalizeIdentifier(dbName);
-      tblName = HiveStringUtils.normalizeIdentifier(tblName);
-      MTable mtbl = getMTable(dbName, tblName);
-      if (mtbl == null) {
-        success = true;
-        return null;
-      }
       query = pm.newQuery(MTableWrite.class,
               "table.tableName == t1 && table.database.name == t2 && writeId == t3");
       query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.Long t3");
+      @SuppressWarnings("unchecked")
       List<MTableWrite> writes = (List<MTableWrite>) query.execute(tblName, dbName, writeId);
       pm.retrieveAll(writes);
       success = true;
@@ -8723,4 +8773,34 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
+  @Override
+  public List<Long> getWriteIds(String dbName, String tblName,
+      long watermarkId, long nextWriteId, char state) throws MetaException {
+    boolean success = false;
+    Query query = null;
+    try {
+      openTransaction();
+      query = pm.newQuery("select writeId from org.apache.hadoop.hive.metastore.model.MTableWrite"
+          + " where table.tableName == t1 && table.database.name == t2 && writeId >= t3"
+          + " && writeId < t4 && state == t5");
+      query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.Long t3, "
+          + "java.lang.Long t4, java.lang.String t5");
+      query.setResult("writeId");
+      query.setOrdering("writeId asc");
+      @SuppressWarnings("unchecked")
+      List<Long> writes = (List<Long>) query.executeWithArray(
+          tblName, dbName, watermarkId, nextWriteId, String.valueOf(state));
+      success = true;
+      return (writes == null || writes.isEmpty()) ? null : new ArrayList<>(writes);
+    } finally {
+      if (success) {
+        commitTransaction();
+      } else {
+        rollbackTransaction();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 c5359cf..170c07d 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -94,6 +94,15 @@ public interface RawStore extends Configurable {
   public abstract boolean commitTransaction();
 
   /**
+   * Commits transaction and detects if the failure to do so is a deadlock or not.
+   * Must be called on the top level with regard to openTransaction calls; attempting to
+   * call this after several nested openTransaction calls will throw.
+   * @return true or false - same as commitTransaction; null in case of deadlock.
+   */
+  @CanNotRetry
+  public abstract Boolean commitTransactionExpectDeadlock();
+
+  /**
    * Rolls back the current transaction if it is active
    */
   @CanNotRetry
@@ -687,4 +696,6 @@ public interface RawStore extends Configurable {
   MTableWrite getTableWrite(String dbName, String tblName, long writeId) throws MetaException;
 
   void createTableWrite(Table tbl, long writeId, char state, long heartbeat);
+
+  List<Long> getWriteIds(String dbName, String tblName, long watermarkId, long nextWriteId, char state) throws MetaException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 4fbeb9e..829f0ae 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
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.metastore.PartFilterExprUtil;
 import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
 import org.apache.hadoop.hive.metastore.RawStore;
 import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.RawStore.CanNotRetry;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
@@ -131,13 +132,27 @@ public class HBaseStore implements RawStore {
   @Override
   public boolean commitTransaction() {
     if (--txnNestLevel == 0) {
-      LOG.debug("Committing HBase transaction");
-      getHBase().commit();
+      commitInternal();
     }
     return true;
   }
 
   @Override
+  @CanNotRetry
+  public Boolean commitTransactionExpectDeadlock() {
+    if (--txnNestLevel != 0) {
+      throw new AssertionError("Cannot be called on a nested transaction");
+    }
+    commitInternal();
+    return true;
+  }
+
+  private void commitInternal() {
+    LOG.debug("Committing HBase transaction");
+    getHBase().commit();
+  }
+
+  @Override
   public void rollbackTransaction() {
     txnNestLevel = 0;
     LOG.debug("Rolling back HBase transaction");
@@ -2741,4 +2756,12 @@ public class HBaseStore implements RawStore {
     // TODO: Auto-generated method stub
     throw new UnsupportedOperationException();
   }
+ 
+
+  @Override
+  public List<Long> getWriteIds(
+      String dbName, String tblName, long watermarkId, long nextWriteId, char state) {
+    // TODO: Auto-generated method stub
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/src/model/package.jdo
----------------------------------------------------------------------
diff --git a/metastore/src/model/package.jdo b/metastore/src/model/package.jdo
index 5126556..bd71056 100644
--- a/metastore/src/model/package.jdo
+++ b/metastore/src/model/package.jdo
@@ -53,7 +53,7 @@
            <column name="PARAM_VALUE" length="4000" jdbc-type="VARCHAR"/>
         </value>
       </field>
-      <field name="ownerName">	
+      <field name="ownerName">    
         <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="true"/>
       </field>
        <field name="ownerType">
@@ -183,10 +183,10 @@
         <column name="TBL_TYPE" length="128" jdbc-type="VARCHAR"/>
       </field>
       <field name="mmNextWriteId">
-        <column name="MM_NEXT_WRITE_ID" jdbc-type="BIGINT"/>
+        <column name="MM_NEXT_WRITE_ID" jdbc-type="BIGINT" default-value="0" />
       </field>
       <field name="mmWatermarkWriteId">
-        <column name="MM_WATERMARK_WRITE_ID" jdbc-type="BIGINT"/>
+        <column name="MM_WATERMARK_WRITE_ID" jdbc-type="BIGINT" default-value="-1" />
       </field>
     </class>
 
@@ -210,7 +210,7 @@
         <column name="PARENT_CD_ID"/>
       </field>
       <field name="parentIntegerIndex">
-	<column name="PARENT_INTEGER_IDX"/>
+    <column name="PARENT_INTEGER_IDX"/>
       </field>
       <field name="parentTable">
         <column name="PARENT_TBL_ID"/>
@@ -219,7 +219,7 @@
         <column name="CONSTRAINT_TYPE"/>
       </field>
       <field name="deleteRule">
-	<column name="DELETE_RULE"/>
+        <column name="DELETE_RULE"/>
       </field>
       <field name="updateRule">
         <column name="UPDATE_RULE"/>
@@ -288,7 +288,7 @@
           </embedded>
         </element>
       </field>
-	</class>
+    </class>
 
    <class name="MStringList" identity-type="datastore" table="Skewed_STRING_LIST" detachable="true">
      <datastore-identity>
@@ -308,7 +308,7 @@
         <column name="SD_ID"/>
       </datastore-identity>
       <field name="cd">
-      	<column name="CD_ID"/>
+          <column name="CD_ID"/>
       </field>
       <field name="location">
         <column name="LOCATION" length="4000" jdbc-type="VARCHAR"/>
@@ -1003,7 +1003,7 @@
       <field name="className">
         <column name="CLASS_NAME" length="4000" jdbc-type="VARCHAR"/>
       </field>
-      <field name="ownerName">	
+      <field name="ownerName">    
         <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR"/>
       </field>
        <field name="ownerType">

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 9fffd3f..98c543f 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -878,4 +878,16 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   public MTableWrite getTableWrite(String dbName, String tblName, long writeId) {
     return null;
   }
+
+  @Override
+  @CanNotRetry
+  public Boolean commitTransactionExpectDeadlock() {
+    return null;
+  }
+
+  @Override
+  public List<Long> getWriteIds(
+      String dbName, String tblName, long watermarkId, long nextWriteId, char state) {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 a763085..8e54b16 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -104,14 +104,17 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   @Override
   public boolean commitTransaction() {
+    return false;
+  }
 
+  @Override
+  @CanNotRetry
+  public Boolean commitTransactionExpectDeadlock() {
     return false;
   }
 
   @Override
   public void rollbackTransaction() {
-
-
   }
 
   @Override
@@ -893,6 +896,12 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   public MTableWrite getTableWrite(String dbName, String tblName, long writeId) {
     return null;
   }
+
+  @Override
+  public List<Long> getWriteIds(
+      String dbName, String tblName, long watermarkId, long nextWriteId, char state) {
+    return null;
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 42d398d..45a80e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -39,6 +39,7 @@ import java.util.concurrent.locks.ReentrantLock;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.common.ValidWriteIds;
 import org.apache.hadoop.hive.common.metrics.common.Metrics;
 import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
@@ -46,8 +47,11 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.conf.HiveVariableSource;
 import org.apache.hadoop.hive.conf.VariableSubstitution;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
 import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.ql.exec.ConditionalTask;
 import org.apache.hadoop.hive.ql.exec.ExplainTask;
@@ -71,6 +75,7 @@ import org.apache.hadoop.hive.ql.hooks.QueryLifeTimeHookContext;
 import org.apache.hadoop.hive.ql.hooks.QueryLifeTimeHookContextImpl;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.lockmgr.HiveLock;
 import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
 import org.apache.hadoop.hive.ql.lockmgr.LockException;
@@ -1416,6 +1421,11 @@ public class Driver implements CommandProcessor {
         return rollback(createProcessorResponse(ret));
       }
     }
+    try {
+      acquireWriteIds(plan, conf);
+    } catch (HiveException e) {
+      return handleHiveException(e, 1);
+    }
     ret = execute();
     if (ret != 0) {
       //if needRequireLock is false, the release here will do nothing because there is no lock
@@ -1458,6 +1468,34 @@ public class Driver implements CommandProcessor {
     return createProcessorResponse(ret);
   }
 
+  private static void acquireWriteIds(QueryPlan plan, HiveConf conf) throws HiveException {
+    // Output IDs are put directly into FileSinkDesc; here, we only need to take care of inputs.
+    for (ReadEntity input : plan.getInputs()) {
+      Table t = extractMmTable(input);
+      if (t == null) continue;
+      ValidWriteIds ids = Hive.get().getValidWriteIdsForTable(t.getDbName(), t.getTableName());
+      ids.addToConf(conf, t.getDbName(), t.getTableName());
+      if (plan.getFetchTask() != null) {
+        ids.addToConf(plan.getFetchTask().getFetchConf(), t.getDbName(), t.getTableName());
+      }
+    }
+  }
+
+  private static Table extractMmTable(ReadEntity input) {
+    Table t = null;
+    switch (input.getType()) {
+      case TABLE:
+        t = input.getTable();
+        break;
+      case DUMMYPARTITION:
+      case PARTITION:
+        t = input.getPartition().getTable();
+        break;
+      default: return null;
+    }
+    return (t != null && !t.isTemporary() && AcidUtils.isMmTable(t)) ? t : null;
+  }
+
   private CommandProcessorResponse rollback(CommandProcessorResponse cpr) {
     //console.printError(cpr.toString());
     try {

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
index 601ad08..7375cd4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
@@ -30,6 +30,7 @@ import java.util.Properties;
 
 import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -695,4 +696,8 @@ public class FetchOperator implements Serializable {
       return inputFormat.getRecordReader(getInputSplit(), job, Reporter.NULL);
     }
   }
+
+  public Configuration getJobConf() {
+    return job;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
index 8c7d99d..93c03fd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
@@ -24,6 +24,7 @@ import java.util.List;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
@@ -193,4 +194,8 @@ public class FetchTask extends Task<FetchWork> implements Serializable {
     }
   }
 
+  public Configuration getFetchConf() {
+    return fetch.getJobConf();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index 6a0143a..e4e0153 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -40,10 +40,10 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.HiveStatsUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.ValidWriteIds;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
@@ -239,7 +239,8 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
         }
       }
       if (isMmTable) {
-        Path manifestPath = new Path(specPath, "_tmp." + getMmPrefixedTaskId() + MANIFEST_EXTENSION);
+        Path manifestPath = new Path(specPath, "_tmp." + ValidWriteIds.getMmFilePrefix(
+            conf.getMmWriteId()) + "_" + taskId + MANIFEST_EXTENSION);
         Utilities.LOG14535.info("Writing manifest to " + manifestPath + " with " + commitPaths);
         try {
           try (FSDataOutputStream out = fs.create(manifestPath)) {
@@ -323,11 +324,12 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
           }
           outPaths[filesIdx] = getTaskOutPath(taskId);
         } else {
+          String subdirPath = ValidWriteIds.getMmFilePrefix(conf.getMmWriteId()) + "/" + taskId;
           if (!bDynParts && !isSkewedStoredAsSubDirectories) {
-            finalPaths[filesIdx] = getFinalPath(getMmPrefixedTaskId(), specPath, extension);
+            finalPaths[filesIdx] = getFinalPath(subdirPath, specPath, extension);
           } else {
-            // TODO# wrong!
-            finalPaths[filesIdx] = getFinalPath(getMmPrefixedTaskId(), specPath, extension);
+            // TODO# wrong! special case #N bucketing
+            finalPaths[filesIdx] = getFinalPath(subdirPath, specPath, extension);
           }
           outPaths[filesIdx] = finalPaths[filesIdx];
         }
@@ -721,10 +723,6 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     }
   }
 
-  private String getMmPrefixedTaskId() {
-    return AcidUtils.getMmFilePrefix(conf.getMmWriteId()) + taskId;
-  }
-
   protected Writable recordValue;
 
 
@@ -1195,21 +1193,6 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     super.jobCloseOp(hconf, success);
   }
 
-  private static class ExecPrefixPathFilter implements PathFilter {
-    private final String prefix, tmpPrefix;
-    public ExecPrefixPathFilter(String prefix) {
-      this.prefix = prefix;
-      this.tmpPrefix = "_tmp." + prefix;
-    }
-
-    @Override
-    public boolean accept(Path path) {
-      String name = path.getName();
-      return name.startsWith(prefix) || name.startsWith(tmpPrefix);
-    }
-  }
-
-
   private void handleMmTable(Path specPath, Configuration hconf, boolean success,
       DynamicPartitionCtx dpCtx, FileSinkDesc conf, Reporter reporter)
           throws IOException, HiveException {
@@ -1217,7 +1200,7 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     int targetLevel = (dpCtx == null) ? 1 : dpCtx.getNumDPCols();
     if (!success) {
       FileStatus[] statuses = HiveStatsUtils.getFileStatusRecurse(specPath, targetLevel, fs,
-          new ExecPrefixPathFilter(AcidUtils.getMmFilePrefix(conf.getMmWriteId())));
+          new ValidWriteIds.IdPathFilter(conf.getMmWriteId(), true));
       for (FileStatus status : statuses) {
         Utilities.LOG14535.info("Deleting " + status.getPath() + " on failure");
         tryDelete(fs, status.getPath());
@@ -1225,15 +1208,19 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
       return;
     }
     FileStatus[] statuses = HiveStatsUtils.getFileStatusRecurse(specPath, targetLevel, fs,
-        new ExecPrefixPathFilter(AcidUtils.getMmFilePrefix(conf.getMmWriteId())));
+        new ValidWriteIds.IdPathFilter(conf.getMmWriteId(), true));
     if (statuses == null) return;
     LinkedList<FileStatus> results = new LinkedList<>();
     List<Path> manifests = new ArrayList<>(statuses.length);
     for (FileStatus status : statuses) {
       if (status.getPath().getName().endsWith(MANIFEST_EXTENSION)) {
         manifests.add(status.getPath());
+      } else if (!status.isDirectory()) {
+        Path path = status.getPath();
+        Utilities.LOG14535.warn("Unknown file found - neither a manifest nor directory: " + path);
+        tryDelete(fs, path);
       } else {
-        results.add(status);
+        results.addAll(Lists.newArrayList(fs.listStatus(status.getPath())));
       }
     }
     HashSet<String> committed = new HashSet<>();
@@ -1254,7 +1241,10 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
       if (!committed.remove(rfs.getPath().toString())) {
         iter.remove();
         Utilities.LOG14535.info("Deleting " + rfs.getPath() + " that was not committed");
-        tryDelete(fs, rfs.getPath());
+        // We should actually succeed here - if we fail, don't commit the query.
+        if (!fs.delete(rfs.getPath(), true)) {
+          throw new HiveException("Failed to delete an uncommitted path " + rfs.getPath());
+        }
       }
     }
     if (!committed.isEmpty()) {
@@ -1268,6 +1258,7 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     if (results.isEmpty()) return;
     FileStatus[] finalResults = results.toArray(new FileStatus[results.size()]);
 
+    // TODO# dp will break - removeTempOrDuplicateFiles assumes dirs in results. Why? We recurse...
     List<Path> emptyBuckets = Utilities.removeTempOrDuplicateFiles(
         fs, finalResults, dpCtx, conf, hconf);
     // create empty buckets if necessary
@@ -1278,7 +1269,7 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
 
   private void tryDelete(FileSystem fs, Path path) {
     try {
-      fs.delete(path, false);
+      fs.delete(path, true);
     } catch (IOException ex) {
       LOG.error("Failed to delete " + path, ex);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index f2389ea..3be21c4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -314,17 +314,9 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
         checkFileFormats(db, tbd, table);
 
         boolean isAcid = work.getLoadTableWork().getWriteType() != AcidUtils.Operation.NOT_ACID;
-        if (tbd.isMmTable()) {
-          if (tbd.getReplace()) {
-            // TODO#: would need a list of new files to support. Then, old ones only would need
-            //        to be removed from MS (and FS). Also, per-partition IOW is problematic for
-            //        the prefix case.
-            throw new HiveException("Replace and MM are not supported");
-          }
-          if (isAcid) {
-            // TODO# need to make sure ACID writes to final directories. Otherwise, might need to move.
-            throw new HiveException("ACID and MM are not supported");
-          }
+        if (tbd.isMmTable() && isAcid) {
+          // TODO# need to make sure ACID writes to final directories. Otherwise, might need to move.
+          throw new HiveException("ACID and MM are not supported");
         }
 
         // Create a data container

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index 9e6a201..03abdc1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -123,6 +123,7 @@ import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
 import org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat;
 import org.apache.hadoop.hive.ql.io.IOConstants;
+import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.io.OneNullRowInputFormat;
 import org.apache.hadoop.hive.ql.io.RCFile;
 import org.apache.hadoop.hive.ql.io.ReworkMapredInputFormat;
@@ -161,6 +162,7 @@ import org.apache.hadoop.hive.ql.stats.StatsFactory;
 import org.apache.hadoop.hive.ql.stats.StatsPublisher;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
 import org.apache.hadoop.hive.serde2.Serializer;
@@ -192,6 +194,7 @@ import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Shell;
 import org.apache.hive.common.util.ReflectionUtil;
@@ -199,6 +202,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.esotericsoftware.kryo.Kryo;
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /**
@@ -651,6 +655,7 @@ public final class Utilities {
     }
   }
 
+  @VisibleForTesting
   public static TableDesc defaultTd;
   static {
     // by default we expect ^A separated strings
@@ -658,7 +663,16 @@ public final class Utilities {
     // PlanUtils.getDefaultTableDesc(String separatorCode, String columns)
     // or getBinarySortableTableDesc(List<FieldSchema> fieldSchemas) when
     // we know the column names.
-    defaultTd = PlanUtils.getDefaultTableDesc("" + Utilities.ctrlaCode);
+    /**
+     * Generate the table descriptor of MetadataTypedColumnsetSerDe with the
+     * separatorCode. MetaDataTypedColumnsetSerDe is used because LazySimpleSerDe
+     * does not support a table with a single column "col" with type
+     * "array<string>".
+     */
+    defaultTd = new TableDesc(TextInputFormat.class, IgnoreKeyTextOutputFormat.class,
+        Utilities.makeProperties(org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT,
+            "" + Utilities.ctrlaCode, serdeConstants.SERIALIZATION_LIB,
+            MetadataTypedColumnsetSerDe.class.getName()));
   }
 
   public static final int carriageReturnCode = 13;
@@ -1528,14 +1542,9 @@ public final class Utilities {
           // get the missing buckets and generate empty buckets
           String taskID1 = taskIDToFile.keySet().iterator().next();
           Path bucketPath = taskIDToFile.values().iterator().next().getPath();
+          Utilities.LOG14535.info("Bucket path " + bucketPath);
           for (int j = 0; j < dpCtx.getNumBuckets(); ++j) {
-            String taskID2 = replaceTaskId(taskID1, j);
-            if (!taskIDToFile.containsKey(taskID2)) {
-              // create empty bucket, file name should be derived from taskID2
-              URI bucketUri = bucketPath.toUri();
-              String path2 = replaceTaskIdFromFilename(bucketUri.getPath().toString(), j);
-              result.add(new Path(bucketUri.getScheme(), bucketUri.getAuthority(), path2));
-            }
+            addBucketFileIfMissing(result, taskIDToFile, taskID1, bucketPath, j);
           }
         }
       }
@@ -1550,14 +1559,9 @@ public final class Utilities {
           // get the missing buckets and generate empty buckets for non-dynamic partition
         String taskID1 = taskIDToFile.keySet().iterator().next();
         Path bucketPath = taskIDToFile.values().iterator().next().getPath();
+        Utilities.LOG14535.info("Bucket path " + bucketPath);
         for (int j = 0; j < conf.getTable().getNumBuckets(); ++j) {
-          String taskID2 = replaceTaskId(taskID1, j);
-          if (!taskIDToFile.containsKey(taskID2)) {
-            // create empty bucket, file name should be derived from taskID2
-            URI bucketUri = bucketPath.toUri();
-            String path2 = replaceTaskIdFromFilename(bucketUri.getPath().toString(), j);
-            result.add(new Path(bucketUri.getScheme(), bucketUri.getAuthority(), path2));
-          }
+          addBucketFileIfMissing(result, taskIDToFile, taskID1, bucketPath, j);
         }
       }
     }
@@ -1565,6 +1569,19 @@ public final class Utilities {
     return result;
   }
 
+  private static void addBucketFileIfMissing(List<Path> result,
+      HashMap<String, FileStatus> taskIDToFile, String taskID1, Path bucketPath, int j) {
+    // TODO# this will probably break with directories cause buckets would be above (or not?)
+    String taskID2 = replaceTaskId(taskID1, j);
+    if (!taskIDToFile.containsKey(taskID2)) {
+      // create empty bucket, file name should be derived from taskID2
+      URI bucketUri = bucketPath.toUri();
+      String path2 = replaceTaskIdFromFilename(bucketUri.getPath().toString(), j);
+      Utilities.LOG14535.info("Creating an empty bucket file " + path2);
+      result.add(new Path(bucketUri.getScheme(), bucketUri.getAuthority(), path2));
+    }
+  }
+
   public static HashMap<String, FileStatus> removeTempOrDuplicateFiles(FileStatus[] items,
       FileSystem fs) throws IOException {
 
@@ -2976,8 +2993,9 @@ public final class Utilities {
 
       // The alias may not have any path
       Path path = null;
-      for (Path file : new LinkedList<Path>(work.getPathToAliases().keySet())) {
-        List<String> aliases = work.getPathToAliases().get(file);
+      for (Map.Entry<Path, ArrayList<String>> e : work.getPathToAliases().entrySet()) {
+        Path file = e.getKey();
+        List<String> aliases = e.getValue();
         if (aliases.contains(alias)) {
           path = file;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index 1ef15cb..70b129e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@ -1167,8 +1167,4 @@ public class AcidUtils {
     }
     return AcidOperationalProperties.parseString(resultStr);
   }
-
-  public static String getMmFilePrefix(long mmWriteId) {
-    return "mm_" + mmWriteId + "_";
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index c4b9940..0510e08 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -23,9 +23,11 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface;
 
 import java.io.DataInput;
 import java.io.DataOutput;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -39,8 +41,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.common.ValidWriteIds;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil;
@@ -345,7 +349,10 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
    */
   private void addSplitsForGroup(List<Path> dirs, TableScanOperator tableScan, JobConf conf,
       InputFormat inputFormat, Class<? extends InputFormat> inputFormatClass, int splits,
-      TableDesc table, List<InputSplit> result) throws IOException {
+      TableDesc table, Map<String, ValidWriteIds> writeIdMap, List<InputSplit> result)
+          throws IOException {
+    ValidWriteIds writeIds = extractWriteIds(writeIdMap, conf, table.getTableName());
+    Utilities.LOG14535.info("Observing " + table.getTableName() + ": " + writeIds);
 
     Utilities.copyTablePropertiesToConf(table, conf);
 
@@ -353,7 +360,19 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
       pushFilters(conf, tableScan);
     }
 
-    FileInputFormat.setInputPaths(conf, dirs.toArray(new Path[dirs.size()]));
+    if (writeIds == null) {
+      FileInputFormat.setInputPaths(conf, dirs.toArray(new Path[dirs.size()]));
+    } else {
+      List<Path> finalPaths = new ArrayList<>(dirs.size());
+      for (Path dir : dirs) {
+        processForWriteIds(dir, conf, writeIds, finalPaths);
+      }
+      if (finalPaths.isEmpty()) {
+        LOG.warn("No valid inputs found in " + dirs);
+        return;
+      }
+      FileInputFormat.setInputPaths(conf, finalPaths.toArray(new Path[finalPaths.size()]));
+    }
     conf.setInputFormat(inputFormat.getClass());
 
     int headerCount = 0;
@@ -373,6 +392,24 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
     }
   }
 
+  private void processForWriteIds(Path dir, JobConf conf,
+      ValidWriteIds writeIds, List<Path> finalPaths) throws IOException {
+    FileStatus[] files = dir.getFileSystem(conf).listStatus(dir); // TODO: batch?
+    for (FileStatus file : files) {
+      Path subdir = file.getPath();
+      if (!file.isDirectory()) {
+        Utilities.LOG14535.warn("Found a file not in subdirectory " + subdir);
+        continue;
+      }
+      if (!writeIds.isValidInput(subdir)) {
+        Utilities.LOG14535.warn("Ignoring an uncommitted directory " + subdir);
+        continue;
+      }
+      Utilities.LOG14535.info("Adding input " + subdir);
+      finalPaths.add(subdir);
+    }
+  }
+
   Path[] getInputPaths(JobConf job) throws IOException {
     Path[] dirs;
     if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) {
@@ -416,6 +453,7 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
     StringBuilder readColumnNamesBuffer = new StringBuilder(newjob.
       get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, ""));
     // for each dir, get the InputFormat, and do getSplits.
+    Map<String, ValidWriteIds> writeIdMap = new HashMap<>();
     for (Path dir : dirs) {
       PartitionDesc part = getPartitionDescFromPath(pathToPartitionInfo, dir);
       Class<? extends InputFormat> inputFormatClass = part.getInputFileFormatClass();
@@ -466,7 +504,7 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
         addSplitsForGroup(currentDirs, currentTableScan, newjob,
             getInputFormatFromCache(currentInputFormatClass, job),
             currentInputFormatClass, currentDirs.size()*(numSplits / dirs.length),
-            currentTable, result);
+            currentTable, writeIdMap, result);
       }
 
       currentDirs.clear();
@@ -488,7 +526,7 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
       addSplitsForGroup(currentDirs, currentTableScan, newjob,
           getInputFormatFromCache(currentInputFormatClass, job),
           currentInputFormatClass, currentDirs.size()*(numSplits / dirs.length),
-          currentTable, result);
+          currentTable, writeIdMap, result);
     }
 
     Utilities.clearWorkMapForConf(job);
@@ -499,6 +537,19 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
     return result.toArray(new HiveInputSplit[result.size()]);
   }
 
+  private static ValidWriteIds extractWriteIds(Map<String, ValidWriteIds> writeIdMap,
+      JobConf newjob, String tableName) {
+    if (StringUtils.isBlank(tableName)) return null;
+    ValidWriteIds writeIds = writeIdMap.get(tableName);
+    if (writeIds == null) {
+      writeIds = ValidWriteIds.createFromConf(newjob, tableName);
+      writeIdMap.put(tableName, writeIds != null ? writeIds : ValidWriteIds.NO_WRITE_IDS);
+    } else if (writeIds == ValidWriteIds.NO_WRITE_IDS) {
+      writeIds = null;
+    }
+    return writeIds;
+  }
+
   private void pushProjection(final JobConf newjob, final StringBuilder readColumnsBuffer,
       final StringBuilder readColumnNamesBuffer) {
     String readColIds = readColumnsBuffer.toString();

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 2ba4fa2..f3609df 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
@@ -67,6 +67,7 @@ import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.HiveStatsUtils;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.ValidWriteIds;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -96,6 +97,7 @@ import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
 import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResult;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 import org.apache.hadoop.hive.metastore.api.HiveObjectType;
@@ -1563,6 +1565,11 @@ public class Hive {
         if (areEventsForDmlNeeded(tbl, oldPart)) {
           newFiles = listFilesCreatedByQuery(loadPath, mmWriteId);
         }
+        if (replace) {
+          Path tableDest = tbl.getPath();
+          deleteOldPathForReplace(newPartPath, oldPartPath,
+              getConf(), new ValidWriteIds.IdPathFilter(mmWriteId, false));
+        }
       } else {
         if (replace || (oldPart == null && !isAcid)) {
           replaceFiles(tbl.getPath(), loadPath, newPartPath, oldPartPath, getConf(),
@@ -1652,7 +1659,7 @@ public class Hive {
 
   private List<Path> listFilesCreatedByQuery(Path loadPath, long mmWriteId) throws HiveException {
     List<Path> newFiles = new ArrayList<Path>();
-    final String filePrefix = AcidUtils.getMmFilePrefix(mmWriteId);
+    final String filePrefix = ValidWriteIds.getMmFilePrefix(mmWriteId);
     FileStatus[] srcs;
     FileSystem srcFs;
     try {
@@ -1920,7 +1927,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       for (Future future : futures) {
         future.get();
       }
-      // TODO# we would commit the txn to metastore here
+      // TODO# special case #N - DP - we would commit the txn to metastore here
     } catch (InterruptedException | ExecutionException e) {
       LOG.debug("Cancelling " + futures.size() + " dynamic loading tasks");
       //cancel other futures
@@ -1993,6 +2000,11 @@ private void constructOneLBLocationMap(FileStatus fSta,
         }
       }
     } else {
+      if (replace) {
+        Path tableDest = tbl.getPath();
+        deleteOldPathForReplace(tableDest, tableDest, sessionConf,
+            new ValidWriteIds.IdPathFilter(mmWriteId, false));
+      }
       newFiles = listFilesCreatedByQuery(loadPath, mmWriteId);
     }
     if (!this.getConf().getBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER)) {
@@ -3376,39 +3388,10 @@ private void constructOneLBLocationMap(FileStatus fSta,
       }
 
       if (oldPath != null) {
-        boolean oldPathDeleted = false;
-        boolean isOldPathUnderDestf = false;
-        FileStatus[] statuses = null;
-        try {
-          FileSystem oldFs = oldPath.getFileSystem(conf);
-          statuses = oldFs.listStatus(oldPath, FileUtils.HIDDEN_FILES_PATH_FILTER);
-          // Do not delete oldPath if:
-          //  - destf is subdir of oldPath
-          isOldPathUnderDestf = isSubDir(oldPath, destf, oldFs, destFs, false);
-          if (isOldPathUnderDestf) {
-            // if oldPath is destf or its subdir, its should definitely be deleted, otherwise its
-            // existing content might result in incorrect (extra) data.
-            // But not sure why we changed not to delete the oldPath in HIVE-8750 if it is
-            // not the destf or its subdir?
-            oldPathDeleted = trashFiles(oldFs, statuses, conf);
-          }
-        } catch (IOException e) {
-          if (isOldPathUnderDestf) {
-            // if oldPath is a subdir of destf but it could not be cleaned
-            throw new HiveException("Directory " + oldPath.toString()
-                + " could not be cleaned up.", e);
-          } else {
-            //swallow the exception since it won't affect the final result
-            LOG.warn("Directory " + oldPath.toString() + " cannot be cleaned: " + e, e);
-          }
-        }
-        if (statuses != null && statuses.length > 0) {
-          if (isOldPathUnderDestf && !oldPathDeleted) {
-            throw new HiveException("Destination directory " + destf + " has not be cleaned up.");
-          }
-        }
+        deleteOldPathForReplace(destf, oldPath, conf, FileUtils.HIDDEN_FILES_PATH_FILTER);
       }
 
+      // TODO# what are the paths that use this? MM tables will need to do this beforehand
       // first call FileUtils.mkdir to make sure that destf directory exists, if not, it creates
       // destf with inherited permissions
       boolean inheritPerms = HiveConf.getBoolVar(conf, HiveConf.ConfVars
@@ -3442,6 +3425,37 @@ private void constructOneLBLocationMap(FileStatus fSta,
   }
 
 
+  private void deleteOldPathForReplace(Path destPath, Path oldPath, HiveConf conf,
+      PathFilter pathFilter) throws HiveException {
+    boolean isOldPathUnderDestf = false;
+    try {
+      FileSystem oldFs = oldPath.getFileSystem(conf);
+      FileSystem destFs = destPath.getFileSystem(conf);
+      // if oldPath is destf or its subdir, its should definitely be deleted, otherwise its
+      // existing content might result in incorrect (extra) data.
+      // But not sure why we changed not to delete the oldPath in HIVE-8750 if it is
+      // not the destf or its subdir?
+      isOldPathUnderDestf = isSubDir(oldPath, destPath, oldFs, destFs, false);
+      if (isOldPathUnderDestf) {
+        FileStatus[] statuses = oldFs.listStatus(oldPath, pathFilter);
+        if (statuses != null && statuses.length > 0 && !trashFiles(oldFs, statuses, conf)) {
+          throw new HiveException("Destination directory " + destPath
+              + " has not been cleaned up.");
+        }
+      }
+    } catch (IOException e) {
+      if (isOldPathUnderDestf) {
+        // if oldPath is a subdir of destf but it could not be cleaned
+        throw new HiveException("Directory " + oldPath.toString()
+            + " could not be cleaned up.", e);
+      } else {
+        //swallow the exception since it won't affect the final result
+        LOG.warn("Directory " + oldPath.toString() + " cannot be cleaned: " + e, e);
+      }
+    }
+  }
+
+
   /**
    * Trashes or deletes all files under a directory. Leaves the directory as is.
    * @param fs FileSystem to use
@@ -4007,7 +4021,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
-
   public long getNextTableWriteId(String dbName, String tableName) throws HiveException {
     try {
       return getMSC().getNextTableWriteId(dbName, tableName);
@@ -4015,4 +4028,17 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throw new HiveException(e);
     }
   }
+
+  public ValidWriteIds getValidWriteIdsForTable(
+      String dbName, String tableName) throws HiveException {
+    try {
+      // TODO: decode ID ranges here if we use that optimization
+      GetValidWriteIdsResult result = getMSC().getValidWriteIds(dbName, tableName);
+      return new ValidWriteIds(result.getLowWatermarkId(), result.getHighWatermarkId(),
+          result.isSetAreIdsValid() && result.isAreIdsValid(),
+          result.isSetIds() ? new HashSet<Long>(result.getIds()) : null);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
 };

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
index bb7001a..675bfd0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
@@ -1808,7 +1808,7 @@ public final class GenMapRedUtils {
 
       // Create the required temporary file in the HDFS location if the destination
       // path of the FileSinkOperator table is a blobstore path.
-      // TODO# HERE
+      // TODO# special case #N - linked FDs (unions?)
       Path tmpDir = baseCtx.getTempDirForPath(fileSinkDesc.getDestPath());
 
       // Change all the linked file sink descriptors

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java
index e2887fd..ee67443 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java
@@ -71,7 +71,6 @@ public class AnnotateRunTimeStatsOptimizer implements PhysicalPlanResolver {
       Task<? extends Serializable> currTask = (Task<? extends Serializable>) nd;
       Set<Operator<? extends OperatorDesc>> ops = new HashSet<>();
 
-      /* TODO# wtf
       if (currTask instanceof MapRedTask) {
         MapRedTask mr = (MapRedTask) currTask;
         ops.addAll(mr.getWork().getAllOperators());
@@ -85,7 +84,7 @@ public class AnnotateRunTimeStatsOptimizer implements PhysicalPlanResolver {
         for (BaseWork w : sparkWork.getAllWork()) {
           ops.addAll(w.getAllOperators());
         }
-      }*/
+      }
 
       setOrAnnotateStats(ops, physicalContext.getParseContext());
       return null;

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
index 422be8e..93fe0e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
@@ -206,7 +206,7 @@ public abstract class TaskCompiler {
       }
     } else if (!isCStats) {
       for (LoadTableDesc ltd : loadTableWork) {
-        // TODO# HERE
+        // TODO# move task is created here; handle MM tables
         Task<MoveWork> tsk = TaskFactory.get(new MoveWork(null, null, ltd, null, false), conf);
         mvTask.add(tsk);
         // Check to see if we are stale'ing any indexes and auto-update them if we want

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
index 5cc3663..1be4d84 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.mapred.JobConf;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Interner;
 
 /**
@@ -375,6 +376,7 @@ public class MapWork extends BaseWork {
   }
 
   @SuppressWarnings("nls")
+  @VisibleForTesting
   public void addMapWork(Path path, String alias, Operator<?> work,
       PartitionDesc pd) {
     ArrayList<String> curAliases = pathToAliases.get(path);

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
index 5dc3aa6..f055cde 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
@@ -381,20 +381,6 @@ public final class PlanUtils {
   }
 
   /**
-   * Generate the table descriptor of MetadataTypedColumnsetSerDe with the
-   * separatorCode. MetaDataTypedColumnsetSerDe is used because LazySimpleSerDe
-   * does not support a table with a single column "col" with type
-   * "array<string>".
-   */
-  public static TableDesc getDefaultTableDesc(String separatorCode) {
-    return new TableDesc(
-        TextInputFormat.class, IgnoreKeyTextOutputFormat.class, Utilities
-        .makeProperties(
-            org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT,separatorCode,
-            serdeConstants.SERIALIZATION_LIB,MetadataTypedColumnsetSerDe.class.getName()));
-  }
-
-  /**
    * Generate the table descriptor for reduce key.
    */
   public static TableDesc getReduceKeyTableDesc(List<FieldSchema> fieldSchemas,


[29/31] hive git commit: HIVE-14644 : use metastore information on the read path appropriately (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 df555ec..573ec6b 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -174,6 +174,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_next_write_id(GetNextWriteIdResult& _return, const GetNextWriteIdRequest& req) = 0;
   virtual void finalize_write_id(FinalizeWriteIdResult& _return, const FinalizeWriteIdRequest& req) = 0;
   virtual void heartbeat_write_id(HeartbeatWriteIdResult& _return, const HeartbeatWriteIdRequest& req) = 0;
+  virtual void get_valid_write_ids(GetValidWriteIdsResult& _return, const GetValidWriteIdsRequest& req) = 0;
 };
 
 class ThriftHiveMetastoreIfFactory : virtual public  ::facebook::fb303::FacebookServiceIfFactory {
@@ -687,6 +688,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void heartbeat_write_id(HeartbeatWriteIdResult& /* _return */, const HeartbeatWriteIdRequest& /* req */) {
     return;
   }
+  void get_valid_write_ids(GetValidWriteIdsResult& /* _return */, const GetValidWriteIdsRequest& /* req */) {
+    return;
+  }
 };
 
 typedef struct _ThriftHiveMetastore_getMetaConf_args__isset {
@@ -19455,6 +19459,110 @@ class ThriftHiveMetastore_heartbeat_write_id_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_get_valid_write_ids_args__isset {
+  _ThriftHiveMetastore_get_valid_write_ids_args__isset() : req(false) {}
+  bool req :1;
+} _ThriftHiveMetastore_get_valid_write_ids_args__isset;
+
+class ThriftHiveMetastore_get_valid_write_ids_args {
+ public:
+
+  ThriftHiveMetastore_get_valid_write_ids_args(const ThriftHiveMetastore_get_valid_write_ids_args&);
+  ThriftHiveMetastore_get_valid_write_ids_args& operator=(const ThriftHiveMetastore_get_valid_write_ids_args&);
+  ThriftHiveMetastore_get_valid_write_ids_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_valid_write_ids_args() throw();
+  GetValidWriteIdsRequest req;
+
+  _ThriftHiveMetastore_get_valid_write_ids_args__isset __isset;
+
+  void __set_req(const GetValidWriteIdsRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_get_valid_write_ids_args & rhs) const
+  {
+    if (!(req == rhs.req))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_valid_write_ids_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_valid_write_ids_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_valid_write_ids_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_valid_write_ids_pargs() throw();
+  const GetValidWriteIdsRequest* req;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_valid_write_ids_result__isset {
+  _ThriftHiveMetastore_get_valid_write_ids_result__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_get_valid_write_ids_result__isset;
+
+class ThriftHiveMetastore_get_valid_write_ids_result {
+ public:
+
+  ThriftHiveMetastore_get_valid_write_ids_result(const ThriftHiveMetastore_get_valid_write_ids_result&);
+  ThriftHiveMetastore_get_valid_write_ids_result& operator=(const ThriftHiveMetastore_get_valid_write_ids_result&);
+  ThriftHiveMetastore_get_valid_write_ids_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_valid_write_ids_result() throw();
+  GetValidWriteIdsResult success;
+
+  _ThriftHiveMetastore_get_valid_write_ids_result__isset __isset;
+
+  void __set_success(const GetValidWriteIdsResult& val);
+
+  bool operator == (const ThriftHiveMetastore_get_valid_write_ids_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_valid_write_ids_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_valid_write_ids_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_valid_write_ids_presult__isset {
+  _ThriftHiveMetastore_get_valid_write_ids_presult__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_get_valid_write_ids_presult__isset;
+
+class ThriftHiveMetastore_get_valid_write_ids_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_valid_write_ids_presult() throw();
+  GetValidWriteIdsResult* success;
+
+  _ThriftHiveMetastore_get_valid_write_ids_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public  ::facebook::fb303::FacebookServiceClient {
  public:
   ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) :
@@ -19922,6 +20030,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void heartbeat_write_id(HeartbeatWriteIdResult& _return, const HeartbeatWriteIdRequest& req);
   void send_heartbeat_write_id(const HeartbeatWriteIdRequest& req);
   void recv_heartbeat_write_id(HeartbeatWriteIdResult& _return);
+  void get_valid_write_ids(GetValidWriteIdsResult& _return, const GetValidWriteIdsRequest& req);
+  void send_get_valid_write_ids(const GetValidWriteIdsRequest& req);
+  void recv_get_valid_write_ids(GetValidWriteIdsResult& _return);
 };
 
 class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceProcessor {
@@ -20084,6 +20195,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_next_write_id(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_finalize_write_id(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_heartbeat_write_id(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_valid_write_ids(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
  public:
   ThriftHiveMetastoreProcessor(boost::shared_ptr<ThriftHiveMetastoreIf> iface) :
      ::facebook::fb303::FacebookServiceProcessor(iface),
@@ -20240,6 +20352,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_next_write_id"] = &ThriftHiveMetastoreProcessor::process_get_next_write_id;
     processMap_["finalize_write_id"] = &ThriftHiveMetastoreProcessor::process_finalize_write_id;
     processMap_["heartbeat_write_id"] = &ThriftHiveMetastoreProcessor::process_heartbeat_write_id;
+    processMap_["get_valid_write_ids"] = &ThriftHiveMetastoreProcessor::process_get_valid_write_ids;
   }
 
   virtual ~ThriftHiveMetastoreProcessor() {}
@@ -21730,6 +21843,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void get_valid_write_ids(GetValidWriteIdsResult& _return, const GetValidWriteIdsRequest& req) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_valid_write_ids(_return, req);
+    }
+    ifaces_[i]->get_valid_write_ids(_return, req);
+    return;
+  }
+
 };
 
 // The 'concurrent' client is a thread safe client that correctly handles
@@ -22202,6 +22325,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void heartbeat_write_id(HeartbeatWriteIdResult& _return, const HeartbeatWriteIdRequest& req);
   int32_t send_heartbeat_write_id(const HeartbeatWriteIdRequest& req);
   void recv_heartbeat_write_id(HeartbeatWriteIdResult& _return, const int32_t seqid);
+  void get_valid_write_ids(GetValidWriteIdsResult& _return, const GetValidWriteIdsRequest& req);
+  int32_t send_get_valid_write_ids(const GetValidWriteIdsRequest& req);
+  void recv_get_valid_write_ids(GetValidWriteIdsResult& _return, const int32_t seqid);
 };
 
 #ifdef _WIN32

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 317598e..f938da4 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -782,6 +782,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("heartbeat_write_id\n");
   }
 
+  void get_valid_write_ids(GetValidWriteIdsResult& _return, const GetValidWriteIdsRequest& req) {
+    // Your implementation goes here
+    printf("get_valid_write_ids\n");
+  }
+
 };
 
 int main(int argc, char **argv) {

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 bdfa35b..356477e 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -18260,6 +18260,291 @@ void HeartbeatWriteIdResult::printTo(std::ostream& out) const {
 }
 
 
+GetValidWriteIdsRequest::~GetValidWriteIdsRequest() throw() {
+}
+
+
+void GetValidWriteIdsRequest::__set_dbName(const std::string& val) {
+  this->dbName = val;
+}
+
+void GetValidWriteIdsRequest::__set_tblName(const std::string& val) {
+  this->tblName = val;
+}
+
+uint32_t GetValidWriteIdsRequest::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_tblName = 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->tblName);
+          isset_tblName = 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_tblName)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t GetValidWriteIdsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("GetValidWriteIdsRequest");
+
+  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->dbName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->tblName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(GetValidWriteIdsRequest &a, GetValidWriteIdsRequest &b) {
+  using ::std::swap;
+  swap(a.dbName, b.dbName);
+  swap(a.tblName, b.tblName);
+}
+
+GetValidWriteIdsRequest::GetValidWriteIdsRequest(const GetValidWriteIdsRequest& other751) {
+  dbName = other751.dbName;
+  tblName = other751.tblName;
+}
+GetValidWriteIdsRequest& GetValidWriteIdsRequest::operator=(const GetValidWriteIdsRequest& other752) {
+  dbName = other752.dbName;
+  tblName = other752.tblName;
+  return *this;
+}
+void GetValidWriteIdsRequest::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "GetValidWriteIdsRequest(";
+  out << "dbName=" << to_string(dbName);
+  out << ", " << "tblName=" << to_string(tblName);
+  out << ")";
+}
+
+
+GetValidWriteIdsResult::~GetValidWriteIdsResult() throw() {
+}
+
+
+void GetValidWriteIdsResult::__set_lowWatermarkId(const int64_t val) {
+  this->lowWatermarkId = val;
+}
+
+void GetValidWriteIdsResult::__set_highWatermarkId(const int64_t val) {
+  this->highWatermarkId = val;
+}
+
+void GetValidWriteIdsResult::__set_areIdsValid(const bool val) {
+  this->areIdsValid = val;
+__isset.areIdsValid = true;
+}
+
+void GetValidWriteIdsResult::__set_ids(const std::vector<int64_t> & val) {
+  this->ids = val;
+__isset.ids = true;
+}
+
+uint32_t GetValidWriteIdsResult::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_lowWatermarkId = false;
+  bool isset_highWatermarkId = 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_I64) {
+          xfer += iprot->readI64(this->lowWatermarkId);
+          isset_lowWatermarkId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->highWatermarkId);
+          isset_highWatermarkId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->areIdsValid);
+          this->__isset.areIdsValid = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->ids.clear();
+            uint32_t _size753;
+            ::apache::thrift::protocol::TType _etype756;
+            xfer += iprot->readListBegin(_etype756, _size753);
+            this->ids.resize(_size753);
+            uint32_t _i757;
+            for (_i757 = 0; _i757 < _size753; ++_i757)
+            {
+              xfer += iprot->readI64(this->ids[_i757]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.ids = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_lowWatermarkId)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_highWatermarkId)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t GetValidWriteIdsResult::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("GetValidWriteIdsResult");
+
+  xfer += oprot->writeFieldBegin("lowWatermarkId", ::apache::thrift::protocol::T_I64, 1);
+  xfer += oprot->writeI64(this->lowWatermarkId);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("highWatermarkId", ::apache::thrift::protocol::T_I64, 2);
+  xfer += oprot->writeI64(this->highWatermarkId);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.areIdsValid) {
+    xfer += oprot->writeFieldBegin("areIdsValid", ::apache::thrift::protocol::T_BOOL, 3);
+    xfer += oprot->writeBool(this->areIdsValid);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.ids) {
+    xfer += oprot->writeFieldBegin("ids", ::apache::thrift::protocol::T_LIST, 4);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->ids.size()));
+      std::vector<int64_t> ::const_iterator _iter758;
+      for (_iter758 = this->ids.begin(); _iter758 != this->ids.end(); ++_iter758)
+      {
+        xfer += oprot->writeI64((*_iter758));
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(GetValidWriteIdsResult &a, GetValidWriteIdsResult &b) {
+  using ::std::swap;
+  swap(a.lowWatermarkId, b.lowWatermarkId);
+  swap(a.highWatermarkId, b.highWatermarkId);
+  swap(a.areIdsValid, b.areIdsValid);
+  swap(a.ids, b.ids);
+  swap(a.__isset, b.__isset);
+}
+
+GetValidWriteIdsResult::GetValidWriteIdsResult(const GetValidWriteIdsResult& other759) {
+  lowWatermarkId = other759.lowWatermarkId;
+  highWatermarkId = other759.highWatermarkId;
+  areIdsValid = other759.areIdsValid;
+  ids = other759.ids;
+  __isset = other759.__isset;
+}
+GetValidWriteIdsResult& GetValidWriteIdsResult::operator=(const GetValidWriteIdsResult& other760) {
+  lowWatermarkId = other760.lowWatermarkId;
+  highWatermarkId = other760.highWatermarkId;
+  areIdsValid = other760.areIdsValid;
+  ids = other760.ids;
+  __isset = other760.__isset;
+  return *this;
+}
+void GetValidWriteIdsResult::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "GetValidWriteIdsResult(";
+  out << "lowWatermarkId=" << to_string(lowWatermarkId);
+  out << ", " << "highWatermarkId=" << to_string(highWatermarkId);
+  out << ", " << "areIdsValid="; (__isset.areIdsValid ? (out << to_string(areIdsValid)) : (out << "<null>"));
+  out << ", " << "ids="; (__isset.ids ? (out << to_string(ids)) : (out << "<null>"));
+  out << ")";
+}
+
+
 GetAllFunctionsResponse::~GetAllFunctionsResponse() throw() {
 }
 
@@ -18294,14 +18579,14 @@ uint32_t GetAllFunctionsResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->functions.clear();
-            uint32_t _size751;
-            ::apache::thrift::protocol::TType _etype754;
-            xfer += iprot->readListBegin(_etype754, _size751);
-            this->functions.resize(_size751);
-            uint32_t _i755;
-            for (_i755 = 0; _i755 < _size751; ++_i755)
+            uint32_t _size761;
+            ::apache::thrift::protocol::TType _etype764;
+            xfer += iprot->readListBegin(_etype764, _size761);
+            this->functions.resize(_size761);
+            uint32_t _i765;
+            for (_i765 = 0; _i765 < _size761; ++_i765)
             {
-              xfer += this->functions[_i755].read(iprot);
+              xfer += this->functions[_i765].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -18331,10 +18616,10 @@ uint32_t GetAllFunctionsResponse::write(::apache::thrift::protocol::TProtocol* o
     xfer += oprot->writeFieldBegin("functions", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->functions.size()));
-      std::vector<Function> ::const_iterator _iter756;
-      for (_iter756 = this->functions.begin(); _iter756 != this->functions.end(); ++_iter756)
+      std::vector<Function> ::const_iterator _iter766;
+      for (_iter766 = this->functions.begin(); _iter766 != this->functions.end(); ++_iter766)
       {
-        xfer += (*_iter756).write(oprot);
+        xfer += (*_iter766).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -18351,13 +18636,13 @@ void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other757) {
-  functions = other757.functions;
-  __isset = other757.__isset;
+GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other767) {
+  functions = other767.functions;
+  __isset = other767.__isset;
 }
-GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other758) {
-  functions = other758.functions;
-  __isset = other758.__isset;
+GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other768) {
+  functions = other768.functions;
+  __isset = other768.__isset;
   return *this;
 }
 void GetAllFunctionsResponse::printTo(std::ostream& out) const {
@@ -18499,19 +18784,19 @@ void swap(TableMeta &a, TableMeta &b) {
   swap(a.__isset, b.__isset);
 }
 
-TableMeta::TableMeta(const TableMeta& other759) {
-  dbName = other759.dbName;
-  tableName = other759.tableName;
-  tableType = other759.tableType;
-  comments = other759.comments;
-  __isset = other759.__isset;
+TableMeta::TableMeta(const TableMeta& other769) {
+  dbName = other769.dbName;
+  tableName = other769.tableName;
+  tableType = other769.tableType;
+  comments = other769.comments;
+  __isset = other769.__isset;
 }
-TableMeta& TableMeta::operator=(const TableMeta& other760) {
-  dbName = other760.dbName;
-  tableName = other760.tableName;
-  tableType = other760.tableType;
-  comments = other760.comments;
-  __isset = other760.__isset;
+TableMeta& TableMeta::operator=(const TableMeta& other770) {
+  dbName = other770.dbName;
+  tableName = other770.tableName;
+  tableType = other770.tableType;
+  comments = other770.comments;
+  __isset = other770.__isset;
   return *this;
 }
 void TableMeta::printTo(std::ostream& out) const {
@@ -18594,13 +18879,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other761) : TException() {
-  message = other761.message;
-  __isset = other761.__isset;
+MetaException::MetaException(const MetaException& other771) : TException() {
+  message = other771.message;
+  __isset = other771.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other762) {
-  message = other762.message;
-  __isset = other762.__isset;
+MetaException& MetaException::operator=(const MetaException& other772) {
+  message = other772.message;
+  __isset = other772.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -18691,13 +18976,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other763) : TException() {
-  message = other763.message;
-  __isset = other763.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other773) : TException() {
+  message = other773.message;
+  __isset = other773.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other764) {
-  message = other764.message;
-  __isset = other764.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other774) {
+  message = other774.message;
+  __isset = other774.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -18788,13 +19073,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other765) : TException() {
-  message = other765.message;
-  __isset = other765.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other775) : TException() {
+  message = other775.message;
+  __isset = other775.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other766) {
-  message = other766.message;
-  __isset = other766.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other776) {
+  message = other776.message;
+  __isset = other776.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -18885,13 +19170,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other767) : TException() {
-  message = other767.message;
-  __isset = other767.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other777) : TException() {
+  message = other777.message;
+  __isset = other777.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other768) {
-  message = other768.message;
-  __isset = other768.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other778) {
+  message = other778.message;
+  __isset = other778.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -18982,13 +19267,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other769) : TException() {
-  message = other769.message;
-  __isset = other769.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other779) : TException() {
+  message = other779.message;
+  __isset = other779.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other770) {
-  message = other770.message;
-  __isset = other770.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other780) {
+  message = other780.message;
+  __isset = other780.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -19079,13 +19364,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other771) : TException() {
-  message = other771.message;
-  __isset = other771.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other781) : TException() {
+  message = other781.message;
+  __isset = other781.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other772) {
-  message = other772.message;
-  __isset = other772.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other782) {
+  message = other782.message;
+  __isset = other782.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -19176,13 +19461,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other773) : TException() {
-  message = other773.message;
-  __isset = other773.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other783) : TException() {
+  message = other783.message;
+  __isset = other783.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other774) {
-  message = other774.message;
-  __isset = other774.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other784) {
+  message = other784.message;
+  __isset = other784.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -19273,13 +19558,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other775) : TException() {
-  message = other775.message;
-  __isset = other775.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other785) : TException() {
+  message = other785.message;
+  __isset = other785.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other776) {
-  message = other776.message;
-  __isset = other776.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other786) {
+  message = other786.message;
+  __isset = other786.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -19370,13 +19655,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other777) : TException() {
-  message = other777.message;
-  __isset = other777.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other787) : TException() {
+  message = other787.message;
+  __isset = other787.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other778) {
-  message = other778.message;
-  __isset = other778.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other788) {
+  message = other788.message;
+  __isset = other788.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -19467,13 +19752,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other779) : TException() {
-  message = other779.message;
-  __isset = other779.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other789) : TException() {
+  message = other789.message;
+  __isset = other789.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other780) {
-  message = other780.message;
-  __isset = other780.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other790) {
+  message = other790.message;
+  __isset = other790.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -19564,13 +19849,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other781) : TException() {
-  message = other781.message;
-  __isset = other781.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other791) : TException() {
+  message = other791.message;
+  __isset = other791.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other782) {
-  message = other782.message;
-  __isset = other782.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other792) {
+  message = other792.message;
+  __isset = other792.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -19661,13 +19946,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other783) : TException() {
-  message = other783.message;
-  __isset = other783.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other793) : TException() {
+  message = other793.message;
+  __isset = other793.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other784) {
-  message = other784.message;
-  __isset = other784.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other794) {
+  message = other794.message;
+  __isset = other794.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -19758,13 +20043,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other785) : TException() {
-  message = other785.message;
-  __isset = other785.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other795) : TException() {
+  message = other795.message;
+  __isset = other795.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other786) {
-  message = other786.message;
-  __isset = other786.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other796) {
+  message = other796.message;
+  __isset = other796.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -19855,13 +20140,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other787) : TException() {
-  message = other787.message;
-  __isset = other787.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other797) : TException() {
+  message = other797.message;
+  __isset = other797.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other788) {
-  message = other788.message;
-  __isset = other788.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other798) {
+  message = other798.message;
+  __isset = other798.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -19952,13 +20237,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other789) : TException() {
-  message = other789.message;
-  __isset = other789.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other799) : TException() {
+  message = other799.message;
+  __isset = other799.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other790) {
-  message = other790.message;
-  __isset = other790.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other800) {
+  message = other800.message;
+  __isset = other800.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -20049,13 +20334,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other791) : TException() {
-  message = other791.message;
-  __isset = other791.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other801) : TException() {
+  message = other801.message;
+  __isset = other801.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other792) {
-  message = other792.message;
-  __isset = other792.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other802) {
+  message = other802.message;
+  __isset = other802.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 beddd4c..b510dc9 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -396,6 +396,10 @@ class HeartbeatWriteIdRequest;
 
 class HeartbeatWriteIdResult;
 
+class GetValidWriteIdsRequest;
+
+class GetValidWriteIdsResult;
+
 class GetAllFunctionsResponse;
 
 class TableMeta;
@@ -7445,6 +7449,117 @@ inline std::ostream& operator<<(std::ostream& out, const HeartbeatWriteIdResult&
   return out;
 }
 
+
+class GetValidWriteIdsRequest {
+ public:
+
+  GetValidWriteIdsRequest(const GetValidWriteIdsRequest&);
+  GetValidWriteIdsRequest& operator=(const GetValidWriteIdsRequest&);
+  GetValidWriteIdsRequest() : dbName(), tblName() {
+  }
+
+  virtual ~GetValidWriteIdsRequest() throw();
+  std::string dbName;
+  std::string tblName;
+
+  void __set_dbName(const std::string& val);
+
+  void __set_tblName(const std::string& val);
+
+  bool operator == (const GetValidWriteIdsRequest & rhs) const
+  {
+    if (!(dbName == rhs.dbName))
+      return false;
+    if (!(tblName == rhs.tblName))
+      return false;
+    return true;
+  }
+  bool operator != (const GetValidWriteIdsRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const GetValidWriteIdsRequest & ) 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(GetValidWriteIdsRequest &a, GetValidWriteIdsRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const GetValidWriteIdsRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _GetValidWriteIdsResult__isset {
+  _GetValidWriteIdsResult__isset() : areIdsValid(false), ids(false) {}
+  bool areIdsValid :1;
+  bool ids :1;
+} _GetValidWriteIdsResult__isset;
+
+class GetValidWriteIdsResult {
+ public:
+
+  GetValidWriteIdsResult(const GetValidWriteIdsResult&);
+  GetValidWriteIdsResult& operator=(const GetValidWriteIdsResult&);
+  GetValidWriteIdsResult() : lowWatermarkId(0), highWatermarkId(0), areIdsValid(0) {
+  }
+
+  virtual ~GetValidWriteIdsResult() throw();
+  int64_t lowWatermarkId;
+  int64_t highWatermarkId;
+  bool areIdsValid;
+  std::vector<int64_t>  ids;
+
+  _GetValidWriteIdsResult__isset __isset;
+
+  void __set_lowWatermarkId(const int64_t val);
+
+  void __set_highWatermarkId(const int64_t val);
+
+  void __set_areIdsValid(const bool val);
+
+  void __set_ids(const std::vector<int64_t> & val);
+
+  bool operator == (const GetValidWriteIdsResult & rhs) const
+  {
+    if (!(lowWatermarkId == rhs.lowWatermarkId))
+      return false;
+    if (!(highWatermarkId == rhs.highWatermarkId))
+      return false;
+    if (__isset.areIdsValid != rhs.__isset.areIdsValid)
+      return false;
+    else if (__isset.areIdsValid && !(areIdsValid == rhs.areIdsValid))
+      return false;
+    if (__isset.ids != rhs.__isset.ids)
+      return false;
+    else if (__isset.ids && !(ids == rhs.ids))
+      return false;
+    return true;
+  }
+  bool operator != (const GetValidWriteIdsResult &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const GetValidWriteIdsResult & ) 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(GetValidWriteIdsResult &a, GetValidWriteIdsResult &b);
+
+inline std::ostream& operator<<(std::ostream& out, const GetValidWriteIdsResult& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _GetAllFunctionsResponse__isset {
   _GetAllFunctionsResponse__isset() : functions(false) {}
   bool functions :1;

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index f427a3a..49a1be2 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -346,14 +346,14 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list634.size);
-                Function _elem635;
-                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
+                org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list642.size);
+                Function _elem643;
+                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
                 {
-                  _elem635 = new Function();
-                  _elem635.read(iprot);
-                  struct.functions.add(_elem635);
+                  _elem643 = new Function();
+                  _elem643.read(iprot);
+                  struct.functions.add(_elem643);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter637 : struct.functions)
+            for (Function _iter645 : struct.functions)
             {
-              _iter637.write(oprot);
+              _iter645.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter638 : struct.functions)
+          for (Function _iter646 : struct.functions)
           {
-            _iter638.write(oprot);
+            _iter646.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list639.size);
-          Function _elem640;
-          for (int _i641 = 0; _i641 < _list639.size; ++_i641)
+          org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list647.size);
+          Function _elem648;
+          for (int _i649 = 0; _i649 < _list647.size; ++_i649)
           {
-            _elem640 = new Function();
-            _elem640.read(iprot);
-            struct.functions.add(_elem640);
+            _elem648 = new Function();
+            _elem648.read(iprot);
+            struct.functions.add(_elem648);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
new file mode 100644
index 0000000..90f103a
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
@@ -0,0 +1,490 @@
+/**
+ * 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 GetValidWriteIdsRequest implements org.apache.thrift.TBase<GetValidWriteIdsRequest, GetValidWriteIdsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetValidWriteIdsRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetValidWriteIdsRequest");
+
+  private static final org.apache.thrift.protocol.TField DB_NAME_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 TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetValidWriteIdsRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetValidWriteIdsRequestTupleSchemeFactory());
+  }
+
+  private String dbName; // required
+  private String tblName; // 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 {
+    DB_NAME((short)1, "dbName"),
+    TBL_NAME((short)2, "tblName");
+
+    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: // DB_NAME
+          return DB_NAME;
+        case 2: // TBL_NAME
+          return TBL_NAME;
+        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.DB_NAME, 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.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tblName", 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(GetValidWriteIdsRequest.class, metaDataMap);
+  }
+
+  public GetValidWriteIdsRequest() {
+  }
+
+  public GetValidWriteIdsRequest(
+    String dbName,
+    String tblName)
+  {
+    this();
+    this.dbName = dbName;
+    this.tblName = tblName;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetValidWriteIdsRequest(GetValidWriteIdsRequest other) {
+    if (other.isSetDbName()) {
+      this.dbName = other.dbName;
+    }
+    if (other.isSetTblName()) {
+      this.tblName = other.tblName;
+    }
+  }
+
+  public GetValidWriteIdsRequest deepCopy() {
+    return new GetValidWriteIdsRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.dbName = null;
+    this.tblName = 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 getTblName() {
+    return this.tblName;
+  }
+
+  public void setTblName(String tblName) {
+    this.tblName = tblName;
+  }
+
+  public void unsetTblName() {
+    this.tblName = null;
+  }
+
+  /** Returns true if field tblName is set (has been assigned a value) and false otherwise */
+  public boolean isSetTblName() {
+    return this.tblName != null;
+  }
+
+  public void setTblNameIsSet(boolean value) {
+    if (!value) {
+      this.tblName = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case DB_NAME:
+      if (value == null) {
+        unsetDbName();
+      } else {
+        setDbName((String)value);
+      }
+      break;
+
+    case TBL_NAME:
+      if (value == null) {
+        unsetTblName();
+      } else {
+        setTblName((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDbName();
+
+    case TBL_NAME:
+      return getTblName();
+
+    }
+    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 DB_NAME:
+      return isSetDbName();
+    case TBL_NAME:
+      return isSetTblName();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetValidWriteIdsRequest)
+      return this.equals((GetValidWriteIdsRequest)that);
+    return false;
+  }
+
+  public boolean equals(GetValidWriteIdsRequest 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_tblName = true && this.isSetTblName();
+    boolean that_present_tblName = true && that.isSetTblName();
+    if (this_present_tblName || that_present_tblName) {
+      if (!(this_present_tblName && that_present_tblName))
+        return false;
+      if (!this.tblName.equals(that.tblName))
+        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_tblName = true && (isSetTblName());
+    list.add(present_tblName);
+    if (present_tblName)
+      list.add(tblName);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetValidWriteIdsRequest 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(isSetTblName()).compareTo(other.isSetTblName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTblName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tblName, other.tblName);
+      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("GetValidWriteIdsRequest(");
+    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("tblName:");
+    if (this.tblName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tblName);
+    }
+    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 (!isSetTblName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tblName' 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 GetValidWriteIdsRequestStandardSchemeFactory implements SchemeFactory {
+    public GetValidWriteIdsRequestStandardScheme getScheme() {
+      return new GetValidWriteIdsRequestStandardScheme();
+    }
+  }
+
+  private static class GetValidWriteIdsRequestStandardScheme extends StandardScheme<GetValidWriteIdsRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetValidWriteIdsRequest 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: // DB_NAME
+            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: // TBL_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tblName = iprot.readString();
+              struct.setTblNameIsSet(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, GetValidWriteIdsRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.dbName != null) {
+        oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.dbName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tblName != null) {
+        oprot.writeFieldBegin(TBL_NAME_FIELD_DESC);
+        oprot.writeString(struct.tblName);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetValidWriteIdsRequestTupleSchemeFactory implements SchemeFactory {
+    public GetValidWriteIdsRequestTupleScheme getScheme() {
+      return new GetValidWriteIdsRequestTupleScheme();
+    }
+  }
+
+  private static class GetValidWriteIdsRequestTupleScheme extends TupleScheme<GetValidWriteIdsRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.dbName);
+      oprot.writeString(struct.tblName);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.dbName = iprot.readString();
+      struct.setDbNameIsSet(true);
+      struct.tblName = iprot.readString();
+      struct.setTblNameIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResult.java
new file mode 100644
index 0000000..a51f321
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResult.java
@@ -0,0 +1,740 @@
+/**
+ * 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 GetValidWriteIdsResult implements org.apache.thrift.TBase<GetValidWriteIdsResult, GetValidWriteIdsResult._Fields>, java.io.Serializable, Cloneable, Comparable<GetValidWriteIdsResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetValidWriteIdsResult");
+
+  private static final org.apache.thrift.protocol.TField LOW_WATERMARK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("lowWatermarkId", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField HIGH_WATERMARK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("highWatermarkId", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField ARE_IDS_VALID_FIELD_DESC = new org.apache.thrift.protocol.TField("areIdsValid", org.apache.thrift.protocol.TType.BOOL, (short)3);
+  private static final org.apache.thrift.protocol.TField IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("ids", org.apache.thrift.protocol.TType.LIST, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetValidWriteIdsResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetValidWriteIdsResultTupleSchemeFactory());
+  }
+
+  private long lowWatermarkId; // required
+  private long highWatermarkId; // required
+  private boolean areIdsValid; // optional
+  private List<Long> ids; // 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 {
+    LOW_WATERMARK_ID((short)1, "lowWatermarkId"),
+    HIGH_WATERMARK_ID((short)2, "highWatermarkId"),
+    ARE_IDS_VALID((short)3, "areIdsValid"),
+    IDS((short)4, "ids");
+
+    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: // LOW_WATERMARK_ID
+          return LOW_WATERMARK_ID;
+        case 2: // HIGH_WATERMARK_ID
+          return HIGH_WATERMARK_ID;
+        case 3: // ARE_IDS_VALID
+          return ARE_IDS_VALID;
+        case 4: // IDS
+          return IDS;
+        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
+  private static final int __LOWWATERMARKID_ISSET_ID = 0;
+  private static final int __HIGHWATERMARKID_ISSET_ID = 1;
+  private static final int __AREIDSVALID_ISSET_ID = 2;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.ARE_IDS_VALID,_Fields.IDS};
+  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.LOW_WATERMARK_ID, new org.apache.thrift.meta_data.FieldMetaData("lowWatermarkId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.HIGH_WATERMARK_ID, new org.apache.thrift.meta_data.FieldMetaData("highWatermarkId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.ARE_IDS_VALID, new org.apache.thrift.meta_data.FieldMetaData("areIdsValid", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.IDS, new org.apache.thrift.meta_data.FieldMetaData("ids", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetValidWriteIdsResult.class, metaDataMap);
+  }
+
+  public GetValidWriteIdsResult() {
+  }
+
+  public GetValidWriteIdsResult(
+    long lowWatermarkId,
+    long highWatermarkId)
+  {
+    this();
+    this.lowWatermarkId = lowWatermarkId;
+    setLowWatermarkIdIsSet(true);
+    this.highWatermarkId = highWatermarkId;
+    setHighWatermarkIdIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetValidWriteIdsResult(GetValidWriteIdsResult other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.lowWatermarkId = other.lowWatermarkId;
+    this.highWatermarkId = other.highWatermarkId;
+    this.areIdsValid = other.areIdsValid;
+    if (other.isSetIds()) {
+      List<Long> __this__ids = new ArrayList<Long>(other.ids);
+      this.ids = __this__ids;
+    }
+  }
+
+  public GetValidWriteIdsResult deepCopy() {
+    return new GetValidWriteIdsResult(this);
+  }
+
+  @Override
+  public void clear() {
+    setLowWatermarkIdIsSet(false);
+    this.lowWatermarkId = 0;
+    setHighWatermarkIdIsSet(false);
+    this.highWatermarkId = 0;
+    setAreIdsValidIsSet(false);
+    this.areIdsValid = false;
+    this.ids = null;
+  }
+
+  public long getLowWatermarkId() {
+    return this.lowWatermarkId;
+  }
+
+  public void setLowWatermarkId(long lowWatermarkId) {
+    this.lowWatermarkId = lowWatermarkId;
+    setLowWatermarkIdIsSet(true);
+  }
+
+  public void unsetLowWatermarkId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LOWWATERMARKID_ISSET_ID);
+  }
+
+  /** Returns true if field lowWatermarkId is set (has been assigned a value) and false otherwise */
+  public boolean isSetLowWatermarkId() {
+    return EncodingUtils.testBit(__isset_bitfield, __LOWWATERMARKID_ISSET_ID);
+  }
+
+  public void setLowWatermarkIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LOWWATERMARKID_ISSET_ID, value);
+  }
+
+  public long getHighWatermarkId() {
+    return this.highWatermarkId;
+  }
+
+  public void setHighWatermarkId(long highWatermarkId) {
+    this.highWatermarkId = highWatermarkId;
+    setHighWatermarkIdIsSet(true);
+  }
+
+  public void unsetHighWatermarkId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HIGHWATERMARKID_ISSET_ID);
+  }
+
+  /** Returns true if field highWatermarkId is set (has been assigned a value) and false otherwise */
+  public boolean isSetHighWatermarkId() {
+    return EncodingUtils.testBit(__isset_bitfield, __HIGHWATERMARKID_ISSET_ID);
+  }
+
+  public void setHighWatermarkIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HIGHWATERMARKID_ISSET_ID, value);
+  }
+
+  public boolean isAreIdsValid() {
+    return this.areIdsValid;
+  }
+
+  public void setAreIdsValid(boolean areIdsValid) {
+    this.areIdsValid = areIdsValid;
+    setAreIdsValidIsSet(true);
+  }
+
+  public void unsetAreIdsValid() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __AREIDSVALID_ISSET_ID);
+  }
+
+  /** Returns true if field areIdsValid is set (has been assigned a value) and false otherwise */
+  public boolean isSetAreIdsValid() {
+    return EncodingUtils.testBit(__isset_bitfield, __AREIDSVALID_ISSET_ID);
+  }
+
+  public void setAreIdsValidIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __AREIDSVALID_ISSET_ID, value);
+  }
+
+  public int getIdsSize() {
+    return (this.ids == null) ? 0 : this.ids.size();
+  }
+
+  public java.util.Iterator<Long> getIdsIterator() {
+    return (this.ids == null) ? null : this.ids.iterator();
+  }
+
+  public void addToIds(long elem) {
+    if (this.ids == null) {
+      this.ids = new ArrayList<Long>();
+    }
+    this.ids.add(elem);
+  }
+
+  public List<Long> getIds() {
+    return this.ids;
+  }
+
+  public void setIds(List<Long> ids) {
+    this.ids = ids;
+  }
+
+  public void unsetIds() {
+    this.ids = null;
+  }
+
+  /** Returns true if field ids is set (has been assigned a value) and false otherwise */
+  public boolean isSetIds() {
+    return this.ids != null;
+  }
+
+  public void setIdsIsSet(boolean value) {
+    if (!value) {
+      this.ids = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case LOW_WATERMARK_ID:
+      if (value == null) {
+        unsetLowWatermarkId();
+      } else {
+        setLowWatermarkId((Long)value);
+      }
+      break;
+
+    case HIGH_WATERMARK_ID:
+      if (value == null) {
+        unsetHighWatermarkId();
+      } else {
+        setHighWatermarkId((Long)value);
+      }
+      break;
+
+    case ARE_IDS_VALID:
+      if (value == null) {
+        unsetAreIdsValid();
+      } else {
+        setAreIdsValid((Boolean)value);
+      }
+      break;
+
+    case IDS:
+      if (value == null) {
+        unsetIds();
+      } else {
+        setIds((List<Long>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case LOW_WATERMARK_ID:
+      return getLowWatermarkId();
+
+    case HIGH_WATERMARK_ID:
+      return getHighWatermarkId();
+
+    case ARE_IDS_VALID:
+      return isAreIdsValid();
+
+    case IDS:
+      return getIds();
+
+    }
+    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 LOW_WATERMARK_ID:
+      return isSetLowWatermarkId();
+    case HIGH_WATERMARK_ID:
+      return isSetHighWatermarkId();
+    case ARE_IDS_VALID:
+      return isSetAreIdsValid();
+    case IDS:
+      return isSetIds();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetValidWriteIdsResult)
+      return this.equals((GetValidWriteIdsResult)that);
+    return false;
+  }
+
+  public boolean equals(GetValidWriteIdsResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_lowWatermarkId = true;
+    boolean that_present_lowWatermarkId = true;
+    if (this_present_lowWatermarkId || that_present_lowWatermarkId) {
+      if (!(this_present_lowWatermarkId && that_present_lowWatermarkId))
+        return false;
+      if (this.lowWatermarkId != that.lowWatermarkId)
+        return false;
+    }
+
+    boolean this_present_highWatermarkId = true;
+    boolean that_present_highWatermarkId = true;
+    if (this_present_highWatermarkId || that_present_highWatermarkId) {
+      if (!(this_present_highWatermarkId && that_present_highWatermarkId))
+        return false;
+      if (this.highWatermarkId != that.highWatermarkId)
+        return false;
+    }
+
+    boolean this_present_areIdsValid = true && this.isSetAreIdsValid();
+    boolean that_present_areIdsValid = true && that.isSetAreIdsValid();
+    if (this_present_areIdsValid || that_present_areIdsValid) {
+      if (!(this_present_areIdsValid && that_present_areIdsValid))
+        return false;
+      if (this.areIdsValid != that.areIdsValid)
+        return false;
+    }
+
+    boolean this_present_ids = true && this.isSetIds();
+    boolean that_present_ids = true && that.isSetIds();
+    if (this_present_ids || that_present_ids) {
+      if (!(this_present_ids && that_present_ids))
+        return false;
+      if (!this.ids.equals(that.ids))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_lowWatermarkId = true;
+    list.add(present_lowWatermarkId);
+    if (present_lowWatermarkId)
+      list.add(lowWatermarkId);
+
+    boolean present_highWatermarkId = true;
+    list.add(present_highWatermarkId);
+    if (present_highWatermarkId)
+      list.add(highWatermarkId);
+
+    boolean present_areIdsValid = true && (isSetAreIdsValid());
+    list.add(present_areIdsValid);
+    if (present_areIdsValid)
+      list.add(areIdsValid);
+
+    boolean present_ids = true && (isSetIds());
+    list.add(present_ids);
+    if (present_ids)
+      list.add(ids);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetValidWriteIdsResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetLowWatermarkId()).compareTo(other.isSetLowWatermarkId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetLowWatermarkId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lowWatermarkId, other.lowWatermarkId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetHighWatermarkId()).compareTo(other.isSetHighWatermarkId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetHighWatermarkId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.highWatermarkId, other.highWatermarkId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetAreIdsValid()).compareTo(other.isSetAreIdsValid());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetAreIdsValid()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.areIdsValid, other.areIdsValid);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIds()).compareTo(other.isSetIds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ids, other.ids);
+      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("GetValidWriteIdsResult(");
+    boolean first = true;
+
+    sb.append("lowWatermarkId:");
+    sb.append(this.lowWatermarkId);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("highWatermarkId:");
+    sb.append(this.highWatermarkId);
+    first = false;
+    if (isSetAreIdsValid()) {
+      if (!first) sb.append(", ");
+      sb.append("areIdsValid:");
+      sb.append(this.areIdsValid);
+      first = false;
+    }
+    if (isSetIds()) {
+      if (!first) sb.append(", ");
+      sb.append("ids:");
+      if (this.ids == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.ids);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetLowWatermarkId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'lowWatermarkId' is unset! Struct:" + toString());
+    }
+
+    if (!isSetHighWatermarkId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'highWatermarkId' 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 {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      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 GetValidWriteIdsResultStandardSchemeFactory implements SchemeFactory {
+    public GetValidWriteIdsResultStandardScheme getScheme() {
+      return new GetValidWriteIdsResultStandardScheme();
+    }
+  }
+
+  private static class GetValidWriteIdsResultStandardScheme extends StandardScheme<GetValidWriteIdsResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetValidWriteIdsResult 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: // LOW_WATERMARK_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.lowWatermarkId = iprot.readI64();
+              struct.setLowWatermarkIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // HIGH_WATERMARK_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.highWatermarkId = iprot.readI64();
+              struct.setHighWatermarkIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ARE_IDS_VALID
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.areIdsValid = iprot.readBool();
+              struct.setAreIdsValidIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
+                struct.ids = new ArrayList<Long>(_list634.size);
+                long _elem635;
+                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
+                {
+                  _elem635 = iprot.readI64();
+                  struct.ids.add(_elem635);
+                }
+                iprot.readListEnd();
+              }
+              struct.setIdsIsSet(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, GetValidWriteIdsResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(LOW_WATERMARK_ID_FIELD_DESC);
+      oprot.writeI64(struct.lowWatermarkId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(HIGH_WATERMARK_ID_FIELD_DESC);
+      oprot.writeI64(struct.highWatermarkId);
+      oprot.writeFieldEnd();
+      if (struct.isSetAreIdsValid()) {
+        oprot.writeFieldBegin(ARE_IDS_VALID_FIELD_DESC);
+        oprot.writeBool(struct.areIdsValid);
+        oprot.writeFieldEnd();
+      }
+      if (struct.ids != null) {
+        if (struct.isSetIds()) {
+          oprot.writeFieldBegin(IDS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.ids.size()));
+            for (long _iter637 : struct.ids)
+            {
+              oprot.writeI64(_iter637);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetValidWriteIdsResultTupleSchemeFactory implements SchemeFactory {
+    public GetValidWriteIdsResultTupleScheme getScheme() {
+      return new GetValidWriteIdsResultTupleScheme();
+    }
+  }
+
+  private static class GetValidWriteIdsResultTupleScheme extends TupleScheme<GetValidWriteIdsResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.lowWatermarkId);
+      oprot.writeI64(struct.highWatermarkId);
+      BitSet optionals = new BitSet();
+      if (struct.isSetAreIdsValid()) {
+        optionals.set(0);
+      }
+      if (struct.isSetIds()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetAreIdsValid()) {
+        oprot.writeBool(struct.areIdsValid);
+      }
+      if (struct.isSetIds()) {
+        {
+          oprot.writeI32(struct.ids.size());
+          for (long _iter638 : struct.ids)
+          {
+            oprot.writeI64(_iter638);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.lowWatermarkId = iprot.readI64();
+      struct.setLowWatermarkIdIsSet(true);
+      struct.highWatermarkId = iprot.readI64();
+      struct.setHighWatermarkIdIsSet(true);
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.areIdsValid = iprot.readBool();
+        struct.setAreIdsValidIsSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.ids = new ArrayList<Long>(_list639.size);
+          long _elem640;
+          for (int _i641 = 0; _i641 < _list639.size; ++_i641)
+          {
+            _elem640 = iprot.readI64();
+            struct.ids.add(_elem640);
+          }
+        }
+        struct.setIdsIsSet(true);
+      }
+    }
+  }
+
+}
+


[05/31] hive git commit: HIVE-14217: Druid integration (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/druid_basic2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_basic2.q.out b/ql/src/test/results/clientpositive/druid_basic2.q.out
new file mode 100644
index 0000000..3205905
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid_basic2.q.out
@@ -0,0 +1,533 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_1
+PREHOOK: query: DESCRIBE FORMATTED druid_table_1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_1
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_1
+# col_name            	data_type           	comment             
+	 	 
+__time              	timestamp           	from deserializer   
+robot               	string              	from deserializer   
+namespace           	string              	from deserializer   
+anonymous           	string              	from deserializer   
+unpatrolled         	string              	from deserializer   
+page                	string              	from deserializer   
+language            	string              	from deserializer   
+newpage             	string              	from deserializer   
+user                	string              	from deserializer   
+count               	float               	from deserializer   
+added               	float               	from deserializer   
+delta               	float               	from deserializer   
+variation           	float               	from deserializer   
+deleted             	float               	from deserializer   
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	EXTERNAL_TABLE      	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	EXTERNAL            	TRUE                
+	druid.datasource    	wikipedia           
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	storage_handler     	org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+	totalSize           	0                   
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.druid.QTestDruidSerDe	 
+InputFormat:        	null                	 
+OutputFormat:       	null                	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+PREHOOK: query: -- dimension
+EXPLAIN EXTENDED
+SELECT robot FROM druid_table_1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- dimension
+EXPLAIN EXTENDED
+SELECT robot FROM druid_table_1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          GatherStats: false
+          Select Operator
+            expressions: robot (type: string)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- metric
+EXPLAIN EXTENDED
+SELECT delta FROM druid_table_1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- metric
+EXPLAIN EXTENDED
+SELECT delta FROM druid_table_1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":[],"metrics":["delta"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          GatherStats: false
+          Select Operator
+            expressions: delta (type: float)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: EXPLAIN EXTENDED
+SELECT robot
+FROM druid_table_1
+WHERE language = 'en'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN EXTENDED
+SELECT robot
+FROM druid_table_1
+WHERE language = 'en'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          GatherStats: false
+          Select Operator
+            expressions: robot (type: string)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: EXPLAIN EXTENDED
+SELECT DISTINCT robot
+FROM druid_table_1
+WHERE language = 'en'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN EXTENDED
+SELECT DISTINCT robot
+FROM druid_table_1
+WHERE language = 'en'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"ALL","dimensions":["robot"],"limitSpec":{"type":"default"},"filter":{"type":"selector","dimension":"language","value":"en"},"aggregations":[{"type":"longSum","name":"dummy_agg","fieldName":"dummy_agg"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type groupBy
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          GatherStats: false
+          Select Operator
+            expressions: robot (type: string)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- TODO: currently nothing is pushed - ISNOTNULL
+EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+  (SELECT robot, language
+  FROM druid_table_1) a
+  JOIN
+  (SELECT language
+  FROM druid_table_1) b
+  ON a.language = b.language
+)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- TODO: currently nothing is pushed - ISNOTNULL
+EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+  (SELECT robot, language
+  FROM druid_table_1) a
+  JOIN
+  (SELECT language
+  FROM druid_table_1) b
+  ON a.language = b.language
+)
+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: druid_table_1
+            filterExpr: language is not null (type: boolean)
+            properties:
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+              druid.query.type select
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: language is not null (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: robot (type: string), language (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: string)
+                  null sort order: a
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: string)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  tag: 0
+                  value expressions: _col0 (type: string)
+                  auto parallelism: false
+          TableScan
+            alias: druid_table_1
+            filterExpr: language is not null (type: boolean)
+            properties:
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+              druid.query.type select
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: language is not null (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: language (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: a
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  tag: 1
+                  auto parallelism: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: druid_table_1
+            input format: org.apache.hadoop.hive.druid.HiveDruidQueryBasedInputFormat
+            output format: org.apache.hadoop.hive.druid.HiveDruidOutputFormat
+            properties:
+              COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
+              EXTERNAL TRUE
+              bucket_count -1
+              columns __time,robot,namespace,anonymous,unpatrolled,page,language,newpage,user,count,added,delta,variation,deleted
+              columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
+              columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
+              druid.datasource wikipedia
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+              druid.query.type select
+#### A masked pattern was here ####
+              name default.druid_table_1
+              numFiles 0
+              numRows 0
+              rawDataSize 0
+              serialization.ddl struct druid_table_1 { timestamp __time, string robot, string namespace, string anonymous, string unpatrolled, string page, string language, string newpage, string user, float count, float added, float delta, float variation, float deleted}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.druid.QTestDruidSerDe
+              storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+              totalSize 0
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.druid.QTestDruidSerDe
+          
+              input format: org.apache.hadoop.hive.druid.HiveDruidQueryBasedInputFormat
+              output format: org.apache.hadoop.hive.druid.HiveDruidOutputFormat
+              properties:
+                COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
+                EXTERNAL TRUE
+                bucket_count -1
+                columns __time,robot,namespace,anonymous,unpatrolled,page,language,newpage,user,count,added,delta,variation,deleted
+                columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
+                columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
+                druid.datasource wikipedia
+                druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+                druid.query.type select
+#### A masked pattern was here ####
+                name default.druid_table_1
+                numFiles 0
+                numRows 0
+                rawDataSize 0
+                serialization.ddl struct druid_table_1 { timestamp __time, string robot, string namespace, string anonymous, string unpatrolled, string page, string language, string newpage, string user, float count, float added, float delta, float variation, float deleted}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.druid.QTestDruidSerDe
+                storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+                totalSize 0
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.druid.QTestDruidSerDe
+              name: default.druid_table_1
+            name: default.druid_table_1
+      Truncated Path -> Alias:
+        /druid_table_1 [$hdt$_0:druid_table_1, $hdt$_1:druid_table_1]
+      Needs Tagging: true
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col1 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col2
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: string), _col2 (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+#### A masked pattern was here ####
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  properties:
+                    columns _col0,_col1
+                    columns.types string:string
+                    escape.delim \
+                    hive.serialization.extend.additional.nesting.levels true
+                    serialization.escape.crlf true
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[5][tables = [druid_table_1, $hdt$_0]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+  (SELECT robot, language
+  FROM druid_table_1
+  WHERE language = 'en') a
+  JOIN
+  (SELECT language
+  FROM druid_table_1) b
+  ON a.language = b.language
+)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+  (SELECT robot, language
+  FROM druid_table_1
+  WHERE language = 'en') a
+  JOIN
+  (SELECT language
+  FROM druid_table_1) b
+  ON a.language = b.language
+)
+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: druid_table_1
+            properties:
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+              druid.query.type select
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+            GatherStats: false
+            Select Operator
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              Reduce Output Operator
+                null sort order: 
+                sort order: 
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                tag: 1
+                auto parallelism: false
+          TableScan
+            alias: druid_table_1
+            properties:
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+              druid.query.type select
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Reduce Output Operator
+              null sort order: 
+              sort order: 
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              tag: 0
+              value expressions: robot (type: string)
+              auto parallelism: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: druid_table_1
+            input format: org.apache.hadoop.hive.druid.HiveDruidQueryBasedInputFormat
+            output format: org.apache.hadoop.hive.druid.HiveDruidOutputFormat
+            properties:
+              COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
+              EXTERNAL TRUE
+              bucket_count -1
+              columns __time,robot,namespace,anonymous,unpatrolled,page,language,newpage,user,count,added,delta,variation,deleted
+              columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
+              columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
+              druid.datasource wikipedia
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+              druid.query.type select
+#### A masked pattern was here ####
+              name default.druid_table_1
+              numFiles 0
+              numRows 0
+              rawDataSize 0
+              serialization.ddl struct druid_table_1 { timestamp __time, string robot, string namespace, string anonymous, string unpatrolled, string page, string language, string newpage, string user, float count, float added, float delta, float variation, float deleted}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.druid.QTestDruidSerDe
+              storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+              totalSize 0
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.druid.QTestDruidSerDe
+          
+              input format: org.apache.hadoop.hive.druid.HiveDruidQueryBasedInputFormat
+              output format: org.apache.hadoop.hive.druid.HiveDruidOutputFormat
+              properties:
+                COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
+                EXTERNAL TRUE
+                bucket_count -1
+                columns __time,robot,namespace,anonymous,unpatrolled,page,language,newpage,user,count,added,delta,variation,deleted
+                columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
+                columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
+                druid.datasource wikipedia
+                druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+                druid.query.type select
+#### A masked pattern was here ####
+                name default.druid_table_1
+                numFiles 0
+                numRows 0
+                rawDataSize 0
+                serialization.ddl struct druid_table_1 { timestamp __time, string robot, string namespace, string anonymous, string unpatrolled, string page, string language, string newpage, string user, float count, float added, float delta, float variation, float deleted}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.druid.QTestDruidSerDe
+                storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+                totalSize 0
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.druid.QTestDruidSerDe
+              name: default.druid_table_1
+            name: default.druid_table_1
+      Truncated Path -> Alias:
+        /druid_table_1 [$hdt$_0:druid_table_1, druid_table_1]
+      Needs Tagging: true
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col1
+          Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col1 (type: string), 'en' (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
+#### A masked pattern was here ####
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  properties:
+                    columns _col0,_col1
+                    columns.types string:string
+                    escape.delim \
+                    hive.serialization.extend.additional.nesting.levels true
+                    serialization.escape.crlf true
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/druid_intervals.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_intervals.q.out b/ql/src/test/results/clientpositive/druid_intervals.q.out
new file mode 100644
index 0000000..984bb79
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid_intervals.q.out
@@ -0,0 +1,398 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_1
+PREHOOK: query: DESCRIBE FORMATTED druid_table_1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_1
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_1
+# col_name            	data_type           	comment             
+	 	 
+__time              	timestamp           	from deserializer   
+robot               	string              	from deserializer   
+namespace           	string              	from deserializer   
+anonymous           	string              	from deserializer   
+unpatrolled         	string              	from deserializer   
+page                	string              	from deserializer   
+language            	string              	from deserializer   
+newpage             	string              	from deserializer   
+user                	string              	from deserializer   
+count               	float               	from deserializer   
+added               	float               	from deserializer   
+delta               	float               	from deserializer   
+variation           	float               	from deserializer   
+deleted             	float               	from deserializer   
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	EXTERNAL_TABLE      	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	EXTERNAL            	TRUE                
+	druid.datasource    	wikipedia           
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	storage_handler     	org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+	totalSize           	0                   
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.druid.QTestDruidSerDe	 
+InputFormat:        	null                	 
+OutputFormat:       	null                	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+PREHOOK: query: -- (-\u221e\u2025+\u221e)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- (-\u221e\u2025+\u221e)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- (-\u221e\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` < '2012-03-01 00:00:00'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- (-\u221e\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` < '2012-03-01 00:00:00'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/2012-03-01T00:00:00.000-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- [2010-01-01 00:00:00\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- [2010-01-01 00:00:00\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2012-03-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00'
+    AND `__time` < '2011-01-01 00:00:00'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00'
+    AND `__time` < '2011-01-01 00:00:00'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.000-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00],[2012-01-01 00:00:00\u20252013-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+    OR (`__time` BETWEEN '2012-01-01 00:00:00' AND '2013-01-01 00:00:00')
+PREHOOK: type: QUERY
+POSTHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00],[2012-01-01 00:00:00\u20252013-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+    OR (`__time` BETWEEN '2012-01-01 00:00:00' AND '2013-01-01 00:00:00')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00","2012-01-01T00:00:00.000-08:00/2013-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- OVERLAP [2010-01-01 00:00:00\u20252012-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+    OR (`__time` BETWEEN '2010-06-01 00:00:00' AND '2012-01-01 00:00:00')
+PREHOOK: type: QUERY
+POSTHOOK: query: -- OVERLAP [2010-01-01 00:00:00\u20252012-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+    OR (`__time` BETWEEN '2010-06-01 00:00:00' AND '2012-01-01 00:00:00')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2012-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- IN: MULTIPLE INTERVALS [2010-01-01 00:00:00\u20252010-01-01 00:00:00),[2011-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+PREHOOK: type: QUERY
+POSTHOOK: query: -- IN: MULTIPLE INTERVALS [2010-01-01 00:00:00\u20252010-01-01 00:00:00),[2011-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2010-01-01T00:00:00.001-08:00","2011-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' AND `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' AND `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2010-01-01T00:00:00.001-08:00","2011-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00"],"filter":{"type":"selector","dimension":"robot","value":"user1"},"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+            druid.query.type select
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), 'user1' (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' OR `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' OR `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+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: druid_table_1
+            filterExpr: ((__time) IN ('2010-01-01 00:00:00', '2011-01-01 00:00:00') or (robot = 'user1')) (type: boolean)
+            properties:
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+              druid.query.type select
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: ((__time) IN ('2010-01-01 00:00:00', '2011-01-01 00:00:00') or (robot = 'user1')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: __time (type: timestamp), robot (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 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/58d1befa/ql/src/test/results/clientpositive/druid_timeseries.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_timeseries.q.out b/ql/src/test/results/clientpositive/druid_timeseries.q.out
new file mode 100644
index 0000000..8d974a4
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid_timeseries.q.out
@@ -0,0 +1,566 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_1
+PREHOOK: query: DESCRIBE FORMATTED druid_table_1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_1
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_1
+# col_name            	data_type           	comment             
+	 	 
+__time              	timestamp           	from deserializer   
+robot               	string              	from deserializer   
+namespace           	string              	from deserializer   
+anonymous           	string              	from deserializer   
+unpatrolled         	string              	from deserializer   
+page                	string              	from deserializer   
+language            	string              	from deserializer   
+newpage             	string              	from deserializer   
+user                	string              	from deserializer   
+count               	float               	from deserializer   
+added               	float               	from deserializer   
+delta               	float               	from deserializer   
+variation           	float               	from deserializer   
+deleted             	float               	from deserializer   
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	EXTERNAL_TABLE      	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	EXTERNAL            	TRUE                
+	druid.datasource    	wikipedia           
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	storage_handler     	org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+	totalSize           	0                   
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.druid.QTestDruidSerDe	 
+InputFormat:        	null                	 
+OutputFormat:       	null                	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+PREHOOK: query: -- GRANULARITY: ALL
+EXPLAIN
+SELECT max(added), sum(variation)
+FROM druid_table_1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: ALL
+EXPLAIN
+SELECT max(added), sum(variation)
+FROM druid_table_1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"ALL","aggregations":[{"type":"longMax","name":"$f0","fieldName":"added"},{"type":"doubleSum","name":"$f1","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: $f0 (type: bigint), $f1 (type: float)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: NONE
+EXPLAIN
+SELECT `__time`, max(added), sum(variation)
+FROM druid_table_1
+GROUP BY `__time`
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: NONE
+EXPLAIN
+SELECT `__time`, max(added), sum(variation)
+FROM druid_table_1
+GROUP BY `__time`
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"NONE","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: YEAR
+EXPLAIN
+SELECT floor_year(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_year(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: YEAR
+EXPLAIN
+SELECT floor_year(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_year(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"YEAR","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: QUARTER
+EXPLAIN
+SELECT floor_quarter(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_quarter(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: QUARTER
+EXPLAIN
+SELECT floor_quarter(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_quarter(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"QUARTER","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: MONTH
+EXPLAIN
+SELECT floor_month(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_month(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: MONTH
+EXPLAIN
+SELECT floor_month(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_month(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"MONTH","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: WEEK
+EXPLAIN
+SELECT floor_week(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_week(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: WEEK
+EXPLAIN
+SELECT floor_week(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_week(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"WEEK","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: DAY
+EXPLAIN
+SELECT floor_day(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_day(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: DAY
+EXPLAIN
+SELECT floor_day(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_day(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"DAY","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: HOUR
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_hour(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: HOUR
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_hour(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"HOUR","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: MINUTE
+EXPLAIN
+SELECT floor_minute(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_minute(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: MINUTE
+EXPLAIN
+SELECT floor_minute(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_minute(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"MINUTE","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: SECOND
+EXPLAIN
+SELECT floor_second(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_second(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: SECOND
+EXPLAIN
+SELECT floor_second(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_second(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"SECOND","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- WITH FILTER ON DIMENSION
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY floor_hour(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- WITH FILTER ON DIMENSION
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY floor_hour(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"HOUR","filter":{"type":"selector","dimension":"robot","value":"1"},"aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type timeseries
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+    BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+        AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY floor_hour(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+    BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+        AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY floor_hour(`__time`)
+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: druid_table_1
+            filterExpr: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+            properties:
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+              druid.query.type select
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: floor_hour(__time) (type: timestamp), added (type: float), variation (type: float)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Group By Operator
+                  aggregations: max(_col1), sum(_col2)
+                  keys: _col0 (type: timestamp)
+                  mode: hash
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: timestamp)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: timestamp)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    value expressions: _col1 (type: float), _col2 (type: double)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: max(VALUE._col0), sum(VALUE._col1)
+          keys: KEY._col0 (type: timestamp)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 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: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT subq.h, subq.m, subq.s
+FROM
+(
+  SELECT floor_hour(`__time`) as h, max(added) as m, sum(variation) as s
+  FROM druid_table_1
+  GROUP BY floor_hour(`__time`)
+) subq
+WHERE subq.h BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+        AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT subq.h, subq.m, subq.s
+FROM
+(
+  SELECT floor_hour(`__time`) as h, max(added) as m, sum(variation) as s
+  FROM druid_table_1
+  GROUP BY floor_hour(`__time`)
+) subq
+WHERE subq.h BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+        AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+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: druid_table_1
+            filterExpr: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+            properties:
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+              druid.query.type select
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: floor_hour(__time) (type: timestamp), added (type: float), variation (type: float)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Group By Operator
+                  aggregations: max(_col1), sum(_col2)
+                  keys: _col0 (type: timestamp)
+                  mode: hash
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: timestamp)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: timestamp)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    value expressions: _col1 (type: float), _col2 (type: double)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: max(VALUE._col0), sum(VALUE._col1)
+          keys: KEY._col0 (type: timestamp)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 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/58d1befa/ql/src/test/results/clientpositive/druid_topn.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_topn.q.out b/ql/src/test/results/clientpositive/druid_topn.q.out
new file mode 100644
index 0000000..17bdaed
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid_topn.q.out
@@ -0,0 +1,419 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_1
+PREHOOK: query: DESCRIBE FORMATTED druid_table_1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_1
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_1
+# col_name            	data_type           	comment             
+	 	 
+__time              	timestamp           	from deserializer   
+robot               	string              	from deserializer   
+namespace           	string              	from deserializer   
+anonymous           	string              	from deserializer   
+unpatrolled         	string              	from deserializer   
+page                	string              	from deserializer   
+language            	string              	from deserializer   
+newpage             	string              	from deserializer   
+user                	string              	from deserializer   
+count               	float               	from deserializer   
+added               	float               	from deserializer   
+delta               	float               	from deserializer   
+variation           	float               	from deserializer   
+deleted             	float               	from deserializer   
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	EXTERNAL_TABLE      	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	EXTERNAL            	TRUE                
+	druid.datasource    	wikipedia           
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	storage_handler     	org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+	totalSize           	0                   
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.druid.QTestDruidSerDe	 
+InputFormat:        	null                	 
+OutputFormat:       	null                	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+PREHOOK: query: -- GRANULARITY: ALL
+EXPLAIN
+SELECT robot, max(added) as m, sum(variation)
+FROM druid_table_1
+GROUP BY robot
+ORDER BY m DESC
+LIMIT 100
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: ALL
+EXPLAIN
+SELECT robot, max(added) as m, sum(variation)
+FROM druid_table_1
+GROUP BY robot
+ORDER BY m DESC
+LIMIT 100
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"ALL","dimension":"robot","metric":"$f1","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"threshold":100}
+            druid.query.type topN
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: robot (type: string), $f1 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: NONE
+EXPLAIN
+SELECT robot, `__time`, max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, `__time`
+ORDER BY s DESC
+LIMIT 100
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: NONE
+EXPLAIN
+SELECT robot, `__time`, max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, `__time`
+ORDER BY s DESC
+LIMIT 100
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"NONE","dimension":"robot","metric":"$f3","aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"threshold":100}
+            druid.query.type topN
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: robot (type: string), __time (type: timestamp), $f2 (type: bigint), $f3 (type: float)
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- GRANULARITY: YEAR
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s DESC
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: YEAR
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s DESC
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"YEAR","dimension":"robot","metric":"$f3","aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"threshold":10}
+            druid.query.type topN
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: robot (type: string), __time (type: timestamp), $f2 (type: bigint), $f3 (type: float)
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- ASC: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_month(`__time`)
+ORDER BY s
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- ASC: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_month(`__time`)
+ORDER BY s
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f3","direction":"ascending"}]},"aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type groupBy
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: robot (type: string), __time (type: timestamp), $f2 (type: bigint), $f3 (type: float)
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- MULTIPLE ORDER: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY s DESC, m DESC
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- MULTIPLE ORDER: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY s DESC, m DESC
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot","namespace"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f4","direction":"descending"},{"dimension":"$f3","direction":"descending"}]},"aggregations":[{"type":"longMax","name":"$f3","fieldName":"added"},{"type":"doubleSum","name":"$f4","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type groupBy
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: robot (type: string), __time (type: timestamp), $f3 (type: bigint), $f4 (type: float)
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- MULTIPLE ORDER MIXED: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY robot ASC, m DESC
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- MULTIPLE ORDER MIXED: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY robot ASC, m DESC
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot","namespace"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"robot","direction":"ascending"},{"dimension":"$f3","direction":"descending"}]},"aggregations":[{"type":"longMax","name":"$f3","fieldName":"added"},{"type":"doubleSum","name":"$f4","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type groupBy
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: robot (type: string), __time (type: timestamp), $f3 (type: bigint), $f4 (type: float)
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- WITH FILTER ON DIMENSION: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- WITH FILTER ON DIMENSION: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: druid_table_1
+          properties:
+            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"YEAR","dimensions":[],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f2","direction":"ascending"}]},"filter":{"type":"selector","dimension":"robot","value":"1"},"aggregations":[{"type":"longMax","name":"$f1_0","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+            druid.query.type groupBy
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: '1' (type: string), __time (type: timestamp), $f1_0 (type: bigint), $f2 (type: float)
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            ListSink
+
+PREHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT robot, floor_hour(`__time`), max(added) as m, sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+    BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+        AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY robot, floor_hour(`__time`)
+ORDER BY m
+LIMIT 100
+PREHOOK: type: QUERY
+POSTHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT robot, floor_hour(`__time`), max(added) as m, sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+    BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+        AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY robot, floor_hour(`__time`)
+ORDER BY m
+LIMIT 100
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: druid_table_1
+            filterExpr: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+            properties:
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+              druid.query.type select
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: robot (type: string), floor_hour(__time) (type: timestamp), added (type: float), variation (type: float)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Group By Operator
+                  aggregations: max(_col2), sum(_col3)
+                  keys: _col0 (type: string), _col1 (type: timestamp)
+                  mode: hash
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: string), _col1 (type: timestamp)
+                    sort order: ++
+                    Map-reduce partition columns: _col0 (type: string), _col1 (type: timestamp)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    value expressions: _col2 (type: float), _col3 (type: double)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: max(VALUE._col0), sum(VALUE._col1)
+          keys: KEY._col0 (type: string), KEY._col1 (type: timestamp)
+          mode: mergepartial
+          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-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col2 (type: float)
+              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: timestamp), _col3 (type: double)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: timestamp), KEY.reducesinkkey0 (type: float), VALUE._col2 (type: double)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Limit
+            Number of rows: 100
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1 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: 100
+      Processor Tree:
+        ListSink
+


[07/31] hive git commit: HIVE-14217: Druid integration (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java
new file mode 100644
index 0000000..43982aa
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java
@@ -0,0 +1,1053 @@
+/**
+ * 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.druid;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelFieldCollation.Direction;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveDateGranularity;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+/**
+ * Relational expression representing a scan of a Druid data set.
+ *
+ * TODO: to be removed when Calcite is upgraded to 1.9
+ */
+public class DruidQuery extends TableScan {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(DruidQuery.class);
+
+  protected QuerySpec querySpec;
+
+  final DruidTable druidTable;
+  final List<Interval> intervals;
+  final ImmutableList<RelNode> rels;
+
+  private static final Pattern VALID_SIG = Pattern.compile("sf?p?a?l?");
+
+  /**
+   * Creates a DruidQuery.
+   *
+   * @param cluster        Cluster
+   * @param traitSet       Traits
+   * @param table          Table
+   * @param druidTable     Druid table
+   * @param interval       Interval for the query
+   * @param rels           Internal relational expressions
+   */
+  private DruidQuery(RelOptCluster cluster, RelTraitSet traitSet,
+      RelOptTable table, DruidTable druidTable,
+      List<Interval> intervals, List<RelNode> rels) {
+    super(cluster, traitSet, table);
+    this.druidTable = druidTable;
+    this.intervals = ImmutableList.copyOf(intervals);
+    this.rels = ImmutableList.copyOf(rels);
+
+    assert isValid(Litmus.THROW);
+  }
+
+  /** Returns a string describing the operations inside this query.
+   *
+   * <p>For example, "sfpal" means {@link TableScan} (s)
+   * followed by {@link Filter} (f)
+   * followed by {@link Project} (p)
+   * followed by {@link Aggregate} (a)
+   * followed by {@link Sort} (l).
+   *
+   * @see #isValidSignature(String)
+   */
+  String signature() {
+    final StringBuilder b = new StringBuilder();
+    for (RelNode rel : rels) {
+      b.append(rel instanceof TableScan ? 's'
+          : rel instanceof Project ? 'p'
+          : rel instanceof Filter ? 'f'
+          : rel instanceof Aggregate ? 'a'
+          : rel instanceof Sort ? 'l'
+          : '!');
+    }
+    return b.toString();
+  }
+
+  @Override public boolean isValid(Litmus litmus) {
+    if (!super.isValid(litmus)) {
+      return false;
+    }
+    final String signature = signature();
+    if (!isValidSignature(signature)) {
+      return litmus.fail("invalid signature");
+    }
+    if (rels.isEmpty()) {
+      return litmus.fail("must have at least one rel");
+    }
+    for (int i = 0; i < rels.size(); i++) {
+      final RelNode r = rels.get(i);
+      if (i == 0) {
+        if (!(r instanceof TableScan)) {
+          return litmus.fail("first rel must be TableScan");
+        }
+        if (r.getTable() != table) {
+          return litmus.fail("first rel must be based on table table");
+        }
+      } else {
+        final List<RelNode> inputs = r.getInputs();
+        if (inputs.size() != 1 || inputs.get(0) != rels.get(i - 1)) {
+          return litmus.fail("each rel must have a single input");
+        }
+        if (r instanceof Aggregate) {
+          final Aggregate aggregate = (Aggregate) r;
+          if (aggregate.getGroupSets().size() != 1
+              || aggregate.indicator) {
+            return litmus.fail("no grouping sets");
+          }
+          for (AggregateCall call : aggregate.getAggCallList()) {
+            if (call.filterArg >= 0) {
+              return litmus.fail("no filtered aggregate functions");
+            }
+          }
+        }
+        if (r instanceof Filter) {
+          final Filter filter = (Filter) r;
+          if (!isValidFilter(filter.getCondition())) {
+            return litmus.fail("invalid filter");
+          }
+        }
+        if (r instanceof Sort) {
+          final Sort sort = (Sort) r;
+          if (sort.offset != null && RexLiteral.intValue(sort.offset) != 0) {
+            return litmus.fail("offset not supported");
+          }
+        }
+      }
+    }
+    return true;
+  }
+
+  boolean isValidFilter(RexNode e) {
+    switch (e.getKind()) {
+    case INPUT_REF:
+    case LITERAL:
+      return true;
+    case AND:
+    case OR:
+    case NOT:
+    case EQUALS:
+    case LESS_THAN:
+    case LESS_THAN_OR_EQUAL:
+    case GREATER_THAN:
+    case GREATER_THAN_OR_EQUAL:
+    case BETWEEN:
+    case IN:
+    case CAST:
+      return areValidFilters(((RexCall) e).getOperands());
+    default:
+      return false;
+    }
+  }
+
+  private boolean areValidFilters(List<RexNode> es) {
+    for (RexNode e : es) {
+      if (!isValidFilter(e)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /** Returns whether a signature represents an sequence of relational operators
+   * that can be translated into a valid Druid query. */
+  static boolean isValidSignature(String signature) {
+    return VALID_SIG.matcher(signature).matches();
+  }
+
+  /** Creates a DruidQuery. */
+  public static DruidQuery create(RelOptCluster cluster, RelTraitSet traitSet,
+      RelOptTable table, DruidTable druidTable, List<RelNode> rels) {
+    return new DruidQuery(cluster, traitSet, table, druidTable, druidTable.intervals, rels);
+  }
+
+  /** Creates a DruidQuery. */
+  private static DruidQuery create(RelOptCluster cluster, RelTraitSet traitSet,
+      RelOptTable table, DruidTable druidTable, List<Interval> intervals, List<RelNode> rels) {
+    return new DruidQuery(cluster, traitSet, table, druidTable, intervals, rels);
+  }
+
+  /** Extends a DruidQuery. */
+  public static DruidQuery extendQuery(DruidQuery query, RelNode r) {
+    final ImmutableList.Builder<RelNode> builder = ImmutableList.builder();
+    return DruidQuery.create(query.getCluster(), query.getTraitSet(), query.getTable(),
+            query.druidTable, query.intervals, builder.addAll(query.rels).add(r).build());
+  }
+
+  /** Extends a DruidQuery. */
+  public static DruidQuery extendQuery(DruidQuery query, List<Interval> intervals) {
+    return DruidQuery.create(query.getCluster(), query.getTraitSet(), query.getTable(),
+            query.druidTable, intervals, query.rels);
+  }
+
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    assert inputs.isEmpty();
+    return this;
+  }
+
+  @Override public RelDataType deriveRowType() {
+    return getCluster().getTypeFactory().createStructType(
+            Pair.right(Util.last(rels).getRowType().getFieldList()),
+            getQuerySpec().fieldNames);
+  }
+
+  public TableScan getTableScan() {
+    return (TableScan) rels.get(0);
+  }
+
+  public RelNode getTopNode() {
+    return Util.last(rels);
+  }
+
+  @Override public RelOptTable getTable() {
+    return table;
+  }
+
+  @Override public RelWriter explainTerms(RelWriter pw) {
+    for (RelNode rel : rels) {
+      if (rel instanceof TableScan) {
+        TableScan tableScan = (TableScan) rel;
+        pw.item("table", tableScan.getTable().getQualifiedName());
+        pw.item("intervals", intervals);
+      } else if (rel instanceof Filter) {
+        pw.item("filter", ((Filter) rel).getCondition());
+      } else if (rel instanceof Project) {
+        pw.item("projects", ((Project) rel).getProjects());
+      } else if (rel instanceof Aggregate) {
+        final Aggregate aggregate = (Aggregate) rel;
+        pw.item("groups", aggregate.getGroupSet())
+            .item("aggs", aggregate.getAggCallList());
+      } else if (rel instanceof Sort) {
+        final Sort sort = (Sort) rel;
+        for (Ord<RelFieldCollation> ord
+                : Ord.zip(sort.collation.getFieldCollations())) {
+          pw.item("sort" + ord.i, ord.e.getFieldIndex());
+        }
+        for (Ord<RelFieldCollation> ord
+            : Ord.zip(sort.collation.getFieldCollations())) {
+          pw.item("dir" + ord.i, ord.e.shortString());
+        }
+        pw.itemIf("fetch", sort.fetch, sort.fetch != null);
+      } else {
+        throw new AssertionError("rel type not supported in Druid query "
+            + rel);
+      }
+    }
+    return pw;
+  }
+
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    // Heuristic: we assume pushing query to Druid reduces cost by 90%
+    return Util.last(rels).computeSelfCost(planner, mq).multiplyBy(.1);
+  }
+
+  @Override public RelNode project(ImmutableBitSet fieldsUsed,
+      Set<RelDataTypeField> extraFields,
+      RelBuilder relBuilder) {
+    final int fieldCount = getRowType().getFieldCount();
+    if (fieldsUsed.equals(ImmutableBitSet.range(fieldCount))
+        && extraFields.isEmpty()) {
+      return this;
+    }
+    final List<RexNode> exprList = new ArrayList<>();
+    final List<String> nameList = new ArrayList<>();
+    final RexBuilder rexBuilder = getCluster().getRexBuilder();
+    final List<RelDataTypeField> fields = getRowType().getFieldList();
+
+    // Project the subset of fields.
+    for (int i : fieldsUsed) {
+      RelDataTypeField field = fields.get(i);
+      exprList.add(rexBuilder.makeInputRef(this, i));
+      nameList.add(field.getName());
+    }
+
+    // Project nulls for the extra fields. (Maybe a sub-class table has
+    // extra fields, but we don't.)
+    for (RelDataTypeField extraField : extraFields) {
+      exprList.add(
+          rexBuilder.ensureType(
+              extraField.getType(),
+              rexBuilder.constantNull(),
+              true));
+      nameList.add(extraField.getName());
+    }
+
+    HiveProject hp = (HiveProject) relBuilder.push(this).project(exprList, nameList).build();
+    hp.setSynthetic();
+    return hp;
+  }
+
+  public QuerySpec getQuerySpec() {
+    if (querySpec == null) {
+      querySpec = deriveQuerySpec();
+      assert querySpec != null : this;
+    }
+    return querySpec;
+  }
+
+  protected QuerySpec deriveQuerySpec() {
+    final RelDataType rowType = table.getRowType();
+    int i = 1;
+
+    RexNode filter = null;
+    if (i < rels.size() && rels.get(i) instanceof Filter) {
+      final Filter filterRel = (Filter) rels.get(i++);
+      filter = filterRel.getCondition();
+    }
+
+    List<RexNode> projects = null;
+    if (i < rels.size() && rels.get(i) instanceof Project) {
+      final Project project = (Project) rels.get(i++);
+      projects = project.getProjects();
+    }
+
+    ImmutableBitSet groupSet = null;
+    List<AggregateCall> aggCalls = null;
+    List<String> aggNames = null;
+    if (i < rels.size() && rels.get(i) instanceof Aggregate) {
+      final Aggregate aggregate = (Aggregate) rels.get(i++);
+      groupSet = aggregate.getGroupSet();
+      aggCalls = aggregate.getAggCallList();
+      aggNames = Util.skip(aggregate.getRowType().getFieldNames(),
+          groupSet.cardinality());
+    }
+
+    List<Integer> collationIndexes = null;
+    List<Direction> collationDirections = null;
+    Integer fetch = null;
+    if (i < rels.size() && rels.get(i) instanceof Sort) {
+      final Sort sort = (Sort) rels.get(i++);
+      collationIndexes = new ArrayList<>();
+      collationDirections = new ArrayList<>();
+      for (RelFieldCollation fCol: sort.collation.getFieldCollations()) {
+        collationIndexes.add(fCol.getFieldIndex());
+        collationDirections.add(fCol.getDirection());
+      }
+      fetch = sort.fetch != null ? RexLiteral.intValue(sort.fetch) : null;
+    }
+
+    if (i != rels.size()) {
+      throw new AssertionError("could not implement all rels");
+    }
+
+    return getQuery(rowType, filter, projects, groupSet, aggCalls, aggNames,
+            collationIndexes, collationDirections, fetch);
+  }
+
+  public String getQueryType() {
+    return getQuerySpec().queryType.getQueryName();
+  }
+
+  public String getQueryString() {
+    return getQuerySpec().queryString;
+  }
+
+  private QuerySpec getQuery(RelDataType rowType, RexNode filter, List<RexNode> projects,
+      ImmutableBitSet groupSet, List<AggregateCall> aggCalls, List<String> aggNames,
+      List<Integer> collationIndexes, List<Direction> collationDirections, Integer fetch) {
+    DruidQueryType queryType = DruidQueryType.SELECT;
+    final Translator translator = new Translator(druidTable, rowType);
+    List<String> fieldNames = rowType.getFieldNames();
+
+    // Handle filter
+    Json jsonFilter = null;
+    if (filter != null) {
+      jsonFilter = translator.translateFilter(filter);
+    }
+
+    // Then we handle project
+    if (projects != null) {
+      translator.metrics.clear();
+      translator.dimensions.clear();
+      final ImmutableList.Builder<String> builder = ImmutableList.builder();
+      for (RexNode project : projects) {
+        builder.add(translator.translate(project, true));
+      }
+      fieldNames = builder.build();
+    }
+
+    // Finally we handle aggregate and sort. Handling of these
+    // operators is more complex, since we need to extract
+    // the conditions to know whether the query will be
+    // executed as a Timeseries, TopN, or GroupBy in Druid
+    final List<String> dimensions = new ArrayList<>();
+    final List<JsonAggregation> aggregations = new ArrayList<>();
+    String granularity = "ALL";
+    Direction timeSeriesDirection = null;
+    JsonLimit limit = null;
+    if (groupSet != null) {
+      assert aggCalls != null;
+      assert aggNames != null;
+      assert aggCalls.size() == aggNames.size();
+
+      int timePositionIdx = -1;
+      final ImmutableList.Builder<String> builder = ImmutableList.builder();
+      if (projects != null) {
+        for (int groupKey : groupSet) {
+          final String s = fieldNames.get(groupKey);
+          final RexNode project = projects.get(groupKey);
+          if (project instanceof RexInputRef) {
+            // Reference, it could be to the timestamp column or any other dimension
+            final RexInputRef ref = (RexInputRef) project;
+            final String origin = druidTable.rowType.getFieldList().get(ref.getIndex()).getName();
+            if (origin.equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
+              granularity = "NONE";
+              builder.add(s);
+              assert timePositionIdx == -1;
+              timePositionIdx = groupKey;
+            } else {
+              dimensions.add(s);
+              builder.add(s);
+            }
+          } else if (project instanceof RexCall) {
+            // Call, check if we should infer granularity
+            RexCall call = (RexCall) project;
+            if (HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator())) {
+              granularity = call.getOperator().getName();
+              builder.add(s);
+              assert timePositionIdx == -1;
+              timePositionIdx = groupKey;
+            } else {
+              dimensions.add(s);
+              builder.add(s);
+            }
+          } else {
+            throw new AssertionError("incompatible project expression: " + project);
+          }
+        }
+      } else {
+        for (int groupKey : groupSet) {
+          final String s = fieldNames.get(groupKey);
+          if (s.equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
+            granularity = "NONE";
+            builder.add(s);
+            assert timePositionIdx == -1;
+            timePositionIdx = groupKey;
+          } else {
+            dimensions.add(s);
+            builder.add(s);
+          }
+        }
+      }
+
+      for (Pair<AggregateCall, String> agg : Pair.zip(aggCalls, aggNames)) {
+        final JsonAggregation jsonAggregation =
+            getJsonAggregation(fieldNames, agg.right, agg.left);
+        aggregations.add(jsonAggregation);
+        builder.add(jsonAggregation.name);
+      }
+
+      fieldNames = builder.build();
+
+      ImmutableList<JsonCollation> collations = null;
+      boolean sortsMetric = false;
+      if (collationIndexes != null) {
+        assert collationDirections != null;
+        ImmutableList.Builder<JsonCollation> colBuilder = new ImmutableList.Builder<JsonCollation>();
+        for (Pair<Integer,Direction> p : Pair.zip(collationIndexes, collationDirections)) {
+          colBuilder.add(new JsonCollation(fieldNames.get(p.left),
+                  p.right == Direction.DESCENDING ? "descending" : "ascending"));
+          if (p.left >= groupSet.cardinality() && p.right == Direction.DESCENDING) {
+            // Currently only support for DESC in TopN
+            sortsMetric = true;
+          } else if (p.left == timePositionIdx) {
+            assert timeSeriesDirection == null;
+            timeSeriesDirection = p.right;
+          }
+        }
+        collations = colBuilder.build();
+      }
+
+      limit = new JsonLimit("default", fetch, collations);
+
+      if (dimensions.isEmpty() && (collations == null || timeSeriesDirection != null)) {
+        queryType = DruidQueryType.TIMESERIES;
+        assert fetch == null;
+      } else if (dimensions.size() == 1 && sortsMetric && collations.size() == 1 && fetch != null) {
+        queryType = DruidQueryType.TOP_N;
+      } else {
+        queryType = DruidQueryType.GROUP_BY;
+      }
+    } else {
+      assert aggCalls == null;
+      assert aggNames == null;
+      assert collationIndexes == null || collationIndexes.isEmpty();
+      assert collationDirections == null || collationDirections.isEmpty();
+    }
+
+    final StringWriter sw = new StringWriter();
+    final JsonFactory factory = new JsonFactory();
+    try {
+      final JsonGenerator generator = factory.createGenerator(sw);
+
+      switch (queryType) {
+      case TIMESERIES:
+        generator.writeStartObject();
+
+        generator.writeStringField("queryType", "timeseries");
+        generator.writeStringField("dataSource", druidTable.dataSource);
+        generator.writeStringField("descending", timeSeriesDirection != null &&
+            timeSeriesDirection == Direction.DESCENDING ? "true" : "false");
+        generator.writeStringField("granularity", granularity);
+        writeFieldIf(generator, "filter", jsonFilter);
+        writeField(generator, "aggregations", aggregations);
+        writeFieldIf(generator, "postAggregations", null);
+        writeField(generator, "intervals", intervals);
+
+        generator.writeEndObject();
+        break;
+
+      case TOP_N:
+        generator.writeStartObject();
+
+        generator.writeStringField("queryType", "topN");
+        generator.writeStringField("dataSource", druidTable.dataSource);
+        generator.writeStringField("granularity", granularity);
+        generator.writeStringField("dimension", dimensions.get(0));
+        generator.writeStringField("metric", fieldNames.get(collationIndexes.get(0)));
+        writeFieldIf(generator, "filter", jsonFilter);
+        writeField(generator, "aggregations", aggregations);
+        writeFieldIf(generator, "postAggregations", null);
+        writeField(generator, "intervals", intervals);
+        generator.writeNumberField("threshold", fetch);
+
+        generator.writeEndObject();
+        break;
+
+      case GROUP_BY:
+        generator.writeStartObject();
+
+        if (aggregations.isEmpty()) {
+          // Druid requires at least one aggregation, otherwise gives:
+          //   Must have at least one AggregatorFactory
+          aggregations.add(
+              new JsonAggregation("longSum", "dummy_agg", "dummy_agg"));
+        }
+
+        generator.writeStringField("queryType", "groupBy");
+        generator.writeStringField("dataSource", druidTable.dataSource);
+        generator.writeStringField("granularity", granularity);
+        writeField(generator, "dimensions", dimensions);
+        writeFieldIf(generator, "limitSpec", limit);
+        writeFieldIf(generator, "filter", jsonFilter);
+        writeField(generator, "aggregations", aggregations);
+        writeFieldIf(generator, "postAggregations", null);
+        writeField(generator, "intervals", intervals);
+        writeFieldIf(generator, "having", null);
+
+        generator.writeEndObject();
+        break;
+
+      case SELECT:
+        generator.writeStartObject();
+
+        generator.writeStringField("queryType", "select");
+        generator.writeStringField("dataSource", druidTable.dataSource);
+        generator.writeStringField("descending", "false");
+        writeField(generator, "intervals", intervals);
+        writeFieldIf(generator, "filter", jsonFilter);
+        writeField(generator, "dimensions", translator.dimensions);
+        writeField(generator, "metrics", translator.metrics);
+        generator.writeStringField("granularity", granularity);
+
+        generator.writeFieldName("pagingSpec");
+        generator.writeStartObject();
+        generator.writeNumberField("threshold", fetch != null ? fetch : 1);
+        generator.writeEndObject();
+
+        generator.writeFieldName("context");
+        generator.writeStartObject();
+        generator.writeBooleanField(Constants.DRUID_QUERY_FETCH, fetch != null);
+        generator.writeEndObject();
+
+        generator.writeEndObject();
+        break;
+
+      default:
+        throw new AssertionError("unknown query type " + queryType);
+      }
+
+      generator.close();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    return new QuerySpec(queryType, sw.toString(), fieldNames);
+  }
+
+  private JsonAggregation getJsonAggregation(List<String> fieldNames,
+      String name, AggregateCall aggCall) {
+    final List<String> list = new ArrayList<>();
+    for (Integer arg : aggCall.getArgList()) {
+      list.add(fieldNames.get(arg));
+    }
+    final String only = Iterables.getFirst(list, null);
+    final boolean b = aggCall.getType().getSqlTypeName() == SqlTypeName.DOUBLE;
+    switch (aggCall.getAggregation().getKind()) {
+    case COUNT:
+      if (aggCall.isDistinct()) {
+        return new JsonCardinalityAggregation("cardinality", name, list);
+      }
+      return new JsonAggregation("count", name, only);
+    case SUM:
+    case SUM0:
+      return new JsonAggregation(b ? "doubleSum" : "longSum", name, only);
+    case MIN:
+      return new JsonAggregation(b ? "doubleMin" : "longMin", name, only);
+    case MAX:
+      return new JsonAggregation(b ? "doubleMax" : "longMax", name, only);
+    default:
+      throw new AssertionError("unknown aggregate " + aggCall);
+    }
+  }
+
+  private static void writeField(JsonGenerator generator, String fieldName,
+      Object o) throws IOException {
+    generator.writeFieldName(fieldName);
+    writeObject(generator, o);
+  }
+
+  private static void writeFieldIf(JsonGenerator generator, String fieldName,
+      Object o) throws IOException {
+    if (o != null) {
+      writeField(generator, fieldName, o);
+    }
+  }
+
+  private static void writeArray(JsonGenerator generator, List<?> elements)
+      throws IOException {
+    generator.writeStartArray();
+    for (Object o : elements) {
+      writeObject(generator, o);
+    }
+    generator.writeEndArray();
+  }
+
+  private static void writeObject(JsonGenerator generator, Object o)
+      throws IOException {
+    if (o instanceof String) {
+      String s = (String) o;
+      generator.writeString(s);
+    } else if (o instanceof Interval) {
+      Interval i = (Interval) o;
+      generator.writeString(i.toString());
+    } else if (o instanceof Integer) {
+      Integer i = (Integer) o;
+      generator.writeNumber(i);
+    } else if (o instanceof List) {
+      writeArray(generator, (List<?>) o);
+    } else if (o instanceof Json) {
+      ((Json) o).write(generator);
+    } else {
+      throw new AssertionError("not a json object: " + o);
+    }
+  }
+
+  /** Druid query specification. */
+  public static class QuerySpec {
+    final DruidQueryType queryType;
+    final String queryString;
+    final List<String> fieldNames;
+
+    QuerySpec(DruidQueryType queryType, String queryString,
+        List<String> fieldNames) {
+      this.queryType = Preconditions.checkNotNull(queryType);
+      this.queryString = Preconditions.checkNotNull(queryString);
+      this.fieldNames = ImmutableList.copyOf(fieldNames);
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(queryType, queryString, fieldNames);
+    }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof QuerySpec
+          && queryType == ((QuerySpec) obj).queryType
+          && queryString.equals(((QuerySpec) obj).queryString)
+          && fieldNames.equals(((QuerySpec) obj).fieldNames);
+    }
+
+    @Override public String toString() {
+      return "{queryType: " + queryType
+          + ", queryString: " + queryString
+          + ", fieldNames: " + fieldNames + "}";
+    }
+
+    String getQueryString(String pagingIdentifier, int offset) {
+      if (pagingIdentifier == null) {
+        return queryString;
+      }
+      return queryString.replace("\"threshold\":",
+          "\"pagingIdentifiers\":{\"" + pagingIdentifier + "\":" + offset
+              + "},\"threshold\":");
+    }
+  }
+
+  /** Translates scalar expressions to Druid field references. */
+  private static class Translator {
+    final List<String> dimensions = new ArrayList<>();
+    final List<String> metrics = new ArrayList<>();
+    final DruidTable druidTable;
+    final RelDataType rowType;
+
+    Translator(DruidTable druidTable, RelDataType rowType) {
+      this.druidTable = druidTable;
+      this.rowType = rowType;
+      for (RelDataTypeField f : rowType.getFieldList()) {
+        final String fieldName = f.getName();
+        if (druidTable.metricFieldNames.contains(fieldName)) {
+          metrics.add(fieldName);
+        } else if (!DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(fieldName)) {
+          dimensions.add(fieldName);
+        }
+      }
+    }
+
+    String translate(RexNode e, boolean set) {
+      switch (e.getKind()) {
+      case INPUT_REF:
+        final RexInputRef ref = (RexInputRef) e;
+        final String fieldName =
+            rowType.getFieldList().get(ref.getIndex()).getName();
+        if (set) {
+          if (druidTable.metricFieldNames.contains(fieldName)) {
+            metrics.add(fieldName);
+          } else if (!DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(fieldName)) {
+            dimensions.add(fieldName);
+          }
+        }
+        return fieldName;
+
+      case CAST:
+       return tr(e, 0, set);
+
+      case LITERAL:
+        return ((RexLiteral) e).getValue2().toString();
+
+      case OTHER_FUNCTION:
+        final RexCall call = (RexCall) e;
+        assert HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator());
+        return tr(call, 0, set);
+
+      default:
+        throw new AssertionError("invalid expression " + e);
+      }
+    }
+
+    @SuppressWarnings("incomplete-switch")
+    private JsonFilter translateFilter(RexNode e) {
+      RexCall call;
+      switch (e.getKind()) {
+      case EQUALS:
+      case NOT_EQUALS:
+      case GREATER_THAN:
+      case GREATER_THAN_OR_EQUAL:
+      case LESS_THAN:
+      case LESS_THAN_OR_EQUAL:
+        call = (RexCall) e;
+        int posRef;
+        int posConstant;
+        if (RexUtil.isConstant(call.getOperands().get(1))) {
+          posRef = 0;
+          posConstant = 1;
+        } else if (RexUtil.isConstant(call.getOperands().get(0))) {
+          posRef = 1;
+          posConstant = 0;
+        } else {
+          throw new AssertionError("it is not a valid comparison: " + e);
+        }
+        switch (e.getKind()) {
+        case EQUALS:
+          return new JsonSelector("selector", tr(e, posRef), tr(e, posConstant));
+        case NOT_EQUALS:
+          return new JsonCompositeFilter("not",
+              ImmutableList.of(new JsonSelector("selector", tr(e, posRef), tr(e, posConstant))));
+        case GREATER_THAN:
+          return new JsonBound("bound", tr(e, posRef), tr(e, posConstant), true, null, false,
+              false);
+        case GREATER_THAN_OR_EQUAL:
+          return new JsonBound("bound", tr(e, posRef), tr(e, posConstant), false, null, false,
+              false);
+        case LESS_THAN:
+          return new JsonBound("bound", tr(e, posRef), null, false, tr(e, posConstant), true,
+              false);
+        case LESS_THAN_OR_EQUAL:
+          return new JsonBound("bound", tr(e, posRef), null, false, tr(e, posConstant), false,
+              false);
+        }
+      case AND:
+      case OR:
+      case NOT:
+        call = (RexCall) e;
+        return new JsonCompositeFilter(e.getKind().toString().toLowerCase(),
+            translateFilters(call.getOperands()));
+      default:
+        throw new AssertionError("cannot translate filter: " + e);
+      }
+    }
+
+    private String tr(RexNode call, int index) {
+      return tr(call, index, false);
+    }
+
+    private String tr(RexNode call, int index, boolean set) {
+      return translate(((RexCall) call).getOperands().get(index), set);
+    }
+
+    private List<JsonFilter> translateFilters(List<RexNode> operands) {
+      final ImmutableList.Builder<JsonFilter> builder =
+          ImmutableList.builder();
+      for (RexNode operand : operands) {
+        builder.add(translateFilter(operand));
+      }
+      return builder.build();
+    }
+  }
+
+  /** Object that knows how to write itself to a
+   * {@link com.fasterxml.jackson.core.JsonGenerator}. */
+  private interface Json {
+    void write(JsonGenerator generator) throws IOException;
+  }
+
+  /** Aggregation element of a Druid "groupBy" or "topN" query. */
+  private static class JsonAggregation implements Json {
+    final String type;
+    final String name;
+    final String fieldName;
+
+    private JsonAggregation(String type, String name, String fieldName) {
+      this.type = type;
+      this.name = name;
+      this.fieldName = fieldName;
+    }
+
+    public void write(JsonGenerator generator) throws IOException {
+      generator.writeStartObject();
+      generator.writeStringField("type", type);
+      generator.writeStringField("name", name);
+      writeFieldIf(generator, "fieldName", fieldName);
+      generator.writeEndObject();
+    }
+  }
+
+  /** Collation element of a Druid "groupBy" query. */
+  private static class JsonLimit implements Json {
+    final String type;
+    final Integer limit;
+    final ImmutableList<JsonCollation> collations;
+
+    private JsonLimit(String type, Integer limit, ImmutableList<JsonCollation> collations) {
+      this.type = type;
+      this.limit = limit;
+      this.collations = collations;
+    }
+
+    public void write(JsonGenerator generator) throws IOException {
+      generator.writeStartObject();
+      generator.writeStringField("type", type);
+      writeFieldIf(generator, "limit", limit);
+      writeFieldIf(generator, "columns", collations);
+      generator.writeEndObject();
+    }
+  }
+
+  /** Collation element of a Druid "groupBy" query. */
+  private static class JsonCollation implements Json {
+    final String dimension;
+    final String direction;
+
+    private JsonCollation(String dimension, String direction) {
+      this.dimension = dimension;
+      this.direction = direction;
+    }
+
+    public void write(JsonGenerator generator) throws IOException {
+      generator.writeStartObject();
+      generator.writeStringField("dimension", dimension);
+      writeFieldIf(generator, "direction", direction);
+      generator.writeEndObject();
+    }
+  }
+
+  /** Aggregation element that calls the "cardinality" function. */
+  private static class JsonCardinalityAggregation extends JsonAggregation {
+    final List<String> fieldNames;
+
+    private JsonCardinalityAggregation(String type, String name,
+        List<String> fieldNames) {
+      super(type, name, null);
+      this.fieldNames = fieldNames;
+    }
+
+    public void write(JsonGenerator generator) throws IOException {
+      generator.writeStartObject();
+      generator.writeStringField("type", type);
+      generator.writeStringField("name", name);
+      writeFieldIf(generator, "fieldNames", fieldNames);
+      generator.writeEndObject();
+    }
+  }
+
+  /** Filter element of a Druid "groupBy" or "topN" query. */
+  private abstract static class JsonFilter implements Json {
+    final String type;
+
+    private JsonFilter(String type) {
+      this.type = type;
+    }
+  }
+
+  /** Equality filter. */
+  private static class JsonSelector extends JsonFilter {
+    private final String dimension;
+    private final String value;
+
+    private JsonSelector(String type, String dimension, String value) {
+      super(type);
+      this.dimension = dimension;
+      this.value = value;
+    }
+
+    public void write(JsonGenerator generator) throws IOException {
+      generator.writeStartObject();
+      generator.writeStringField("type", type);
+      generator.writeStringField("dimension", dimension);
+      generator.writeStringField("value", value);
+      generator.writeEndObject();
+    }
+  }
+
+  /** Bound filter. */
+  private static class JsonBound extends JsonFilter {
+    private final String dimension;
+    private final String lower;
+    private final boolean lowerStrict;
+    private final String upper;
+    private final boolean upperStrict;
+    private final boolean alphaNumeric;
+
+    private JsonBound(String type, String dimension, String lower,
+        boolean lowerStrict, String upper, boolean upperStrict,
+        boolean alphaNumeric) {
+      super(type);
+      this.dimension = dimension;
+      this.lower = lower;
+      this.lowerStrict = lowerStrict;
+      this.upper = upper;
+      this.upperStrict = upperStrict;
+      this.alphaNumeric = alphaNumeric;
+    }
+
+    public void write(JsonGenerator generator) throws IOException {
+      generator.writeStartObject();
+      generator.writeStringField("type", type);
+      generator.writeStringField("dimension", dimension);
+      if (lower != null) {
+        generator.writeStringField("lower", lower);
+        generator.writeBooleanField("lowerStrict", lowerStrict);
+      }
+      if (upper != null) {
+        generator.writeStringField("upper", upper);
+        generator.writeBooleanField("upperStrict", upperStrict);
+      }
+      generator.writeBooleanField("alphaNumeric", alphaNumeric);
+      generator.writeEndObject();
+    }
+  }
+
+  /** Filter that combines other filters using a boolean operator. */
+  private static class JsonCompositeFilter extends JsonFilter {
+    private final List<? extends JsonFilter> fields;
+
+    private JsonCompositeFilter(String type,
+        List<? extends JsonFilter> fields) {
+      super(type);
+      this.fields = fields;
+    }
+
+    public void write(JsonGenerator generator) throws IOException {
+      generator.writeStartObject();
+      generator.writeStringField("type", type);
+      switch (type) {
+      case "NOT":
+        writeField(generator, "field", fields.get(0));
+        break;
+      default:
+        writeField(generator, "fields", fields);
+      }
+      generator.writeEndObject();
+    }
+  }
+
+}
+
+// End DruidQuery.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java
new file mode 100644
index 0000000..228b307
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java
@@ -0,0 +1,42 @@
+/**
+ * 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.druid;
+
+/**
+ * Type of Druid query.
+ *
+ * TODO: to be removed when Calcite is upgraded to 1.9
+ */
+public enum DruidQueryType {
+  SELECT("select"),
+  TOP_N("topN"),
+  GROUP_BY("groupBy"),
+  TIMESERIES("timeseries");
+
+  private final String queryName;
+
+  private DruidQueryType(String queryName) {
+    this.queryName = queryName;
+  }
+
+  public String getQueryName() {
+    return this.queryName;
+  }
+}
+
+// End QueryType.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java
new file mode 100644
index 0000000..f68ffa5
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java
@@ -0,0 +1,591 @@
+/**
+ * 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.druid;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveDateGranularity;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectSortTransposeRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortProjectTransposeRule;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+/**
+ * Rules and relational operators for {@link DruidQuery}.
+ *
+ * TODO: to be removed when Calcite is upgraded to 1.9
+ */
+public class DruidRules {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(DruidRules.class);
+
+  // Avoid instantiation
+  private DruidRules() {
+  }
+
+  public static final DruidFilterRule FILTER = new DruidFilterRule();
+  public static final DruidProjectRule PROJECT = new DruidProjectRule();
+  public static final DruidAggregateRule AGGREGATE = new DruidAggregateRule();
+  public static final DruidProjectAggregateRule PROJECT_AGGREGATE = new DruidProjectAggregateRule();
+  public static final DruidSortRule SORT = new DruidSortRule();
+  public static final DruidProjectSortRule PROJECT_SORT = new DruidProjectSortRule();
+  public static final DruidSortProjectRule SORT_PROJECT = new DruidSortProjectRule();
+
+  /** Predicate that returns whether Druid can not handle an aggregate. */
+  private static final Predicate<AggregateCall> BAD_AGG = new Predicate<AggregateCall>() {
+    public boolean apply(AggregateCall aggregateCall) {
+      switch (aggregateCall.getAggregation().getKind()) {
+        case COUNT:
+        case SUM:
+        case SUM0:
+        case MIN:
+        case MAX:
+          return false;
+        default:
+          return true;
+      }
+    }
+  };
+
+  /**
+   * Rule to push a {@link org.apache.calcite.rel.core.Filter} into a {@link DruidQuery}.
+   */
+  private static class DruidFilterRule extends RelOptRule {
+    private DruidFilterRule() {
+      super(operand(Filter.class,
+              operand(DruidQuery.class, none())));
+    }
+
+    public void onMatch(RelOptRuleCall call) {
+      final Filter filter = call.rel(0);
+      final DruidQuery query = call.rel(1);
+      if (!DruidQuery.isValidSignature(query.signature() + 'f')
+              || !query.isValidFilter(filter.getCondition())) {
+        return;
+      }
+      // Timestamp
+      int timestampFieldIdx = -1;
+      for (int i = 0; i < query.getRowType().getFieldCount(); i++) {
+        if (DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(
+                query.getRowType().getFieldList().get(i).getName())) {
+          timestampFieldIdx = i;
+          break;
+        }
+      }
+      final Pair<List<RexNode>, List<RexNode>> pair = splitFilters(
+              filter.getCluster().getRexBuilder(), query, filter.getCondition(), timestampFieldIdx);
+      if (pair == null) {
+        // We can't push anything useful to Druid.
+        return;
+      }
+      List<Interval> intervals = null;
+      if (!pair.left.isEmpty()) {
+        intervals = DruidIntervalUtils.createInterval(
+                query.getRowType().getFieldList().get(timestampFieldIdx).getType(),
+                pair.left);
+        if (intervals == null) {
+          // We can't push anything useful to Druid.
+          return;
+        }
+      }
+      DruidQuery newDruidQuery = query;
+      if (!pair.right.isEmpty()) {
+        if (!validConditions(pair.right)) {
+          return;
+        }
+        final RelNode newFilter = filter.copy(filter.getTraitSet(), Util.last(query.rels),
+                RexUtil.composeConjunction(filter.getCluster().getRexBuilder(), pair.right, false));
+        newDruidQuery = DruidQuery.extendQuery(query, newFilter);
+      }
+      if (intervals != null) {
+        newDruidQuery = DruidQuery.extendQuery(newDruidQuery, intervals);
+      }
+      call.transformTo(newDruidQuery);
+    }
+
+    /* Splits the filter condition in two groups: those that filter on the timestamp column
+     * and those that filter on other fields */
+    private static Pair<List<RexNode>, List<RexNode>> splitFilters(final RexBuilder rexBuilder,
+            final DruidQuery input, RexNode cond, final int timestampFieldIdx) {
+      final List<RexNode> timeRangeNodes = new ArrayList<>();
+      final List<RexNode> otherNodes = new ArrayList<>();
+      List<RexNode> conjs = RelOptUtil.conjunctions(cond);
+      if (conjs.isEmpty()) {
+        // We do not transform
+        return null;
+      }
+      // Number of columns with the dimensions and timestamp
+      int max = input.getRowType().getFieldCount() - input.druidTable.metricFieldNames.size();
+      for (RexNode conj : conjs) {
+        final RelOptUtil.InputReferencedVisitor visitor = new RelOptUtil.InputReferencedVisitor();
+        conj.accept(visitor);
+        if (visitor.inputPosReferenced.contains(timestampFieldIdx)) {
+          if (visitor.inputPosReferenced.size() != 1) {
+            // Complex predicate, transformation currently not supported
+            return null;
+          }
+          timeRangeNodes.add(conj);
+        } else if (!visitor.inputPosReferenced.tailSet(max).isEmpty()) {
+          // Filter on metrics, not supported in Druid
+          return null;
+        } else {
+          otherNodes.add(conj);
+        }
+      }
+      return Pair.of(timeRangeNodes, otherNodes);
+    }
+
+    /* Checks that all conditions are on ref + literal*/
+    private static boolean validConditions(List<RexNode> nodes) {
+      for (RexNode node: nodes) {
+        try {
+          node.accept(
+              new RexVisitorImpl<Void>(true) {
+                @SuppressWarnings("incomplete-switch")
+                @Override public Void visitCall(RexCall call) {
+                  switch (call.getKind()) {
+                    case CAST:
+                      // Only if on top of ref or literal
+                      if (call.getOperands().get(0) instanceof RexInputRef ||
+                              call.getOperands().get(0) instanceof RexLiteral) {
+                        break;
+                      }
+                      // Not supported
+                      throw Util.FoundOne.NULL;
+                    case EQUALS:
+                    case LESS_THAN:
+                    case LESS_THAN_OR_EQUAL:
+                    case GREATER_THAN:
+                    case GREATER_THAN_OR_EQUAL:
+                      // Check cast
+                      RexNode left = call.getOperands().get(0);
+                      if (left.getKind() == SqlKind.CAST) {
+                        left = ((RexCall)left).getOperands().get(0);
+                      }
+                      RexNode right = call.getOperands().get(1);
+                      if (right.getKind() == SqlKind.CAST) {
+                        right = ((RexCall)right).getOperands().get(0);
+                      }
+                      if (left instanceof RexInputRef && right instanceof RexLiteral) {
+                        break;
+                      }
+                      if (right instanceof RexInputRef && left instanceof RexLiteral) {
+                        break;
+                      }
+                      // Not supported if it is not ref + literal
+                      throw Util.FoundOne.NULL;
+                    case BETWEEN:
+                    case IN:
+                      // Not supported here yet
+                      throw Util.FoundOne.NULL;
+                  }
+                  return super.visitCall(call);
+                }
+              });
+        } catch (Util.FoundOne e) {
+          return false;
+        }
+      }
+      return true;
+    }
+  }
+
+  /**
+   * Rule to push a {@link org.apache.calcite.rel.core.Project} into a {@link DruidQuery}.
+   */
+  private static class DruidProjectRule extends RelOptRule {
+    private DruidProjectRule() {
+      super(operand(Project.class,
+              operand(DruidQuery.class, none())));
+    }
+
+    public void onMatch(RelOptRuleCall call) {
+      final Project project = call.rel(0);
+      final DruidQuery query = call.rel(1);
+      if (!DruidQuery.isValidSignature(query.signature() + 'p')) {
+        return;
+      }
+
+      if (canProjectAll(project.getProjects())) {
+        // All expressions can be pushed to Druid in their entirety.
+        final RelNode newProject = project.copy(project.getTraitSet(),
+                ImmutableList.of(Util.last(query.rels)));
+        RelNode newNode = DruidQuery.extendQuery(query, newProject);
+        call.transformTo(newNode);
+        return;
+      }
+      final Pair<List<RexNode>, List<RexNode>> pair = splitProjects(
+              project.getCluster().getRexBuilder(), query, project.getProjects());
+      if (pair == null) {
+        // We can't push anything useful to Druid.
+        return;
+      }
+      final List<RexNode> above = pair.left;
+      final List<RexNode> below = pair.right;
+      final RelDataTypeFactory.FieldInfoBuilder builder = project.getCluster().getTypeFactory()
+              .builder();
+      final RelNode input = Util.last(query.rels);
+      for (RexNode e : below) {
+        final String name;
+        if (e instanceof RexInputRef) {
+          name = input.getRowType().getFieldNames().get(((RexInputRef) e).getIndex());
+        } else {
+          name = null;
+        }
+        builder.add(name, e.getType());
+      }
+      final RelNode newProject = project.copy(project.getTraitSet(), input, below, builder.build());
+      final DruidQuery newQuery = DruidQuery.extendQuery(query, newProject);
+      final RelNode newProject2 = project.copy(project.getTraitSet(), newQuery, above,
+              project.getRowType());
+      call.transformTo(newProject2);
+    }
+
+    private static boolean canProjectAll(List<RexNode> nodes) {
+      for (RexNode e : nodes) {
+        if (!(e instanceof RexInputRef)) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    private static Pair<List<RexNode>, List<RexNode>> splitProjects(final RexBuilder rexBuilder,
+            final RelNode input, List<RexNode> nodes) {
+      final RelOptUtil.InputReferencedVisitor visitor = new RelOptUtil.InputReferencedVisitor();
+      for (RexNode node : nodes) {
+        node.accept(visitor);
+      }
+      if (visitor.inputPosReferenced.size() == input.getRowType().getFieldCount()) {
+        // All inputs are referenced
+        return null;
+      }
+      final List<RexNode> belowNodes = new ArrayList<>();
+      final List<RelDataType> belowTypes = new ArrayList<>();
+      final List<Integer> positions = Lists.newArrayList(visitor.inputPosReferenced);
+      for (int i : positions) {
+        final RexNode node = rexBuilder.makeInputRef(input, i);
+        belowNodes.add(node);
+        belowTypes.add(node.getType());
+      }
+      final List<RexNode> aboveNodes = new ArrayList<>();
+      for (RexNode node : nodes) {
+        aboveNodes.add(node.accept(new RexShuttle() {
+          @Override
+          public RexNode visitInputRef(RexInputRef ref) {
+            final int index = positions.indexOf(ref.getIndex());
+            return rexBuilder.makeInputRef(belowTypes.get(index), index);
+          }
+        }));
+      }
+      return Pair.of(aboveNodes, belowNodes);
+    }
+  }
+
+  /**
+   * Rule to push an {@link org.apache.calcite.rel.core.Aggregate} into a {@link DruidQuery}.
+   */
+  private static class DruidAggregateRule extends RelOptRule {
+    private DruidAggregateRule() {
+      super(operand(Aggregate.class,
+              operand(DruidQuery.class, none())));
+    }
+
+    public void onMatch(RelOptRuleCall call) {
+      final Aggregate aggregate = call.rel(0);
+      final DruidQuery query = call.rel(1);
+      if (!DruidQuery.isValidSignature(query.signature() + 'a')) {
+        return;
+      }
+      if (aggregate.indicator
+              || aggregate.getGroupSets().size() != 1
+              || Iterables.any(aggregate.getAggCallList(), BAD_AGG)
+              || !validAggregate(aggregate, query)) {
+        return;
+      }
+      final RelNode newAggregate = aggregate.copy(aggregate.getTraitSet(),
+              ImmutableList.of(Util.last(query.rels)));
+      call.transformTo(DruidQuery.extendQuery(query, newAggregate));
+    }
+
+    /* Check whether agg functions reference timestamp */
+    private static boolean validAggregate(Aggregate aggregate, DruidQuery query) {
+      ImmutableBitSet.Builder builder = ImmutableBitSet.builder();
+      for (AggregateCall aggCall : aggregate.getAggCallList()) {
+        builder.addAll(aggCall.getArgList());
+      }
+      return !checkTimestampRefOnQuery(builder.build(), query.getTopNode());
+    }
+  }
+
+  /**
+   * Rule to push an {@link org.apache.calcite.rel.core.Aggregate} and
+   * {@link org.apache.calcite.rel.core.Project} into a {@link DruidQuery}.
+   */
+  private static class DruidProjectAggregateRule extends RelOptRule {
+    private DruidProjectAggregateRule() {
+      super(operand(Aggregate.class,
+              operand(Project.class,
+                      operand(DruidQuery.class, none()))));
+    }
+
+    public void onMatch(RelOptRuleCall call) {
+      final Aggregate aggregate = call.rel(0);
+      final Project project = call.rel(1);
+      final DruidQuery query = call.rel(2);
+      if (!DruidQuery.isValidSignature(query.signature() + 'p' + 'a')) {
+        return;
+      }
+      int timestampIdx;
+      if ((timestampIdx = validProject(project, query)) == -1) {
+        return;
+      }
+      if (aggregate.indicator
+              || aggregate.getGroupSets().size() != 1
+              || Iterables.any(aggregate.getAggCallList(), BAD_AGG)
+              || !validAggregate(aggregate, timestampIdx)) {
+        return;
+      }
+
+      final RelNode newProject = project.copy(project.getTraitSet(),
+              ImmutableList.of(Util.last(query.rels)));
+      final DruidQuery projectDruidQuery = DruidQuery.extendQuery(query, newProject);
+      final RelNode newAggregate = aggregate.copy(aggregate.getTraitSet(),
+              ImmutableList.of(Util.last(projectDruidQuery.rels)));
+      call.transformTo(DruidQuery.extendQuery(projectDruidQuery, newAggregate));
+    }
+
+    /* To be a valid Project, we allow it to contain references, and a single call
+     * to an EXTRACT function on the timestamp column. Returns the reference to
+     * the timestamp, if any. */
+    private static int validProject(Project project, DruidQuery query) {
+      List<RexNode> nodes = project.getProjects();
+      int idxTimestamp = -1;
+      for (int i = 0; i < nodes.size(); i++) {
+        final RexNode e = nodes.get(i);
+        if (e instanceof RexCall) {
+          // It is a call, check that it is EXTRACT and follow-up conditions
+          final RexCall call = (RexCall) e;
+          if (!HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator())) {
+            return -1;
+          }
+          if (idxTimestamp != -1) {
+            // Already one usage of timestamp column
+            return -1;
+          }
+          if (!(call.getOperands().get(0) instanceof RexInputRef)) {
+            return -1;
+          }
+          final RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+          if (!(checkTimestampRefOnQuery(ImmutableBitSet.of(ref.getIndex()), query.getTopNode()))) {
+            return -1;
+          }
+          idxTimestamp = i;
+          continue;
+        }
+        if (!(e instanceof RexInputRef)) {
+          // It needs to be a reference
+          return -1;
+        }
+        final RexInputRef ref = (RexInputRef) e;
+        if (checkTimestampRefOnQuery(ImmutableBitSet.of(ref.getIndex()), query.getTopNode())) {
+          if (idxTimestamp != -1) {
+            // Already one usage of timestamp column
+            return -1;
+          }
+          idxTimestamp = i;
+        }
+      }
+      return idxTimestamp;
+    }
+
+    private static boolean validAggregate(Aggregate aggregate, int idx) {
+      if (!aggregate.getGroupSet().get(idx)) {
+        return false;
+      }
+      for (AggregateCall aggCall : aggregate.getAggCallList()) {
+        if (aggCall.getArgList().contains(idx)) {
+          return false;
+        }
+      }
+      return true;
+    }
+  }
+
+  /**
+   * Rule to push an {@link org.apache.calcite.rel.core.Sort} through a
+   * {@link org.apache.calcite.rel.core.Project}. Useful to transform
+   * to complex Druid queries.
+   */
+  private static class DruidProjectSortRule extends HiveSortProjectTransposeRule {
+    private DruidProjectSortRule() {
+      super(operand(Sort.class,
+              operand(Project.class,
+                      operand(DruidQuery.class, none()))));
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+      return true;
+    }
+
+  }
+
+  /**
+   * Rule to push back {@link org.apache.calcite.rel.core.Project} through a
+   * {@link org.apache.calcite.rel.core.Sort}. Useful if after pushing Sort,
+   * we could not push it inside DruidQuery.
+   */
+  private static class DruidSortProjectRule extends HiveProjectSortTransposeRule {
+    private DruidSortProjectRule() {
+      super(operand(Project.class,
+              operand(Sort.class,
+                      operand(DruidQuery.class, none()))));
+    }
+  }
+
+  /**
+   * Rule to push an {@link org.apache.calcite.rel.core.Aggregate} into a {@link DruidQuery}.
+   */
+  private static class DruidSortRule extends RelOptRule {
+    private DruidSortRule() {
+      super(operand(Sort.class,
+              operand(DruidQuery.class, none())));
+    }
+
+    public void onMatch(RelOptRuleCall call) {
+      final Sort sort = call.rel(0);
+      final DruidQuery query = call.rel(1);
+      if (!DruidQuery.isValidSignature(query.signature() + 'l')) {
+        return;
+      }
+      // Either it is:
+      // - a sort without limit on the time column on top of
+      //     Agg operator (transformable to timeseries query), or
+      // - it is a sort w/o limit on columns that do not include
+      //     the time column on top of Agg operator, or
+      // - a simple limit on top of other operator than Agg
+      if (!validSortLimit(sort, query)) {
+        return;
+      }
+      final RelNode newSort = sort.copy(sort.getTraitSet(),
+              ImmutableList.of(Util.last(query.rels)));
+      call.transformTo(DruidQuery.extendQuery(query, newSort));
+    }
+
+    /* Check sort valid */
+    private static boolean validSortLimit(Sort sort, DruidQuery query) {
+      if (sort.offset != null && RexLiteral.intValue(sort.offset) != 0) {
+        // offset not supported by Druid
+        return false;
+      }
+      if (query.getTopNode() instanceof Aggregate) {
+        final Aggregate topAgg = (Aggregate) query.getTopNode();
+        final ImmutableBitSet.Builder positionsReferenced = ImmutableBitSet.builder();
+        int metricsRefs = 0;
+        for (RelFieldCollation col : sort.collation.getFieldCollations()) {
+          int idx = col.getFieldIndex();
+          if (idx >= topAgg.getGroupCount()) {
+            metricsRefs++;
+            continue;
+          }
+          positionsReferenced.set(topAgg.getGroupSet().nth(idx));
+        }
+        boolean refsTimestamp =
+                checkTimestampRefOnQuery(positionsReferenced.build(), topAgg.getInput());
+        if (refsTimestamp && metricsRefs != 0) {
+          return false;
+        }
+        return true;
+      }
+      // If it is going to be a Druid select operator, we push the limit iff
+      // 1) it does not contain a sort specification (required by Druid) and
+      // 2) limit is smaller than select threshold, as otherwise it might be
+      //   better to obtain some parallelization and let global limit
+      //   optimizer kick in
+      HiveDruidConf conf = sort.getCluster().getPlanner()
+              .getContext().unwrap(HiveDruidConf.class);
+      return HiveCalciteUtil.pureLimitRelNode(sort) &&
+              RexLiteral.intValue(sort.fetch) <= conf.getSelectThreshold();
+    }
+  }
+
+  /* Check if any of the references leads to the timestamp column */
+  private static boolean checkTimestampRefOnQuery(ImmutableBitSet set, RelNode top) {
+    if (top instanceof Project) {
+      ImmutableBitSet.Builder newSet = ImmutableBitSet.builder();
+      final Project project = (Project) top;
+      for (int index : set) {
+        RexNode node = project.getProjects().get(index);
+        if (node instanceof RexInputRef) {
+          newSet.set(((RexInputRef)node).getIndex());
+        } else if (node instanceof RexCall) {
+          RexCall call = (RexCall) node;
+          assert HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator());
+          newSet.set(((RexInputRef)call.getOperands().get(0)).getIndex());
+        }
+      }
+      top = project.getInput();
+      set = newSet.build();
+    }
+
+    // Check if any references the timestamp column
+    for (int index : set) {
+      if (DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(top.getRowType().getFieldNames().get(index))) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+}
+
+// End DruidRules.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java
new file mode 100644
index 0000000..3b3f68a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java
@@ -0,0 +1,51 @@
+/**
+ * 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.druid;
+
+import java.util.Map;
+
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * Schema mapped onto a Druid instance.
+ *
+ * TODO: to be removed when Calcite is upgraded to 1.9
+ */
+public class DruidSchema extends AbstractSchema {
+  final String url;
+
+  /**
+   * Creates a Druid schema.
+   *
+   * @param url URL of query REST service
+   */
+  public DruidSchema(String url) {
+    this.url = Preconditions.checkNotNull(url);
+  }
+
+  @Override protected Map<String, Table> getTableMap() {
+    final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
+    return builder.build();
+  }
+}
+
+// End DruidSchema.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java
new file mode 100644
index 0000000..7288291
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java
@@ -0,0 +1,121 @@
+/**
+ * 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.druid;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.calcite.interpreter.BindableConvention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
+/**
+ * Table mapped onto a Druid table.
+ *
+ * TODO: to be removed when Calcite is upgraded to 1.9
+ */
+public class DruidTable extends AbstractTable implements TranslatableTable {
+
+  public static final String DEFAULT_TIMESTAMP_COLUMN = "__time";
+  public static final Interval DEFAULT_INTERVAL = new Interval(
+          new DateTime("1900-01-01"),
+          new DateTime("3000-01-01")
+  );
+
+  final DruidSchema schema;
+  final String dataSource;
+  final RelDataType rowType;
+  final RelProtoDataType protoRowType;
+  final ImmutableSet<String> metricFieldNames;
+  final ImmutableList<Interval> intervals;
+  final String timestampFieldName;
+
+  /**
+   * Creates a Druid table.
+   *
+   * @param schema Druid schema that contains this table
+   * @param dataSource Druid data source name
+   * @param protoRowType Field names and types
+   * @param metricFieldNames Names of fields that are metrics
+   * @param interval Default interval if query does not constrain the time
+   * @param timestampFieldName Name of the column that contains the time
+   */
+  public DruidTable(DruidSchema schema, String dataSource,
+      RelProtoDataType protoRowType, Set<String> metricFieldNames,
+      List<Interval> intervals, String timestampFieldName) {
+    this.schema = Preconditions.checkNotNull(schema);
+    this.dataSource = Preconditions.checkNotNull(dataSource);
+    this.rowType = null;
+    this.protoRowType = protoRowType;
+    this.metricFieldNames = ImmutableSet.copyOf(metricFieldNames);
+    this.intervals = ImmutableList.copyOf(intervals);
+    this.timestampFieldName = Preconditions.checkNotNull(timestampFieldName);
+  }
+
+  public DruidTable(DruidSchema schema, String dataSource,
+      RelDataType rowType, Set<String> metricFieldNames,
+      List<Interval> intervals, String timestampFieldName) {
+    this.schema = Preconditions.checkNotNull(schema);
+    this.dataSource = Preconditions.checkNotNull(dataSource);
+    this.rowType = Preconditions.checkNotNull(rowType);
+    this.protoRowType = null;
+    this.metricFieldNames = ImmutableSet.copyOf(metricFieldNames);
+    this.intervals = ImmutableList.copyOf(intervals);
+    this.timestampFieldName = Preconditions.checkNotNull(timestampFieldName);
+  }
+
+  public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+    final RelDataType thisRowType;
+    if (rowType != null) {
+      thisRowType = rowType;
+    } else {
+      // Generate
+      thisRowType = protoRowType.apply(typeFactory);
+    }
+    final List<String> fieldNames = thisRowType.getFieldNames();
+    Preconditions.checkArgument(fieldNames.contains(timestampFieldName));
+    Preconditions.checkArgument(fieldNames.containsAll(metricFieldNames));
+    return thisRowType;
+  }
+
+  public RelNode toRel(RelOptTable.ToRelContext context,
+      RelOptTable relOptTable) {
+    final RelOptCluster cluster = context.getCluster();
+    final TableScan scan = LogicalTableScan.create(cluster, relOptTable);
+    return DruidQuery.create(cluster,
+        cluster.traitSetOf(BindableConvention.INSTANCE), relOptTable, this,
+        ImmutableList.<RelNode>of(scan));
+  }
+
+}
+
+// End DruidTable.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java
new file mode 100644
index 0000000..0686dff
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java
@@ -0,0 +1,33 @@
+/**
+ * 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.druid;
+
+public class HiveDruidConf {
+
+  private int selectThreshold;
+
+
+  public HiveDruidConf(int selectThreshold) {
+    this.selectThreshold = selectThreshold;
+  }
+
+  public int getSelectThreshold() {
+    return selectThreshold;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlCountAggFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlCountAggFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlCountAggFunction.java
index bc48707..75b7ad2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlCountAggFunction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlCountAggFunction.java
@@ -41,7 +41,7 @@ public class HiveSqlCountAggFunction extends SqlAggFunction implements CanAggreg
       SqlOperandTypeInference operandTypeInference, SqlOperandTypeChecker operandTypeChecker) {
     super(
         "count",
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.COUNT,
         returnTypeInference,
         operandTypeInference,
         operandTypeChecker,

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlMinMaxAggFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlMinMaxAggFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlMinMaxAggFunction.java
index 77dca1f..834fc3e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlMinMaxAggFunction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlMinMaxAggFunction.java
@@ -32,7 +32,7 @@ public class HiveSqlMinMaxAggFunction extends SqlAggFunction {
       SqlOperandTypeInference operandTypeInference, SqlOperandTypeChecker operandTypeChecker, boolean isMin) {
     super(
         isMin ? "min" : "max",
-        SqlKind.OTHER_FUNCTION,
+        isMin ? SqlKind.MIN : SqlKind.MAX,
         returnTypeInference,
         operandTypeInference,
         operandTypeChecker,

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlSumAggFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlSumAggFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlSumAggFunction.java
index dc286a2..1d551a6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlSumAggFunction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlSumAggFunction.java
@@ -58,7 +58,7 @@ public class HiveSqlSumAggFunction extends SqlAggFunction implements CanAggregat
     SqlOperandTypeInference operandTypeInference, SqlOperandTypeChecker operandTypeChecker) {
     super(
         "sum",
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.SUM,
         returnTypeInference,
         operandTypeInference,
         operandTypeChecker,

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java
new file mode 100644
index 0000000..b3f8d9b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.reloperators;
+
+import java.util.Set;
+
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+
+import com.google.common.collect.Sets;
+
+public class HiveDateGranularity extends SqlFunction {
+
+  public static final SqlFunction YEAR = new HiveDateGranularity("YEAR");
+  public static final SqlFunction QUARTER = new HiveDateGranularity("QUARTER");
+  public static final SqlFunction MONTH = new HiveDateGranularity("MONTH");
+  public static final SqlFunction WEEK = new HiveDateGranularity("WEEK");
+  public static final SqlFunction DAY = new HiveDateGranularity("DAY");
+  public static final SqlFunction HOUR = new HiveDateGranularity("HOUR");
+  public static final SqlFunction MINUTE = new HiveDateGranularity("MINUTE");
+  public static final SqlFunction SECOND = new HiveDateGranularity("SECOND");
+
+  public static final Set<SqlFunction> ALL_FUNCTIONS =
+          Sets.newHashSet(YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE, SECOND);
+
+  private HiveDateGranularity(String name) {
+    super(
+        name,
+        SqlKind.OTHER_FUNCTION,
+        ReturnTypes.TIME_NULLABLE,
+        null,
+        OperandTypes.ANY,
+        SqlFunctionCategory.TIMEDATE);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
index aac6126..fd19d99 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
@@ -48,6 +49,10 @@ public class HiveProjectSortTransposeRule extends RelOptRule {
             operand(HiveSortLimit.class, any())));
   }
 
+  protected HiveProjectSortTransposeRule(RelOptRuleOperand operand) {
+    super(operand);
+  }
+
   //~ Methods ----------------------------------------------------------------
 
   // implement RelOptRule

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
index feec3c2..fe29850 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
@@ -49,6 +50,10 @@ public class HiveSortProjectTransposeRule extends RelOptRule {
             operand(HiveProject.class, any())));
   }
 
+  protected HiveSortProjectTransposeRule(RelOptRuleOperand operand) {
+    super(operand);
+  }
+
   //~ Methods ----------------------------------------------------------------
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
index 78c76ab..9a5becb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
@@ -22,19 +22,21 @@ import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.Calendar;
 
-import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidQuery;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
 
 class ASTBuilder {
 
@@ -62,14 +64,32 @@ class ASTBuilder {
         ASTBuilder.construct(HiveParser.TOK_TABNAME, "TOK_TABNAME")
             .add(HiveParser.Identifier, hTbl.getHiveTableMD().getDbName())
             .add(HiveParser.Identifier, hTbl.getHiveTableMD().getTableName()));
-    // we need to carry the insideView information from calcite into the ast.
-    if (((HiveTableScan) scan).isInsideView()) {
-      b.add(ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTIES, "TOK_TABLEPROPERTIES").add(
-          ASTBuilder.construct(HiveParser.TOK_TABLEPROPLIST, "TOK_TABLEPROPLIST").add(
-              ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTY, "TOK_TABLEPROPERTY")
-                  .add(HiveParser.StringLiteral, "\"insideView\"")
-                  .add(HiveParser.StringLiteral, "\"TRUE\""))));
+
+    HiveTableScan hts;
+    if (scan instanceof DruidQuery) {
+      hts = (HiveTableScan) ((DruidQuery)scan).getTableScan();
+    } else {
+      hts = (HiveTableScan) scan;
+    }
+    ASTBuilder propList = ASTBuilder.construct(HiveParser.TOK_TABLEPROPLIST, "TOK_TABLEPROPLIST");
+    if (scan instanceof DruidQuery) {
+      // Pass possible query to Druid
+      DruidQuery dq = (DruidQuery) scan;
+      propList.add(ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTY, "TOK_TABLEPROPERTY")
+              .add(HiveParser.StringLiteral, "\"" + Constants.DRUID_QUERY_JSON + "\"")
+              .add(HiveParser.StringLiteral, "\"" + SemanticAnalyzer.escapeSQLString(
+                      dq.getQueryString()) + "\""));
+      propList.add(ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTY, "TOK_TABLEPROPERTY")
+              .add(HiveParser.StringLiteral, "\"" + Constants.DRUID_QUERY_TYPE + "\"")
+              .add(HiveParser.StringLiteral, "\"" + dq.getQueryType() + "\""));
+    }
+    if (hts.isInsideView()) {
+      // We need to carry the insideView information from calcite into the ast.
+      propList.add(ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTY, "TOK_TABLEPROPERTY")
+              .add(HiveParser.StringLiteral, "\"insideView\"")
+              .add(HiveParser.StringLiteral, "\"TRUE\""));
     }
+    b.add(ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTIES, "TOK_TABLEPROPERTIES").add(propList));
 
     // NOTE: Calcite considers tbls to be equal if their names are the same. Hence
     // we need to provide Calcite the fully qualified table name (dbname.tblname)
@@ -77,7 +97,7 @@ class ASTBuilder {
     // However in HIVE DB name can not appear in select list; in case of join
     // where table names differ only in DB name, Hive would require user
     // introducing explicit aliases for tbl.
-    b.add(HiveParser.Identifier, ((HiveTableScan)scan).getTableAlias());
+    b.add(HiveParser.Identifier, hts.getTableAlias());
     return b.node();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
index 40215a2..9f5e733 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
@@ -56,6 +56,7 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidQuery;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
@@ -625,7 +626,13 @@ public class ASTConverter {
     private static final long serialVersionUID = 1L;
 
     Schema(TableScan scan) {
-      String tabName = ((HiveTableScan) scan).getTableAlias();
+      HiveTableScan hts;
+      if (scan instanceof DruidQuery) {
+        hts = (HiveTableScan) ((DruidQuery)scan).getTableScan();
+      } else {
+        hts = (HiveTableScan) scan;
+      }
+      String tabName = hts.getTableAlias();
       for (RelDataTypeField field : scan.getRowType().getFieldList()) {
         add(new ColumnInfo(tabName, field.getName()));
       }


[28/31] hive git commit: HIVE-14644 : use metastore information on the read path appropriately (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 40907b3..4134483 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
@@ -346,6 +346,8 @@ public class ThriftHiveMetastore {
 
     public HeartbeatWriteIdResult heartbeat_write_id(HeartbeatWriteIdRequest req) throws org.apache.thrift.TException;
 
+    public GetValidWriteIdsResult get_valid_write_ids(GetValidWriteIdsRequest req) throws org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface extends com.facebook.fb303.FacebookService .AsyncIface {
@@ -654,6 +656,8 @@ public class ThriftHiveMetastore {
 
     public void heartbeat_write_id(HeartbeatWriteIdRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_valid_write_ids(GetValidWriteIdsRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends com.facebook.fb303.FacebookService.Client implements Iface {
@@ -5044,6 +5048,29 @@ public class ThriftHiveMetastore {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "heartbeat_write_id failed: unknown result");
     }
 
+    public GetValidWriteIdsResult get_valid_write_ids(GetValidWriteIdsRequest req) throws org.apache.thrift.TException
+    {
+      send_get_valid_write_ids(req);
+      return recv_get_valid_write_ids();
+    }
+
+    public void send_get_valid_write_ids(GetValidWriteIdsRequest req) throws org.apache.thrift.TException
+    {
+      get_valid_write_ids_args args = new get_valid_write_ids_args();
+      args.setReq(req);
+      sendBase("get_valid_write_ids", args);
+    }
+
+    public GetValidWriteIdsResult recv_get_valid_write_ids() throws org.apache.thrift.TException
+    {
+      get_valid_write_ids_result result = new get_valid_write_ids_result();
+      receiveBase(result, "get_valid_write_ids");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_valid_write_ids failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends com.facebook.fb303.FacebookService.AsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -10409,6 +10436,38 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void get_valid_write_ids(GetValidWriteIdsRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_valid_write_ids_call method_call = new get_valid_write_ids_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_valid_write_ids_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private GetValidWriteIdsRequest req;
+      public get_valid_write_ids_call(GetValidWriteIdsRequest 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("get_valid_write_ids", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_valid_write_ids_args args = new get_valid_write_ids_args();
+        args.setReq(req);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public GetValidWriteIdsResult getResult() throws 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);
+        return (new Client(prot)).recv_get_valid_write_ids();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends com.facebook.fb303.FacebookService.Processor<I> implements org.apache.thrift.TProcessor {
@@ -10574,6 +10633,7 @@ public class ThriftHiveMetastore {
       processMap.put("get_next_write_id", new get_next_write_id());
       processMap.put("finalize_write_id", new finalize_write_id());
       processMap.put("heartbeat_write_id", new heartbeat_write_id());
+      processMap.put("get_valid_write_ids", new get_valid_write_ids());
       return processMap;
     }
 
@@ -14443,6 +14503,26 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_valid_write_ids<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_valid_write_ids_args> {
+      public get_valid_write_ids() {
+        super("get_valid_write_ids");
+      }
+
+      public get_valid_write_ids_args getEmptyArgsInstance() {
+        return new get_valid_write_ids_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_valid_write_ids_result getResult(I iface, get_valid_write_ids_args args) throws org.apache.thrift.TException {
+        get_valid_write_ids_result result = new get_valid_write_ids_result();
+        result.success = iface.get_valid_write_ids(args.req);
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends com.facebook.fb303.FacebookService.AsyncProcessor<I> {
@@ -14608,6 +14688,7 @@ public class ThriftHiveMetastore {
       processMap.put("get_next_write_id", new get_next_write_id());
       processMap.put("finalize_write_id", new finalize_write_id());
       processMap.put("heartbeat_write_id", new heartbeat_write_id());
+      processMap.put("get_valid_write_ids", new get_valid_write_ids());
       return processMap;
     }
 
@@ -23843,6 +23924,57 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_valid_write_ids<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_valid_write_ids_args, GetValidWriteIdsResult> {
+      public get_valid_write_ids() {
+        super("get_valid_write_ids");
+      }
+
+      public get_valid_write_ids_args getEmptyArgsInstance() {
+        return new get_valid_write_ids_args();
+      }
+
+      public AsyncMethodCallback<GetValidWriteIdsResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<GetValidWriteIdsResult>() { 
+          public void onComplete(GetValidWriteIdsResult o) {
+            get_valid_write_ids_result result = new get_valid_write_ids_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_valid_write_ids_result result = new get_valid_write_ids_result();
+            {
+              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_valid_write_ids_args args, org.apache.thrift.async.AsyncMethodCallback<GetValidWriteIdsResult> resultHandler) throws TException {
+        iface.get_valid_write_ids(args.req,resultHandler);
+      }
+    }
+
   }
 
   public static class getMetaConf_args implements org.apache.thrift.TBase<getMetaConf_args, getMetaConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_args>   {
@@ -29238,13 +29370,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list642.size);
-                  String _elem643;
-                  for (int _i644 = 0; _i644 < _list642.size; ++_i644)
+                  org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list650.size);
+                  String _elem651;
+                  for (int _i652 = 0; _i652 < _list650.size; ++_i652)
                   {
-                    _elem643 = iprot.readString();
-                    struct.success.add(_elem643);
+                    _elem651 = iprot.readString();
+                    struct.success.add(_elem651);
                   }
                   iprot.readListEnd();
                 }
@@ -29279,9 +29411,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter645 : struct.success)
+            for (String _iter653 : struct.success)
             {
-              oprot.writeString(_iter645);
+              oprot.writeString(_iter653);
             }
             oprot.writeListEnd();
           }
@@ -29320,9 +29452,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter646 : struct.success)
+            for (String _iter654 : struct.success)
             {
-              oprot.writeString(_iter646);
+              oprot.writeString(_iter654);
             }
           }
         }
@@ -29337,13 +29469,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list647.size);
-            String _elem648;
-            for (int _i649 = 0; _i649 < _list647.size; ++_i649)
+            org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list655.size);
+            String _elem656;
+            for (int _i657 = 0; _i657 < _list655.size; ++_i657)
             {
-              _elem648 = iprot.readString();
-              struct.success.add(_elem648);
+              _elem656 = iprot.readString();
+              struct.success.add(_elem656);
             }
           }
           struct.setSuccessIsSet(true);
@@ -29997,13 +30129,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list650.size);
-                  String _elem651;
-                  for (int _i652 = 0; _i652 < _list650.size; ++_i652)
+                  org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list658.size);
+                  String _elem659;
+                  for (int _i660 = 0; _i660 < _list658.size; ++_i660)
                   {
-                    _elem651 = iprot.readString();
-                    struct.success.add(_elem651);
+                    _elem659 = iprot.readString();
+                    struct.success.add(_elem659);
                   }
                   iprot.readListEnd();
                 }
@@ -30038,9 +30170,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter653 : struct.success)
+            for (String _iter661 : struct.success)
             {
-              oprot.writeString(_iter653);
+              oprot.writeString(_iter661);
             }
             oprot.writeListEnd();
           }
@@ -30079,9 +30211,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter654 : struct.success)
+            for (String _iter662 : struct.success)
             {
-              oprot.writeString(_iter654);
+              oprot.writeString(_iter662);
             }
           }
         }
@@ -30096,13 +30228,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list655.size);
-            String _elem656;
-            for (int _i657 = 0; _i657 < _list655.size; ++_i657)
+            org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list663.size);
+            String _elem664;
+            for (int _i665 = 0; _i665 < _list663.size; ++_i665)
             {
-              _elem656 = iprot.readString();
-              struct.success.add(_elem656);
+              _elem664 = iprot.readString();
+              struct.success.add(_elem664);
             }
           }
           struct.setSuccessIsSet(true);
@@ -34709,16 +34841,16 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map658 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map658.size);
-                  String _key659;
-                  Type _val660;
-                  for (int _i661 = 0; _i661 < _map658.size; ++_i661)
+                  org.apache.thrift.protocol.TMap _map666 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map666.size);
+                  String _key667;
+                  Type _val668;
+                  for (int _i669 = 0; _i669 < _map666.size; ++_i669)
                   {
-                    _key659 = iprot.readString();
-                    _val660 = new Type();
-                    _val660.read(iprot);
-                    struct.success.put(_key659, _val660);
+                    _key667 = iprot.readString();
+                    _val668 = new Type();
+                    _val668.read(iprot);
+                    struct.success.put(_key667, _val668);
                   }
                   iprot.readMapEnd();
                 }
@@ -34753,10 +34885,10 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter662 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter670 : struct.success.entrySet())
             {
-              oprot.writeString(_iter662.getKey());
-              _iter662.getValue().write(oprot);
+              oprot.writeString(_iter670.getKey());
+              _iter670.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -34795,10 +34927,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter663 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter671 : struct.success.entrySet())
             {
-              oprot.writeString(_iter663.getKey());
-              _iter663.getValue().write(oprot);
+              oprot.writeString(_iter671.getKey());
+              _iter671.getValue().write(oprot);
             }
           }
         }
@@ -34813,16 +34945,16 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map664 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new HashMap<String,Type>(2*_map664.size);
-            String _key665;
-            Type _val666;
-            for (int _i667 = 0; _i667 < _map664.size; ++_i667)
+            org.apache.thrift.protocol.TMap _map672 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new HashMap<String,Type>(2*_map672.size);
+            String _key673;
+            Type _val674;
+            for (int _i675 = 0; _i675 < _map672.size; ++_i675)
             {
-              _key665 = iprot.readString();
-              _val666 = new Type();
-              _val666.read(iprot);
-              struct.success.put(_key665, _val666);
+              _key673 = iprot.readString();
+              _val674 = new Type();
+              _val674.read(iprot);
+              struct.success.put(_key673, _val674);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35857,14 +35989,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list668 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list668.size);
-                  FieldSchema _elem669;
-                  for (int _i670 = 0; _i670 < _list668.size; ++_i670)
+                  org.apache.thrift.protocol.TList _list676 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list676.size);
+                  FieldSchema _elem677;
+                  for (int _i678 = 0; _i678 < _list676.size; ++_i678)
                   {
-                    _elem669 = new FieldSchema();
-                    _elem669.read(iprot);
-                    struct.success.add(_elem669);
+                    _elem677 = new FieldSchema();
+                    _elem677.read(iprot);
+                    struct.success.add(_elem677);
                   }
                   iprot.readListEnd();
                 }
@@ -35917,9 +36049,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter671 : struct.success)
+            for (FieldSchema _iter679 : struct.success)
             {
-              _iter671.write(oprot);
+              _iter679.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -35974,9 +36106,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter672 : struct.success)
+            for (FieldSchema _iter680 : struct.success)
             {
-              _iter672.write(oprot);
+              _iter680.write(oprot);
             }
           }
         }
@@ -35997,14 +36129,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list673 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list673.size);
-            FieldSchema _elem674;
-            for (int _i675 = 0; _i675 < _list673.size; ++_i675)
+            org.apache.thrift.protocol.TList _list681 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list681.size);
+            FieldSchema _elem682;
+            for (int _i683 = 0; _i683 < _list681.size; ++_i683)
             {
-              _elem674 = new FieldSchema();
-              _elem674.read(iprot);
-              struct.success.add(_elem674);
+              _elem682 = new FieldSchema();
+              _elem682.read(iprot);
+              struct.success.add(_elem682);
             }
           }
           struct.setSuccessIsSet(true);
@@ -37158,14 +37290,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list676 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list676.size);
-                  FieldSchema _elem677;
-                  for (int _i678 = 0; _i678 < _list676.size; ++_i678)
+                  org.apache.thrift.protocol.TList _list684 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list684.size);
+                  FieldSchema _elem685;
+                  for (int _i686 = 0; _i686 < _list684.size; ++_i686)
                   {
-                    _elem677 = new FieldSchema();
-                    _elem677.read(iprot);
-                    struct.success.add(_elem677);
+                    _elem685 = new FieldSchema();
+                    _elem685.read(iprot);
+                    struct.success.add(_elem685);
                   }
                   iprot.readListEnd();
                 }
@@ -37218,9 +37350,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter679 : struct.success)
+            for (FieldSchema _iter687 : struct.success)
             {
-              _iter679.write(oprot);
+              _iter687.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -37275,9 +37407,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter680 : struct.success)
+            for (FieldSchema _iter688 : struct.success)
             {
-              _iter680.write(oprot);
+              _iter688.write(oprot);
             }
           }
         }
@@ -37298,14 +37430,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list681 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list681.size);
-            FieldSchema _elem682;
-            for (int _i683 = 0; _i683 < _list681.size; ++_i683)
+            org.apache.thrift.protocol.TList _list689 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list689.size);
+            FieldSchema _elem690;
+            for (int _i691 = 0; _i691 < _list689.size; ++_i691)
             {
-              _elem682 = new FieldSchema();
-              _elem682.read(iprot);
-              struct.success.add(_elem682);
+              _elem690 = new FieldSchema();
+              _elem690.read(iprot);
+              struct.success.add(_elem690);
             }
           }
           struct.setSuccessIsSet(true);
@@ -38350,14 +38482,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list684 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list684.size);
-                  FieldSchema _elem685;
-                  for (int _i686 = 0; _i686 < _list684.size; ++_i686)
+                  org.apache.thrift.protocol.TList _list692 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list692.size);
+                  FieldSchema _elem693;
+                  for (int _i694 = 0; _i694 < _list692.size; ++_i694)
                   {
-                    _elem685 = new FieldSchema();
-                    _elem685.read(iprot);
-                    struct.success.add(_elem685);
+                    _elem693 = new FieldSchema();
+                    _elem693.read(iprot);
+                    struct.success.add(_elem693);
                   }
                   iprot.readListEnd();
                 }
@@ -38410,9 +38542,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter687 : struct.success)
+            for (FieldSchema _iter695 : struct.success)
             {
-              _iter687.write(oprot);
+              _iter695.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -38467,9 +38599,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter688 : struct.success)
+            for (FieldSchema _iter696 : struct.success)
             {
-              _iter688.write(oprot);
+              _iter696.write(oprot);
             }
           }
         }
@@ -38490,14 +38622,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list689 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list689.size);
-            FieldSchema _elem690;
-            for (int _i691 = 0; _i691 < _list689.size; ++_i691)
+            org.apache.thrift.protocol.TList _list697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list697.size);
+            FieldSchema _elem698;
+            for (int _i699 = 0; _i699 < _list697.size; ++_i699)
             {
-              _elem690 = new FieldSchema();
-              _elem690.read(iprot);
-              struct.success.add(_elem690);
+              _elem698 = new FieldSchema();
+              _elem698.read(iprot);
+              struct.success.add(_elem698);
             }
           }
           struct.setSuccessIsSet(true);
@@ -39651,14 +39783,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list692 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list692.size);
-                  FieldSchema _elem693;
-                  for (int _i694 = 0; _i694 < _list692.size; ++_i694)
+                  org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list700.size);
+                  FieldSchema _elem701;
+                  for (int _i702 = 0; _i702 < _list700.size; ++_i702)
                   {
-                    _elem693 = new FieldSchema();
-                    _elem693.read(iprot);
-                    struct.success.add(_elem693);
+                    _elem701 = new FieldSchema();
+                    _elem701.read(iprot);
+                    struct.success.add(_elem701);
                   }
                   iprot.readListEnd();
                 }
@@ -39711,9 +39843,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter695 : struct.success)
+            for (FieldSchema _iter703 : struct.success)
             {
-              _iter695.write(oprot);
+              _iter703.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -39768,9 +39900,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter696 : struct.success)
+            for (FieldSchema _iter704 : struct.success)
             {
-              _iter696.write(oprot);
+              _iter704.write(oprot);
             }
           }
         }
@@ -39791,14 +39923,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list697.size);
-            FieldSchema _elem698;
-            for (int _i699 = 0; _i699 < _list697.size; ++_i699)
+            org.apache.thrift.protocol.TList _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list705.size);
+            FieldSchema _elem706;
+            for (int _i707 = 0; _i707 < _list705.size; ++_i707)
             {
-              _elem698 = new FieldSchema();
-              _elem698.read(iprot);
-              struct.success.add(_elem698);
+              _elem706 = new FieldSchema();
+              _elem706.read(iprot);
+              struct.success.add(_elem706);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42523,14 +42655,14 @@ public class ThriftHiveMetastore {
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list700.size);
-                  SQLPrimaryKey _elem701;
-                  for (int _i702 = 0; _i702 < _list700.size; ++_i702)
+                  org.apache.thrift.protocol.TList _list708 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list708.size);
+                  SQLPrimaryKey _elem709;
+                  for (int _i710 = 0; _i710 < _list708.size; ++_i710)
                   {
-                    _elem701 = new SQLPrimaryKey();
-                    _elem701.read(iprot);
-                    struct.primaryKeys.add(_elem701);
+                    _elem709 = new SQLPrimaryKey();
+                    _elem709.read(iprot);
+                    struct.primaryKeys.add(_elem709);
                   }
                   iprot.readListEnd();
                 }
@@ -42542,14 +42674,14 @@ public class ThriftHiveMetastore {
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list703 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list703.size);
-                  SQLForeignKey _elem704;
-                  for (int _i705 = 0; _i705 < _list703.size; ++_i705)
+                  org.apache.thrift.protocol.TList _list711 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list711.size);
+                  SQLForeignKey _elem712;
+                  for (int _i713 = 0; _i713 < _list711.size; ++_i713)
                   {
-                    _elem704 = new SQLForeignKey();
-                    _elem704.read(iprot);
-                    struct.foreignKeys.add(_elem704);
+                    _elem712 = new SQLForeignKey();
+                    _elem712.read(iprot);
+                    struct.foreignKeys.add(_elem712);
                   }
                   iprot.readListEnd();
                 }
@@ -42580,9 +42712,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-            for (SQLPrimaryKey _iter706 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter714 : struct.primaryKeys)
             {
-              _iter706.write(oprot);
+              _iter714.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -42592,9 +42724,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-            for (SQLForeignKey _iter707 : struct.foreignKeys)
+            for (SQLForeignKey _iter715 : struct.foreignKeys)
             {
-              _iter707.write(oprot);
+              _iter715.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -42634,18 +42766,18 @@ public class ThriftHiveMetastore {
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter708 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter716 : struct.primaryKeys)
             {
-              _iter708.write(oprot);
+              _iter716.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter709 : struct.foreignKeys)
+            for (SQLForeignKey _iter717 : struct.foreignKeys)
             {
-              _iter709.write(oprot);
+              _iter717.write(oprot);
             }
           }
         }
@@ -42662,28 +42794,28 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list710 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list710.size);
-            SQLPrimaryKey _elem711;
-            for (int _i712 = 0; _i712 < _list710.size; ++_i712)
+            org.apache.thrift.protocol.TList _list718 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list718.size);
+            SQLPrimaryKey _elem719;
+            for (int _i720 = 0; _i720 < _list718.size; ++_i720)
             {
-              _elem711 = new SQLPrimaryKey();
-              _elem711.read(iprot);
-              struct.primaryKeys.add(_elem711);
+              _elem719 = new SQLPrimaryKey();
+              _elem719.read(iprot);
+              struct.primaryKeys.add(_elem719);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list713.size);
-            SQLForeignKey _elem714;
-            for (int _i715 = 0; _i715 < _list713.size; ++_i715)
+            org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list721.size);
+            SQLForeignKey _elem722;
+            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
             {
-              _elem714 = new SQLForeignKey();
-              _elem714.read(iprot);
-              struct.foreignKeys.add(_elem714);
+              _elem722 = new SQLForeignKey();
+              _elem722.read(iprot);
+              struct.foreignKeys.add(_elem722);
             }
           }
           struct.setForeignKeysIsSet(true);
@@ -48882,13 +49014,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list716.size);
-                  String _elem717;
-                  for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list724.size);
+                  String _elem725;
+                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
                   {
-                    _elem717 = iprot.readString();
-                    struct.success.add(_elem717);
+                    _elem725 = iprot.readString();
+                    struct.success.add(_elem725);
                   }
                   iprot.readListEnd();
                 }
@@ -48923,9 +49055,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter719 : struct.success)
+            for (String _iter727 : struct.success)
             {
-              oprot.writeString(_iter719);
+              oprot.writeString(_iter727);
             }
             oprot.writeListEnd();
           }
@@ -48964,9 +49096,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter720 : struct.success)
+            for (String _iter728 : struct.success)
             {
-              oprot.writeString(_iter720);
+              oprot.writeString(_iter728);
             }
           }
         }
@@ -48981,13 +49113,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list721.size);
-            String _elem722;
-            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list729.size);
+            String _elem730;
+            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
             {
-              _elem722 = iprot.readString();
-              struct.success.add(_elem722);
+              _elem730 = iprot.readString();
+              struct.success.add(_elem730);
             }
           }
           struct.setSuccessIsSet(true);
@@ -49492,13 +49624,13 @@ public class ThriftHiveMetastore {
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list724.size);
-                  String _elem725;
-                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list732.size);
+                  String _elem733;
+                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                   {
-                    _elem725 = iprot.readString();
-                    struct.tbl_types.add(_elem725);
+                    _elem733 = iprot.readString();
+                    struct.tbl_types.add(_elem733);
                   }
                   iprot.readListEnd();
                 }
@@ -49534,9 +49666,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
-            for (String _iter727 : struct.tbl_types)
+            for (String _iter735 : struct.tbl_types)
             {
-              oprot.writeString(_iter727);
+              oprot.writeString(_iter735);
             }
             oprot.writeListEnd();
           }
@@ -49579,9 +49711,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter728 : struct.tbl_types)
+            for (String _iter736 : struct.tbl_types)
             {
-              oprot.writeString(_iter728);
+              oprot.writeString(_iter736);
             }
           }
         }
@@ -49601,13 +49733,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list729.size);
-            String _elem730;
-            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
+            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list737.size);
+            String _elem738;
+            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
             {
-              _elem730 = iprot.readString();
-              struct.tbl_types.add(_elem730);
+              _elem738 = iprot.readString();
+              struct.tbl_types.add(_elem738);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -50013,14 +50145,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list732.size);
-                  TableMeta _elem733;
-                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
+                  org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list740.size);
+                  TableMeta _elem741;
+                  for (int _i742 = 0; _i742 < _list740.size; ++_i742)
                   {
-                    _elem733 = new TableMeta();
-                    _elem733.read(iprot);
-                    struct.success.add(_elem733);
+                    _elem741 = new TableMeta();
+                    _elem741.read(iprot);
+                    struct.success.add(_elem741);
                   }
                   iprot.readListEnd();
                 }
@@ -50055,9 +50187,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter735 : struct.success)
+            for (TableMeta _iter743 : struct.success)
             {
-              _iter735.write(oprot);
+              _iter743.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -50096,9 +50228,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter736 : struct.success)
+            for (TableMeta _iter744 : struct.success)
             {
-              _iter736.write(oprot);
+              _iter744.write(oprot);
             }
           }
         }
@@ -50113,14 +50245,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list737.size);
-            TableMeta _elem738;
-            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
+            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list745.size);
+            TableMeta _elem746;
+            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
             {
-              _elem738 = new TableMeta();
-              _elem738.read(iprot);
-              struct.success.add(_elem738);
+              _elem746 = new TableMeta();
+              _elem746.read(iprot);
+              struct.success.add(_elem746);
             }
           }
           struct.setSuccessIsSet(true);
@@ -50886,13 +51018,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list740.size);
-                  String _elem741;
-                  for (int _i742 = 0; _i742 < _list740.size; ++_i742)
+                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list748.size);
+                  String _elem749;
+                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
                   {
-                    _elem741 = iprot.readString();
-                    struct.success.add(_elem741);
+                    _elem749 = iprot.readString();
+                    struct.success.add(_elem749);
                   }
                   iprot.readListEnd();
                 }
@@ -50927,9 +51059,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter743 : struct.success)
+            for (String _iter751 : struct.success)
             {
-              oprot.writeString(_iter743);
+              oprot.writeString(_iter751);
             }
             oprot.writeListEnd();
           }
@@ -50968,9 +51100,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter744 : struct.success)
+            for (String _iter752 : struct.success)
             {
-              oprot.writeString(_iter744);
+              oprot.writeString(_iter752);
             }
           }
         }
@@ -50985,13 +51117,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list745.size);
-            String _elem746;
-            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
+            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list753.size);
+            String _elem754;
+            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
             {
-              _elem746 = iprot.readString();
-              struct.success.add(_elem746);
+              _elem754 = iprot.readString();
+              struct.success.add(_elem754);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52444,13 +52576,13 @@ public class ThriftHiveMetastore {
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list748.size);
-                  String _elem749;
-                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
+                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list756.size);
+                  String _elem757;
+                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                   {
-                    _elem749 = iprot.readString();
-                    struct.tbl_names.add(_elem749);
+                    _elem757 = iprot.readString();
+                    struct.tbl_names.add(_elem757);
                   }
                   iprot.readListEnd();
                 }
@@ -52481,9 +52613,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter751 : struct.tbl_names)
+            for (String _iter759 : struct.tbl_names)
             {
-              oprot.writeString(_iter751);
+              oprot.writeString(_iter759);
             }
             oprot.writeListEnd();
           }
@@ -52520,9 +52652,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter752 : struct.tbl_names)
+            for (String _iter760 : struct.tbl_names)
             {
-              oprot.writeString(_iter752);
+              oprot.writeString(_iter760);
             }
           }
         }
@@ -52538,13 +52670,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list753.size);
-            String _elem754;
-            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
+            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list761.size);
+            String _elem762;
+            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
             {
-              _elem754 = iprot.readString();
-              struct.tbl_names.add(_elem754);
+              _elem762 = iprot.readString();
+              struct.tbl_names.add(_elem762);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -53112,14 +53244,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list756.size);
-                  Table _elem757;
-                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
+                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list764.size);
+                  Table _elem765;
+                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
                   {
-                    _elem757 = new Table();
-                    _elem757.read(iprot);
-                    struct.success.add(_elem757);
+                    _elem765 = new Table();
+                    _elem765.read(iprot);
+                    struct.success.add(_elem765);
                   }
                   iprot.readListEnd();
                 }
@@ -53172,9 +53304,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter759 : struct.success)
+            for (Table _iter767 : struct.success)
             {
-              _iter759.write(oprot);
+              _iter767.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -53229,9 +53361,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter760 : struct.success)
+            for (Table _iter768 : struct.success)
             {
-              _iter760.write(oprot);
+              _iter768.write(oprot);
             }
           }
         }
@@ -53252,14 +53384,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list761.size);
-            Table _elem762;
-            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
+            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list769.size);
+            Table _elem770;
+            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
             {
-              _elem762 = new Table();
-              _elem762.read(iprot);
-              struct.success.add(_elem762);
+              _elem770 = new Table();
+              _elem770.read(iprot);
+              struct.success.add(_elem770);
             }
           }
           struct.setSuccessIsSet(true);
@@ -54405,13 +54537,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list764.size);
-                  String _elem765;
-                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
+                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list772.size);
+                  String _elem773;
+                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
                   {
-                    _elem765 = iprot.readString();
-                    struct.success.add(_elem765);
+                    _elem773 = iprot.readString();
+                    struct.success.add(_elem773);
                   }
                   iprot.readListEnd();
                 }
@@ -54464,9 +54596,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter767 : struct.success)
+            for (String _iter775 : struct.success)
             {
-              oprot.writeString(_iter767);
+              oprot.writeString(_iter775);
             }
             oprot.writeListEnd();
           }
@@ -54521,9 +54653,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter768 : struct.success)
+            for (String _iter776 : struct.success)
             {
-              oprot.writeString(_iter768);
+              oprot.writeString(_iter776);
             }
           }
         }
@@ -54544,13 +54676,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list769.size);
-            String _elem770;
-            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
+            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list777.size);
+            String _elem778;
+            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
             {
-              _elem770 = iprot.readString();
-              struct.success.add(_elem770);
+              _elem778 = iprot.readString();
+              struct.success.add(_elem778);
             }
           }
           struct.setSuccessIsSet(true);
@@ -60409,14 +60541,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list772.size);
-                  Partition _elem773;
-                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
+                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list780.size);
+                  Partition _elem781;
+                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
                   {
-                    _elem773 = new Partition();
-                    _elem773.read(iprot);
-                    struct.new_parts.add(_elem773);
+                    _elem781 = new Partition();
+                    _elem781.read(iprot);
+                    struct.new_parts.add(_elem781);
                   }
                   iprot.readListEnd();
                 }
@@ -60442,9 +60574,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (Partition _iter775 : struct.new_parts)
+            for (Partition _iter783 : struct.new_parts)
             {
-              _iter775.write(oprot);
+              _iter783.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60475,9 +60607,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter776 : struct.new_parts)
+            for (Partition _iter784 : struct.new_parts)
             {
-              _iter776.write(oprot);
+              _iter784.write(oprot);
             }
           }
         }
@@ -60489,14 +60621,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list777.size);
-            Partition _elem778;
-            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
+            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list785.size);
+            Partition _elem786;
+            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
             {
-              _elem778 = new Partition();
-              _elem778.read(iprot);
-              struct.new_parts.add(_elem778);
+              _elem786 = new Partition();
+              _elem786.read(iprot);
+              struct.new_parts.add(_elem786);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -61497,14 +61629,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list780.size);
-                  PartitionSpec _elem781;
-                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
+                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list788.size);
+                  PartitionSpec _elem789;
+                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
                   {
-                    _elem781 = new PartitionSpec();
-                    _elem781.read(iprot);
-                    struct.new_parts.add(_elem781);
+                    _elem789 = new PartitionSpec();
+                    _elem789.read(iprot);
+                    struct.new_parts.add(_elem789);
                   }
                   iprot.readListEnd();
                 }
@@ -61530,9 +61662,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (PartitionSpec _iter783 : struct.new_parts)
+            for (PartitionSpec _iter791 : struct.new_parts)
             {
-              _iter783.write(oprot);
+              _iter791.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -61563,9 +61695,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter784 : struct.new_parts)
+            for (PartitionSpec _iter792 : struct.new_parts)
             {
-              _iter784.write(oprot);
+              _iter792.write(oprot);
             }
           }
         }
@@ -61577,14 +61709,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list785.size);
-            PartitionSpec _elem786;
-            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
+            org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list793.size);
+            PartitionSpec _elem794;
+            for (int _i795 = 0; _i795 < _list793.size; ++_i795)
             {
-              _elem786 = new PartitionSpec();
-              _elem786.read(iprot);
-              struct.new_parts.add(_elem786);
+              _elem794 = new PartitionSpec();
+              _elem794.read(iprot);
+              struct.new_parts.add(_elem794);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -62760,13 +62892,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list788.size);
-                  String _elem789;
-                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
+                  org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list796.size);
+                  String _elem797;
+                  for (int _i798 = 0; _i798 < _list796.size; ++_i798)
                   {
-                    _elem789 = iprot.readString();
-                    struct.part_vals.add(_elem789);
+                    _elem797 = iprot.readString();
+                    struct.part_vals.add(_elem797);
                   }
                   iprot.readListEnd();
                 }
@@ -62802,9 +62934,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter791 : struct.part_vals)
+            for (String _iter799 : struct.part_vals)
             {
-              oprot.writeString(_iter791);
+              oprot.writeString(_iter799);
             }
             oprot.writeListEnd();
           }
@@ -62847,9 +62979,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter792 : struct.part_vals)
+            for (String _iter800 : struct.part_vals)
             {
-              oprot.writeString(_iter792);
+              oprot.writeString(_iter800);
             }
           }
         }
@@ -62869,13 +63001,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list793.size);
-            String _elem794;
-            for (int _i795 = 0; _i795 < _list793.size; ++_i795)
+            org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list801.size);
+            String _elem802;
+            for (int _i803 = 0; _i803 < _list801.size; ++_i803)
             {
-              _elem794 = iprot.readString();
-              struct.part_vals.add(_elem794);
+              _elem802 = iprot.readString();
+              struct.part_vals.add(_elem802);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -65184,13 +65316,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list796.size);
-                  String _elem797;
-                  for (int _i798 = 0; _i798 < _list796.size; ++_i798)
+                  org.apache.thrift.protocol.TList _list804 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list804.size);
+                  String _elem805;
+                  for (int _i806 = 0; _i806 < _list804.size; ++_i806)
                   {
-                    _elem797 = iprot.readString();
-                    struct.part_vals.add(_elem797);
+                    _elem805 = iprot.readString();
+                    struct.part_vals.add(_elem805);
                   }
                   iprot.readListEnd();
                 }
@@ -65235,9 +65367,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter799 : struct.part_vals)
+            for (String _iter807 : struct.part_vals)
             {
-              oprot.writeString(_iter799);
+              oprot.writeString(_iter807);
             }
             oprot.writeListEnd();
           }
@@ -65288,9 +65420,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter800 : struct.part_vals)
+            for (String _iter808 : struct.part_vals)
             {
-              oprot.writeString(_iter800);
+              oprot.writeString(_iter808);
             }
           }
         }
@@ -65313,13 +65445,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list801.size);
-            String _elem802;
-            for (int _i803 = 0; _i803 < _list801.size; ++_i803)
+            org.apache.thrift.protocol.TList _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list809.size);
+            String _elem810;
+            for (int _i811 = 0; _i811 < _list809.size; ++_i811)
             {
-              _elem802 = iprot.readString();
-              struct.part_vals.add(_elem802);
+              _elem810 = iprot.readString();
+              struct.part_vals.add(_elem810);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -69189,13 +69321,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list804 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list804.size);
-                  String _elem805;
-                  for (int _i806 = 0; _i806 < _list804.size; ++_i806)
+                  org.apache.thrift.protocol.TList _list812 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list812.size);
+                  String _elem813;
+                  for (int _i814 = 0; _i814 < _list812.size; ++_i814)
                   {
-                    _elem805 = iprot.readString();
-                    struct.part_vals.add(_elem805);
+                    _elem813 = iprot.readString();
+                    struct.part_vals.add(_elem813);
                   }
                   iprot.readListEnd();
                 }
@@ -69239,9 +69371,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter807 : struct.part_vals)
+            for (String _iter815 : struct.part_vals)
             {
-              oprot.writeString(_iter807);
+              oprot.writeString(_iter815);
             }
             oprot.writeListEnd();
           }
@@ -69290,9 +69422,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter808 : struct.part_vals)
+            for (String _iter816 : struct.part_vals)
             {
-              oprot.writeString(_iter808);
+              oprot.writeString(_iter816);
             }
           }
         }
@@ -69315,13 +69447,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list809.size);
-            String _elem810;
-            for (int _i811 = 0; _i811 < _list809.size; ++_i811)
+            org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list817.size);
+            String _elem818;
+            for (int _i819 = 0; _i819 < _list817.size; ++_i819)
             {
-              _elem810 = iprot.readString();
-              struct.part_vals.add(_elem810);
+              _elem818 = iprot.readString();
+              struct.part_vals.add(_elem818);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -70560,13 +70692,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list812 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list812.size);
-                  String _elem813;
-                  for (int _i814 = 0; _i814 < _list812.size; ++_i814)
+                  org.apache.thrift.protocol.TList _list820 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list820.size);
+                  String _elem821;
+                  for (int _i822 = 0; _i822 < _list820.size; ++_i822)
                   {
-                    _elem813 = iprot.readString();
-                    struct.part_vals.add(_elem813);
+                    _elem821 = iprot.readString();
+                    struct.part_vals.add(_elem821);
                   }
                   iprot.readListEnd();
                 }
@@ -70619,9 +70751,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter815 : struct.part_vals)
+            for (String _iter823 : struct.part_vals)
             {
-              oprot.writeString(_iter815);
+              oprot.writeString(_iter823);
             }
             oprot.writeListEnd();
           }
@@ -70678,9 +70810,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter816 : struct.part_vals)
+            for (String _iter824 : struct.part_vals)
             {
-              oprot.writeString(_iter816);
+              oprot.writeString(_iter824);
             }
           }
         }
@@ -70706,13 +70838,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list817.size);
-            String _elem818;
-            for (int _i819 = 0; _i819 < _list817.size; ++_i819)
+            org.apache.thrift.protocol.TList _list825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list825.size);
+            String _elem826;
+            for (int _i827 = 0; _i827 < _list825.size; ++_i827)
             {
-              _elem818 = iprot.readString();
-              struct.part_vals.add(_elem818);
+              _elem826 = iprot.readString();
+              struct.part_vals.add(_elem826);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -75314,13 +75446,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list820 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list820.size);
-                  String _elem821;
-                  for (int _i822 = 0; _i822 < _list820.size; ++_i822)
+                  org.apache.thrift.protocol.TList _list828 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list828.size);
+                  String _elem829;
+                  for (int _i830 = 0; _i830 < _list828.size; ++_i830)
                   {
-                    _elem821 = iprot.readString();
-                    struct.part_vals.add(_elem821);
+                    _elem829 = iprot.readString();
+                    struct.part_vals.add(_elem829);
                   }
                   iprot.readListEnd();
                 }
@@ -75356,9 +75488,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter823 : struct.part_vals)
+            for (String _iter831 : struct.part_vals)
             {
-              oprot.writeString(_iter823);
+              oprot.writeString(_iter831);
             }
             oprot.writeListEnd();
           }
@@ -75401,9 +75533,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter824 : struct.part_vals)
+            for (String _iter832 : struct.part_vals)
             {
-              oprot.writeString(_iter824);
+              oprot.writeString(_iter832);
             }
           }
         }
@@ -75423,13 +75555,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list825.size);
-            String _elem826;
-            for (int _i827 = 0; _i827 < _list825.size; ++_i827)
+            org.apache.thrift.protocol.TList _list833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list833.size);
+            String _elem834;
+            for (int _i835 = 0; _i835 < _list833.size; ++_i835)
             {
-              _elem826 = iprot.readString();
-              struct.part_vals.add(_elem826);
+              _elem834 = iprot.readString();
+              struct.part_vals.add(_elem834);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -76647,15 +76779,15 @@ public class ThriftHiveMetastore {
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map828 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map828.size);
-                  String _key829;
-                  String _val830;
-                  for (int _i831 = 0; _i831 < _map828.size; ++_i831)
+                  org.apache.thrift.protocol.TMap _map836 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map836.size);
+                  String _key837;
+                  String _val838;
+                  for (int _i839 = 0; _i839 < _map836.size; ++_i839)
                   {
-                    _key829 = iprot.readString();
-                    _val830 = iprot.readString();
-                    struct.partitionSpecs.put(_key829, _val830);
+                    _key837 = iprot.readString();
+                    _val838 = iprot.readString();
+                    struct.partitionSpecs.put(_key837, _val838);
                   }
                   iprot.readMapEnd();
                 }
@@ -76713,10 +76845,10 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
-            for (Map.Entry<String, String> _iter832 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter840 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter832.getKey());
-              oprot.writeString(_iter832.getValue());
+              oprot.writeString(_iter840.getKey());
+              oprot.writeString(_iter840.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -76779,10 +76911,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter833 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter841 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter833.getKey());
-              oprot.writeString(_iter833.getValue());
+              oprot.writeString(_iter841.getKey());
+              oprot.writeString(_iter841.getValue());
             }
           }
         }
@@ -76806,15 +76938,15 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map834 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partitionSpecs = new HashMap<String,String>(2*_map834.size);
-            String _key835;
-            String _val836;
-            for (int _i837 = 0; _i837 < _map834.size; ++_i837)
+            org.apache.thrift.protocol.TMap _map842 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partitionSpecs = new HashMap<String,String>(2*_map842.size);
+            String _key843;
+            String _val844;
+            for (int _i845 = 0; _i845 < _map842.size; ++_i845)
             {
-              _key835 = iprot.readString();
-              _val836 = iprot.readString();
-              struct.partitionSpecs.put(_key835, _val836);
+              _key843 = iprot.readString();
+              _val844 = iprot.readString();
+              struct.partitionSpecs.put(_key843, _val844);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -78260,15 +78392,15 @@ public class ThriftHiveMetastore {
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map838 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map838.size);
-                  String _key839;
-                  String _val840;
-                  for (int _i841 = 0; _i841 < _map838.size; ++_i841)
+                  org.apache.thrift.protocol.TMap _map846 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map846.size);
+                  String _key847;
+                  String _val848;
+                  for (int _i849 = 0; _i849 < _map846.size; ++_i849)
                   {
-                    _key839 = iprot.readString();
-                    _val840 = iprot.readString();
-                    struct.partitionSpecs.put(_key839, _val840);
+                    _key847 = iprot.readString();
+                    _val848 = iprot.readString();
+                    struct.partitionSpecs.put(_key847, _val848);
                   }
                   iprot.readMapEnd();
                 }
@@ -78326,10 +78458,10 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
-            for (Map.Entry<String, String> _iter842 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter850 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter842.getKey());
-              oprot.writeString(_iter842.getValue());
+              oprot.writeString(_iter850.getKey());
+              oprot.writeString(_iter850.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -78392,10 +78524,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter843 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter851 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter843.getKey());
-              oprot.writeString(_iter843.getValue());
+              oprot.writeString(_iter851.getKey());
+              oprot.writeString(_iter851.getValue());
             }
           }
         }
@@ -78419,15 +78551,15 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map844 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partitionSpecs = new HashMap<String,String>(2*_map844.size);
-            String _key845;
-            String _val846;
-            for (int _i847 = 0; _i847 < _map844.size; ++_i847)
+            org.apache.thrift.protocol.TMap _map852 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partitionSpecs = new HashMap<String,String>(2*_map852.size);
+            String _key853;
+            String _val854;
+            for (int _i855 = 0; _i855 < _map852.size; ++_i855)
       

<TRUNCATED>

[22/31] hive git commit: HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: 815e069889e06c8509c00678c6d14bc431fa8037
Parents: 30fd19f 49f103c
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Sep 12 13:12:42 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Sep 12 13:12:42 2016 -0700

----------------------------------------------------------------------
 beeline/pom.xml                                 |    10 -
 .../java/org/apache/hive/beeline/BeeLine.java   |    15 +-
 .../java/org/apache/hive/beeline/Commands.java  |     2 +-
 .../org/apache/hadoop/hive/conf/Constants.java  |    10 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    11 +
 druid-handler/pom.xml                           |   201 +
 .../hadoop/hive/druid/DruidStorageHandler.java  |   109 +
 .../hive/druid/DruidStorageHandlerUtils.java    |    90 +
 .../hive/druid/HiveDruidOutputFormat.java       |    55 +
 .../druid/HiveDruidQueryBasedInputFormat.java   |   369 +
 .../hadoop/hive/druid/HiveDruidSplit.java       |    83 +
 .../serde/DruidGroupByQueryRecordReader.java    |   199 +
 .../druid/serde/DruidQueryRecordReader.java     |   142 +
 .../serde/DruidSelectQueryRecordReader.java     |   106 +
 .../hadoop/hive/druid/serde/DruidSerDe.java     |   343 +
 .../hive/druid/serde/DruidSerDeUtils.java       |    83 +
 .../serde/DruidTimeseriesQueryRecordReader.java |    93 +
 .../druid/serde/DruidTopNQueryRecordReader.java |   106 +
 .../hadoop/hive/druid/serde/DruidWritable.java  |    81 +
 .../hadoop/hive/druid/QTestDruidSerDe.java      |    88 +
 .../hive/druid/QTestDruidStorageHandler.java    |    34 +
 .../hadoop/hive/druid/TestDruidSerDe.java       |   576 +
 .../TestHiveDruidQueryBasedInputFormat.java     |   101 +
 .../hive/hcatalog/messaging/MessageFactory.java |     2 +-
 .../hive/jdbc/miniHS2/AbstractHiveService.java  |     2 +-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |     4 +
 .../apache/hive/jdbc/TestJdbcWithMiniHS2.java   |    44 +
 .../thrift/TestMiniHS2StateWithNoZookeeper.java |    82 +
 itests/qtest/pom.xml                            |    13 +
 .../resources/testconfiguration.properties.orig |  1377 --
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |     3 +-
 .../hadoop/hive/llap/cache/BuddyAllocator.java  |    18 +-
 .../tezplugins/LlapTaskSchedulerService.java    |    35 +-
 .../hadoop/hive/metastore/DatabaseProduct.java  |    75 +
 .../hive/metastore/MetaStoreDirectSql.java      |    58 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |    52 +-
 .../hadoop/hive/metastore/txn/TestTxnUtils.java |     7 +-
 packaging/pom.xml                               |     5 +
 pom.xml                                         |     2 +
 .../org/apache/hadoop/hive/ql/Context.java.orig |   829 --
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    41 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    22 +-
 .../apache/hadoop/hive/ql/exec/MapOperator.java |   110 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |     2 +-
 .../hadoop/hive/ql/metadata/VirtualColumn.java  |    41 +-
 .../optimizer/calcite/HivePlannerContext.java   |    17 +-
 .../calcite/druid/DruidIntervalUtils.java       |   466 +
 .../ql/optimizer/calcite/druid/DruidQuery.java  |  1053 ++
 .../optimizer/calcite/druid/DruidQueryType.java |    42 +
 .../ql/optimizer/calcite/druid/DruidRules.java  |   591 +
 .../ql/optimizer/calcite/druid/DruidSchema.java |    51 +
 .../ql/optimizer/calcite/druid/DruidTable.java  |   121 +
 .../optimizer/calcite/druid/HiveDruidConf.java  |    33 +
 .../functions/HiveSqlCountAggFunction.java      |     2 +-
 .../functions/HiveSqlMinMaxAggFunction.java     |     2 +-
 .../functions/HiveSqlSumAggFunction.java        |     2 +-
 .../reloperators/HiveDateGranularity.java       |    54 +
 .../rules/HiveProjectSortTransposeRule.java     |     5 +
 .../rules/HiveSortProjectTransposeRule.java     |     5 +
 .../calcite/translator/ASTBuilder.java          |    38 +-
 .../calcite/translator/ASTConverter.java        |     9 +-
 .../translator/SqlFunctionConverter.java        |    23 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   119 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    40 +-
 .../hive/ql/parse/SemanticAnalyzer.java.orig    | 13038 -----------------
 .../hadoop/hive/ql/parse/TaskCompiler.java      |     2 +-
 .../hadoop/hive/ql/plan/CreateTableDesc.java    |     8 +-
 .../hadoop/hive/ql/plan/TableScanDesc.java      |     7 +
 .../hadoop/hive/ql/session/SessionState.java    |    13 +
 .../apache/hadoop/hive/ql/udf/UDFDateFloor.java |   506 +
 .../hadoop/hive/ql/udf/UDFDateFloorDay.java     |    39 +
 .../hadoop/hive/ql/udf/UDFDateFloorHour.java    |    39 +
 .../hadoop/hive/ql/udf/UDFDateFloorMinute.java  |    39 +
 .../hadoop/hive/ql/udf/UDFDateFloorMonth.java   |    39 +
 .../hadoop/hive/ql/udf/UDFDateFloorQuarter.java |    39 +
 .../hadoop/hive/ql/udf/UDFDateFloorSecond.java  |    39 +
 .../hadoop/hive/ql/udf/UDFDateFloorWeek.java    |    39 +
 .../hadoop/hive/ql/udf/UDFDateFloorYear.java    |    39 +
 .../calcite/TestCBORuleFiredOnlyOnce.java       |     2 +-
 .../ql/udf/TestUDFDateFormatGranularity.java    |    85 +
 .../test/queries/clientnegative/druid_address.q |     5 +
 .../test/queries/clientnegative/druid_buckets.q |     6 +
 .../queries/clientnegative/druid_datasource.q   |     3 +
 .../queries/clientnegative/druid_external.q     |     5 +
 .../queries/clientnegative/druid_location.q     |     6 +
 .../queries/clientnegative/druid_partitions.q   |     6 +
 .../test/queries/clientpositive/druid_basic1.q  |    18 +
 .../test/queries/clientpositive/druid_basic2.q  |    52 +
 .../queries/clientpositive/druid_intervals.q    |    67 +
 .../queries/clientpositive/druid_timeseries.q   |    94 +
 ql/src/test/queries/clientpositive/druid_topn.q |    75 +
 .../clientpositive/encryption_drop_partition.q  |     7 +
 .../clientpositive/encryption_drop_table.q      |    23 +
 .../queries/clientpositive/groupby_nullvalues.q |    29 +
 .../results/clientnegative/druid_address.q.out  |     7 +
 .../results/clientnegative/druid_buckets.q.out  |     8 +
 .../clientnegative/druid_datasource.q.out       |     7 +
 .../results/clientnegative/druid_external.q.out |     7 +
 .../results/clientnegative/druid_location.q.out |     9 +
 .../clientnegative/druid_partitions.q.out       |     8 +
 .../results/clientpositive/create_view.q.out    |     2 +
 ql/src/test/results/clientpositive/ctas.q.out   |     8 +-
 .../results/clientpositive/druid_basic1.q.out   |   142 +
 .../results/clientpositive/druid_basic2.q.out   |   533 +
 .../clientpositive/druid_intervals.q.out        |   398 +
 .../clientpositive/druid_timeseries.q.out       |   566 +
 .../results/clientpositive/druid_topn.q.out     |   419 +
 .../encrypted/encryption_drop_partition.q.out   |    33 +
 .../encrypted/encryption_drop_table.q.out       |   110 +
 .../results/clientpositive/explain_ddl.q.out    |     2 +
 .../clientpositive/explain_logical.q.out        |    16 +
 .../clientpositive/groupby_nullvalues.q.out     |    83 +
 .../test/results/clientpositive/join_view.q.out |     4 +
 .../test/results/clientpositive/llap/ctas.q.out |     8 +-
 .../clientpositive/llap/explainuser_1.q.out     |    10 +-
 .../test/results/clientpositive/masking_2.q.out |    14 +
 .../test/results/clientpositive/masking_6.q.out |     8 +
 .../test/results/clientpositive/masking_7.q.out |     8 +
 .../clientpositive/serde_user_properties.q.out  |     4 +
 .../results/clientpositive/show_functions.q.out |     9 +
 .../results/clientpositive/spark/ctas.q.out     |     8 +-
 .../clientpositive/spark/join_view.q.out        |     4 +
 .../results/clientpositive/subquery_notin.q.out |     6 +
 .../results/clientpositive/subquery_views.q.out |     4 +
 .../service/cli/session/HiveSessionImpl.java    |     9 +-
 .../service/cli/session/SessionManager.java     |     2 +-
 .../apache/hive/service/server/HiveServer2.java |    17 +-
 127 files changed, 9688 insertions(+), 15558 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/815e0698/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/815e0698/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/815e0698/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------


[13/31] hive git commit: HIVE-14591: HS2 is shut down unexpectedly during the startup time (Tao Li reviewed by Vaibhav Gumashta)

Posted by se...@apache.org.
HIVE-14591: HS2 is shut down unexpectedly during the startup time (Tao Li reviewed by Vaibhav Gumashta)


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

Branch: refs/heads/hive-14535
Commit: 5edf7c8335279a8393300bfb11743c144d23836d
Parents: 0755348
Author: Vaibhav Gumashta <vg...@hortonworks.com>
Authored: Fri Sep 9 02:35:37 2016 -0700
Committer: Vaibhav Gumashta <vg...@hortonworks.com>
Committed: Fri Sep 9 02:35:37 2016 -0700

----------------------------------------------------------------------
 .../hive/jdbc/miniHS2/AbstractHiveService.java  |  2 +-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |  4 +
 .../thrift/TestMiniHS2StateWithNoZookeeper.java | 82 ++++++++++++++++++++
 .../service/cli/session/SessionManager.java     |  2 +-
 .../apache/hive/service/server/HiveServer2.java | 17 ++--
 5 files changed, 97 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5edf7c83/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
index 2c1cd07..98c5d01 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
@@ -140,7 +140,7 @@ public abstract class AbstractHiveService {
     return startedHiveService;
   }
 
-  protected void setStarted(boolean hiveServiceStatus) {
+  public void setStarted(boolean hiveServiceStatus) {
     this.startedHiveService =  hiveServiceStatus;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/5edf7c83/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
index de1ce76..abb80a2 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
@@ -502,4 +502,8 @@ public class MiniHS2 extends AbstractHiveService {
       break;
     } while (true);
   }
+  
+  public Service.STATE getState() {
+    return hiveServer2.getServiceState();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5edf7c83/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestMiniHS2StateWithNoZookeeper.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestMiniHS2StateWithNoZookeeper.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestMiniHS2StateWithNoZookeeper.java
new file mode 100644
index 0000000..5114803
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestMiniHS2StateWithNoZookeeper.java
@@ -0,0 +1,82 @@
+/**
+ * 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.service.cli.thrift;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.apache.hive.service.Service;
+import org.apache.hive.service.cli.CLIServiceClient;
+import org.apache.hive.service.cli.SessionHandle;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+*
+* TestMiniHS2StateWithNoZookeeper.
+* This tests HS2 shutdown is not triggered by CloseSession operation 
+* while HS2 has never been registered with ZooKeeper.
+*
+*/
+
+public class TestMiniHS2StateWithNoZookeeper {
+  
+  private static final Logger LOG = LoggerFactory.getLogger(TestMiniHS2StateWithNoZookeeper.class);
+  private static MiniHS2 miniHS2 = null;
+  private static HiveConf hiveConf = null;
+
+  @BeforeClass
+  public static void beforeTest() throws Exception   { 
+    hiveConf = new HiveConf();
+    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY, true);
+    hiveConf.setIntVar(ConfVars.HIVE_ZOOKEEPER_CONNECTION_MAX_RETRIES, 0);
+    hiveConf.setTimeVar(ConfVars.HIVE_ZOOKEEPER_CONNECTION_BASESLEEPTIME, 0, TimeUnit.MILLISECONDS);
+    miniHS2 = new MiniHS2(hiveConf);
+    Map<String, String> confOverlay = new HashMap<String, String>();
+    try {
+      miniHS2.start(confOverlay);
+    } catch (Exception ex) {
+      LOG.warn("Zookeeper is not set up intentionally, so the error is expected (unless it's not related to ZK): " + ex);
+      miniHS2.setStarted(true);
+    }
+  }
+
+  @AfterClass
+  public static void afterTest() throws Exception {
+    miniHS2.stop();
+  }
+
+  @Test
+  public void openSessionAndClose() throws Exception {
+    CLIServiceClient client = miniHS2.getServiceClient();
+    SessionHandle sessionHandle = client.openSession(null, null, null);
+    client.closeSession(sessionHandle);
+    Thread.sleep(100);
+
+    Assert.assertEquals(Service.STATE.STARTED, miniHS2.getState());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/5edf7c83/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
index 88dee48..15bab06 100644
--- a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
+++ b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
@@ -366,7 +366,7 @@ public class SessionManager extends CompositeService {
     } finally {
       // Shutdown HiveServer2 if it has been deregistered from ZooKeeper and has no active sessions
       if (!(hiveServer2 == null) && (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY))
-          && (!hiveServer2.isRegisteredWithZooKeeper())) {
+          && (hiveServer2.isDeregisteredWithZooKeeper())) {
         // Asynchronously shutdown this instance of HiveServer2,
         // if there are no active client sessions
         if (getOpenSessionCount() == 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/5edf7c83/service/src/java/org/apache/hive/service/server/HiveServer2.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index 0ecaa76..9822a85 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -94,7 +94,7 @@ public class HiveServer2 extends CompositeService {
   private PersistentEphemeralNode znode;
   private String znodePath;
   private CuratorFramework zooKeeperClient;
-  private boolean registeredWithZooKeeper = false;
+  private boolean deregisteredWithZooKeeper = false; // Set to true only when deregistration happens
   private HttpServer webServer; // Web UI
 
   public HiveServer2() {
@@ -324,7 +324,7 @@ public class HiveServer2 extends CompositeService {
       if (!znode.waitForInitialCreate(znodeCreationTimeout, TimeUnit.SECONDS)) {
         throw new Exception("Max znode creation wait time: " + znodeCreationTimeout + "s exhausted");
       }
-      setRegisteredWithZooKeeper(true);
+      setDeregisteredWithZooKeeper(false);
       znodePath = znode.getActualPath();
       // Set a watch on the znode
       if (zooKeeperClient.checkExists().usingWatcher(new DeRegisterWatcher()).forPath(znodePath) == null) {
@@ -415,7 +415,7 @@ public class HiveServer2 extends CompositeService {
           } catch (IOException e) {
             LOG.error("Failed to close the persistent ephemeral znode", e);
           } finally {
-            HiveServer2.this.setRegisteredWithZooKeeper(false);
+            HiveServer2.this.setDeregisteredWithZooKeeper(true);
             // If there are no more active client sessions, stop the server
             if (cliService.getSessionManager().getOpenSessionCount() == 0) {
               LOG.warn("This instance of HiveServer2 has been removed from the list of server "
@@ -430,7 +430,8 @@ public class HiveServer2 extends CompositeService {
   }
 
   private void removeServerInstanceFromZooKeeper() throws Exception {
-    setRegisteredWithZooKeeper(false);
+    setDeregisteredWithZooKeeper(true);
+    
     if (znode != null) {
       znode.close();
     }
@@ -438,12 +439,12 @@ public class HiveServer2 extends CompositeService {
     LOG.info("Server instance removed from ZooKeeper.");
   }
 
-  public boolean isRegisteredWithZooKeeper() {
-    return registeredWithZooKeeper;
+  public boolean isDeregisteredWithZooKeeper() {
+    return deregisteredWithZooKeeper;
   }
 
-  private void setRegisteredWithZooKeeper(boolean registeredWithZooKeeper) {
-    this.registeredWithZooKeeper = registeredWithZooKeeper;
+  private void setDeregisteredWithZooKeeper(boolean deregisteredWithZooKeeper) {
+    this.deregisteredWithZooKeeper = deregisteredWithZooKeeper;
   }
 
   private String getServerInstanceURI() throws Exception {


[15/31] hive git commit: HIVE-14686: Get unexpected command type when execute query "CREATE TABLE IF NOT EXISTS ... AS" (Yunbo via Rui)

Posted by se...@apache.org.
HIVE-14686: Get unexpected command type when execute query "CREATE TABLE IF NOT EXISTS ... AS" (Yunbo via Rui)


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

Branch: refs/heads/hive-14535
Commit: 407cfe1856a234eae40b410db19381ae4cfa56e3
Parents: d2e294c
Author: Yunbo Fan <44...@qq.com>
Authored: Fri Sep 9 19:02:52 2016 +0800
Committer: Rui Li <sh...@cn.ibm.com>
Committed: Fri Sep 9 19:02:52 2016 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/parse/SemanticAnalyzer.java   | 12 ++++++++----
 ql/src/test/results/clientpositive/ctas.q.out           |  8 ++++----
 ql/src/test/results/clientpositive/llap/ctas.q.out      |  8 ++++----
 .../results/clientpositive/llap/explainuser_1.q.out     |  8 ++++----
 ql/src/test/results/clientpositive/spark/ctas.q.out     |  8 ++++----
 5 files changed, 24 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/407cfe18/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 489e70f..e4a37f4 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
@@ -11620,6 +11620,14 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       }
     }
 
+    if (command_type == CREATE_TABLE || command_type == CTLT) {
+        queryState.setCommandType(HiveOperation.CREATETABLE);
+    } else if (command_type == CTAS) {
+        queryState.setCommandType(HiveOperation.CREATETABLE_AS_SELECT);
+    } else {
+        throw new SemanticException("Unrecognized command.");
+    }
+
     storageFormat.fillDefaultStorageFormat(isExt);
 
     if ((command_type == CTAS) && (storageFormat.getStorageHandler() != null)) {
@@ -11681,7 +11689,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       crtTblDesc.validate(conf);
       // outputs is empty, which means this create table happens in the current
       // database.
-      queryState.setCommandType(HiveOperation.CREATETABLE);
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
           crtTblDesc), conf));
       break;
@@ -11700,7 +11707,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           storageFormat.getInputFormat(), storageFormat.getOutputFormat(), location,
           storageFormat.getSerde(), storageFormat.getSerdeProps(), tblProps, ifNotExists,
           likeTableName, isUserStorageFormat);
-      queryState.setCommandType(HiveOperation.CREATETABLE);
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
           crtTblLikeDesc), conf));
       break;
@@ -11774,8 +11780,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       tableDesc.setNullFormat(rowFormatParams.nullFormat);
       qb.setTableDesc(tableDesc);
 
-      queryState.setCommandType(HiveOperation.CREATETABLE_AS_SELECT);
-
       return selectStmt;
     default:
       throw new SemanticException("Unrecognized command.");

http://git-wip-us.apache.org/repos/asf/hive/blob/407cfe18/ql/src/test/results/clientpositive/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ctas.q.out b/ql/src/test/results/clientpositive/ctas.q.out
index afd6d7c..7f1c912 100644
--- a/ql/src/test/results/clientpositive/ctas.q.out
+++ b/ql/src/test/results/clientpositive/ctas.q.out
@@ -474,16 +474,16 @@ Sort Columns:       	[]
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-PREHOOK: type: CREATETABLE
+PREHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-POSTHOOK: type: CREATETABLE
+POSTHOOK: type: CREATETABLE_AS_SELECT
 STAGE DEPENDENCIES:
 
 STAGE PLANS:
 PREHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-PREHOOK: type: CREATETABLE
+PREHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-POSTHOOK: type: CREATETABLE
+POSTHOOK: type: CREATETABLE_AS_SELECT
 PREHOOK: query: select * from nzhang_ctas3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@nzhang_ctas3

http://git-wip-us.apache.org/repos/asf/hive/blob/407cfe18/ql/src/test/results/clientpositive/llap/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/ctas.q.out b/ql/src/test/results/clientpositive/llap/ctas.q.out
index f48d431..dab0251 100644
--- a/ql/src/test/results/clientpositive/llap/ctas.q.out
+++ b/ql/src/test/results/clientpositive/llap/ctas.q.out
@@ -489,16 +489,16 @@ Sort Columns:       	[]
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-PREHOOK: type: CREATETABLE
+PREHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-POSTHOOK: type: CREATETABLE
+POSTHOOK: type: CREATETABLE_AS_SELECT
 STAGE DEPENDENCIES:
 
 STAGE PLANS:
 PREHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-PREHOOK: type: CREATETABLE
+PREHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-POSTHOOK: type: CREATETABLE
+POSTHOOK: type: CREATETABLE_AS_SELECT
 PREHOOK: query: select * from nzhang_ctas3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@nzhang_ctas3

http://git-wip-us.apache.org/repos/asf/hive/blob/407cfe18/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
index ef01018..70ec02f 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -3393,14 +3393,14 @@ POSTHOOK: Output: default@nzhang_ctas3
 POSTHOOK: Lineage: nzhang_ctas3.conb EXPRESSION [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 POSTHOOK: Lineage: nzhang_ctas3.half_key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 PREHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-PREHOOK: type: CREATETABLE
+PREHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-POSTHOOK: type: CREATETABLE
+POSTHOOK: type: CREATETABLE_AS_SELECT
 
 PREHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-PREHOOK: type: CREATETABLE
+PREHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-POSTHOOK: type: CREATETABLE
+POSTHOOK: type: CREATETABLE_AS_SELECT
 PREHOOK: query: explain create temporary table acid_dtt(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 POSTHOOK: query: explain create temporary table acid_dtt(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')

http://git-wip-us.apache.org/repos/asf/hive/blob/407cfe18/ql/src/test/results/clientpositive/spark/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/ctas.q.out b/ql/src/test/results/clientpositive/spark/ctas.q.out
index f7165fc..7a44cc7 100644
--- a/ql/src/test/results/clientpositive/spark/ctas.q.out
+++ b/ql/src/test/results/clientpositive/spark/ctas.q.out
@@ -462,16 +462,16 @@ Sort Columns:       	[]
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-PREHOOK: type: CREATETABLE
+PREHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-POSTHOOK: type: CREATETABLE
+POSTHOOK: type: CREATETABLE_AS_SELECT
 STAGE DEPENDENCIES:
 
 STAGE PLANS:
 PREHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-PREHOOK: type: CREATETABLE
+PREHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-POSTHOOK: type: CREATETABLE
+POSTHOOK: type: CREATETABLE_AS_SELECT
 PREHOOK: query: select * from nzhang_ctas3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@nzhang_ctas3


[09/31] hive git commit: HIVE-14217: Druid integration (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-14217: Druid integration (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Close apache/hive#98


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

Branch: refs/heads/hive-14535
Commit: 58d1befa2131254b53122b3573189ac1c5022217
Parents: 63fdb51
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri Aug 12 12:55:46 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu Sep 8 09:42:26 2016 +0100

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/Constants.java  |   10 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   11 +
 druid-handler/pom.xml                           |  201 ++++
 .../hadoop/hive/druid/DruidStorageHandler.java  |  109 ++
 .../hive/druid/DruidStorageHandlerUtils.java    |   90 ++
 .../hive/druid/HiveDruidOutputFormat.java       |   55 +
 .../druid/HiveDruidQueryBasedInputFormat.java   |  369 ++++++
 .../hadoop/hive/druid/HiveDruidSplit.java       |   83 ++
 .../serde/DruidGroupByQueryRecordReader.java    |  199 ++++
 .../druid/serde/DruidQueryRecordReader.java     |  142 +++
 .../serde/DruidSelectQueryRecordReader.java     |  106 ++
 .../hadoop/hive/druid/serde/DruidSerDe.java     |  343 ++++++
 .../hive/druid/serde/DruidSerDeUtils.java       |   83 ++
 .../serde/DruidTimeseriesQueryRecordReader.java |   93 ++
 .../druid/serde/DruidTopNQueryRecordReader.java |  106 ++
 .../hadoop/hive/druid/serde/DruidWritable.java  |   81 ++
 .../hadoop/hive/druid/QTestDruidSerDe.java      |   88 ++
 .../hive/druid/QTestDruidStorageHandler.java    |   34 +
 .../hadoop/hive/druid/TestDruidSerDe.java       |  576 ++++++++++
 .../TestHiveDruidQueryBasedInputFormat.java     |  101 ++
 itests/qtest/pom.xml                            |   13 +
 packaging/pom.xml                               |    5 +
 pom.xml                                         |    2 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    8 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   22 +-
 .../optimizer/calcite/HivePlannerContext.java   |   17 +-
 .../calcite/druid/DruidIntervalUtils.java       |  466 ++++++++
 .../ql/optimizer/calcite/druid/DruidQuery.java  | 1053 ++++++++++++++++++
 .../optimizer/calcite/druid/DruidQueryType.java |   42 +
 .../ql/optimizer/calcite/druid/DruidRules.java  |  591 ++++++++++
 .../ql/optimizer/calcite/druid/DruidSchema.java |   51 +
 .../ql/optimizer/calcite/druid/DruidTable.java  |  121 ++
 .../optimizer/calcite/druid/HiveDruidConf.java  |   33 +
 .../functions/HiveSqlCountAggFunction.java      |    2 +-
 .../functions/HiveSqlMinMaxAggFunction.java     |    2 +-
 .../functions/HiveSqlSumAggFunction.java        |    2 +-
 .../reloperators/HiveDateGranularity.java       |   54 +
 .../rules/HiveProjectSortTransposeRule.java     |    5 +
 .../rules/HiveSortProjectTransposeRule.java     |    5 +
 .../calcite/translator/ASTBuilder.java          |   38 +-
 .../calcite/translator/ASTConverter.java        |    9 +-
 .../translator/SqlFunctionConverter.java        |   23 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  119 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   22 +-
 .../hadoop/hive/ql/plan/CreateTableDesc.java    |    8 +-
 .../hadoop/hive/ql/plan/TableScanDesc.java      |    7 +
 .../apache/hadoop/hive/ql/udf/UDFDateFloor.java |  506 +++++++++
 .../hadoop/hive/ql/udf/UDFDateFloorDay.java     |   39 +
 .../hadoop/hive/ql/udf/UDFDateFloorHour.java    |   39 +
 .../hadoop/hive/ql/udf/UDFDateFloorMinute.java  |   39 +
 .../hadoop/hive/ql/udf/UDFDateFloorMonth.java   |   39 +
 .../hadoop/hive/ql/udf/UDFDateFloorQuarter.java |   39 +
 .../hadoop/hive/ql/udf/UDFDateFloorSecond.java  |   39 +
 .../hadoop/hive/ql/udf/UDFDateFloorWeek.java    |   39 +
 .../hadoop/hive/ql/udf/UDFDateFloorYear.java    |   39 +
 .../calcite/TestCBORuleFiredOnlyOnce.java       |    2 +-
 .../ql/udf/TestUDFDateFormatGranularity.java    |   85 ++
 .../test/queries/clientnegative/druid_address.q |    5 +
 .../test/queries/clientnegative/druid_buckets.q |    6 +
 .../queries/clientnegative/druid_datasource.q   |    3 +
 .../queries/clientnegative/druid_external.q     |    5 +
 .../queries/clientnegative/druid_location.q     |    6 +
 .../queries/clientnegative/druid_partitions.q   |    6 +
 .../test/queries/clientpositive/druid_basic1.q  |   18 +
 .../test/queries/clientpositive/druid_basic2.q  |   52 +
 .../queries/clientpositive/druid_intervals.q    |   67 ++
 .../queries/clientpositive/druid_timeseries.q   |   94 ++
 ql/src/test/queries/clientpositive/druid_topn.q |   75 ++
 .../results/clientnegative/druid_address.q.out  |    7 +
 .../results/clientnegative/druid_buckets.q.out  |    8 +
 .../clientnegative/druid_datasource.q.out       |    7 +
 .../results/clientnegative/druid_external.q.out |    7 +
 .../results/clientnegative/druid_location.q.out |    9 +
 .../clientnegative/druid_partitions.q.out       |    8 +
 .../results/clientpositive/create_view.q.out    |    2 +
 .../results/clientpositive/druid_basic1.q.out   |  142 +++
 .../results/clientpositive/druid_basic2.q.out   |  533 +++++++++
 .../clientpositive/druid_intervals.q.out        |  398 +++++++
 .../clientpositive/druid_timeseries.q.out       |  566 ++++++++++
 .../results/clientpositive/druid_topn.q.out     |  419 +++++++
 .../results/clientpositive/explain_ddl.q.out    |    2 +
 .../clientpositive/explain_logical.q.out        |   16 +
 .../test/results/clientpositive/join_view.q.out |    4 +
 .../clientpositive/llap/explainuser_1.q.out     |    2 +-
 .../test/results/clientpositive/masking_2.q.out |   14 +
 .../test/results/clientpositive/masking_6.q.out |    8 +
 .../test/results/clientpositive/masking_7.q.out |    8 +
 .../clientpositive/serde_user_properties.q.out  |    4 +
 .../results/clientpositive/show_functions.q.out |    9 +
 .../clientpositive/spark/join_view.q.out        |    4 +
 .../results/clientpositive/subquery_notin.q.out |    6 +
 .../results/clientpositive/subquery_views.q.out |    4 +
 92 files changed, 8969 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/common/src/java/org/apache/hadoop/hive/conf/Constants.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/Constants.java b/common/src/java/org/apache/hadoop/hive/conf/Constants.java
index 00ec8c0..77c6aa5 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/Constants.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/Constants.java
@@ -15,8 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.hadoop.hive.conf;
 
 public class Constants {
@@ -24,4 +22,12 @@ public class Constants {
   public static final String LLAP_LOGGER_NAME_QUERY_ROUTING = "query-routing";
   public static final String LLAP_LOGGER_NAME_CONSOLE = "console";
   public static final String LLAP_LOGGER_NAME_RFA = "RFA";
+
+  /* Constants for Druid storage handler */
+  public static final String DRUID_HIVE_STORAGE_HANDLER_ID =
+          "org.apache.hadoop.hive.druid.DruidStorageHandler";
+  public static final String DRUID_DATA_SOURCE = "druid.datasource";
+  public static final String DRUID_QUERY_JSON = "druid.query.json";
+  public static final String DRUID_QUERY_TYPE = "druid.query.type";
+  public static final String DRUID_QUERY_FETCH = "druid.query.fetch";
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/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 13cfdf1..d6944ee 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1884,6 +1884,17 @@ public class HiveConf extends Configuration {
     WRITE_SET_REAPER_INTERVAL("hive.writeset.reaper.interval", "60s",
       new TimeValidator(TimeUnit.MILLISECONDS), "Frequency of WriteSet reaper runs"),
 
+    // For Druid storage handler
+    HIVE_DRUID_BROKER_DEFAULT_ADDRESS("hive.druid.broker.address.default", "localhost:8082",
+        "Address of the Druid broker. If we are querying Druid from Hive, this address needs to be\n" +
+        "declared"),
+    HIVE_DRUID_SELECT_THRESHOLD("hive.druid.select.threshold", 10000,
+        "When we can split a Select query, this is the maximum number of rows that we try to retrieve\n" +
+        "per query. In order to do that, we obtain the estimated size for the complete result. If the\n" +
+        "number of records of the query results is larger than this threshold, we split the query in\n" +
+        "total number of rows/threshold parts across the time dimension. Note that we assume the\n" +
+        "records to be split uniformly across the time dimension"),
+
     // For HBase storage handler
     HIVE_HBASE_WAL_ENABLED("hive.hbase.wal.enabled", true,
         "Whether writes to HBase should be forced to the write-ahead log. \n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/pom.xml
----------------------------------------------------------------------
diff --git a/druid-handler/pom.xml b/druid-handler/pom.xml
new file mode 100644
index 0000000..2173cdc
--- /dev/null
+++ b/druid-handler/pom.xml
@@ -0,0 +1,201 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive</artifactId>
+    <version>2.2.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>hive-druid-handler</artifactId>
+  <packaging>jar</packaging>
+  <name>Hive Druid Handler</name>
+
+  <properties>
+    <hive.path.to.root>..</hive.path.to.root>
+  </properties>
+
+  <dependencies>
+    <!-- dependencies are always listed in sorted order by groupId, artifectId -->
+    <!-- intra-project -->
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-exec</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-all</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- inter-project -->
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+      <version>${commons-lang.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>${hadoop.version}</version>
+      <optional>true</optional>
+        <exclusions>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>io.druid</groupId>
+      <artifactId>druid-processing</artifactId>
+      <version>${druid.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-databind</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- test inter-project -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <sourceDirectory>${basedir}/src/java</sourceDirectory>
+    <testSourceDirectory>${basedir}/src/test</testSourceDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>${maven.shade.plugin.version}</version>
+        <executions>
+          <!-- we need to shade netty, as there is a conflict between versions
+          used by Hadoop (3.6.2.Final) and Druid (3.10.4.Final) -->
+          <!-- we need to shade jackson, as there is a conflict between versions
+          used by Hive (2.4.2) and Druid (2.4.6) -->
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <shadeTestJar>true</shadeTestJar>
+              <createDependencyReducedPom>false</createDependencyReducedPom>
+              <relocations>
+                <relocation>
+                  <pattern>io.druid</pattern>
+                  <shadedPattern>org.apache.hive.druid.io.druid</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>com.metamx.emitter</pattern>
+                  <shadedPattern>org.apache.hive.druid.com.metamx.emitter</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>com.metamx.http.client</pattern>
+                  <shadedPattern>org.apache.hive.druid.com.metamx.http.client</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>io.netty</pattern>
+                  <shadedPattern>org.apache.hive.druid.io.netty</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.jboss.netty</pattern>
+                  <shadedPattern>org.apache.hive.druid.org.jboss.netty</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>com.fasterxml.jackson</pattern>
+                  <shadedPattern>org.apache.hive.druid.com.fasterxml.jackson</shadedPattern>
+                </relocation>
+              </relocations>
+              <artifactSet>
+                <includes>
+                  <include>io.druid:*</include>
+                  <include>com.metamx:emitter:*</include>
+                  <include>com.metamx:http-client:*</include>
+                  <include>io.netty:*</include>
+                  <include>com.fasterxml.jackson.core:*</include>
+                  <include>com.fasterxml.jackson.datatype:*</include>
+                  <include>com.fasterxml.jackson.dataformat:*</include>
+                </includes>
+              </artifactSet>
+              <filters>
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>META-INF/*.SF</exclude>
+                    <exclude>META-INF/*.DSA</exclude>
+                    <exclude>META-INF/*.RSA</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
new file mode 100644
index 0000000..ac03099
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
@@ -0,0 +1,109 @@
+/**
+ * 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.druid;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.druid.serde.DruidSerDe;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.OutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DruidStorageHandler provides a HiveStorageHandler implementation for Druid.
+ */
+@SuppressWarnings({"deprecation","rawtypes"})
+public class DruidStorageHandler extends DefaultStorageHandler implements HiveMetaHook {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(DruidStorageHandler.class);
+
+  @Override
+  public Class<? extends InputFormat> getInputFormatClass() {
+    return HiveDruidQueryBasedInputFormat.class;
+  }
+
+  @Override
+  public Class<? extends OutputFormat> getOutputFormatClass() {
+    return HiveDruidOutputFormat.class;
+  }
+
+  @Override
+  public Class<? extends SerDe> getSerDeClass() {
+    return DruidSerDe.class;
+  }
+
+  @Override
+  public HiveMetaHook getMetaHook() {
+    return this;
+  }
+
+  @Override
+  public void preCreateTable(Table table) throws MetaException {
+    // Do safety checks
+    if (!MetaStoreUtils.isExternalTable(table)) {
+      throw new MetaException("Table in Druid needs to be declared as EXTERNAL");
+    }
+    if (!StringUtils.isEmpty(table.getSd().getLocation())) {
+      throw new MetaException("LOCATION may not be specified for Druid");
+    }
+    if (table.getPartitionKeysSize() != 0) {
+      throw new MetaException("PARTITIONED BY may not be specified for Druid");
+    }
+    if (table.getSd().getBucketColsSize() != 0) {
+      throw new MetaException("CLUSTERED BY may not be specified for Druid");
+    }
+  }
+
+  @Override
+  public void rollbackCreateTable(Table table) throws MetaException {
+    // Nothing to do
+  }
+
+  @Override
+  public void commitCreateTable(Table table) throws MetaException {
+    // Nothing to do
+  }
+
+  @Override
+  public void preDropTable(Table table) throws MetaException {
+    // Nothing to do
+  }
+
+  @Override
+  public void rollbackDropTable(Table table) throws MetaException {
+    // Nothing to do
+  }
+
+  @Override
+  public void commitDropTable(Table table, boolean deleteData) throws MetaException {
+    // Nothing to do
+  }
+
+  @Override
+  public String toString() {
+    return Constants.DRUID_HIVE_STORAGE_HANDLER_ID;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
new file mode 100644
index 0000000..c6b8024
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.druid;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.concurrent.ExecutionException;
+
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.smile.SmileFactory;
+import com.metamx.http.client.HttpClient;
+import com.metamx.http.client.Request;
+import com.metamx.http.client.response.InputStreamResponseHandler;
+
+import io.druid.jackson.DefaultObjectMapper;
+import io.druid.query.BaseQuery;
+
+/**
+ * Utils class for Druid storage handler.
+ */
+public final class DruidStorageHandlerUtils {
+
+  private static final String SMILE_CONTENT_TYPE = "application/x-jackson-smile";
+
+  /**
+   * Mapper to use to serialize/deserialize Druid objects (JSON)
+   */
+  public static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper();
+
+  /**
+   * Mapper to use to serialize/deserialize Druid objects (SMILE)
+   */
+  public static final ObjectMapper SMILE_MAPPER = new DefaultObjectMapper(new SmileFactory());
+
+  /**
+   * Method that creates a request for Druid JSON query (using SMILE).
+   * @param mapper
+   * @param address
+   * @param query
+   * @return
+   * @throws IOException
+   */
+  public static Request createRequest(String address, BaseQuery<?> query)
+          throws IOException {
+    return new Request(HttpMethod.POST, new URL(String.format("%s/druid/v2/", "http://" + address)))
+            .setContent(SMILE_MAPPER.writeValueAsBytes(query))
+            .setHeader(HttpHeaders.Names.CONTENT_TYPE, SMILE_CONTENT_TYPE);
+  }
+
+  /**
+   * Method that submits a request to an Http address and retrieves the result.
+   * The caller is responsible for closing the stream once it finishes consuming it.
+   * @param client
+   * @param request
+   * @return
+   * @throws IOException
+   */
+  public static InputStream submitRequest(HttpClient client, Request request)
+          throws IOException {
+    InputStream response;
+    try {
+      response = client.go(request, new InputStreamResponseHandler()).get();
+    } catch (ExecutionException e) {
+      throw new IOException(e.getCause());
+    } catch (InterruptedException e) {
+      throw new IOException(e.getCause());
+    }
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java
new file mode 100644
index 0000000..45e31d6
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java
@@ -0,0 +1,55 @@
+/**
+ * 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.druid;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Place holder for Druid output format. Currently not implemented.
+ */
+@SuppressWarnings("rawtypes")
+public class HiveDruidOutputFormat implements HiveOutputFormat {
+
+  @Override
+  public RecordWriter getRecordWriter(FileSystem ignored, JobConf job, String name,
+          Progressable progress) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jc, Path finalOutPath,
+          Class valueClass, boolean isCompressed, Properties tableProperties, Progressable progress)
+                  throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java
new file mode 100644
index 0000000..3df1452
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java
@@ -0,0 +1,369 @@
+/**
+ * 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.druid;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang3.StringEscapeUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.druid.serde.DruidGroupByQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidSelectQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidTimeseriesQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidTopNQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidWritable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidIntervalUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.metamx.common.lifecycle.Lifecycle;
+import com.metamx.http.client.HttpClient;
+import com.metamx.http.client.HttpClientConfig;
+import com.metamx.http.client.HttpClientInit;
+
+import io.druid.query.Druids;
+import io.druid.query.Druids.SegmentMetadataQueryBuilder;
+import io.druid.query.Druids.SelectQueryBuilder;
+import io.druid.query.Druids.TimeBoundaryQueryBuilder;
+import io.druid.query.Query;
+import io.druid.query.Result;
+import io.druid.query.metadata.metadata.SegmentAnalysis;
+import io.druid.query.metadata.metadata.SegmentMetadataQuery;
+import io.druid.query.select.PagingSpec;
+import io.druid.query.select.SelectQuery;
+import io.druid.query.spec.MultipleIntervalSegmentSpec;
+import io.druid.query.timeboundary.TimeBoundaryQuery;
+import io.druid.query.timeboundary.TimeBoundaryResultValue;
+
+/**
+ * Druid query based input format.
+ * 
+ * Given a query and the Druid broker address, it will send it, and retrieve
+ * and parse the results.
+ */
+public class HiveDruidQueryBasedInputFormat extends InputFormat<NullWritable, DruidWritable>
+        implements org.apache.hadoop.mapred.InputFormat<NullWritable, DruidWritable> {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(HiveDruidQueryBasedInputFormat.class);
+
+  @Override
+  public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf job, int numSplits)
+          throws IOException {
+    return getInputSplits(job);
+  }
+
+  @Override
+  public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException {
+    return Arrays.<InputSplit> asList(getInputSplits(context.getConfiguration()));
+  }
+
+  @SuppressWarnings("deprecation")
+  private HiveDruidSplit[] getInputSplits(Configuration conf) throws IOException {
+    String address = HiveConf.getVar(conf,
+            HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS);
+    if (StringUtils.isEmpty(address)) {
+      throw new IOException("Druid broker address not specified in configuration");
+    }
+    String druidQuery = StringEscapeUtils.unescapeJava(conf.get(Constants.DRUID_QUERY_JSON));
+    String druidQueryType;
+    if (StringUtils.isEmpty(druidQuery)) {
+      // Empty, maybe because CBO did not run; we fall back to
+      // full Select query
+      if (LOG.isWarnEnabled()) {
+        LOG.warn("Druid query is empty; creating Select query");
+      }
+      String dataSource = conf.get(Constants.DRUID_DATA_SOURCE);
+      if (dataSource == null) {
+        throw new IOException("Druid data source cannot be empty");
+      }
+      druidQuery = createSelectStarQuery(address, dataSource);
+      druidQueryType = Query.SELECT;
+    } else {
+      druidQueryType = conf.get(Constants.DRUID_QUERY_TYPE);
+      if (druidQueryType == null) {
+        throw new IOException("Druid query type not recognized");
+      }
+    }
+
+    // hive depends on FileSplits
+    Job job = new Job(conf);
+    JobContext jobContext = ShimLoader.getHadoopShims().newJobContext(job);
+    Path [] paths = FileInputFormat.getInputPaths(jobContext);
+
+    switch (druidQueryType) {
+      case Query.TIMESERIES:
+      case Query.TOPN:
+      case Query.GROUP_BY:
+        return new HiveDruidSplit[] { new HiveDruidSplit(address, druidQuery, paths[0]) };
+      case Query.SELECT:
+        return splitSelectQuery(conf, address, druidQuery, paths[0]);
+      default:
+        throw new IOException("Druid query type not recognized");
+    }
+  }
+
+  private static String createSelectStarQuery(String address, String dataSource) throws IOException {
+    // Create Select query
+    SelectQueryBuilder builder = new Druids.SelectQueryBuilder();
+    builder.dataSource(dataSource);
+    builder.intervals(Arrays.asList(DruidTable.DEFAULT_INTERVAL));
+    builder.pagingSpec(PagingSpec.newSpec(1));
+    Map<String, Object> context = new HashMap<>();
+    context.put(Constants.DRUID_QUERY_FETCH, false);
+    builder.context(context);
+    return DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(builder.build());
+  }
+
+  /* Method that splits Select query depending on the threshold so read can be
+   * parallelized */
+  private static HiveDruidSplit[] splitSelectQuery(Configuration conf, String address,
+          String druidQuery, Path dummyPath) throws IOException {
+    final int selectThreshold = (int) HiveConf.getIntVar(
+            conf, HiveConf.ConfVars.HIVE_DRUID_SELECT_THRESHOLD);
+
+    SelectQuery query;
+    try {
+      query = DruidStorageHandlerUtils.JSON_MAPPER.readValue(druidQuery, SelectQuery.class);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+
+    final boolean isFetch = query.getContextBoolean(Constants.DRUID_QUERY_FETCH, false);
+    if (isFetch) {
+      // If it has a limit, we use it and we do not split the query
+      return new HiveDruidSplit[] { new HiveDruidSplit(
+              address, DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath) };
+    }
+
+    // We do not have the number of rows, thus we need to execute a
+    // Segment Metadata query to obtain number of rows
+    SegmentMetadataQueryBuilder metadataBuilder = new Druids.SegmentMetadataQueryBuilder();
+    metadataBuilder.dataSource(query.getDataSource());
+    metadataBuilder.intervals(query.getIntervals());
+    metadataBuilder.merge(true);
+    metadataBuilder.analysisTypes();
+    SegmentMetadataQuery metadataQuery = metadataBuilder.build();
+
+    HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), new Lifecycle());
+    InputStream response;
+    try {
+      response = DruidStorageHandlerUtils.submitRequest(client,
+              DruidStorageHandlerUtils.createRequest(address, metadataQuery));
+    } catch (Exception e) {
+      throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+    }
+
+    // Retrieve results
+    List<SegmentAnalysis> metadataList;
+    try {
+      metadataList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response,
+            new TypeReference<List<SegmentAnalysis>>() {});
+    } catch (Exception e) {
+      response.close();
+      throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+    }
+    if (metadataList == null || metadataList.isEmpty()) {
+      throw new IOException("Connected to Druid but could not retrieve datasource information");
+    }
+    if (metadataList.size() != 1) {
+      throw new IOException("Information about segments should have been merged");
+    }
+
+    final long numRows = metadataList.get(0).getNumRows();
+
+    query = query.withPagingSpec(PagingSpec.newSpec(selectThreshold));
+    if (numRows <= selectThreshold) {
+      // We are not going to split it
+      return new HiveDruidSplit[] { new HiveDruidSplit(address,
+              DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath) };
+    }
+
+    // If the query does not specify a timestamp, we obtain the total time using
+    // a Time Boundary query. Then, we use the information to split the query
+    // following the Select threshold configuration property
+    final List<Interval> intervals = new ArrayList<>();
+    if (query.getIntervals().size() == 1 &&
+            query.getIntervals().get(0).equals(DruidTable.DEFAULT_INTERVAL)) {
+      // Default max and min, we should execute a time boundary query to get a
+      // more precise range
+      TimeBoundaryQueryBuilder timeBuilder = new Druids.TimeBoundaryQueryBuilder();
+      timeBuilder.dataSource(query.getDataSource());
+      TimeBoundaryQuery timeQuery = timeBuilder.build();
+
+      try {
+        response = DruidStorageHandlerUtils.submitRequest(client,
+                DruidStorageHandlerUtils.createRequest(address, timeQuery));
+      } catch (Exception e) {
+        throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+      }
+
+      // Retrieve results
+      List<Result<TimeBoundaryResultValue>> timeList;
+      try {
+        timeList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response,
+              new TypeReference<List<Result<TimeBoundaryResultValue>>>() {});
+      } catch (Exception e) {
+        response.close();
+        throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+      }
+      if (timeList == null || timeList.isEmpty()) {
+        throw new IOException("Connected to Druid but could not retrieve time boundary information");
+      }
+      if (timeList.size() != 1) {
+        throw new IOException("We should obtain a single time boundary");
+      }
+
+      intervals.add(new Interval(timeList.get(0).getValue().getMinTime().getMillis(),
+              timeList.get(0).getValue().getMaxTime().getMillis()));
+    } else {
+      intervals.addAll(query.getIntervals());
+    }
+
+    // Create (numRows/default threshold) input splits
+    int numSplits = (int) Math.ceil((double) numRows / selectThreshold);
+    List<List<Interval>> newIntervals = createSplitsIntervals(intervals, numSplits);
+    HiveDruidSplit[] splits = new HiveDruidSplit[numSplits];
+    for (int i = 0; i < numSplits; i++) {
+      // Create partial Select query
+      final SelectQuery partialQuery = query.withQuerySegmentSpec(
+              new MultipleIntervalSegmentSpec(newIntervals.get(i)));
+      splits[i] = new HiveDruidSplit(address,
+              DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery), dummyPath);
+    }
+    return splits;
+  }
+
+  private static List<List<Interval>> createSplitsIntervals(List<Interval> intervals, int numSplits) {
+    final long totalTime = DruidIntervalUtils.extractTotalTime(intervals);
+    long startTime = intervals.get(0).getStartMillis();
+    long endTime = startTime;
+    long currTime = 0;
+    List<List<Interval>> newIntervals = new ArrayList<>();
+    for (int i = 0, posIntervals = 0; i < numSplits; i++) {
+      final long rangeSize = Math.round( (double) (totalTime * (i + 1)) / numSplits) -
+              Math.round( (double) (totalTime * i) / numSplits);
+      // Create the new interval(s)
+      List<Interval> currentIntervals = new ArrayList<>();
+      while (posIntervals < intervals.size()) {
+        final Interval interval = intervals.get(posIntervals);
+        final long expectedRange = rangeSize - currTime;
+        if (interval.getEndMillis() - startTime >= expectedRange) {
+          endTime = startTime + expectedRange;
+          currentIntervals.add(new Interval(startTime, endTime));
+          startTime = endTime;
+          currTime = 0;
+          break;
+        }
+        endTime = interval.getEndMillis();
+        currentIntervals.add(new Interval(startTime, endTime));
+        currTime += (endTime - startTime);
+        startTime = intervals.get(++posIntervals).getStartMillis();
+      }
+      newIntervals.add(currentIntervals);
+    }
+    assert endTime == intervals.get(intervals.size()-1).getEndMillis();
+    return newIntervals;
+  }
+
+  @Override
+  public org.apache.hadoop.mapred.RecordReader<NullWritable, DruidWritable> getRecordReader(
+          org.apache.hadoop.mapred.InputSplit split, JobConf job, Reporter reporter)
+                  throws IOException {
+    // We need to provide a different record reader for every type of Druid query.
+    // The reason is that Druid results format is different for each type.
+    final DruidQueryRecordReader<?,?> reader;
+    final String druidQueryType = job.get(Constants.DRUID_QUERY_TYPE);
+    if (druidQueryType == null) {
+      reader = new DruidSelectQueryRecordReader(); // By default
+      reader.initialize((HiveDruidSplit)split, job);
+      return reader;
+    }
+    switch (druidQueryType) {
+      case Query.TIMESERIES:
+        reader = new DruidTimeseriesQueryRecordReader();
+        break;
+      case Query.TOPN:
+        reader = new DruidTopNQueryRecordReader();
+        break;
+      case Query.GROUP_BY:
+        reader = new DruidGroupByQueryRecordReader();
+        break;
+      case Query.SELECT:
+        reader = new DruidSelectQueryRecordReader();
+        break;
+      default:
+        throw new IOException("Druid query type not recognized");
+    }
+    reader.initialize((HiveDruidSplit)split, job);
+    return reader;
+  }
+
+  @Override
+  public RecordReader<NullWritable, DruidWritable> createRecordReader(InputSplit split,
+          TaskAttemptContext context) throws IOException, InterruptedException {
+    // We need to provide a different record reader for every type of Druid query.
+    // The reason is that Druid results format is different for each type.
+    final String druidQueryType = context.getConfiguration().get(Constants.DRUID_QUERY_TYPE);
+    if (druidQueryType == null) {
+      return new DruidSelectQueryRecordReader(); // By default
+    }
+    final DruidQueryRecordReader<?,?> reader;
+    switch (druidQueryType) {
+      case Query.TIMESERIES:
+        reader = new DruidTimeseriesQueryRecordReader();
+        break;
+      case Query.TOPN:
+        reader = new DruidTopNQueryRecordReader();
+        break;
+      case Query.GROUP_BY:
+        reader = new DruidGroupByQueryRecordReader();
+        break;
+      case Query.SELECT:
+        reader = new DruidSelectQueryRecordReader();
+        break;
+      default:
+        throw new IOException("Druid query type not recognized");
+    }
+    return reader;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java
new file mode 100644
index 0000000..3fba5d0
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java
@@ -0,0 +1,83 @@
+/**
+ * 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.druid;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+
+/**
+ * Druid split. Its purpose is to trigger query execution in Druid.
+ */
+public class HiveDruidSplit extends FileSplit implements org.apache.hadoop.mapred.InputSplit {
+
+  private String address;
+  private String druidQuery;
+
+  // required for deserialization
+  public HiveDruidSplit() {
+    super((Path) null, 0, 0, (String[]) null);
+  }
+
+  public HiveDruidSplit(String address, String druidQuery, Path dummyPath) {
+    super(dummyPath, 0, 0, (String[]) null);
+    this.address = address;
+    this.druidQuery = druidQuery;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    out.writeUTF(address);
+    out.writeUTF(druidQuery);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    address = in.readUTF();
+    druidQuery = in.readUTF();
+  }
+
+  @Override
+  public long getLength() {
+    return 0L;
+  }
+
+  @Override
+  public String[] getLocations() {
+    return new String[] {""} ;
+  }
+
+  public String getAddress() {
+    return address;
+  }
+
+  public String getDruidQuery() {
+    return druidQuery;
+  }
+
+  @Override
+  public String toString() {
+    return "HiveDruidSplit{" + address + ", " + druidQuery + "}";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
new file mode 100644
index 0000000..226060f
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
@@ -0,0 +1,199 @@
+/**
+ * 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.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+
+import io.druid.data.input.Row;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.aggregation.PostAggregator;
+import io.druid.query.dimension.DimensionSpec;
+import io.druid.query.groupby.GroupByQuery;
+
+/**
+ * Record reader for results for Druid GroupByQuery.
+ */
+public class DruidGroupByQueryRecordReader
+        extends DruidQueryRecordReader<GroupByQuery, Row> {
+
+  private Row current;
+  private int[] indexes = new int[0];
+  // Row objects returned by GroupByQuery have different access paths depending on
+  // whether the result for the metric is a Float or a Long, thus we keep track
+  // using these converters
+  private Extract[] extractors;
+
+  @Override
+  public void initialize(InputSplit split, Configuration conf) throws IOException {
+    super.initialize(split, conf);
+    initExtractors();
+  }
+
+  @Override
+  protected GroupByQuery createQuery(String content) throws IOException {
+    return DruidStorageHandlerUtils.JSON_MAPPER.readValue(content, GroupByQuery.class);
+  }
+
+  @Override
+  protected List<Row> createResultsList(InputStream content) throws IOException {
+    return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content,
+            new TypeReference<List<Row>>(){});
+  }
+
+  private void initExtractors() throws IOException {
+    extractors = new Extract[query.getAggregatorSpecs().size() + query.getPostAggregatorSpecs().size()];
+    int counter = 0;
+    for (int i = 0; i < query.getAggregatorSpecs().size(); i++, counter++) {
+      AggregatorFactory af = query.getAggregatorSpecs().get(i);
+      switch (af.getTypeName().toUpperCase()) {
+        case DruidSerDeUtils.FLOAT_TYPE:
+          extractors[counter] = Extract.FLOAT;
+          break;
+        case DruidSerDeUtils.LONG_TYPE:
+          extractors[counter] = Extract.LONG;
+          break;
+        default:
+          throw new IOException("Type not supported");
+      }
+    }
+    for (int i = 0; i < query.getPostAggregatorSpecs().size(); i++, counter++) {
+      extractors[counter] = Extract.FLOAT;
+    }
+  }
+
+  @Override
+  public boolean nextKeyValue() {
+    // Refresh indexes
+    for (int i = indexes.length - 1; i >= 0; i--) {
+      if (indexes[i] > 0) {
+        indexes[i]--;
+        for (int j = i + 1; j < indexes.length; j++) {
+          indexes[j] = current.getDimension(
+                  query.getDimensions().get(j).getDimension()).size() - 1;
+        }
+        return true;
+      }
+    }
+    // Results
+    if (results.hasNext()) {
+      current = results.next();
+      indexes = new int[query.getDimensions().size()];
+      for (int i=0; i < query.getDimensions().size(); i++) {
+        DimensionSpec ds = query.getDimensions().get(i);
+        indexes[i] = current.getDimension(ds.getDimension()).size() - 1;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public NullWritable getCurrentKey() throws IOException, InterruptedException {
+    return NullWritable.get();
+  }
+
+  @Override
+  public DruidWritable getCurrentValue() throws IOException, InterruptedException {
+    // Create new value
+    DruidWritable value = new DruidWritable();
+    // 1) The timestamp column
+    value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+    // 2) The dimension columns
+    for (int i=0; i < query.getDimensions().size(); i++) {
+      DimensionSpec ds = query.getDimensions().get(i);
+      List<String> dims = current.getDimension(ds.getDimension());
+      int pos = dims.size() - indexes[i] - 1;
+      value.getValue().put(ds.getOutputName(), dims.get(pos));
+    }
+    int counter = 0;
+    // 3) The aggregation columns
+    for (AggregatorFactory af : query.getAggregatorSpecs()) {
+      switch (extractors[counter++]) {
+        case FLOAT:
+          value.getValue().put(af.getName(), current.getFloatMetric(af.getName()));
+          break;
+        case LONG:
+          value.getValue().put(af.getName(), current.getLongMetric(af.getName()));
+          break;
+      }
+    }
+    // 4) The post-aggregation columns
+    for (PostAggregator pa : query.getPostAggregatorSpecs()) {
+      assert extractors[counter++] == Extract.FLOAT;
+      value.getValue().put(pa.getName(), current.getFloatMetric(pa.getName()));
+    }
+    return value;
+  }
+
+  @Override
+  public boolean next(NullWritable key, DruidWritable value) {
+    if (nextKeyValue()) {
+      // Update value
+      value.getValue().clear();
+      // 1) The timestamp column
+      value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+      // 2) The dimension columns
+      for (int i=0; i < query.getDimensions().size(); i++) {
+        DimensionSpec ds = query.getDimensions().get(i);
+        List<String> dims = current.getDimension(ds.getDimension());
+        int pos = dims.size() - indexes[i] - 1;
+        value.getValue().put(ds.getOutputName(), dims.get(pos));
+      }
+      int counter = 0;
+      // 3) The aggregation columns
+      for (AggregatorFactory af : query.getAggregatorSpecs()) {
+        switch (extractors[counter++]) {
+          case FLOAT:
+            value.getValue().put(af.getName(), current.getFloatMetric(af.getName()));
+            break;
+          case LONG:
+            value.getValue().put(af.getName(), current.getLongMetric(af.getName()));
+            break;
+        }
+      }
+      // 4) The post-aggregation columns
+      for (PostAggregator pa : query.getPostAggregatorSpecs()) {
+        assert extractors[counter++] == Extract.FLOAT;
+        value.getValue().put(pa.getName(), current.getFloatMetric(pa.getName()));
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public float getProgress() throws IOException {
+    return results.hasNext() ? 0 : 1;
+  }
+
+  private enum Extract {
+    FLOAT,
+    LONG
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
new file mode 100644
index 0000000..96bcee8
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.druid.HiveDruidSplit;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Iterators;
+import com.metamx.common.lifecycle.Lifecycle;
+import com.metamx.http.client.HttpClient;
+import com.metamx.http.client.HttpClientConfig;
+import com.metamx.http.client.HttpClientInit;
+
+import io.druid.query.BaseQuery;
+
+/**
+ * Base record reader for given a Druid query. This class contains the logic to
+ * send the query to the broker and retrieve the results. The transformation to
+ * emit records needs to be done by the classes that extend the reader.
+ * 
+ * The key for each record will be a NullWritable, while the value will be a
+ * DruidWritable containing the timestamp as well as all values resulting from
+ * the query.
+ */
+public abstract class DruidQueryRecordReader<T extends BaseQuery<R>,R extends Comparable<R>>
+        extends RecordReader<NullWritable, DruidWritable>
+        implements org.apache.hadoop.mapred.RecordReader<NullWritable, DruidWritable> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DruidQueryRecordReader.class);
+
+  /**
+   * Query that Druid executes.
+   */
+  protected T query;
+
+  /**
+   * Query results.
+   */
+  protected Iterator<R> results = Iterators.emptyIterator();
+
+  @Override
+  public void initialize(InputSplit split, TaskAttemptContext context) throws IOException {
+    initialize(split, context.getConfiguration());
+  }
+
+  public void initialize(InputSplit split, Configuration conf) throws IOException {
+    HiveDruidSplit hiveDruidSplit = (HiveDruidSplit) split;
+
+    // Create query
+    query = createQuery(hiveDruidSplit.getDruidQuery());
+
+    // Execute query
+    if (LOG.isInfoEnabled()) {
+      LOG.info("Retrieving from druid using query:\n " + query);
+    }
+
+    HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), new Lifecycle());
+    InputStream response = DruidStorageHandlerUtils.submitRequest(client,
+            DruidStorageHandlerUtils.createRequest(hiveDruidSplit.getAddress(), query));
+
+    // Retrieve results
+    List<R> resultsList;
+    try {
+      resultsList = createResultsList(response);
+    } catch (IOException e) {
+      response.close();
+      throw e;
+    }
+    if (resultsList == null || resultsList.isEmpty()) {
+      return;
+    }
+    results = resultsList.iterator();
+  }
+
+  protected abstract T createQuery(String content) throws IOException;
+
+  protected abstract List<R> createResultsList(InputStream content) throws IOException;
+
+  @Override
+  public NullWritable createKey() {
+    return NullWritable.get();
+  }
+
+  @Override
+  public DruidWritable createValue() {
+    return new DruidWritable();
+  }
+
+  @Override
+  public abstract boolean next(NullWritable key, DruidWritable value) throws IOException;
+
+  @Override
+  public long getPos() {
+    return 0;
+  }
+
+  @Override
+  public abstract boolean nextKeyValue() throws IOException;
+
+  @Override
+  public abstract NullWritable getCurrentKey() throws IOException, InterruptedException;
+
+  @Override
+  // TODO: we could generate vector row batches so that vectorized execution may get triggered
+  public abstract DruidWritable getCurrentValue() throws IOException, InterruptedException;
+
+  @Override
+  public abstract float getProgress() throws IOException;
+
+  @Override
+  public void close() {
+    // Nothing to do
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
new file mode 100644
index 0000000..70b493c
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
@@ -0,0 +1,106 @@
+/**
+ * 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.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.io.NullWritable;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.Iterators;
+
+import io.druid.query.Result;
+import io.druid.query.select.EventHolder;
+import io.druid.query.select.SelectQuery;
+import io.druid.query.select.SelectResultValue;
+
+/**
+ * Record reader for results for Druid SelectQuery.
+ */
+public class DruidSelectQueryRecordReader
+        extends DruidQueryRecordReader<SelectQuery, Result<SelectResultValue>> {
+
+  private Result<SelectResultValue> current;
+  private Iterator<EventHolder> values = Iterators.emptyIterator();
+
+  @Override
+  protected SelectQuery createQuery(String content) throws IOException {
+    return DruidStorageHandlerUtils.JSON_MAPPER.readValue(content, SelectQuery.class);
+  }
+
+  @Override
+  protected List<Result<SelectResultValue>> createResultsList(InputStream content) throws IOException {
+    return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content,
+            new TypeReference<List<Result<SelectResultValue>>>(){});
+  }
+
+  @Override
+  public boolean nextKeyValue() throws IOException {
+    if (values.hasNext()) {
+      return true;
+    }
+    if (results.hasNext()) {
+      current = results.next();
+      values = current.getValue().getEvents().iterator();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public NullWritable getCurrentKey() throws IOException, InterruptedException {
+    return NullWritable.get();
+  }
+
+  @Override
+  public DruidWritable getCurrentValue() throws IOException, InterruptedException {
+    // Create new value
+    DruidWritable value = new DruidWritable();
+    value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+    if (values.hasNext()) {
+      value.getValue().putAll(values.next().getEvent());
+      return value;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean next(NullWritable key, DruidWritable value) throws IOException {
+    if (nextKeyValue()) {
+      // Update value
+      value.getValue().clear();
+      value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+      if (values.hasNext()) {
+        value.getValue().putAll(values.next().getEvent());
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public float getProgress() {
+    return results.hasNext() || values.hasNext() ? 0 : 1;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
new file mode 100644
index 0000000..8f53d4a
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
@@ -0,0 +1,343 @@
+/**
+ * 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.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeSpec;
+import org.apache.hadoop.hive.serde2.SerDeStats;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.Lists;
+import com.metamx.common.lifecycle.Lifecycle;
+import com.metamx.http.client.HttpClient;
+import com.metamx.http.client.HttpClientConfig;
+import com.metamx.http.client.HttpClientInit;
+
+import io.druid.query.Druids;
+import io.druid.query.Druids.SegmentMetadataQueryBuilder;
+import io.druid.query.Query;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.aggregation.PostAggregator;
+import io.druid.query.dimension.DimensionSpec;
+import io.druid.query.groupby.GroupByQuery;
+import io.druid.query.metadata.metadata.ColumnAnalysis;
+import io.druid.query.metadata.metadata.SegmentAnalysis;
+import io.druid.query.metadata.metadata.SegmentMetadataQuery;
+import io.druid.query.select.SelectQuery;
+import io.druid.query.timeseries.TimeseriesQuery;
+import io.druid.query.topn.TopNQuery;
+
+/**
+ * DruidSerDe that is used to  deserialize objects from a Druid data source.
+ */
+@SerDeSpec(schemaProps = {Constants.DRUID_DATA_SOURCE})
+public class DruidSerDe extends AbstractSerDe {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(DruidSerDe.class);
+
+  private String[] columns;
+  private PrimitiveTypeInfo[] types;
+  private ObjectInspector inspector;
+
+
+  @Override
+  public void initialize(Configuration configuration, Properties properties) throws SerDeException {
+    final List<String> columnNames = new ArrayList<>();
+    final List<PrimitiveTypeInfo> columnTypes = new ArrayList<>();
+    List<ObjectInspector> inspectors = new ArrayList<>();
+
+    // Druid query
+    String druidQuery = properties.getProperty(Constants.DRUID_QUERY_JSON);
+    if (druidQuery == null) {
+      // No query. We need to create a Druid Segment Metadata query that retrieves all
+      // columns present in the data source (dimensions and metrics).
+      // Create Segment Metadata Query
+      String dataSource = properties.getProperty(Constants.DRUID_DATA_SOURCE);
+      if (dataSource == null) {
+        throw new SerDeException("Druid data source not specified; use " +
+                Constants.DRUID_DATA_SOURCE + " in table properties");
+      }
+      SegmentMetadataQueryBuilder builder = new Druids.SegmentMetadataQueryBuilder();
+      builder.dataSource(dataSource);
+      builder.merge(true);
+      builder.analysisTypes();
+      SegmentMetadataQuery query = builder.build();
+
+      // Execute query in Druid
+      String address = HiveConf.getVar(configuration,
+              HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS);
+      if (org.apache.commons.lang3.StringUtils.isEmpty(address)) {
+        throw new SerDeException("Druid broker address not specified in configuration");
+      }
+
+      // Infer schema
+      SegmentAnalysis schemaInfo;
+      try {
+        schemaInfo = submitMetadataRequest(address, query);
+      } catch (IOException e) {
+        throw new SerDeException(e);
+      }
+      for (Entry<String,ColumnAnalysis> columnInfo : schemaInfo.getColumns().entrySet()) {
+        if (columnInfo.getKey().equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
+          // Special handling for timestamp column
+          columnNames.add(columnInfo.getKey()); // field name
+          PrimitiveTypeInfo type = TypeInfoFactory.timestampTypeInfo; // field type
+          columnTypes.add(type);
+          inspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(type));
+          continue;
+        }
+        columnNames.add(columnInfo.getKey()); // field name
+        PrimitiveTypeInfo type = DruidSerDeUtils.convertDruidToHiveType(
+                columnInfo.getValue().getType()); // field type
+        columnTypes.add(type);
+        inspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(type));
+      }
+      columns = columnNames.toArray(new String[columnNames.size()]);
+      types = columnTypes.toArray(new PrimitiveTypeInfo[columnTypes.size()]);
+      inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors);
+    } else {
+      // Query is specified, we can extract the results schema from the query
+      Query<?> query;
+      try {
+        query = DruidStorageHandlerUtils.JSON_MAPPER.readValue(druidQuery, Query.class);
+      } catch (Exception e) {
+        throw new SerDeException(e);
+      }
+
+      switch (query.getType()) {
+        case Query.TIMESERIES:
+          inferSchema((TimeseriesQuery) query, columnNames, columnTypes);
+          break;
+        case Query.TOPN:
+          inferSchema((TopNQuery) query, columnNames, columnTypes);
+          break;
+        case Query.SELECT:
+          inferSchema((SelectQuery) query, columnNames, columnTypes);
+          break;
+        case Query.GROUP_BY:
+          inferSchema((GroupByQuery) query, columnNames, columnTypes);
+          break;
+        default:
+          throw new SerDeException("Not supported Druid query");
+      }
+    
+      columns = new String[columnNames.size()];
+      types = new PrimitiveTypeInfo[columnNames.size()];
+      for (int i = 0; i < columnTypes.size(); ++i) {
+        columns[i] = columnNames.get(i);
+        types[i] = columnTypes.get(i);
+        inspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(types[i]));
+      }
+      inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("DruidSerDe initialized with\n"
+              + "\t columns: " + columnNames
+              + "\n\t types: " + columnTypes);
+    }
+  }
+
+  /* Submits the request and returns */
+  protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQuery query)
+          throws SerDeException, IOException {
+    HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), new Lifecycle());
+    InputStream response;
+    try {
+      response = DruidStorageHandlerUtils.submitRequest(client,
+              DruidStorageHandlerUtils.createRequest(address, query));
+    } catch (Exception e) {
+      throw new SerDeException(StringUtils.stringifyException(e));
+    }
+
+    // Retrieve results
+    List<SegmentAnalysis> resultsList;
+    try {
+      resultsList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response,
+              new TypeReference<List<SegmentAnalysis>>() {});
+    } catch (Exception e) {
+      response.close();
+      throw new SerDeException(StringUtils.stringifyException(e));
+    }
+    if (resultsList == null || resultsList.isEmpty()) {
+      throw new SerDeException("Connected to Druid but could not retrieve datasource information");
+    }
+    if (resultsList.size() != 1) {
+      throw new SerDeException("Information about segments should have been merged");
+    }
+
+    return resultsList.get(0);
+  }
+
+  /* Timeseries query */
+  private void inferSchema(TimeseriesQuery query, List<String> columnNames,
+          List<PrimitiveTypeInfo> columnTypes) {
+    // Timestamp column
+    columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+    columnTypes.add(TypeInfoFactory.timestampTypeInfo);
+    // Aggregator columns
+    for (AggregatorFactory af : query.getAggregatorSpecs()) {
+      columnNames.add(af.getName());
+      columnTypes.add(DruidSerDeUtils.convertDruidToHiveType(af.getTypeName()));
+    }
+    // Post-aggregator columns
+    for (PostAggregator pa : query.getPostAggregatorSpecs()) {
+      columnNames.add(pa.getName());
+      columnTypes.add(TypeInfoFactory.floatTypeInfo);
+    }
+  }
+
+  /* TopN query */
+  private void inferSchema(TopNQuery query, List<String> columnNames, List<PrimitiveTypeInfo> columnTypes) {
+    // Timestamp column
+    columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+    columnTypes.add(TypeInfoFactory.timestampTypeInfo);
+    // Dimension column
+    columnNames.add(query.getDimensionSpec().getOutputName());
+    columnTypes.add(TypeInfoFactory.stringTypeInfo);
+    // Aggregator columns
+    for (AggregatorFactory af : query.getAggregatorSpecs()) {
+      columnNames.add(af.getName());
+      columnTypes.add(DruidSerDeUtils.convertDruidToHiveType(af.getTypeName()));
+    }
+    // Post-aggregator columns
+    for (PostAggregator pa : query.getPostAggregatorSpecs()) {
+      columnNames.add(pa.getName());
+      columnTypes.add(TypeInfoFactory.floatTypeInfo);
+    }
+  }
+
+  /* Select query */
+  private void inferSchema(SelectQuery query, List<String> columnNames,
+          List<PrimitiveTypeInfo> columnTypes) {
+    // Timestamp column
+    columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+    columnTypes.add(TypeInfoFactory.timestampTypeInfo);
+    // Dimension columns
+    for (DimensionSpec ds : query.getDimensions()) {
+      columnNames.add(ds.getOutputName());
+      columnTypes.add(TypeInfoFactory.stringTypeInfo);
+    }
+    // Metric columns
+    for (String metric : query.getMetrics()) {
+      columnNames.add(metric);
+      columnTypes.add(TypeInfoFactory.floatTypeInfo);
+    }
+  }
+
+  /* GroupBy query */
+  private void inferSchema(GroupByQuery query, List<String> columnNames, List<PrimitiveTypeInfo> columnTypes) {
+    // Timestamp column
+    columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+    columnTypes.add(TypeInfoFactory.timestampTypeInfo);
+    // Dimension columns
+    for (DimensionSpec ds : query.getDimensions()) {
+      columnNames.add(ds.getOutputName());
+      columnTypes.add(TypeInfoFactory.stringTypeInfo);
+    }
+    // Aggregator columns
+    for (AggregatorFactory af : query.getAggregatorSpecs()) {
+      columnNames.add(af.getName());
+      columnTypes.add(DruidSerDeUtils.convertDruidToHiveType(af.getTypeName()));
+    }
+    // Post-aggregator columns
+    for (PostAggregator pa : query.getPostAggregatorSpecs()) {
+      columnNames.add(pa.getName());
+      columnTypes.add(TypeInfoFactory.floatTypeInfo);
+    }
+  }
+
+  @Override
+  public Class<? extends Writable> getSerializedClass() {
+    return NullWritable.class;
+  }
+
+  @Override
+  public Writable serialize(Object o, ObjectInspector objectInspector) throws SerDeException {
+    return NullWritable.get();
+  }
+
+  @Override
+  public SerDeStats getSerDeStats() {
+    throw new UnsupportedOperationException("SerdeStats not supported.");
+  }
+
+  @Override
+  public Object deserialize(Writable writable) throws SerDeException {
+    DruidWritable input = (DruidWritable) writable;
+    List<Object> output = Lists.newArrayListWithExpectedSize(columns.length);
+    for (int i = 0; i < columns.length; i++) {
+      final Object value = input.getValue().get(columns[i]);
+      if (value == null) {
+        output.add(null);
+        continue;
+      }
+      switch (types[i].getPrimitiveCategory()) {
+        case TIMESTAMP:
+          output.add(new TimestampWritable(new Timestamp((Long)value)));
+          break;
+        case LONG:
+          output.add(new LongWritable(((Number)value).longValue()));
+          break;
+        case FLOAT:
+          output.add(new FloatWritable(((Number)value).floatValue()));
+          break;
+        case STRING:
+          output.add(new Text(value.toString()));
+          break;
+        default:
+          throw new SerDeException("Unknown type: " + types[i].getPrimitiveCategory());
+      }
+    }
+    return output;
+  }
+
+  @Override
+  public ObjectInspector getObjectInspector() throws SerDeException {
+    return inspector;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java
new file mode 100644
index 0000000..29b8845
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java
@@ -0,0 +1,83 @@
+/**
+ * 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.druid.serde;
+
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utils class for Druid SerDe.
+ */
+public final class DruidSerDeUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DruidSerDeUtils.class);
+
+  protected static final String FLOAT_TYPE = "FLOAT";
+  protected static final String LONG_TYPE = "LONG";
+  protected static final String STRING_TYPE = "STRING";
+
+  /* This method converts from the String representation of Druid type
+   * to the corresponding Hive type */
+  public static PrimitiveTypeInfo convertDruidToHiveType(String typeName) {
+    typeName = typeName.toUpperCase();
+    switch(typeName) {
+      case FLOAT_TYPE:
+        return TypeInfoFactory.floatTypeInfo;
+      case LONG_TYPE:
+        return TypeInfoFactory.longTypeInfo;
+      case STRING_TYPE:
+        return TypeInfoFactory.stringTypeInfo;
+      default:
+        // This is a guard for special Druid types e.g. hyperUnique
+        // (http://druid.io/docs/0.9.1.1/querying/aggregations.html#hyperunique-aggregator).
+        // Currently, we do not support doing anything special with them in Hive.
+        // However, those columns are there, and they can be actually read as normal
+        // dimensions e.g. with a select query. Thus, we print the warning and just read them
+        // as String.
+        LOG.warn("Transformation to STRING for unknown type " + typeName);
+        return TypeInfoFactory.stringTypeInfo;
+    }
+  }
+
+  /* This method converts from the String representation of Druid type
+   * to the String representation of the corresponding Hive type */
+  public static String convertDruidToHiveTypeString(String typeName) {
+    typeName = typeName.toUpperCase();
+    switch(typeName) {
+      case FLOAT_TYPE:
+        return serdeConstants.FLOAT_TYPE_NAME;
+      case LONG_TYPE:
+        return serdeConstants.BIGINT_TYPE_NAME;
+      case STRING_TYPE:
+        return serdeConstants.STRING_TYPE_NAME;
+      default:
+        // This is a guard for special Druid types e.g. hyperUnique
+        // (http://druid.io/docs/0.9.1.1/querying/aggregations.html#hyperunique-aggregator).
+        // Currently, we do not support doing anything special with them in Hive.
+        // However, those columns are there, and they can be actually read as normal
+        // dimensions e.g. with a select query. Thus, we print the warning and just read them
+        // as String.
+        LOG.warn("Transformation to STRING for unknown type " + typeName);
+        return serdeConstants.STRING_TYPE_NAME;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
new file mode 100644
index 0000000..812ae03
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
@@ -0,0 +1,93 @@
+/**
+ * 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.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.io.NullWritable;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+
+import io.druid.query.Result;
+import io.druid.query.timeseries.TimeseriesQuery;
+import io.druid.query.timeseries.TimeseriesResultValue;
+
+/**
+ * Record reader for results for Druid TimeseriesQuery.
+ */
+public class DruidTimeseriesQueryRecordReader
+        extends DruidQueryRecordReader<TimeseriesQuery, Result<TimeseriesResultValue>> {
+
+  private Result<TimeseriesResultValue> current;
+
+  @Override
+  protected TimeseriesQuery createQuery(String content) throws IOException {
+    return DruidStorageHandlerUtils.JSON_MAPPER.readValue(content, TimeseriesQuery.class);
+  }
+
+  @Override
+  protected List<Result<TimeseriesResultValue>> createResultsList(InputStream content) throws IOException {
+    return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content,
+            new TypeReference<List<Result<TimeseriesResultValue>>>(){});
+  }
+
+  @Override
+  public boolean nextKeyValue() {
+    if (results.hasNext()) {
+      current = results.next();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public NullWritable getCurrentKey() throws IOException, InterruptedException {
+    return NullWritable.get();
+  }
+
+  @Override
+  public DruidWritable getCurrentValue() throws IOException, InterruptedException {
+    // Create new value
+    DruidWritable value = new DruidWritable();
+    value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+    value.getValue().putAll(current.getValue().getBaseObject());
+    return value;
+  }
+
+  @Override
+  public boolean next(NullWritable key, DruidWritable value) {
+    if (nextKeyValue()) {
+      // Update value
+      value.getValue().clear();
+      value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+      value.getValue().putAll(current.getValue().getBaseObject());
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public float getProgress() throws IOException {
+    return results.hasNext() ? 0 : 1;
+  }
+
+}


[23/31] hive git commit: HIVE-14644 : use metastore information on the read path appropriately (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java
index 1da8e91..0a611f9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java
@@ -147,8 +147,7 @@ public class TableDesc implements Serializable, Cloneable {
 
   @Explain(displayName = "name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
   public String getTableName() {
-    return properties
-        .getProperty(hive_metastoreConstants.META_TABLE_NAME);
+    return properties.getProperty(hive_metastoreConstants.META_TABLE_NAME);
   }
 
   @Explain(displayName = "input format")

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/test/queries/clientpositive/mm_current.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/mm_current.q b/ql/src/test/queries/clientpositive/mm_current.q
index 8d19df6..7c3e138 100644
--- a/ql/src/test/queries/clientpositive/mm_current.q
+++ b/ql/src/test/queries/clientpositive/mm_current.q
@@ -24,7 +24,8 @@ select * from part_mm;
 
 create table simple_mm(key int) stored as orc tblproperties ('hivecommit'='true');
 insert into table simple_mm select key from intermediate;
-insert into table simple_mm select key from intermediate;
+insert overwrite table simple_mm select key from intermediate;
+
 select * from simple_mm;
 
 drop table part_mm;

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/ql/src/test/results/clientpositive/llap/mm_current.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mm_current.q.out b/ql/src/test/results/clientpositive/llap/mm_current.q.out
index f357020..ece6cbf 100644
--- a/ql/src/test/results/clientpositive/llap/mm_current.q.out
+++ b/ql/src/test/results/clientpositive/llap/mm_current.q.out
@@ -182,13 +182,13 @@ POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
 POSTHOOK: Output: default@simple_mm
 POSTHOOK: Lineage: simple_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: insert into table simple_mm select key from intermediate
+PREHOOK: query: insert overwrite table simple_mm select key from intermediate
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
 PREHOOK: Input: default@intermediate@p=455
 PREHOOK: Input: default@intermediate@p=456
 PREHOOK: Output: default@simple_mm
-POSTHOOK: query: insert into table simple_mm select key from intermediate
+POSTHOOK: query: insert overwrite table simple_mm select key from intermediate
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
@@ -207,10 +207,6 @@ POSTHOOK: Input: default@simple_mm
 455
 0
 455
-0
-455
-0
-455
 PREHOOK: query: drop table part_mm
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@part_mm


[02/31] hive git commit: HIVE-14608 : LLAP: slow scheduling due to LlapTaskScheduler not removing nodes on kill (Sergey Shelukhin, reviewed by Gopal V)

Posted by se...@apache.org.
HIVE-14608 : LLAP: slow scheduling due to LlapTaskScheduler not removing nodes on kill (Sergey Shelukhin, reviewed by Gopal V)


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

Branch: refs/heads/hive-14535
Commit: cd6c3cdf91c466a01cd08b108601f8a654a192a8
Parents: 146a918
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Sep 7 19:00:43 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed Sep 7 19:00:43 2016 -0700

----------------------------------------------------------------------
 .../tezplugins/LlapTaskSchedulerService.java    | 35 ++++++++++----------
 1 file changed, 18 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cd6c3cdf/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 10d9ad1..9fc43b3 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
@@ -15,7 +15,9 @@
 package org.apache.hadoop.hive.llap.tezplugins;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -781,27 +783,26 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         }
       }
       /* fall through - miss in locality (random scheduling) or no locality-requested */
-      Entry<String, NodeInfo>[] all = instanceToNodeMap.entrySet().toArray(new Entry[0]);
-      // Check again
+      Collection<ServiceInstance> instances = activeInstances.getAll();
+      ArrayList<NodeInfo> all = new ArrayList<>(instances.size());
+      for (ServiceInstance inst : instances) {
+        NodeInfo nodeInfo = instanceToNodeMap.get(inst.getWorkerIdentity());
+        if (nodeInfo != null && nodeInfo.canAcceptTask()) {
+          all.add(nodeInfo);
+        }
+      }
       if (LOG.isDebugEnabled()) {
         LOG.debug("Attempting random allocation for task={}", request.task);
       }
-      if (all.length > 0) {
-        int n = random.nextInt(all.length);
-        // start at random offset and iterate whole list
-        for (int i = 0; i < all.length; i++) {
-          Entry<String, NodeInfo> inst = all[(i + n) % all.length];
-          if (inst.getValue().canAcceptTask()) {
-            LOG.info(
-                "Assigning " + nodeToString(inst.getValue().getServiceInstance(), inst.getValue()) +
-                    " when looking for any host, from #hosts=" + all.length + ", requestedHosts=" +
-                    ((requestedHosts == null || requestedHosts.length == 0) ? "null" :
-                        Arrays.toString(requestedHosts)));
-            return new SelectHostResult(inst.getValue().getServiceInstance(), inst.getValue());
-          }
-        }
+      if (all.isEmpty()) {
+        return SELECT_HOST_RESULT_DELAYED_RESOURCES;
       }
-      return SELECT_HOST_RESULT_DELAYED_RESOURCES;
+      NodeInfo randomNode = all.get(random.nextInt(all.size()));
+      LOG.info("Assigning " + nodeToString(randomNode.getServiceInstance(), randomNode)
+          + " when looking for any host, from #hosts=" + all.size() + ", requestedHosts="
+          + ((requestedHosts == null || requestedHosts.length == 0)
+              ? "null" : Arrays.toString(requestedHosts)));
+      return new SelectHostResult(randomNode.getServiceInstance(), randomNode);
     } finally {
       readLock.unlock();
     }


[26/31] hive git commit: HIVE-14644 : use metastore information on the read path appropriately (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 78eb365..1418c2e 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -17978,6 +17978,274 @@ class HeartbeatWriteIdResult {
 
 }
 
+class GetValidWriteIdsRequest {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $tblName = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tblName',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['tblName'])) {
+        $this->tblName = $vals['tblName'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetValidWriteIdsRequest';
+  }
+
+  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->tblName);
+          } 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('GetValidWriteIdsRequest');
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tblName !== null) {
+      $xfer += $output->writeFieldBegin('tblName', TType::STRING, 2);
+      $xfer += $output->writeString($this->tblName);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetValidWriteIdsResult {
+  static $_TSPEC;
+
+  /**
+   * @var int
+   */
+  public $lowWatermarkId = null;
+  /**
+   * @var int
+   */
+  public $highWatermarkId = null;
+  /**
+   * @var bool
+   */
+  public $areIdsValid = null;
+  /**
+   * @var int[]
+   */
+  public $ids = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'lowWatermarkId',
+          'type' => TType::I64,
+          ),
+        2 => array(
+          'var' => 'highWatermarkId',
+          'type' => TType::I64,
+          ),
+        3 => array(
+          'var' => 'areIdsValid',
+          'type' => TType::BOOL,
+          ),
+        4 => array(
+          'var' => 'ids',
+          'type' => TType::LST,
+          'etype' => TType::I64,
+          'elem' => array(
+            'type' => TType::I64,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['lowWatermarkId'])) {
+        $this->lowWatermarkId = $vals['lowWatermarkId'];
+      }
+      if (isset($vals['highWatermarkId'])) {
+        $this->highWatermarkId = $vals['highWatermarkId'];
+      }
+      if (isset($vals['areIdsValid'])) {
+        $this->areIdsValid = $vals['areIdsValid'];
+      }
+      if (isset($vals['ids'])) {
+        $this->ids = $vals['ids'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetValidWriteIdsResult';
+  }
+
+  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::I64) {
+            $xfer += $input->readI64($this->lowWatermarkId);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->highWatermarkId);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->areIdsValid);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::LST) {
+            $this->ids = array();
+            $_size562 = 0;
+            $_etype565 = 0;
+            $xfer += $input->readListBegin($_etype565, $_size562);
+            for ($_i566 = 0; $_i566 < $_size562; ++$_i566)
+            {
+              $elem567 = null;
+              $xfer += $input->readI64($elem567);
+              $this->ids []= $elem567;
+            }
+            $xfer += $input->readListEnd();
+          } 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('GetValidWriteIdsResult');
+    if ($this->lowWatermarkId !== null) {
+      $xfer += $output->writeFieldBegin('lowWatermarkId', TType::I64, 1);
+      $xfer += $output->writeI64($this->lowWatermarkId);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->highWatermarkId !== null) {
+      $xfer += $output->writeFieldBegin('highWatermarkId', TType::I64, 2);
+      $xfer += $output->writeI64($this->highWatermarkId);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->areIdsValid !== null) {
+      $xfer += $output->writeFieldBegin('areIdsValid', TType::BOOL, 3);
+      $xfer += $output->writeBool($this->areIdsValid);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->ids !== null) {
+      if (!is_array($this->ids)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('ids', TType::LST, 4);
+      {
+        $output->writeListBegin(TType::I64, count($this->ids));
+        {
+          foreach ($this->ids as $iter568)
+          {
+            $xfer += $output->writeI64($iter568);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class GetAllFunctionsResponse {
   static $_TSPEC;
 
@@ -18029,15 +18297,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size562 = 0;
-            $_etype565 = 0;
-            $xfer += $input->readListBegin($_etype565, $_size562);
-            for ($_i566 = 0; $_i566 < $_size562; ++$_i566)
+            $_size569 = 0;
+            $_etype572 = 0;
+            $xfer += $input->readListBegin($_etype572, $_size569);
+            for ($_i573 = 0; $_i573 < $_size569; ++$_i573)
             {
-              $elem567 = null;
-              $elem567 = new \metastore\Function();
-              $xfer += $elem567->read($input);
-              $this->functions []= $elem567;
+              $elem574 = null;
+              $elem574 = new \metastore\Function();
+              $xfer += $elem574->read($input);
+              $this->functions []= $elem574;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18065,9 +18333,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter568)
+          foreach ($this->functions as $iter575)
           {
-            $xfer += $iter568->write($output);
+            $xfer += $iter575->write($output);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 13be322..70fbc08 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -176,6 +176,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  GetNextWriteIdResult get_next_write_id(GetNextWriteIdRequest req)')
   print('  FinalizeWriteIdResult finalize_write_id(FinalizeWriteIdRequest req)')
   print('  HeartbeatWriteIdResult heartbeat_write_id(HeartbeatWriteIdRequest req)')
+  print('  GetValidWriteIdsResult get_valid_write_ids(GetValidWriteIdsRequest req)')
   print('  string getName()')
   print('  string getVersion()')
   print('  fb_status getStatus()')
@@ -1157,6 +1158,12 @@ elif cmd == 'heartbeat_write_id':
     sys.exit(1)
   pp.pprint(client.heartbeat_write_id(eval(args[0]),))
 
+elif cmd == 'get_valid_write_ids':
+  if len(args) != 1:
+    print('get_valid_write_ids requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_valid_write_ids(eval(args[0]),))
+
 elif cmd == 'getName':
   if len(args) != 0:
     print('getName requires 0 args')


[06/31] hive git commit: HIVE-14217: Druid integration (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
index 8b08ae7..911b86b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlCountAggFunc
 import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlMinMaxAggFunction;
 import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlSumAggFunction;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveBetween;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveDateGranularity;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIn;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
@@ -206,6 +207,7 @@ public class SqlFunctionConverter {
         case IS_NOT_NULL:
         case IS_NULL:
         case CASE:
+        case OTHER_FUNCTION:
           node = (ASTNode) ParseDriver.adaptor.create(HiveParser.TOK_FUNCTION, "TOK_FUNCTION");
           node.addChild((ASTNode) ParseDriver.adaptor.create(hToken.type, hToken.text));
           break;
@@ -311,8 +313,8 @@ public class SqlFunctionConverter {
       registerFunction("+", SqlStdOperatorTable.PLUS, hToken(HiveParser.PLUS, "+"));
       registerFunction("-", SqlStdOperatorTable.MINUS, hToken(HiveParser.MINUS, "-"));
       registerFunction("*", SqlStdOperatorTable.MULTIPLY, hToken(HiveParser.STAR, "*"));
-      registerFunction("/", SqlStdOperatorTable.DIVIDE, hToken(HiveParser.STAR, "/"));
-      registerFunction("%", SqlStdOperatorTable.MOD, hToken(HiveParser.STAR, "%"));
+      registerFunction("/", SqlStdOperatorTable.DIVIDE, hToken(HiveParser.DIVIDE, "/"));
+      registerFunction("%", SqlStdOperatorTable.MOD, hToken(HiveParser.Identifier, "%"));
       registerFunction("and", SqlStdOperatorTable.AND, hToken(HiveParser.KW_AND, "and"));
       registerFunction("or", SqlStdOperatorTable.OR, hToken(HiveParser.KW_OR, "or"));
       registerFunction("=", SqlStdOperatorTable.EQUALS, hToken(HiveParser.EQUAL, "="));
@@ -334,6 +336,23 @@ public class SqlFunctionConverter {
       registerFunction("isnull", SqlStdOperatorTable.IS_NULL, hToken(HiveParser.TOK_ISNULL, "TOK_ISNULL"));
       registerFunction("when", SqlStdOperatorTable.CASE, hToken(HiveParser.Identifier, "when"));
       registerDuplicateFunction("case", SqlStdOperatorTable.CASE, hToken(HiveParser.Identifier, "when"));
+      // timebased
+      registerFunction("floor_year", HiveDateGranularity.YEAR,
+          hToken(HiveParser.Identifier, "floor_year"));
+      registerFunction("floor_quarter", HiveDateGranularity.QUARTER,
+          hToken(HiveParser.Identifier, "floor_quarter"));
+      registerFunction("floor_month", HiveDateGranularity.MONTH,
+          hToken(HiveParser.Identifier, "floor_month"));
+      registerFunction("floor_week", HiveDateGranularity.WEEK,
+          hToken(HiveParser.Identifier, "floor_week"));
+      registerFunction("floor_day", HiveDateGranularity.DAY,
+          hToken(HiveParser.Identifier, "floor_day"));
+      registerFunction("floor_hour", HiveDateGranularity.HOUR,
+          hToken(HiveParser.Identifier, "floor_hour"));
+      registerFunction("floor_minute", HiveDateGranularity.MINUTE,
+          hToken(HiveParser.Identifier, "floor_minute"));
+      registerFunction("floor_second", HiveDateGranularity.SECOND,
+          hToken(HiveParser.Identifier, "floor_second"));
     }
 
     private void registerFunction(String name, SqlOperator calciteFn, HiveToken hiveToken) {

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/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 18a55cb..ff94160 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
@@ -24,6 +24,7 @@ import java.lang.reflect.UndeclaredThrowableException;
 import java.math.BigDecimal;
 import java.util.AbstractMap.SimpleEntry;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.BitSet;
 import java.util.Collections;
 import java.util.EnumSet;
@@ -61,6 +62,7 @@ import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -98,6 +100,7 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
@@ -128,6 +131,11 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
 import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveAlgorithmsConf;
 import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveVolcanoPlanner;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidQuery;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidRules;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidSchema;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.HiveDruidConf;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
@@ -208,6 +216,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.joda.time.Interval;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
@@ -931,8 +940,11 @@ public class CalcitePlanner extends SemanticAnalyzer {
       final Double maxMemory = (double) HiveConf.getLongVar(
               conf, HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD);
       HiveAlgorithmsConf algorithmsConf = new HiveAlgorithmsConf(maxSplitSize, maxMemory);
+      final int selectThreshold = (int) HiveConf.getIntVar(
+              conf, HiveConf.ConfVars.HIVE_DRUID_SELECT_THRESHOLD);
+      HiveDruidConf druidConf = new HiveDruidConf(selectThreshold);
       HiveRulesRegistry registry = new HiveRulesRegistry();
-      HivePlannerContext confContext = new HivePlannerContext(algorithmsConf, registry);
+      HivePlannerContext confContext = new HivePlannerContext(algorithmsConf, druidConf, registry);
       RelOptPlanner planner = HiveVolcanoPlanner.createPlanner(confContext);
       final RelOptQuery query = new RelOptQuery(planner);
       final RexBuilder rexBuilder = cluster.getRexBuilder();
@@ -1070,10 +1082,18 @@ public class CalcitePlanner extends SemanticAnalyzer {
         perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: Window fixing rule");
       }
 
-      // 8. Run rules to aid in translation from Calcite tree to Hive tree
+      // 8. Apply Druid transformation rules
+      perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
+      calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
+              HepMatchOrder.BOTTOM_UP, DruidRules.FILTER, DruidRules.PROJECT_AGGREGATE,
+              DruidRules.PROJECT, DruidRules.AGGREGATE, DruidRules.PROJECT_SORT,
+              DruidRules.SORT, DruidRules.SORT_PROJECT);
+      perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: Druid transformation rules");
+
+      // 9. Run rules to aid in translation from Calcite tree to Hive tree
       if (HiveConf.getBoolVar(conf, ConfVars.HIVE_CBO_RETPATH_HIVEOP)) {
         perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
-        // 8.1. Merge join into multijoin operators (if possible)
+        // 9.1. Merge join into multijoin operators (if possible)
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, true, mdProvider.getMetadataProvider(), null,
                 HepMatchOrder.BOTTOM_UP, HiveJoinProjectTransposeRule.BOTH_PROJECT_INCLUDE_OUTER,
                 HiveJoinProjectTransposeRule.LEFT_PROJECT_INCLUDE_OUTER,
@@ -1091,7 +1111,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
             new HiveFilterProjectTSTransposeRule(Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY,
                     HiveProject.class, HiveRelFactories.HIVE_PROJECT_FACTORY, HiveTableScan.class));
 
-        // 8.2.  Introduce exchange operators below join/multijoin operators
+        // 9.2.  Introduce exchange operators below join/multijoin operators
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
                 HepMatchOrder.BOTTOM_UP, HiveInsertExchange4JoinRule.EXCHANGE_BELOW_JOIN,
                 HiveInsertExchange4JoinRule.EXCHANGE_BELOW_MULTIJOIN);
@@ -1251,7 +1271,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       basePlan = hepPlan(basePlan, true, mdProvider, null,
           new HiveFilterProjectTSTransposeRule(
               Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY, HiveProject.class,
-              HiveRelFactories.HIVE_PROJECT_FACTORY, HiveTableScan.class),
+              HiveRelFactories.HIVE_PROJECT_FACTORY, TableScan.class),
           HiveProjectFilterPullUpConstantsRule.INSTANCE);
       perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER,
         "Calcite: Prejoin ordering transformation, Rerun PPD");
@@ -1657,7 +1677,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
 
     private RelNode genTableLogicalPlan(String tableAlias, QB qb) throws SemanticException {
       RowResolver rr = new RowResolver();
-      HiveTableScan tableRel = null;
+      RelNode tableRel = null;
 
       try {
 
@@ -1713,16 +1733,20 @@ public class CalcitePlanner extends SemanticAnalyzer {
           partitionColumns.add(colInfo);
         }
 
+        final TableType tableType = obtainTableType(tabMetaData);
+
         // 3.3 Add column info corresponding to virtual columns
         List<VirtualColumn> virtualCols = new ArrayList<VirtualColumn>();
-        Iterator<VirtualColumn> vcs = VirtualColumn.getRegistry(conf).iterator();
-        while (vcs.hasNext()) {
-          VirtualColumn vc = vcs.next();
-          colInfo = new ColumnInfo(vc.getName(), vc.getTypeInfo(), tableAlias, true,
-              vc.getIsHidden());
-          rr.put(tableAlias, vc.getName().toLowerCase(), colInfo);
-          cInfoLst.add(colInfo);
-          virtualCols.add(vc);
+        if (tableType == TableType.NATIVE) {
+          Iterator<VirtualColumn> vcs = VirtualColumn.getRegistry(conf).iterator();
+          while (vcs.hasNext()) {
+            VirtualColumn vc = vcs.next();
+            colInfo = new ColumnInfo(vc.getName(), vc.getTypeInfo(), tableAlias, true,
+                vc.getIsHidden());
+            rr.put(tableAlias, vc.getName().toLowerCase(), colInfo);
+            cInfoLst.add(colInfo);
+            virtualCols.add(vc);
+          }
         }
 
         // 3.4 Build row type from field <type, name>
@@ -1737,15 +1761,50 @@ public class CalcitePlanner extends SemanticAnalyzer {
           fullyQualifiedTabName = tabMetaData.getTableName();
         }
         RelOptHiveTable optTable = new RelOptHiveTable(relOptSchema, fullyQualifiedTabName,
-            rowType, tabMetaData, nonPartitionColumns, partitionColumns, virtualCols, conf,
-            partitionCache, noColsMissingStats);
-
-        // 5. Build Hive Table Scan Rel
-        tableRel = new HiveTableScan(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION), optTable,
-            null == tableAlias ? tabMetaData.getTableName() : tableAlias,
-            getAliasId(tableAlias, qb), HiveConf.getBoolVar(conf,
-                HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP), qb.isInsideView()
-                || qb.getAliasInsideView().contains(tableAlias.toLowerCase()));
+                rowType, tabMetaData, nonPartitionColumns, partitionColumns, virtualCols, conf,
+                partitionCache, noColsMissingStats);
+
+        // 5. Build operator
+        if (tableType == TableType.DRUID) {
+          // Build Druid query
+          String address = HiveConf.getVar(conf,
+                  HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS);
+          String dataSource = tabMetaData.getParameters().get(Constants.DRUID_DATA_SOURCE);
+          Set<String> metrics = new HashSet<>();
+          List<RelDataType> druidColTypes = new ArrayList<>();
+          List<String> druidColNames = new ArrayList<>();
+          for (RelDataTypeField field : rowType.getFieldList()) {
+            druidColTypes.add(field.getType());
+            druidColNames.add(field.getName());
+            if (field.getName().equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
+              // timestamp
+              continue;
+            }
+            if (field.getType().getSqlTypeName() == SqlTypeName.VARCHAR) {
+              // dimension
+              continue;
+            }
+            metrics.add(field.getName());
+          }
+          List<Interval> intervals = Arrays.asList(DruidTable.DEFAULT_INTERVAL);
+
+          DruidTable druidTable = new DruidTable(new DruidSchema(address),
+                  dataSource, rowType, metrics, intervals, DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+          final TableScan scan = new HiveTableScan(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+                  optTable, null == tableAlias ? tabMetaData.getTableName() : tableAlias,
+                  getAliasId(tableAlias, qb), HiveConf.getBoolVar(conf,
+                      HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP), qb.isInsideView()
+                      || qb.getAliasInsideView().contains(tableAlias.toLowerCase()));
+          tableRel = DruidQuery.create(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+              optTable, druidTable, ImmutableList.<RelNode>of(scan));
+        } else {
+          // Build Hive Table Scan Rel
+          tableRel = new HiveTableScan(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION), optTable,
+              null == tableAlias ? tabMetaData.getTableName() : tableAlias,
+              getAliasId(tableAlias, qb), HiveConf.getBoolVar(conf,
+                  HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP), qb.isInsideView()
+                  || qb.getAliasInsideView().contains(tableAlias.toLowerCase()));
+        }
 
         // 6. Add Schema(RR) to RelNode-Schema map
         ImmutableMap<String, Integer> hiveToCalciteColMap = buildHiveToCalciteColumnMap(rr,
@@ -1763,6 +1822,15 @@ public class CalcitePlanner extends SemanticAnalyzer {
       return tableRel;
     }
 
+    private TableType obtainTableType(Table tabMetaData) {
+      if (tabMetaData.getStorageHandler() != null &&
+              tabMetaData.getStorageHandler().toString().equals(
+                      Constants.DRUID_HIVE_STORAGE_HANDLER_ID)) {
+        return TableType.DRUID;
+      }
+      return TableType.NATIVE;
+    }
+
     private RelNode genFilterRelNode(ASTNode filterExpr, RelNode srcRel,
             boolean useCaching) throws SemanticException {
       ExprNodeDesc filterCondn = genExprNodeDesc(filterExpr, relToHiveRR.get(srcRel), useCaching);
@@ -3366,4 +3434,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
     }
   }
 
+  private enum TableType {
+    DRUID,
+    NATIVE
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/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 ace0e9c..943d9d7 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
@@ -56,7 +56,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hive.common.BlobStorageUtils;
-import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.StatsSetupConst;
@@ -156,6 +155,7 @@ import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFrameSpec;
 import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFunctionSpec;
 import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowSpec;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
 import org.apache.hadoop.hive.ql.plan.CreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.CreateTableLikeDesc;
@@ -9787,11 +9787,24 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     // is the table already present
     TableScanOperator top = topOps.get(alias_id);
 
+    // Obtain table props in query
+    Map<String, String> properties = qb.getTabPropsForAlias(alias);
+
     if (top == null) {
       // Determine row schema for TSOP.
       // Include column names from SerDe, the partition and virtual columns.
       rwsch = new RowResolver();
       try {
+        // Including parameters passed in the query
+        if (properties != null) {
+          for (Entry<String, String> prop : properties.entrySet()) {
+            if (tab.getSerdeParam(prop.getKey()) != null) {
+              LOG.warn("SerDe property in input query overrides stored SerDe property");
+            }
+            tab.setSerdeParam(prop.getKey(), prop.getValue());
+          }
+        }
+        // Obtain inspector for schema
         StructObjectInspector rowObjectInspector = (StructObjectInspector) tab
             .getDeserializer().getObjectInspector();
         List<? extends StructField> fields = rowObjectInspector
@@ -9852,10 +9865,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       // Add a mapping from the table scan operator to Table
       topToTable.put(top, tab);
 
-      Map<String, String> props = qb.getTabPropsForAlias(alias);
-      if (props != null) {
-        topToTableProps.put(top, props);
-        tsDesc.setOpProps(props);
+      if (properties != null) {
+        topToTableProps.put(top, properties);
+        tsDesc.setOpProps(properties);
       }
     } else {
       rwsch = opParseCtx.get(top).getRowResolver();

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
index bf808c3..eafba21 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
@@ -24,14 +24,15 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.DDLTask;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -476,7 +477,8 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
 
     if ((this.getCols() == null) || (this.getCols().size() == 0)) {
       // for now make sure that serde exists
-      if (Table.hasMetastoreBasedSchema(conf, getSerName())) {
+      if (Table.hasMetastoreBasedSchema(conf, serName) &&
+              StringUtils.isEmpty(getStorageHandler())) {
         throw new SemanticException(ErrorMsg.INVALID_TBL_DDL_SERDE.getMsg());
       }
       return;

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
index 47c65bb..ebe613e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
@@ -389,4 +389,11 @@ public class TableScanDesc extends AbstractOperatorDesc {
     }
     return rtn;
   }
+
+  @Override
+  @Explain(displayName = "properties", explainLevels = { Level.DEFAULT, Level.USER, Level.EXTENDED })
+  public Map<String, String> getOpProps() {
+    return opProps;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java
new file mode 100644
index 0000000..08ed9fd
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java
@@ -0,0 +1,506 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.udf;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.joda.time.Chronology;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Period;
+import org.joda.time.ReadableDuration;
+import org.joda.time.chrono.ISOChronology;
+
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * UDFDateFloor.
+ *
+ * Abstract class that converts a timestamp to a timestamp with a given granularity.
+ */
+public abstract class UDFDateFloor extends UDF {
+
+  private final QueryGranularity granularity;
+
+  private final TimestampWritable result;
+
+  public UDFDateFloor(String granularity) {
+    this.granularity = QueryGranularity.fromString(granularity);
+    this.result = new TimestampWritable();
+  }
+
+  public TimestampWritable evaluate(TimestampWritable t) {
+    if (t == null) {
+      return null;
+    }
+    long newTimestamp = granularity.truncate(t.getTimestamp().getTime());
+    result.setTime(newTimestamp);
+    return result;
+  }
+
+  /*
+   * This code that creates the result for the granularity functions has been brought from Druid
+   */
+
+  private static final Map<String, PeriodGranularity> CALENDRIC_GRANULARITIES = ImmutableMap.of(
+          "YEAR", new PeriodGranularity(new Period("P1Y"), null, null),
+          "MONTH", new PeriodGranularity(new Period("P1M"), null, null),
+          "QUARTER", new PeriodGranularity(new Period("P3M"), null, null),
+          "WEEK", new PeriodGranularity(new Period("P1W"), null, null));
+
+  private static abstract class QueryGranularity {
+
+    public abstract long next(long offset);
+
+    public abstract long truncate(long offset);
+
+    public abstract DateTime toDateTime(long offset);
+
+    public abstract Iterable<Long> iterable(final long start, final long end);
+
+    public static QueryGranularity fromString(String str) {
+      String name = str.toUpperCase();
+      if (CALENDRIC_GRANULARITIES.containsKey(name)) {
+        return CALENDRIC_GRANULARITIES.get(name);
+      }
+      return new DurationGranularity(convertValue(str), 0);
+    }
+
+    private static enum MillisIn {
+      SECOND(1000), MINUTE(60 * 1000), FIFTEEN_MINUTE(15 * 60 * 1000), THIRTY_MINUTE(
+              30 * 60 * 1000), HOUR(3600 * 1000), DAY(24 * 3600 * 1000);
+
+      private final long millis;
+
+      MillisIn(final long millis) {
+        this.millis = millis;
+      }
+    }
+
+    private static long convertValue(Object o) {
+      if (o instanceof String) {
+        return MillisIn.valueOf(((String) o).toUpperCase()).millis;
+      } else if (o instanceof ReadableDuration) {
+        return ((ReadableDuration) o).getMillis();
+      } else if (o instanceof Number) {
+        return ((Number) o).longValue();
+      }
+      throw new RuntimeException("Granularity not recognized");
+    }
+  }
+
+  private static abstract class BaseQueryGranularity extends QueryGranularity {
+    public abstract long next(long offset);
+
+    public abstract long truncate(long offset);
+
+    public DateTime toDateTime(long offset) {
+      return new DateTime(offset, DateTimeZone.UTC);
+    }
+
+    public Iterable<Long> iterable(final long start, final long end) {
+      return new Iterable<Long>() {
+        @Override
+        public Iterator<Long> iterator() {
+          return new Iterator<Long>() {
+            long curr = truncate(start);
+
+            long next = BaseQueryGranularity.this.next(curr);
+
+            @Override
+            public boolean hasNext() {
+              return curr < end;
+            }
+
+            @Override
+            public Long next() {
+              if (!hasNext()) {
+                throw new NoSuchElementException();
+              }
+
+              long retVal = curr;
+
+              curr = next;
+              next = BaseQueryGranularity.this.next(curr);
+
+              return retVal;
+            }
+
+            @Override
+            public void remove() {
+              throw new UnsupportedOperationException();
+            }
+          };
+        }
+      };
+    }
+  }
+
+  private static class PeriodGranularity extends BaseQueryGranularity {
+    private final Period period;
+
+    private final Chronology chronology;
+
+    private final long origin;
+
+    private final boolean hasOrigin;
+
+    private final boolean isCompound;
+
+    public PeriodGranularity(Period period, DateTime origin, DateTimeZone tz) {
+      this.period = period;
+      this.chronology = tz == null ? ISOChronology.getInstanceUTC() : ISOChronology.getInstance(tz);
+      if (origin == null) {
+        // default to origin in given time zone when aligning multi-period granularities
+        this.origin = new DateTime(0, DateTimeZone.UTC).withZoneRetainFields(chronology.getZone())
+                .getMillis();
+        this.hasOrigin = false;
+      } else {
+        this.origin = origin.getMillis();
+        this.hasOrigin = true;
+      }
+      this.isCompound = isCompoundPeriod(period);
+    }
+
+    @Override
+    public DateTime toDateTime(long t) {
+      return new DateTime(t, chronology.getZone());
+    }
+
+    @Override
+    public long next(long t) {
+      return chronology.add(period, t, 1);
+    }
+
+    @Override
+    public long truncate(long t) {
+      if (isCompound) {
+        try {
+          return truncateMillisPeriod(t);
+        } catch (UnsupportedOperationException e) {
+          return truncateCompoundPeriod(t);
+        }
+      }
+
+      final int years = period.getYears();
+      if (years > 0) {
+        if (years > 1 || hasOrigin) {
+          int y = chronology.years().getDifference(t, origin);
+          y -= y % years;
+          long tt = chronology.years().add(origin, y);
+          // always round down to the previous period (for timestamps prior to origin)
+          if (t < tt)
+            t = chronology.years().add(tt, -years);
+          else
+            t = tt;
+          return t;
+        } else {
+          return chronology.year().roundFloor(t);
+        }
+      }
+
+      final int months = period.getMonths();
+      if (months > 0) {
+        if (months > 1 || hasOrigin) {
+          int m = chronology.months().getDifference(t, origin);
+          m -= m % months;
+          long tt = chronology.months().add(origin, m);
+          // always round down to the previous period (for timestamps prior to origin)
+          if (t < tt)
+            t = chronology.months().add(tt, -months);
+          else
+            t = tt;
+          return t;
+        } else {
+          return chronology.monthOfYear().roundFloor(t);
+        }
+      }
+
+      final int weeks = period.getWeeks();
+      if (weeks > 0) {
+        if (weeks > 1 || hasOrigin) {
+          // align on multiples from origin
+          int w = chronology.weeks().getDifference(t, origin);
+          w -= w % weeks;
+          long tt = chronology.weeks().add(origin, w);
+          // always round down to the previous period (for timestamps prior to origin)
+          if (t < tt)
+            t = chronology.weeks().add(tt, -weeks);
+          else
+            t = tt;
+          return t;
+        } else {
+          t = chronology.dayOfWeek().roundFloor(t);
+          // default to Monday as beginning of the week
+          return chronology.dayOfWeek().set(t, 1);
+        }
+      }
+
+      final int days = period.getDays();
+      if (days > 0) {
+        if (days > 1 || hasOrigin) {
+          // align on multiples from origin
+          int d = chronology.days().getDifference(t, origin);
+          d -= d % days;
+          long tt = chronology.days().add(origin, d);
+          // always round down to the previous period (for timestamps prior to origin)
+          if (t < tt)
+            t = chronology.days().add(tt, -days);
+          else
+            t = tt;
+          return t;
+        } else {
+          t = chronology.hourOfDay().roundFloor(t);
+          return chronology.hourOfDay().set(t, 0);
+        }
+      }
+
+      final int hours = period.getHours();
+      if (hours > 0) {
+        if (hours > 1 || hasOrigin) {
+          // align on multiples from origin
+          long h = chronology.hours().getDifferenceAsLong(t, origin);
+          h -= h % hours;
+          long tt = chronology.hours().add(origin, h);
+          // always round down to the previous period (for timestamps prior to origin)
+          if (t < tt)
+            t = chronology.hours().add(tt, -hours);
+          else
+            t = tt;
+          return t;
+        } else {
+          t = chronology.minuteOfHour().roundFloor(t);
+          return chronology.minuteOfHour().set(t, 0);
+        }
+      }
+
+      final int minutes = period.getMinutes();
+      if (minutes > 0) {
+        // align on multiples from origin
+        if (minutes > 1 || hasOrigin) {
+          long m = chronology.minutes().getDifferenceAsLong(t, origin);
+          m -= m % minutes;
+          long tt = chronology.minutes().add(origin, m);
+          // always round down to the previous period (for timestamps prior to origin)
+          if (t < tt)
+            t = chronology.minutes().add(tt, -minutes);
+          else
+            t = tt;
+          return t;
+        } else {
+          t = chronology.secondOfMinute().roundFloor(t);
+          return chronology.secondOfMinute().set(t, 0);
+        }
+      }
+
+      final int seconds = period.getSeconds();
+      if (seconds > 0) {
+        // align on multiples from origin
+        if (seconds > 1 || hasOrigin) {
+          long s = chronology.seconds().getDifferenceAsLong(t, origin);
+          s -= s % seconds;
+          long tt = chronology.seconds().add(origin, s);
+          // always round down to the previous period (for timestamps prior to origin)
+          if (t < tt)
+            t = chronology.seconds().add(tt, -seconds);
+          else
+            t = tt;
+          return t;
+        } else {
+          return chronology.millisOfSecond().set(t, 0);
+        }
+      }
+
+      final int millis = period.getMillis();
+      if (millis > 0) {
+        if (millis > 1) {
+          long ms = chronology.millis().getDifferenceAsLong(t, origin);
+          ms -= ms % millis;
+          long tt = chronology.millis().add(origin, ms);
+          // always round down to the previous period (for timestamps prior to origin)
+          if (t < tt)
+            t = chronology.millis().add(tt, -millis);
+          else
+            t = tt;
+          return t;
+        } else {
+          return t;
+        }
+      }
+
+      return t;
+    }
+
+    private static boolean isCompoundPeriod(Period period) {
+      int[] values = period.getValues();
+      boolean single = false;
+      for (int v : values) {
+        if (v > 0) {
+          if (single)
+            return true;
+          single = true;
+        }
+      }
+      return false;
+    }
+
+    private long truncateCompoundPeriod(long t) {
+      long current;
+      if (t >= origin) {
+        long next = origin;
+        do {
+          current = next;
+          next = chronology.add(period, current, 1);
+        } while (t >= next);
+      } else {
+        current = origin;
+        do {
+          current = chronology.add(period, current, -1);
+        } while (t < current);
+      }
+      return current;
+    }
+
+    private long truncateMillisPeriod(final long t) {
+      // toStandardDuration assumes days are always 24h, and hours are always 60 minutes,
+      // which may not always be the case, e.g if there are daylight saving changes.
+      if (chronology.days().isPrecise() && chronology.hours().isPrecise()) {
+        final long millis = period.toStandardDuration().getMillis();
+        long offset = t % millis - origin % millis;
+        if (offset < 0) {
+          offset += millis;
+        }
+        return t - offset;
+      } else {
+        throw new UnsupportedOperationException(
+                "Period cannot be converted to milliseconds as some fields mays vary in length with chronology "
+                        + chronology.toString());
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      PeriodGranularity that = (PeriodGranularity) o;
+
+      if (hasOrigin != that.hasOrigin) {
+        return false;
+      }
+      if (origin != that.origin) {
+        return false;
+      }
+      if (!chronology.equals(that.chronology)) {
+        return false;
+      }
+      if (!period.equals(that.period)) {
+        return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = period.hashCode();
+      result = 31 * result + chronology.hashCode();
+      result = 31 * result + (int) (origin ^ (origin >>> 32));
+      result = 31 * result + (hasOrigin ? 1 : 0);
+      return result;
+    }
+
+    @Override
+    public String toString() {
+      return "PeriodGranularity{" + "period=" + period + ", timeZone=" + chronology.getZone()
+              + ", origin=" + (hasOrigin ? origin : "null") + '}';
+    }
+  }
+
+  private static class DurationGranularity extends BaseQueryGranularity {
+    private final long length;
+
+    private final long origin;
+
+    public DurationGranularity(long millis, long origin) {
+      this.length = millis;
+      this.origin = origin % length;
+    }
+
+    @Override
+    public long next(long t) {
+      return t + getDurationMillis();
+    }
+
+    @Override
+    public long truncate(final long t) {
+      final long duration = getDurationMillis();
+      long offset = t % duration - origin % duration;
+      if (offset < 0) {
+        offset += duration;
+      }
+      return t - offset;
+    }
+
+    public long getDurationMillis() {
+      return length;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      DurationGranularity that = (DurationGranularity) o;
+
+      if (length != that.length) {
+        return false;
+      }
+      if (origin != that.origin) {
+        return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = (int) (length ^ (length >>> 32));
+      result = 31 * result + (int) (origin ^ (origin >>> 32));
+      return result;
+    }
+
+    @Override
+    public String toString() {
+      return "DurationGranularity{" + "length=" + length + ", origin=" + origin + '}';
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorDay.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorDay.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorDay.java
new file mode 100644
index 0000000..e205797
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorDay.java
@@ -0,0 +1,39 @@
+/**
+ * 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.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorDay.
+ *
+ * Converts a timestamp to a timestamp with day granularity.
+ */
+@Description(name = "floor_day",
+    value = "_FUNC_(param) - Returns the timestamp at a day granularity",
+    extended = "param needs to be a timestamp value\n"
+    + "Example:\n "
+    + "  > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+    + "  yyyy-MM-dd 00:00:00")
+public class UDFDateFloorDay extends UDFDateFloor {
+
+  public UDFDateFloorDay() {
+    super("DAY");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorHour.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorHour.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorHour.java
new file mode 100644
index 0000000..84fd394
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorHour.java
@@ -0,0 +1,39 @@
+/**
+ * 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.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorHour.
+ *
+ * Converts a timestamp to a timestamp with hour granularity.
+ */
+@Description(name = "floor_hour",
+    value = "_FUNC_(param) - Returns the timestamp at a hour granularity",
+    extended = "param needs to be a timestamp value\n"
+    + "Example:\n "
+    + "  > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+    + "  yyyy-MM-dd HH:00:00")
+public class UDFDateFloorHour extends UDFDateFloor {
+
+  public UDFDateFloorHour() {
+    super("HOUR");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMinute.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMinute.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMinute.java
new file mode 100644
index 0000000..45b8f7e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMinute.java
@@ -0,0 +1,39 @@
+/**
+ * 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.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorMinute.
+ *
+ * Converts a timestamp to a timestamp with minute granularity.
+ */
+@Description(name = "floor_minute",
+    value = "_FUNC_(param) - Returns the timestamp at a minute granularity",
+    extended = "param needs to be a timestamp value\n"
+    + "Example:\n "
+    + "  > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+    + "  yyyy-MM-dd HH:mm:00")
+public class UDFDateFloorMinute extends UDFDateFloor {
+
+  public UDFDateFloorMinute() {
+    super("MINUTE");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMonth.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMonth.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMonth.java
new file mode 100644
index 0000000..e3c70a1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMonth.java
@@ -0,0 +1,39 @@
+/**
+ * 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.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorMonth.
+ *
+ * Converts a timestamp to a timestamp with month granularity.
+ */
+@Description(name = "floor_month",
+    value = "_FUNC_(param) - Returns the timestamp at a month granularity",
+    extended = "param needs to be a timestamp value\n"
+    + "Example:\n "
+    + "  > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+    + "  yyyy-MM-01 00:00:00")
+public class UDFDateFloorMonth extends UDFDateFloor {
+
+  public UDFDateFloorMonth() {
+    super("MONTH");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorQuarter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorQuarter.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorQuarter.java
new file mode 100644
index 0000000..c017238
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorQuarter.java
@@ -0,0 +1,39 @@
+/**
+ * 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.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorQuarter.
+ *
+ * Converts a timestamp to a timestamp with quarter granularity.
+ */
+@Description(name = "floor_quarter",
+    value = "_FUNC_(param) - Returns the timestamp at a quarter granularity",
+    extended = "param needs to be a timestamp value\n"
+    + "Example:\n "
+    + "  > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+    + "  yyyy-xx-01 00:00:00")
+public class UDFDateFloorQuarter extends UDFDateFloor {
+
+  public UDFDateFloorQuarter() {
+    super("QUARTER");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorSecond.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorSecond.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorSecond.java
new file mode 100644
index 0000000..5e10026
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorSecond.java
@@ -0,0 +1,39 @@
+/**
+ * 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.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorSecond.
+ *
+ * Converts a timestamp to a timestamp with second granularity.
+ */
+@Description(name = "floor_second",
+    value = "_FUNC_(param) - Returns the timestamp at a second granularity",
+    extended = "param needs to be a timestamp value\n"
+    + "Example:\n "
+    + "  > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+    + "  yyyy-MM-dd HH:mm:ss")
+public class UDFDateFloorSecond extends UDFDateFloor {
+
+  public UDFDateFloorSecond() {
+    super("SECOND");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorWeek.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorWeek.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorWeek.java
new file mode 100644
index 0000000..185a84a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorWeek.java
@@ -0,0 +1,39 @@
+/**
+ * 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.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorWeek.
+ *
+ * Converts a timestamp to a timestamp with week granularity.
+ */
+@Description(name = "floor_week",
+    value = "_FUNC_(param) - Returns the timestamp at a week granularity",
+    extended = "param needs to be a timestamp value\n"
+    + "Example:\n "
+    + "  > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+    + "  yyyy-MM-xx 00:00:00")
+public class UDFDateFloorWeek extends UDFDateFloor {
+
+  public UDFDateFloorWeek() {
+    super("WEEK");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorYear.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorYear.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorYear.java
new file mode 100644
index 0000000..e68decf
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorYear.java
@@ -0,0 +1,39 @@
+/**
+ * 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.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorYear.
+ *
+ * Converts a timestamp to a timestamp with year granularity.
+ */
+@Description(name = "floor_year",
+    value = "_FUNC_(param) - Returns the timestamp at a year granularity",
+    extended = "param needs to be a timestamp value\n"
+    + "Example:\n "
+    + "  > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+    + "  yyyy-01-01 00:00:00")
+public class UDFDateFloorYear extends UDFDateFloor {
+
+  public UDFDateFloorYear() {
+    super("YEAR");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
index 44e157b..2830f1f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
@@ -61,7 +61,7 @@ public class TestCBORuleFiredOnlyOnce {
 
     // Create rules registry to not trigger a rule more than once
     HiveRulesRegistry registry = new HiveRulesRegistry();
-    HivePlannerContext context = new HivePlannerContext(null, registry);
+    HivePlannerContext context = new HivePlannerContext(null, null, registry);
     HepPlanner planner = new HepPlanner(programBuilder.build(), context);
 
     // Cluster

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
new file mode 100644
index 0000000..f871de2
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
@@ -0,0 +1,85 @@
+/**
+ * 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.udf;
+
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestUDFDateFormatGranularity extends TestCase {
+  
+  @Test
+  public void testTimestampToTimestampWithGranularity() throws Exception {
+    // Running example
+    // Friday 30th August 1985 02:47:02 AM
+    final TimestampWritable t = new TimestampWritable(new Timestamp(494218022082L));
+    UDFDateFloor g;
+
+    // Year granularity
+    // Tuesday 1st January 1985 12:00:00 AM
+    g = new UDFDateFloorYear();
+    TimestampWritable i1 = g.evaluate(t);
+    assertEquals(473385600000L, i1.getTimestamp().getTime());
+    
+    // Quarter granularity
+    // Monday 1st July 1985 12:00:00 AM
+    g = new UDFDateFloorQuarter();
+    TimestampWritable i2 = g.evaluate(t);
+    assertEquals(489024000000L, i2.getTimestamp().getTime());
+
+    // Month granularity
+    // Thursday 1st August 1985 12:00:00 AM
+    g = new UDFDateFloorMonth();
+    TimestampWritable i3 = g.evaluate(t);
+    assertEquals(491702400000L, i3.getTimestamp().getTime());
+
+    // Week granularity
+    // Monday 26th August 1985 12:00:00 AM
+    g = new UDFDateFloorWeek();
+    TimestampWritable i4 = g.evaluate(t);
+    assertEquals(493862400000L, i4.getTimestamp().getTime());
+
+    // Day granularity
+    // Friday 30th August 1985 12:00:00 AM
+    g = new UDFDateFloorDay();
+    TimestampWritable i5 = g.evaluate(t);
+    assertEquals(494208000000L, i5.getTimestamp().getTime());
+
+    // Hour granularity
+    // Friday 30th August 1985 02:00:00 AM
+    g = new UDFDateFloorHour();
+    TimestampWritable i6 = g.evaluate(t);
+    assertEquals(494215200000L, i6.getTimestamp().getTime());
+
+    // Minute granularity
+    // Friday 30th August 1985 02:47:00 AM
+    g = new UDFDateFloorMinute();
+    TimestampWritable i7 = g.evaluate(t);
+    assertEquals(494218020000L, i7.getTimestamp().getTime());
+
+    // Second granularity
+    // Friday 30th August 1985 02:47:02 AM
+    g = new UDFDateFloorSecond();
+    TimestampWritable i8 = g.evaluate(t);
+    assertEquals(494218022000L, i8.getTimestamp().getTime());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_address.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_address.q b/ql/src/test/queries/clientnegative/druid_address.q
new file mode 100644
index 0000000..35ba06a
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_address.q
@@ -0,0 +1,5 @@
+set hive.druid.broker.address.default=;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_buckets.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_buckets.q b/ql/src/test/queries/clientnegative/druid_buckets.q
new file mode 100644
index 0000000..6fc75d1
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_buckets.q
@@ -0,0 +1,6 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+CLUSTERED BY (robot) INTO 32 BUCKETS
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_datasource.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_datasource.q b/ql/src/test/queries/clientnegative/druid_datasource.q
new file mode 100644
index 0000000..87481fd
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_datasource.q
@@ -0,0 +1,3 @@
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("property" = "localhost");

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_external.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_external.q b/ql/src/test/queries/clientnegative/druid_external.q
new file mode 100644
index 0000000..2de04db
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_external.q
@@ -0,0 +1,5 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_location.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_location.q b/ql/src/test/queries/clientnegative/druid_location.q
new file mode 100644
index 0000000..a9705dc
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_location.q
@@ -0,0 +1,6 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+LOCATION '/testfolder/'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_partitions.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_partitions.q b/ql/src/test/queries/clientnegative/druid_partitions.q
new file mode 100644
index 0000000..e26a3b6
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_partitions.q
@@ -0,0 +1,6 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+PARTITIONED BY (dt string)
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientpositive/druid_basic1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druid_basic1.q b/ql/src/test/queries/clientpositive/druid_basic1.q
new file mode 100644
index 0000000..83f5968
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druid_basic1.q
@@ -0,0 +1,18 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_1;
+
+-- different table, same datasource
+CREATE EXTERNAL TABLE druid_table_2
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_2;
+
+DROP TABLE druid_table_2;
+
+DROP TABLE druid_table_1;

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientpositive/druid_basic2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druid_basic2.q b/ql/src/test/queries/clientpositive/druid_basic2.q
new file mode 100644
index 0000000..fe24410
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druid_basic2.q
@@ -0,0 +1,52 @@
+set hive.strict.checks.cartesian.product=false;
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_1;
+
+-- dimension
+EXPLAIN EXTENDED
+SELECT robot FROM druid_table_1;
+
+-- metric
+EXPLAIN EXTENDED
+SELECT delta FROM druid_table_1;
+
+EXPLAIN EXTENDED
+SELECT robot
+FROM druid_table_1
+WHERE language = 'en';
+
+EXPLAIN EXTENDED
+SELECT DISTINCT robot
+FROM druid_table_1
+WHERE language = 'en';
+
+-- TODO: currently nothing is pushed - ISNOTNULL
+EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+  (SELECT robot, language
+  FROM druid_table_1) a
+  JOIN
+  (SELECT language
+  FROM druid_table_1) b
+  ON a.language = b.language
+);
+
+EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+  (SELECT robot, language
+  FROM druid_table_1
+  WHERE language = 'en') a
+  JOIN
+  (SELECT language
+  FROM druid_table_1) b
+  ON a.language = b.language
+);

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientpositive/druid_intervals.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druid_intervals.q b/ql/src/test/queries/clientpositive/druid_intervals.q
new file mode 100644
index 0000000..140ff82
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druid_intervals.q
@@ -0,0 +1,67 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_1;
+
+-- (-\u221e\u2025+\u221e)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1;
+
+-- (-\u221e\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` < '2012-03-01 00:00:00';
+
+-- [2010-01-01 00:00:00\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00';
+
+-- [2010-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00'
+    AND `__time` < '2011-01-01 00:00:00';
+
+-- [2010-01-01 00:00:00\u20252011-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00';
+
+-- [2010-01-01 00:00:00\u20252011-01-01 00:00:00],[2012-01-01 00:00:00\u20252013-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+    OR (`__time` BETWEEN '2012-01-01 00:00:00' AND '2013-01-01 00:00:00');
+
+-- OVERLAP [2010-01-01 00:00:00\u20252012-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+    OR (`__time` BETWEEN '2010-06-01 00:00:00' AND '2012-01-01 00:00:00');
+
+-- IN: MULTIPLE INTERVALS [2010-01-01 00:00:00\u20252010-01-01 00:00:00),[2011-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00');
+
+EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' AND `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00');
+
+EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' OR `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00');

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientpositive/druid_timeseries.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druid_timeseries.q b/ql/src/test/queries/clientpositive/druid_timeseries.q
new file mode 100644
index 0000000..c0ad60f
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druid_timeseries.q
@@ -0,0 +1,94 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_1;
+
+-- GRANULARITY: ALL
+EXPLAIN
+SELECT max(added), sum(variation)
+FROM druid_table_1;
+
+-- GRANULARITY: NONE
+EXPLAIN
+SELECT `__time`, max(added), sum(variation)
+FROM druid_table_1
+GROUP BY `__time`;
+
+-- GRANULARITY: YEAR
+EXPLAIN
+SELECT floor_year(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_year(`__time`);
+
+-- GRANULARITY: QUARTER
+EXPLAIN
+SELECT floor_quarter(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_quarter(`__time`);
+
+-- GRANULARITY: MONTH
+EXPLAIN
+SELECT floor_month(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_month(`__time`);
+
+-- GRANULARITY: WEEK
+EXPLAIN
+SELECT floor_week(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_week(`__time`);
+
+-- GRANULARITY: DAY
+EXPLAIN
+SELECT floor_day(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_day(`__time`);
+
+-- GRANULARITY: HOUR
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_hour(`__time`);
+
+-- GRANULARITY: MINUTE
+EXPLAIN
+SELECT floor_minute(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_minute(`__time`);
+
+-- GRANULARITY: SECOND
+EXPLAIN
+SELECT floor_second(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_second(`__time`);
+
+-- WITH FILTER ON DIMENSION
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY floor_hour(`__time`);
+
+-- WITH FILTER ON TIME
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+    BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+        AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY floor_hour(`__time`);
+
+-- WITH FILTER ON TIME
+EXPLAIN
+SELECT subq.h, subq.m, subq.s
+FROM
+(
+  SELECT floor_hour(`__time`) as h, max(added) as m, sum(variation) as s
+  FROM druid_table_1
+  GROUP BY floor_hour(`__time`)
+) subq
+WHERE subq.h BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+        AND CAST('2014-01-01 00:00:00' AS TIMESTAMP);

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientpositive/druid_topn.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druid_topn.q b/ql/src/test/queries/clientpositive/druid_topn.q
new file mode 100644
index 0000000..b121b7e
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druid_topn.q
@@ -0,0 +1,75 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_1;
+
+-- GRANULARITY: ALL
+EXPLAIN
+SELECT robot, max(added) as m, sum(variation)
+FROM druid_table_1
+GROUP BY robot
+ORDER BY m DESC
+LIMIT 100;
+
+-- GRANULARITY: NONE
+EXPLAIN
+SELECT robot, `__time`, max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, `__time`
+ORDER BY s DESC
+LIMIT 100;
+
+-- GRANULARITY: YEAR
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s DESC
+LIMIT 10;
+
+-- ASC: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_month(`__time`)
+ORDER BY s
+LIMIT 10;
+
+-- MULTIPLE ORDER: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY s DESC, m DESC
+LIMIT 10;
+
+-- MULTIPLE ORDER MIXED: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY robot ASC, m DESC
+LIMIT 10;
+
+-- WITH FILTER ON DIMENSION: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s
+LIMIT 10;
+
+-- WITH FILTER ON TIME
+EXPLAIN
+SELECT robot, floor_hour(`__time`), max(added) as m, sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+    BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+        AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY robot, floor_hour(`__time`)
+ORDER BY m
+LIMIT 100;

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_address.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_address.q.out b/ql/src/test/results/clientnegative/druid_address.q.out
new file mode 100644
index 0000000..66b7e14
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_address.q.out
@@ -0,0 +1,7 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException Druid broker address not specified in configuration)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_buckets.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_buckets.q.out b/ql/src/test/results/clientnegative/druid_buckets.q.out
new file mode 100644
index 0000000..94e4f70
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_buckets.q.out
@@ -0,0 +1,8 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+CLUSTERED BY (robot) INTO 32 BUCKETS
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:CLUSTERED BY may not be specified for Druid)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_datasource.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_datasource.q.out b/ql/src/test/results/clientnegative/druid_datasource.q.out
new file mode 100644
index 0000000..177ffaa
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_datasource.q.out
@@ -0,0 +1,7 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("property" = "localhost")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException Druid data source not specified; use druid.datasource in table properties)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_external.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_external.q.out b/ql/src/test/results/clientnegative/druid_external.q.out
new file mode 100644
index 0000000..e5fac51
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_external.q.out
@@ -0,0 +1,7 @@
+PREHOOK: query: CREATE TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:Table in Druid needs to be declared as EXTERNAL)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_location.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_location.q.out b/ql/src/test/results/clientnegative/druid_location.q.out
new file mode 100644
index 0000000..5727e8c
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_location.q.out
@@ -0,0 +1,9 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+#### A masked pattern was here ####
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:LOCATION may not be specified for Druid)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_partitions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_partitions.q.out b/ql/src/test/results/clientnegative/druid_partitions.q.out
new file mode 100644
index 0000000..6fb55c1
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_partitions.q.out
@@ -0,0 +1,8 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+PARTITIONED BY (dt string)
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:PARTITIONED BY may not be specified for Druid)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/create_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view.q.out b/ql/src/test/results/clientpositive/create_view.q.out
index d9c1e11..7281185 100644
--- a/ql/src/test/results/clientpositive/create_view.q.out
+++ b/ql/src/test/results/clientpositive/create_view.q.out
@@ -190,6 +190,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (UDFToDouble(key) = 18.0) (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/druid_basic1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_basic1.q.out b/ql/src/test/results/clientpositive/druid_basic1.q.out
new file mode 100644
index 0000000..74ae9ed
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid_basic1.q.out
@@ -0,0 +1,142 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_1
+PREHOOK: query: DESCRIBE FORMATTED druid_table_1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_1
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_1
+# col_name            	data_type           	comment             
+	 	 
+__time              	timestamp           	from deserializer   
+robot               	string              	from deserializer   
+namespace           	string              	from deserializer   
+anonymous           	string              	from deserializer   
+unpatrolled         	string              	from deserializer   
+page                	string              	from deserializer   
+language            	string              	from deserializer   
+newpage             	string              	from deserializer   
+user                	string              	from deserializer   
+count               	float               	from deserializer   
+added               	float               	from deserializer   
+delta               	float               	from deserializer   
+variation           	float               	from deserializer   
+deleted             	float               	from deserializer   
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	EXTERNAL_TABLE      	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	EXTERNAL            	TRUE                
+	druid.datasource    	wikipedia           
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	storage_handler     	org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+	totalSize           	0                   
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.druid.QTestDruidSerDe	 
+InputFormat:        	null                	 
+OutputFormat:       	null                	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+PREHOOK: query: -- different table, same datasource
+CREATE EXTERNAL TABLE druid_table_2
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_2
+POSTHOOK: query: -- different table, same datasource
+CREATE EXTERNAL TABLE druid_table_2
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_2
+PREHOOK: query: DESCRIBE FORMATTED druid_table_2
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_2
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_2
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_2
+# col_name            	data_type           	comment             
+	 	 
+__time              	timestamp           	from deserializer   
+robot               	string              	from deserializer   
+namespace           	string              	from deserializer   
+anonymous           	string              	from deserializer   
+unpatrolled         	string              	from deserializer   
+page                	string              	from deserializer   
+language            	string              	from deserializer   
+newpage             	string              	from deserializer   
+user                	string              	from deserializer   
+count               	float               	from deserializer   
+added               	float               	from deserializer   
+delta               	float               	from deserializer   
+variation           	float               	from deserializer   
+deleted             	float               	from deserializer   
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	EXTERNAL_TABLE      	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	EXTERNAL            	TRUE                
+	druid.datasource    	wikipedia           
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	storage_handler     	org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+	totalSize           	0                   
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.druid.QTestDruidSerDe	 
+InputFormat:        	null                	 
+OutputFormat:       	null                	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+PREHOOK: query: DROP TABLE druid_table_2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@druid_table_2
+PREHOOK: Output: default@druid_table_2
+POSTHOOK: query: DROP TABLE druid_table_2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@druid_table_2
+POSTHOOK: Output: default@druid_table_2
+PREHOOK: query: DROP TABLE druid_table_1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@druid_table_1
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: DROP TABLE druid_table_1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@druid_table_1
+POSTHOOK: Output: default@druid_table_1


[08/31] hive git commit: HIVE-14217: Druid integration (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
new file mode 100644
index 0000000..0b87976
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
@@ -0,0 +1,106 @@
+/**
+ * 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.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.io.NullWritable;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.Iterators;
+
+import io.druid.query.Result;
+import io.druid.query.topn.DimensionAndMetricValueExtractor;
+import io.druid.query.topn.TopNQuery;
+import io.druid.query.topn.TopNResultValue;
+
+/**
+ * Record reader for results for Druid TopNQuery.
+ */
+public class DruidTopNQueryRecordReader
+        extends DruidQueryRecordReader<TopNQuery, Result<TopNResultValue>> {
+
+  private Result<TopNResultValue> current;
+  private Iterator<DimensionAndMetricValueExtractor> values = Iterators.emptyIterator();
+
+  @Override
+  protected TopNQuery createQuery(String content) throws IOException {
+    return DruidStorageHandlerUtils.JSON_MAPPER.readValue(content, TopNQuery.class);
+  }
+
+  @Override
+  protected List<Result<TopNResultValue>> createResultsList(InputStream content) throws IOException {
+    return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content,
+            new TypeReference<List<Result<TopNResultValue>>>(){});
+  }
+
+  @Override
+  public boolean nextKeyValue() {
+    if (values.hasNext()) {
+      return true;
+    }
+    if (results.hasNext()) {
+      current = results.next();
+      values = current.getValue().getValue().iterator();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public NullWritable getCurrentKey() throws IOException, InterruptedException {
+    return NullWritable.get();
+  }
+
+  @Override
+  public DruidWritable getCurrentValue() throws IOException, InterruptedException {
+    // Create new value
+    DruidWritable value = new DruidWritable();
+    value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+    if (values.hasNext()) {
+      value.getValue().putAll(values.next().getBaseObject());
+      return value;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean next(NullWritable key, DruidWritable value) {
+    if (nextKeyValue()) {
+      // Update value
+      value.getValue().clear();
+      value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+      if (values.hasNext()) {
+        value.getValue().putAll(values.next().getBaseObject());
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public float getProgress() {
+    return results.hasNext() || values.hasNext() ? 0 : 1;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java
new file mode 100644
index 0000000..77ffcd4
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java
@@ -0,0 +1,81 @@
+/**
+ * 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.druid.serde;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.io.Writable;
+
+import com.google.common.base.Objects;
+
+/**
+ * Writable for Druid results.
+ */
+public class DruidWritable implements Writable {
+
+  private final Map<String, Object> value;
+
+  public DruidWritable() {
+    value = new HashMap<>();
+  }
+
+  public DruidWritable(Map<String, Object> value) {
+    this.value = value;
+  }
+
+  public Map<String, Object> getValue() {
+    return value;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(value);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    return Objects.equal(value, ((DruidWritable) o).value);
+  }
+
+  @Override
+  public String toString() {
+    return "DruidWritable{value=" + value + '}';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java
new file mode 100644
index 0000000..2b4df78
--- /dev/null
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java
@@ -0,0 +1,88 @@
+/**
+ * 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.druid;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.druid.serde.DruidSerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.util.StringUtils;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+
+import io.druid.query.metadata.metadata.SegmentAnalysis;
+import io.druid.query.metadata.metadata.SegmentMetadataQuery;
+
+/**
+ * Druid SerDe to be used in tests.
+ */
+public class QTestDruidSerDe extends DruidSerDe {
+
+  // Request :
+  //        "{\"queryType\":\"segmentMetadata\",\"dataSource\":{\"type\":\"table\",\"name\":\"wikipedia\"},"
+  //        + "\"intervals\":{\"type\":\"intervals\","
+  //        + "\"intervals\":[\"-146136543-09-08T00:30:34.096-07:52:58/146140482-04-24T08:36:27.903-07:00\"]},"
+  //        + "\"toInclude\":{\"type\":\"all\"},\"merge\":true,\"context\":null,\"analysisTypes\":[],"
+  //        + "\"usingDefaultInterval\":true,\"lenientAggregatorMerge\":false,\"descending\":false}";
+  private static final String RESPONSE =
+          "[ {\r\n "
+          + " \"id\" : \"merged\",\r\n "
+          + " \"intervals\" : [ \"2010-01-01T00:00:00.000Z/2015-12-31T00:00:00.000Z\" ],\r\n "
+          + " \"columns\" : {\r\n  "
+          + "  \"__time\" : { \"type\" : \"LONG\", \"hasMultipleValues\" : false, \"size\" : 407240380, \"cardinality\" : null, \"errorMessage\" : null },\r\n  "
+          + "  \"robot\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n  "
+          + "  \"namespace\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : true, \"size\" : 100000, \"cardinality\" : 1504, \"errorMessage\" : null },\r\n  "
+          + "  \"anonymous\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n  "
+          + "  \"unpatrolled\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n  "
+          + "  \"page\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n  "
+          + "  \"language\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n  "
+          + "  \"newpage\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n  "
+          + "  \"user\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n  "
+          + "  \"count\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n  "
+          + "  \"added\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n  "
+          + "  \"delta\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n  "
+          + "  \"variation\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n  "
+          + "  \"deleted\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null }\r\n "
+          + " },\r\n "
+          + " \"aggregators\" : {\r\n  "
+          + "  \"count\" : { \"type\" : \"longSum\", \"name\" : \"count\", \"fieldName\" : \"count\" },\r\n  "
+          + "  \"added\" : { \"type\" : \"doubleSum\", \"name\" : \"added\", \"fieldName\" : \"added\" },\r\n  "
+          + "  \"delta\" : { \"type\" : \"doubleSum\", \"name\" : \"delta\", \"fieldName\" : \"delta\" },\r\n  "
+          + "  \"variation\" : { \"type\" : \"doubleSum\", \"name\" : \"variation\", \"fieldName\" : \"variation\" },\r\n  "
+          + "  \"deleted\" : { \"type\" : \"doubleSum\", \"name\" : \"deleted\", \"fieldName\" : \"deleted\" }\r\n "
+          + " },\r\n "
+          + " \"queryGranularity\" : {\r\n    \"type\": \"none\"\r\n  },\r\n "
+          + " \"size\" : 300000,\r\n "
+          + " \"numRows\" : 5000000\r\n} ]";
+
+  /* Submits the request and returns */
+  @Override
+  protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQuery query)
+          throws SerDeException {
+    // Retrieve results
+    List<SegmentAnalysis> resultsList;
+    try {
+      resultsList = DruidStorageHandlerUtils.JSON_MAPPER.readValue(RESPONSE,
+            new TypeReference<List<SegmentAnalysis>>() {});
+    } catch (Exception e) {
+      throw new SerDeException(StringUtils.stringifyException(e));
+    }
+    return resultsList.get(0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidStorageHandler.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidStorageHandler.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidStorageHandler.java
new file mode 100644
index 0000000..0a44aaa
--- /dev/null
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidStorageHandler.java
@@ -0,0 +1,34 @@
+/**
+ * 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.druid;
+
+import org.apache.hadoop.hive.serde2.SerDe;
+
+/**
+ * Storage handler for Druid to be used in tests. It cannot connect to
+ * Druid, and thus it cannot execute queries.
+ */
+@SuppressWarnings("deprecation")
+public class QTestDruidStorageHandler extends DruidStorageHandler {
+
+  @Override
+  public Class<? extends SerDe> getSerDeClass() {
+    return QTestDruidSerDe.class;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java
new file mode 100644
index 0000000..9c5c65c
--- /dev/null
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java
@@ -0,0 +1,576 @@
+/**
+ * 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.druid;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.sql.Timestamp;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.druid.serde.DruidGroupByQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidSelectQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidSerDe;
+import org.apache.hadoop.hive.druid.serde.DruidTimeseriesQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidTopNQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidWritable;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import io.druid.data.input.Row;
+import io.druid.jackson.DefaultObjectMapper;
+import io.druid.query.Query;
+import io.druid.query.Result;
+import io.druid.query.groupby.GroupByQuery;
+import io.druid.query.select.SelectQuery;
+import io.druid.query.select.SelectResultValue;
+import io.druid.query.timeseries.TimeseriesQuery;
+import io.druid.query.timeseries.TimeseriesResultValue;
+import io.druid.query.topn.TopNQuery;
+import io.druid.query.topn.TopNResultValue;
+
+/**
+ * Basic tests for Druid SerDe. The examples are taken from Druid 0.9.1.1
+ * documentation.
+ */
+public class TestDruidSerDe {
+
+  // Timeseries query
+  private static final String TIMESERIES_QUERY =
+          "{  \"queryType\": \"timeseries\", "
+          + " \"dataSource\": \"sample_datasource\", "
+          + " \"granularity\": \"day\", "
+          + " \"descending\": \"true\", "
+          + " \"filter\": {  "
+          + "  \"type\": \"and\",  "
+          + "  \"fields\": [   "
+          + "   { \"type\": \"selector\", \"dimension\": \"sample_dimension1\", \"value\": \"sample_value1\" },   "
+          + "   { \"type\": \"or\",    "
+          + "    \"fields\": [     "
+          + "     { \"type\": \"selector\", \"dimension\": \"sample_dimension2\", \"value\": \"sample_value2\" },     "
+          + "     { \"type\": \"selector\", \"dimension\": \"sample_dimension3\", \"value\": \"sample_value3\" }    "
+          + "    ]   "
+          + "   }  "
+          + "  ] "
+          + " }, "
+          + " \"aggregations\": [  "
+          + "  { \"type\": \"longSum\", \"name\": \"sample_name1\", \"fieldName\": \"sample_fieldName1\" },  "
+          + "  { \"type\": \"doubleSum\", \"name\": \"sample_name2\", \"fieldName\": \"sample_fieldName2\" } "
+          + " ], "
+          + " \"postAggregations\": [  "
+          + "  { \"type\": \"arithmetic\",  "
+          + "    \"name\": \"sample_divide\",  "
+          + "    \"fn\": \"/\",  "
+          + "    \"fields\": [   "
+          + "     { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name1\", \"fieldName\": \"sample_name1\" },   "
+          + "     { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name2\", \"fieldName\": \"sample_name2\" }  "
+          + "    ]  "
+          + "  } "
+          + " ], "
+          + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ]}";
+  // Timeseries query results
+  private static final String TIMESERIES_QUERY_RESULTS =
+          "[  "
+          + "{   "
+          + " \"timestamp\": \"2012-01-01T00:00:00.000Z\",   "
+          + " \"result\": { \"sample_name1\": 0, \"sample_name2\": 1.0, \"sample_divide\": 2.2222 }   "
+          + "},  "
+          + "{   "
+          + " \"timestamp\": \"2012-01-02T00:00:00.000Z\",   "
+          + " \"result\": { \"sample_name1\": 2, \"sample_name2\": 3.32, \"sample_divide\": 4 }  "
+          + "}]";
+  // Timeseries query results as records
+  private static final Object[][] TIMESERIES_QUERY_RESULTS_RECORDS = new Object[][] {
+    new Object[] { new TimestampWritable(new Timestamp(1325376000000L)), new LongWritable(0), new FloatWritable(1.0F), new FloatWritable(2.2222F) } ,
+    new Object[] { new TimestampWritable(new Timestamp(1325462400000L)), new LongWritable(2), new FloatWritable(3.32F), new FloatWritable(4F)}
+  };
+
+  // TopN query
+  private static final String TOPN_QUERY =
+          "{  \"queryType\": \"topN\", "
+          + " \"dataSource\": \"sample_data\", "
+          + " \"dimension\": \"sample_dim\", "
+          + " \"threshold\": 5, "
+          + " \"metric\": \"count\", "
+          + " \"granularity\": \"all\", "
+          + " \"filter\": {  "
+          + "  \"type\": \"and\",  "
+          + "  \"fields\": [   "
+          + "   {    "
+          + "    \"type\": \"selector\",    "
+          + "    \"dimension\": \"dim1\",    "
+          + "    \"value\": \"some_value\"   "
+          + "   },   "
+          + "   {    "
+          + "    \"type\": \"selector\",    "
+          + "    \"dimension\": \"dim2\",    "
+          + "    \"value\": \"some_other_val\"   "
+          + "   }  "
+          + "  ] "
+          + " }, "
+          + " \"aggregations\": [  "
+          + "  {   "
+          + "   \"type\": \"longSum\",   "
+          + "   \"name\": \"count\",   "
+          + "   \"fieldName\": \"count\"  "
+          + "  },  "
+          + "  {   "
+          + "   \"type\": \"doubleSum\",   "
+          + "   \"name\": \"some_metric\",   "
+          + "   \"fieldName\": \"some_metric\"  "
+          + "  } "
+          + " ], "
+          + " \"postAggregations\": [  "
+          + "  {   "
+          + "   \"type\": \"arithmetic\",   "
+          + "   \"name\": \"sample_divide\",   "
+          + "   \"fn\": \"/\",   "
+          + "   \"fields\": [    "
+          + "    {     "
+          + "     \"type\": \"fieldAccess\",     "
+          + "     \"name\": \"some_metric\",     "
+          + "     \"fieldName\": \"some_metric\"    "
+          + "    },    "
+          + "    {     "
+          + "     \"type\": \"fieldAccess\",     "
+          + "     \"name\": \"count\",     "
+          + "     \"fieldName\": \"count\"    "
+          + "    }   "
+          + "   ]  "
+          + "  } "
+          + " ], "
+          + " \"intervals\": [  "
+          + "  \"2013-08-31T00:00:00.000/2013-09-03T00:00:00.000\" "
+          + " ]}";
+  // TopN query results
+  private static final String TOPN_QUERY_RESULTS =
+          "[ "
+          + " {  "
+          + "  \"timestamp\": \"2013-08-31T00:00:00.000Z\",  "
+          + "  \"result\": [   "
+          + "   {   "
+          + "     \"sample_dim\": \"dim1_val\",   "
+          + "     \"count\": 111,   "
+          + "     \"some_metric\": 10669,   "
+          + "     \"sample_divide\": 96.11711711711712   "
+          + "   },   "
+          + "   {   "
+          + "     \"sample_dim\": \"another_dim1_val\",   "
+          + "     \"count\": 88,   "
+          + "     \"some_metric\": 28344,   "
+          + "     \"sample_divide\": 322.09090909090907   "
+          + "   },   "
+          + "   {   "
+          + "     \"sample_dim\": \"dim1_val3\",   "
+          + "     \"count\": 70,   "
+          + "     \"some_metric\": 871,   "
+          + "     \"sample_divide\": 12.442857142857143   "
+          + "   },   "
+          + "   {   "
+          + "     \"sample_dim\": \"dim1_val4\",   "
+          + "     \"count\": 62,   "
+          + "     \"some_metric\": 815,   "
+          + "     \"sample_divide\": 13.14516129032258   "
+          + "   },   "
+          + "   {   "
+          + "     \"sample_dim\": \"dim1_val5\",   "
+          + "     \"count\": 60,   "
+          + "     \"some_metric\": 2787,   "
+          + "     \"sample_divide\": 46.45   "
+          + "   }  "
+          + "  ] "
+          + " }]";
+  // TopN query results as records
+  private static final Object[][] TOPN_QUERY_RESULTS_RECORDS = new Object[][] {
+    new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val"), new LongWritable(111), new FloatWritable(10669F), new FloatWritable(96.11711711711712F) } ,
+    new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("another_dim1_val"), new LongWritable(88), new FloatWritable(28344F), new FloatWritable(322.09090909090907F) } ,
+    new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val3"), new LongWritable(70), new FloatWritable(871F), new FloatWritable(12.442857142857143F) } ,
+    new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val4"), new LongWritable(62), new FloatWritable(815F), new FloatWritable(13.14516129032258F) } ,
+    new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val5"), new LongWritable(60), new FloatWritable(2787F), new FloatWritable(46.45F) }
+  };
+
+  // GroupBy query
+  private static final String GROUP_BY_QUERY =
+          "{ "
+          + " \"queryType\": \"groupBy\", "
+          + " \"dataSource\": \"sample_datasource\", "
+          + " \"granularity\": \"day\", "
+          + " \"dimensions\": [\"country\", \"device\"], "
+          + " \"limitSpec\": {"
+          + " \"type\": \"default\","
+          + " \"limit\": 5000,"
+          + " \"columns\": [\"country\", \"data_transfer\"] }, "
+          + " \"filter\": {  "
+          + "  \"type\": \"and\",  "
+          + "  \"fields\": [   "
+          + "   { \"type\": \"selector\", \"dimension\": \"carrier\", \"value\": \"AT&T\" },   "
+          + "   { \"type\": \"or\",     "
+          + "    \"fields\": [     "
+          + "     { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Apple\" },     "
+          + "     { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Samsung\" }    "
+          + "    ]   "
+          + "   }  "
+          + "  ] "
+          + " }, "
+          + " \"aggregations\": [  "
+          + "  { \"type\": \"longSum\", \"name\": \"total_usage\", \"fieldName\": \"user_count\" },  "
+          + "  { \"type\": \"doubleSum\", \"name\": \"data_transfer\", \"fieldName\": \"data_transfer\" } "
+          + " ], "
+          + " \"postAggregations\": [  "
+          + "  { \"type\": \"arithmetic\",  "
+          + "    \"name\": \"avg_usage\",  "
+          + "    \"fn\": \"/\",  "
+          + "    \"fields\": [   "
+          + "     { \"type\": \"fieldAccess\", \"fieldName\": \"data_transfer\" },   "
+          + "     { \"type\": \"fieldAccess\", \"fieldName\": \"total_usage\" }  "
+          + "    ]  "
+          + "  } "
+          + " ], "
+          + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ], "
+          + " \"having\": {  "
+          + "  \"type\": \"greaterThan\",  "
+          + "  \"aggregation\": \"total_usage\",  "
+          + "  \"value\": 100 "
+          + " }}";
+  // GroupBy query results
+  private static final String GROUP_BY_QUERY_RESULTS =
+          "[  "
+          + " {  "
+          + "  \"version\" : \"v1\",  "
+          + "  \"timestamp\" : \"2012-01-01T00:00:00.000Z\",  "
+          + "  \"event\" : {   "
+          + "   \"country\" : \"India\",   "
+          + "   \"device\" : \"phone\",   "
+          + "   \"total_usage\" : 88,   "
+          + "   \"data_transfer\" : 29.91233453,   "
+          + "   \"avg_usage\" : 60.32  "
+          + "  } "
+          + " },  "
+          + " {  "
+          + "  \"version\" : \"v1\",  "
+          + "  \"timestamp\" : \"2012-01-01T00:00:12.000Z\",  "
+          + "  \"event\" : {   "
+          + "   \"country\" : \"Spain\",   "
+          + "   \"device\" : \"pc\",   "
+          + "   \"total_usage\" : 16,   "
+          + "   \"data_transfer\" : 172.93494959,   "
+          + "   \"avg_usage\" : 6.333333  "
+          + "  } "
+          + " }]";
+  // GroupBy query results as records
+  private static final Object[][] GROUP_BY_QUERY_RESULTS_RECORDS = new Object[][] {
+    new Object[] { new TimestampWritable(new Timestamp(1325376000000L)), new Text("India"), new Text("phone"), new LongWritable(88), new FloatWritable(29.91233453F), new FloatWritable(60.32F) } ,
+    new Object[] { new TimestampWritable(new Timestamp(1325376012000L)), new Text("Spain"), new Text("pc"), new LongWritable(16), new FloatWritable(172.93494959F), new FloatWritable(6.333333F) }
+  };
+
+  // Select query
+  private static final String SELECT_QUERY =
+          "{   \"queryType\": \"select\",  "
+          + " \"dataSource\": \"wikipedia\",   \"descending\": \"false\",  "
+          + " \"dimensions\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\",\"newpage\",\"user\"],  "
+          + " \"metrics\":[\"count\",\"added\",\"delta\",\"variation\",\"deleted\"],  "
+          + " \"granularity\": \"all\",  "
+          + " \"intervals\": [     \"2013-01-01/2013-01-02\"   ],  "
+          + " \"pagingSpec\":{\"pagingIdentifiers\": {}, \"threshold\":5} }";
+  // Select query results
+  private static final String SELECT_QUERY_RESULTS =
+          "[{ "
+          + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", "
+          + " \"result\" : {  "
+          + "  \"pagingIdentifiers\" : {   "
+          + "   \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\" : 4    }, "
+          + "   \"events\" : [ {  "
+          + "    \"segmentId\" : \"wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\",  "
+          + "    \"offset\" : 0,  "
+          + "    \"event\" : {   "
+          + "     \"timestamp\" : \"2013-01-01T00:00:00.000Z\",   "
+          + "     \"robot\" : \"1\",   "
+          + "     \"namespace\" : \"article\",   "
+          + "     \"anonymous\" : \"0\",   "
+          + "     \"unpatrolled\" : \"0\",   "
+          + "     \"page\" : \"11._korpus_(NOVJ)\",   "
+          + "     \"language\" : \"sl\",   "
+          + "     \"newpage\" : \"0\",   "
+          + "     \"user\" : \"EmausBot\",   "
+          + "     \"count\" : 1.0,   "
+          + "     \"added\" : 39.0,   "
+          + "     \"delta\" : 39.0,   "
+          + "     \"variation\" : 39.0,   "
+          + "     \"deleted\" : 0.0  "
+          + "    } "
+          + "   }, {  "
+          + "    \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\",  "
+          + "    \"offset\" : 1,  "
+          + "    \"event\" : {   "
+          + "     \"timestamp\" : \"2013-01-01T00:00:00.000Z\",   "
+          + "     \"robot\" : \"0\",   "
+          + "     \"namespace\" : \"article\",   "
+          + "     \"anonymous\" : \"0\",   "
+          + "     \"unpatrolled\" : \"0\",   "
+          + "     \"page\" : \"112_U.S._580\",   "
+          + "     \"language\" : \"en\",   "
+          + "     \"newpage\" : \"1\",   "
+          + "     \"user\" : \"MZMcBride\",   "
+          + "     \"count\" : 1.0,   "
+          + "     \"added\" : 70.0,   "
+          + "     \"delta\" : 70.0,   "
+          + "     \"variation\" : 70.0,   "
+          + "     \"deleted\" : 0.0  "
+          + "    } "
+          + "   }, {  "
+          + "    \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\",  "
+          + "    \"offset\" : 2,  "
+          + "    \"event\" : {   "
+          + "     \"timestamp\" : \"2013-01-01T00:00:00.000Z\",   "
+          + "     \"robot\" : \"0\",   "
+          + "     \"namespace\" : \"article\",   "
+          + "     \"anonymous\" : \"0\",   "
+          + "     \"unpatrolled\" : \"0\",   "
+          + "     \"page\" : \"113_U.S._243\",   "
+          + "     \"language\" : \"en\",   "
+          + "     \"newpage\" : \"1\",   "
+          + "     \"user\" : \"MZMcBride\",   "
+          + "     \"count\" : 1.0,   "
+          + "     \"added\" : 77.0,   "
+          + "     \"delta\" : 77.0,   "
+          + "     \"variation\" : 77.0,   "
+          + "     \"deleted\" : 0.0  "
+          + "    } "
+          + "   }, {  "
+          + "    \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\",  "
+          + "    \"offset\" : 3,  "
+          + "    \"event\" : {   "
+          + "     \"timestamp\" : \"2013-01-01T00:00:00.000Z\",   "
+          + "     \"robot\" : \"0\",   "
+          + "     \"namespace\" : \"article\",   "
+          + "     \"anonymous\" : \"0\",   "
+          + "     \"unpatrolled\" : \"0\",   "
+          + "     \"page\" : \"113_U.S._73\",   "
+          + "     \"language\" : \"en\",   "
+          + "     \"newpage\" : \"1\",   "
+          + "     \"user\" : \"MZMcBride\",   "
+          + "     \"count\" : 1.0,   "
+          + "     \"added\" : 70.0,   "
+          + "     \"delta\" : 70.0,   "
+          + "     \"variation\" : 70.0,   "
+          + "     \"deleted\" : 0.0  "
+          + "    } "
+          + "   }, {  "
+          + "    \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\",  "
+          + "    \"offset\" : 4,  "
+          + "    \"event\" : {   "
+          + "     \"timestamp\" : \"2013-01-01T00:00:00.000Z\",   "
+          + "     \"robot\" : \"0\",   "
+          + "     \"namespace\" : \"article\",   "
+          + "     \"anonymous\" : \"0\",   "
+          + "     \"unpatrolled\" : \"0\",   "
+          + "     \"page\" : \"113_U.S._756\",   "
+          + "     \"language\" : \"en\",   "
+          + "     \"newpage\" : \"1\",   "
+          + "     \"user\" : \"MZMcBride\",   "
+          + "     \"count\" : 1.0,   "
+          + "     \"added\" : 68.0,   "
+          + "     \"delta\" : 68.0,   "
+          + "     \"variation\" : 68.0,   "
+          + "     \"deleted\" : 0.0  "
+          + "    } "
+          + "   } ]  }} ]";
+  // Select query results as records
+  private static final Object[][] SELECT_QUERY_RESULTS_RECORDS = new Object[][] {
+    new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("1"), new Text("article"), new Text("0"), new Text("0"),
+        new Text("11._korpus_(NOVJ)"), new Text("sl"), new Text("0"), new Text("EmausBot"),
+        new FloatWritable(1.0F), new FloatWritable(39.0F), new FloatWritable(39.0F), new FloatWritable(39.0F), new FloatWritable(0.0F) } ,
+    new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"),
+        new Text("112_U.S._580"), new Text("en"), new Text("1"), new Text("MZMcBride"),
+        new FloatWritable(1.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(0.0F) } ,
+    new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"),
+        new Text("113_U.S._243"), new Text("en"), new Text("1"), new Text("MZMcBride"),
+        new FloatWritable(1.0F), new FloatWritable(77.0F), new FloatWritable(77.0F), new FloatWritable(77.0F), new FloatWritable(0.0F) } ,
+    new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"),
+        new Text("113_U.S._73"), new Text("en"), new Text("1"), new Text("MZMcBride"),
+        new FloatWritable(1.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(0.0F) } ,
+    new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"),
+        new Text("113_U.S._756"), new Text("en"), new Text("1"), new Text("MZMcBride"),
+        new FloatWritable(1.0F), new FloatWritable(68.0F), new FloatWritable(68.0F), new FloatWritable(68.0F), new FloatWritable(0.0F) }
+  };
+
+
+  /**
+   * Test the default behavior of the objects and object inspectors.
+   * @throws IOException 
+   * @throws IllegalAccessException 
+   * @throws IllegalArgumentException 
+   * @throws SecurityException 
+   * @throws NoSuchFieldException 
+   * @throws JsonMappingException 
+   * @throws JsonParseException 
+   * @throws InvocationTargetException 
+   * @throws NoSuchMethodException 
+   */
+  @Test
+  public void testDruidSerDe()
+          throws SerDeException, JsonParseException, JsonMappingException,
+          NoSuchFieldException, SecurityException, IllegalArgumentException,
+          IllegalAccessException, IOException, InterruptedException,
+          NoSuchMethodException, InvocationTargetException {
+    // Create, initialize, and test the SerDe
+    DruidSerDe serDe = new DruidSerDe();
+    Configuration conf = new Configuration();
+    Properties tbl;
+    // Timeseries query
+    tbl = createPropertiesQuery("sample_datasource", Query.TIMESERIES, TIMESERIES_QUERY);
+    SerDeUtils.initializeSerDe(serDe, conf, tbl, null);
+    deserializeQueryResults(serDe, Query.TIMESERIES, TIMESERIES_QUERY,
+            TIMESERIES_QUERY_RESULTS, TIMESERIES_QUERY_RESULTS_RECORDS);
+    // TopN query
+    tbl = createPropertiesQuery("sample_data", Query.TOPN, TOPN_QUERY);
+    SerDeUtils.initializeSerDe(serDe, conf, tbl, null);
+    deserializeQueryResults(serDe, Query.TOPN, TOPN_QUERY,
+            TOPN_QUERY_RESULTS, TOPN_QUERY_RESULTS_RECORDS);
+    // GroupBy query
+    tbl = createPropertiesQuery("sample_datasource", Query.GROUP_BY, GROUP_BY_QUERY);
+    SerDeUtils.initializeSerDe(serDe, conf, tbl, null);
+    deserializeQueryResults(serDe, Query.GROUP_BY, GROUP_BY_QUERY,
+            GROUP_BY_QUERY_RESULTS, GROUP_BY_QUERY_RESULTS_RECORDS);
+    // Select query
+    tbl = createPropertiesQuery("wikipedia", Query.SELECT, SELECT_QUERY);
+    SerDeUtils.initializeSerDe(serDe, conf, tbl, null);
+    deserializeQueryResults(serDe, Query.SELECT, SELECT_QUERY,
+            SELECT_QUERY_RESULTS, SELECT_QUERY_RESULTS_RECORDS);
+  }
+
+  private static Properties createPropertiesQuery(String dataSource, String queryType, String jsonQuery) {
+    Properties tbl = new Properties();
+
+    // Set the configuration parameters
+    tbl.setProperty(Constants.DRUID_DATA_SOURCE, dataSource);
+    tbl.setProperty(Constants.DRUID_QUERY_JSON, jsonQuery);
+    tbl.setProperty(Constants.DRUID_QUERY_TYPE, queryType);
+    return tbl;
+  }
+
+  private static void deserializeQueryResults(DruidSerDe serDe, String queryType, String jsonQuery,
+          String resultString, Object[][] records) throws SerDeException, JsonParseException,
+          JsonMappingException, IOException, NoSuchFieldException, SecurityException,
+          IllegalArgumentException, IllegalAccessException, InterruptedException,
+          NoSuchMethodException, InvocationTargetException {
+
+    // Initialize
+    Query<?> query = null;
+    DruidQueryRecordReader<?,?> reader = null;
+    List<?> resultsList = null;
+    ObjectMapper mapper = new DefaultObjectMapper();
+    switch (queryType) {
+      case Query.TIMESERIES:
+        query = mapper.readValue(jsonQuery, TimeseriesQuery.class);
+        reader = new DruidTimeseriesQueryRecordReader();
+        resultsList = mapper.readValue(resultString,
+                new TypeReference<List<Result<TimeseriesResultValue>>>() {});
+        break;
+      case Query.TOPN:
+        query = mapper.readValue(jsonQuery, TopNQuery.class);
+        reader = new DruidTopNQueryRecordReader();
+        resultsList = mapper.readValue(resultString,
+                new TypeReference<List<Result<TopNResultValue>>>() {});
+        break;
+      case Query.GROUP_BY:
+        query = mapper.readValue(jsonQuery, GroupByQuery.class);
+        reader = new DruidGroupByQueryRecordReader();
+        resultsList = mapper.readValue(resultString,
+                new TypeReference<List<Row>>() {});
+        break;
+      case Query.SELECT:
+        query = mapper.readValue(jsonQuery, SelectQuery.class);
+        reader = new DruidSelectQueryRecordReader();
+        resultsList = mapper.readValue(resultString,
+                new TypeReference<List<Result<SelectResultValue>>>() {});
+        break;
+    }
+
+    // Set query and fields access
+    Field field1 = DruidQueryRecordReader.class.getDeclaredField("query");
+    field1.setAccessible(true);
+    field1.set(reader, query);
+    if (reader instanceof DruidGroupByQueryRecordReader) {
+      Method method1 = DruidGroupByQueryRecordReader.class.getDeclaredMethod("initExtractors");
+      method1.setAccessible(true);
+      method1.invoke(reader);
+    }
+    Field field2 = DruidQueryRecordReader.class.getDeclaredField("results");
+    field2.setAccessible(true);
+    
+    // Get the row structure
+    StructObjectInspector oi = (StructObjectInspector) serDe.getObjectInspector();
+    List<? extends StructField> fieldRefs = oi.getAllStructFieldRefs();
+
+    // Check mapred
+    Iterator<?> results = resultsList.iterator();
+    field2.set(reader, results);
+    DruidWritable writable = new DruidWritable();
+    int pos = 0;
+    while (reader.next(NullWritable.get(), writable)) {
+      Object row = serDe.deserialize(writable);
+      Object[] expectedFieldsData = records[pos];
+      assertEquals(expectedFieldsData.length, fieldRefs.size());
+      for (int i = 0; i < fieldRefs.size(); i++) {
+        Object fieldData = oi.getStructFieldData(row, fieldRefs.get(i));
+        assertEquals("Field " + i, expectedFieldsData[i], fieldData);
+      }
+      pos++;
+    }
+    assertEquals(pos, records.length);
+
+    // Check mapreduce
+    results = resultsList.iterator();
+    field2.set(reader, results);
+    pos = 0;
+    while (reader.nextKeyValue()) {
+      Object row = serDe.deserialize(reader.getCurrentValue());
+      Object[] expectedFieldsData = records[pos];
+      assertEquals(expectedFieldsData.length, fieldRefs.size());
+      for (int i = 0; i < fieldRefs.size(); i++) {
+        Object fieldData = oi.getStructFieldData(row, fieldRefs.get(i));
+        assertEquals("Field " + i, expectedFieldsData[i], fieldData);
+      }
+      pos++;
+    }
+    assertEquals(pos, records.length);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
new file mode 100644
index 0000000..b20168d
--- /dev/null
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.druid;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.joda.time.Interval;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestHiveDruidQueryBasedInputFormat extends TestCase {
+  
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testCreateSplitsIntervals() throws Exception {
+    HiveDruidQueryBasedInputFormat input = new HiveDruidQueryBasedInputFormat();
+
+    Method method1 = HiveDruidQueryBasedInputFormat.class.getDeclaredMethod("createSplitsIntervals",
+            List.class, int.class);
+    method1.setAccessible(true);
+
+    List<Interval> intervals;
+    List<List<Interval>> resultList;
+    List<List<Interval>> expectedResultList;
+
+    // Test 1 : single split, create 4
+    intervals = new ArrayList<>();
+    intervals.add(new Interval(1262304000000L, 1293840000000L));
+    resultList = (List<List<Interval>>) method1.invoke(input, intervals, 4);
+    expectedResultList = new ArrayList<>();
+    expectedResultList.add(Arrays.asList(new Interval(1262304000000L, 1270188000000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1270188000000L, 1278072000000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1278072000000L, 1285956000000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1285956000000L, 1293840000000L)));
+    assertEquals(expectedResultList, resultList);
+
+    // Test 2 : two splits, create 4
+    intervals = new ArrayList<>();
+    intervals.add(new Interval(1262304000000L, 1293840000000L));
+    intervals.add(new Interval(1325376000000L, 1356998400000L));
+    resultList = (List<List<Interval>>) method1.invoke(input, intervals, 4);
+    expectedResultList = new ArrayList<>();
+    expectedResultList.add(Arrays.asList(new Interval(1262304000000L, 1278093600000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1278093600000L, 1293840000000L),
+            new Interval(1325376000000L, 1325419200000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1325419200000L, 1341208800000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1341208800000L, 1356998400000L)));
+    assertEquals(expectedResultList, resultList);
+
+    // Test 3 : two splits, create 5
+    intervals = new ArrayList<>();
+    intervals.add(new Interval(1262304000000L, 1293840000000L));
+    intervals.add(new Interval(1325376000000L, 1356998400000L));
+    resultList = (List<List<Interval>>) method1.invoke(input, intervals, 5);
+    expectedResultList = new ArrayList<>();
+    expectedResultList.add(Arrays.asList(new Interval(1262304000000L, 1274935680000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1274935680000L, 1287567360000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1287567360000L, 1293840000000L),
+            new Interval(1325376000000L, 1331735040000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1331735040000L, 1344366720000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1344366720000L, 1356998400000L)));
+    assertEquals(expectedResultList, resultList);
+
+    // Test 4 : three splits, different ranges, create 6
+    intervals = new ArrayList<>();
+    intervals.add(new Interval(1199145600000L, 1201824000000L)); // one month
+    intervals.add(new Interval(1325376000000L, 1356998400000L)); // one year
+    intervals.add(new Interval(1407283200000L, 1407888000000L)); // 7 days
+    resultList = (List<List<Interval>>) method1.invoke(input, intervals, 6);
+    expectedResultList = new ArrayList<>();
+    expectedResultList.add(Arrays.asList(new Interval(1199145600000L, 1201824000000L),
+            new Interval(1325376000000L, 1328515200000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1328515200000L, 1334332800000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1334332800000L, 1340150400000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1340150400000L, 1345968000000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1345968000000L, 1351785600000L)));
+    expectedResultList.add(Arrays.asList(new Interval(1351785600000L, 1356998400000L),
+            new Interval(1407283200000L, 1407888000000L)));
+    assertEquals(expectedResultList, resultList);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index 7fc72b9..e762d0e 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -110,6 +110,19 @@
       <version>${hadoop.version}</version>
       <optional>true</optional>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-druid-handler</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-druid-handler</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
 
     <!-- test inter-project -->
     <dependency>

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/packaging/pom.xml
----------------------------------------------------------------------
diff --git a/packaging/pom.xml b/packaging/pom.xml
index 679dfe8..76e0cff 100644
--- a/packaging/pom.xml
+++ b/packaging/pom.xml
@@ -210,6 +210,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-druid-handler</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-hwi</artifactId>
       <version>${project.version}</version>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4c41200..2fb78cd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -37,6 +37,7 @@
     <module>cli</module>
     <module>common</module>
     <module>contrib</module>
+    <module>druid-handler</module>
     <module>hbase-handler</module>
     <module>hcatalog</module>
     <module>hplsql</module>
@@ -130,6 +131,7 @@
     <derby.version>10.10.2.0</derby.version>
     <dropwizard.version>3.1.0</dropwizard.version>
     <dropwizard-metrics-hadoop-metrics2-reporter.version>0.1.2</dropwizard-metrics-hadoop-metrics2-reporter.version>
+    <druid.version>0.9.1.1</druid.version>
     <guava.version>14.0.1</guava.version>
     <groovy.version>2.4.4</groovy.version>
     <hadoop.version>2.7.2</hadoop.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/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 5722305..66cbdd3 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
@@ -60,6 +60,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.io.HdfsUtils;
@@ -4468,12 +4469,13 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
   }
 
   public static boolean doesTableNeedLocation(Table tbl) {
-    // If we are ok with breaking compatibility of existing 3rd party StorageHandlers,
+    // TODO: If we are ok with breaking compatibility of existing 3rd party StorageHandlers,
     // this method could be moved to the HiveStorageHandler interface.
     boolean retval = true;
     if (tbl.getStorageHandler() != null) {
-      retval = !tbl.getStorageHandler().toString().equals(
-          "org.apache.hadoop.hive.hbase.HBaseStorageHandler");
+      String sh = tbl.getStorageHandler().toString();
+      retval = !sh.equals("org.apache.hadoop.hive.hbase.HBaseStorageHandler")
+              && !sh.equals(Constants.DRUID_HIVE_STORAGE_HANDLER_ID);
     }
     return retval;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 60646ba..4710b8f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -31,14 +31,20 @@ import java.util.TreeSet;
 import java.util.regex.Pattern;
 import java.util.regex.PatternSyntaxException;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionResource;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorDay;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorHour;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorMinute;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorMonth;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorQuarter;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorSecond;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorWeek;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorYear;
 import org.apache.hadoop.hive.ql.udf.SettableUDF;
 import org.apache.hadoop.hive.ql.udf.UDAFPercentile;
 import org.apache.hadoop.hive.ql.udf.UDFAcos;
@@ -141,6 +147,8 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hive.common.util.AnnotationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * FunctionRegistry.
@@ -289,6 +297,16 @@ public final class FunctionRegistry {
     system.registerGenericUDF("trunc", GenericUDFTrunc.class);
     system.registerGenericUDF("date_format", GenericUDFDateFormat.class);
 
+    // Special date formatting functions
+    system.registerUDF("floor_year", UDFDateFloorYear.class, false);
+    system.registerUDF("floor_quarter", UDFDateFloorQuarter.class, false);
+    system.registerUDF("floor_month", UDFDateFloorMonth.class, false);
+    system.registerUDF("floor_day", UDFDateFloorDay.class, false);
+    system.registerUDF("floor_week", UDFDateFloorWeek.class, false);
+    system.registerUDF("floor_hour", UDFDateFloorHour.class, false);
+    system.registerUDF("floor_minute", UDFDateFloorMinute.class, false);
+    system.registerUDF("floor_second", UDFDateFloorSecond.class, false);
+
     system.registerGenericUDF("date_add", GenericUDFDateAdd.class);
     system.registerGenericUDF("date_sub", GenericUDFDateSub.class);
     system.registerGenericUDF("datediff", GenericUDFDateDiff.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
index aeb4e7d..890aea1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
@@ -19,21 +19,28 @@ package org.apache.hadoop.hive.ql.optimizer.calcite;
 
 import org.apache.calcite.plan.Context;
 import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveAlgorithmsConf;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.HiveDruidConf;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRulesRegistry;
 
 
 public class HivePlannerContext implements Context {
-  private HiveAlgorithmsConf config;
+  private HiveAlgorithmsConf algoConfig;
+  private HiveDruidConf druidConf;
   private HiveRulesRegistry registry;
 
-  public HivePlannerContext(HiveAlgorithmsConf config, HiveRulesRegistry registry) {
-    this.config = config;
+  public HivePlannerContext(HiveAlgorithmsConf algoConfig, HiveDruidConf druidConf,
+          HiveRulesRegistry registry) {
+    this.algoConfig = algoConfig;
+    this.druidConf = druidConf;
     this.registry = registry;
   }
 
   public <T> T unwrap(Class<T> clazz) {
-    if (clazz.isInstance(config)) {
-      return clazz.cast(config);
+    if (clazz.isInstance(algoConfig)) {
+      return clazz.cast(algoConfig);
+    }
+    if (clazz.isInstance(druidConf)) {
+      return clazz.cast(druidConf);
     }
     if (clazz.isInstance(registry)) {
       return clazz.cast(registry);

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java
new file mode 100644
index 0000000..82ab4d7
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java
@@ -0,0 +1,466 @@
+/**
+ * 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.druid;
+
+import java.sql.Timestamp;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.TreeSet;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.commons.lang.StringUtils;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+import com.google.common.collect.BoundType;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import com.google.common.collect.Sets;
+
+/** 
+ * Utilities for generating intervals from RexNode.
+ * 
+ * Based on Navis logic implemented on Hive data structures.
+ * See <a href="https://github.com/druid-io/druid/pull/2880">Druid PR-2880</a>
+ * 
+ */
+@SuppressWarnings({"rawtypes","unchecked"})
+public class DruidIntervalUtils {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(DruidIntervalUtils.class);
+
+
+  /**
+   * Given a list of predicates, it generates the equivalent Interval
+   * (if possible). It assumes that all the predicates in the input
+   * reference a single column : the timestamp column.
+   * 
+   * @param conjs list of conditions to use for the transformation
+   * @return interval representing the conditions in the input list
+   */
+  public static List<Interval> createInterval(RelDataType type, List<RexNode> conjs) {
+    List<Range> ranges = new ArrayList<>();
+    for (RexNode child : conjs) {
+      List<Range> extractedRanges = extractRanges(type, child, false);
+      if (extractedRanges == null || extractedRanges.isEmpty()) {
+        // We could not extract, we bail out
+        return null;
+      }
+      if (ranges.isEmpty()) {
+        ranges.addAll(extractedRanges);
+        continue;
+      }
+      List<Range> overlapped = Lists.newArrayList();
+      for (Range current : ranges) {
+        for (Range interval : extractedRanges) {
+          if (current.isConnected(interval)) {
+            overlapped.add(current.intersection(interval));
+          }
+        }
+      }
+      ranges = overlapped;
+    }
+    List<Range> compactRanges = condenseRanges(ranges);
+    LOG.debug("Inferred ranges on interval : " + compactRanges);
+    return toInterval(compactRanges);
+  }
+
+  protected static List<Interval> toInterval(List<Range> ranges) {
+    List<Interval> intervals = Lists.transform(ranges, new Function<Range, Interval>() {
+      @Override
+      public Interval apply(Range range) {
+        if (!range.hasLowerBound() && !range.hasUpperBound()) {
+          return DruidTable.DEFAULT_INTERVAL;
+        }
+        long start = range.hasLowerBound() ? toLong(range.lowerEndpoint()) :
+          DruidTable.DEFAULT_INTERVAL.getStartMillis();
+        long end = range.hasUpperBound() ? toLong(range.upperEndpoint()) :
+          DruidTable.DEFAULT_INTERVAL.getEndMillis();
+        if (range.hasLowerBound() && range.lowerBoundType() == BoundType.OPEN) {
+          start++;
+        }
+        if (range.hasUpperBound() && range.upperBoundType() == BoundType.CLOSED) {
+          end++;
+        }
+        return new Interval(start, end);
+      }
+    });
+    LOG.info("Converted time ranges " + ranges + " to interval " + intervals);
+    return intervals;
+  }
+
+  protected static List<Range> extractRanges(RelDataType type, RexNode node,
+          boolean withNot) {
+    switch (node.getKind()) {
+      case EQUALS:
+      case LESS_THAN:
+      case LESS_THAN_OR_EQUAL:
+      case GREATER_THAN:
+      case GREATER_THAN_OR_EQUAL:
+      case BETWEEN:
+      case IN:
+        return leafToRanges(type, (RexCall) node, withNot);
+
+      case NOT:
+        return extractRanges(type, ((RexCall) node).getOperands().get(0), !withNot);
+
+      case OR:
+        RexCall call = (RexCall) node;
+        List<Range> intervals = Lists.newArrayList();
+        for (RexNode child : call.getOperands()) {
+          List<Range> extracted = extractRanges(type, child, withNot);
+          if (extracted != null) {
+            intervals.addAll(extracted);
+          }
+        }
+        return intervals;
+
+      default:
+        return null;
+    }
+  }
+
+  protected static List<Range> leafToRanges(RelDataType type, RexCall call,
+          boolean withNot) {
+    switch (call.getKind()) {
+      case EQUALS:
+      case LESS_THAN:
+      case LESS_THAN_OR_EQUAL:
+      case GREATER_THAN:
+      case GREATER_THAN_OR_EQUAL:
+      {
+        RexLiteral literal = null;
+        if (call.getOperands().get(0) instanceof RexInputRef &&
+                call.getOperands().get(1) instanceof RexLiteral) {
+          literal = extractLiteral(call.getOperands().get(1));
+        } else if (call.getOperands().get(0) instanceof RexInputRef &&
+                call.getOperands().get(1).getKind() == SqlKind.CAST) {
+          literal = extractLiteral(call.getOperands().get(1));
+        } else if (call.getOperands().get(1) instanceof RexInputRef &&
+                call.getOperands().get(0) instanceof RexLiteral) {
+          literal = extractLiteral(call.getOperands().get(0));
+        } else if (call.getOperands().get(1) instanceof RexInputRef &&
+                call.getOperands().get(0).getKind() == SqlKind.CAST) {
+          literal = extractLiteral(call.getOperands().get(0));
+        }
+        if (literal == null) {
+          return null;
+        }
+        Comparable value = literalToType(literal, type);
+        if (value == null) {
+          return null;
+        }
+        if (call.getKind() == SqlKind.LESS_THAN) {
+          return Arrays.<Range> asList(withNot ? Range.atLeast(value) : Range.lessThan(value));
+        } else if (call.getKind() == SqlKind.LESS_THAN_OR_EQUAL) {
+          return Arrays.<Range> asList(withNot ? Range.greaterThan(value) : Range.atMost(value));
+        } else if (call.getKind() == SqlKind.GREATER_THAN) {
+          return Arrays.<Range> asList(withNot ? Range.atMost(value) : Range.greaterThan(value));
+        } else if (call.getKind() == SqlKind.GREATER_THAN_OR_EQUAL) {
+          return Arrays.<Range> asList(withNot ? Range.lessThan(value) : Range.atLeast(value));
+        } else { //EQUALS
+          if (!withNot) {
+            return Arrays.<Range> asList(Range.closed(value, value));
+          }
+          return Arrays.<Range> asList(Range.lessThan(value), Range.greaterThan(value));
+        }
+      }
+      case BETWEEN:
+      {
+        RexLiteral literal1 = extractLiteral(call.getOperands().get(2));
+        if (literal1 == null) {
+          return null;
+        }
+        RexLiteral literal2 = extractLiteral(call.getOperands().get(3));
+        if (literal2 == null) {
+          return null;
+        }
+        Comparable value1 = literalToType(literal1, type);
+        Comparable value2 = literalToType(literal2, type);
+        if (value1 == null || value2 == null) {
+          return null;
+        }
+        boolean inverted = value1.compareTo(value2) > 0;
+        if (!withNot) {
+          return Arrays.<Range> asList(
+                  inverted ? Range.closed(value2, value1) : Range.closed(value1, value2));
+        }
+        return Arrays.<Range> asList(Range.lessThan(inverted ? value2 : value1),
+                Range.greaterThan(inverted ? value1 : value2));
+      }
+      case IN:
+      {
+        List<Range> ranges = Lists.newArrayList();
+        for (int i = 1; i < call.getOperands().size(); i++) {
+          RexLiteral literal = extractLiteral(call.getOperands().get(i));
+          if (literal == null) {
+            return null;
+          }
+          Comparable element = literalToType(literal, type);
+          if (element == null) {
+            return null;
+          }
+          if (withNot) {
+            ranges.addAll(
+                    Arrays.<Range> asList(Range.lessThan(element), Range.greaterThan(element)));
+          } else {
+            ranges.add(Range.closed(element, element));
+          }
+        }
+        return ranges;
+      }
+      default:
+        return null;
+    }
+  }
+
+  @SuppressWarnings("incomplete-switch")
+  protected static Comparable literalToType(RexLiteral literal, RelDataType type) {
+    // Extract
+    Object value = null;
+    switch (literal.getType().getSqlTypeName()) {
+      case DATE:
+      case TIME:
+      case TIMESTAMP:
+      case INTERVAL_YEAR_MONTH:
+      case INTERVAL_DAY_TIME:
+        value = literal.getValue();
+        break;
+      case TINYINT:
+      case SMALLINT:
+      case INTEGER:
+      case BIGINT:
+      case DOUBLE:
+      case DECIMAL:
+      case FLOAT:
+      case REAL:
+      case VARCHAR:
+      case CHAR:
+      case BOOLEAN:
+        value = literal.getValue3();
+    }
+    if (value == null) {
+      return null;
+    }
+
+    // Convert
+    switch (type.getSqlTypeName()) {
+      case BIGINT:
+        return toLong(value);
+      case INTEGER:
+        return toInt(value);
+      case FLOAT:
+        return toFloat(value);
+      case DOUBLE:
+        return toDouble(value);
+      case VARCHAR:
+      case CHAR:
+        return String.valueOf(value);
+      case TIMESTAMP:
+        return toTimestamp(value);
+    }
+    return null;
+  }
+
+  private static RexLiteral extractLiteral(RexNode node) {
+    RexNode target = node;
+    if (node.getKind() == SqlKind.CAST) {
+      target = ((RexCall)node).getOperands().get(0);
+    }
+    if (!(target instanceof RexLiteral)) {
+      return null;
+    }
+    return (RexLiteral) target;
+  }
+
+  private static Comparable toTimestamp(Object literal) {
+    if (literal instanceof Timestamp) {
+      return (Timestamp) literal;
+    }
+    if (literal instanceof Date) {
+      return new Timestamp(((Date) literal).getTime());
+    }
+    if (literal instanceof Number) {
+      return new Timestamp(((Number) literal).longValue());
+    }
+    if (literal instanceof String) {
+      String string = (String) literal;
+      if (StringUtils.isNumeric(string)) {
+        return new Timestamp(Long.valueOf(string));
+      }
+      try {
+        return Timestamp.valueOf(string);
+      } catch (NumberFormatException e) {
+        // ignore
+      }
+    }
+    return null;
+  }
+
+  private static Long toLong(Object literal) {
+    if (literal instanceof Number) {
+      return ((Number) literal).longValue();
+    }
+    if (literal instanceof Date) {
+      return ((Date) literal).getTime();
+    }
+    if (literal instanceof Timestamp) {
+      return ((Timestamp) literal).getTime();
+    }
+    if (literal instanceof String) {
+      try {
+        return Long.valueOf((String) literal);
+      } catch (NumberFormatException e) {
+        // ignore
+      }
+      try {
+        return DateFormat.getDateInstance().parse((String) literal).getTime();
+      } catch (ParseException e) {
+        // best effort. ignore
+      }
+    }
+    return null;
+  }
+
+
+  private static Integer toInt(Object literal) {
+    if (literal instanceof Number) {
+      return ((Number) literal).intValue();
+    }
+    if (literal instanceof String) {
+      try {
+        return Integer.valueOf((String) literal);
+      } catch (NumberFormatException e) {
+        // ignore
+      }
+    }
+    return null;
+  }
+
+  private static Float toFloat(Object literal) {
+    if (literal instanceof Number) {
+      return ((Number) literal).floatValue();
+    }
+    if (literal instanceof String) {
+      try {
+        return Float.valueOf((String) literal);
+      } catch (NumberFormatException e) {
+        // ignore
+      }
+    }
+    return null;
+  }
+
+  private static Double toDouble(Object literal) {
+    if (literal instanceof Number) {
+      return ((Number) literal).doubleValue();
+    }
+    if (literal instanceof String) {
+      try {
+        return Double.valueOf((String) literal);
+      } catch (NumberFormatException e) {
+        // ignore
+      }
+    }
+    return null;
+  }
+
+  protected static List<Range> condenseRanges(List<Range> ranges) {
+    if (ranges.size() <= 1) {
+      return ranges;
+    }
+
+    Comparator<Range> startThenEnd = new Comparator<Range>() {
+      @Override
+      public int compare(Range lhs, Range rhs) {
+        int compare = 0;
+        if (lhs.hasLowerBound() && rhs.hasLowerBound()) {
+          compare = lhs.lowerEndpoint().compareTo(rhs.lowerEndpoint());
+        } else if (!lhs.hasLowerBound() && rhs.hasLowerBound()) {
+          compare = -1;
+        } else if (lhs.hasLowerBound() && !rhs.hasLowerBound()) {
+          compare = 1;
+        }
+        if (compare != 0) {
+          return compare;
+        }
+        if (lhs.hasUpperBound() && rhs.hasUpperBound()) {
+          compare = lhs.upperEndpoint().compareTo(rhs.upperEndpoint());
+        } else if (!lhs.hasUpperBound() && rhs.hasUpperBound()) {
+          compare = -1;
+        } else if (lhs.hasUpperBound() && !rhs.hasUpperBound()) {
+          compare = 1;
+        }
+        return compare;
+      }
+    };
+
+    TreeSet<Range> sortedIntervals = Sets.newTreeSet(startThenEnd);
+    sortedIntervals.addAll(ranges);
+
+    List<Range> retVal = Lists.newArrayList();
+
+    Iterator<Range> intervalsIter = sortedIntervals.iterator();
+    Range currInterval = intervalsIter.next();
+    while (intervalsIter.hasNext()) {
+      Range next = intervalsIter.next();
+      if (currInterval.encloses(next)) {
+        continue;
+      }
+      if (mergeable(currInterval, next)) {
+        currInterval = currInterval.span(next);
+      } else {
+        retVal.add(currInterval);
+        currInterval = next;
+      }
+    }
+    retVal.add(currInterval);
+
+    return retVal;
+  }
+
+  protected static boolean mergeable(Range range1, Range range2) {
+    Comparable x1 = range1.upperEndpoint();
+    Comparable x2 = range2.lowerEndpoint();
+    int compare = x1.compareTo(x2);
+    return compare > 0 || (compare == 0 && range1.upperBoundType() == BoundType.CLOSED
+            && range2.lowerBoundType() == BoundType.CLOSED);
+  }
+
+  public static long extractTotalTime(List<Interval> intervals) {
+    long totalTime = 0;
+    for (Interval interval : intervals) {
+      totalTime += (interval.getEndMillis() - interval.getStartMillis());
+    }
+    return totalTime;
+  }
+
+}


[16/31] hive git commit: HIVE-14728: Redundant orig files (Rui reviewed by Pengcheng)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bc75e46a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java.orig
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java.orig b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java.orig
deleted file mode 100644
index 699bb11..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java.orig
+++ /dev/null
@@ -1,13038 +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.parse;
-
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESTATSDBCLASS;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.Serializable;
-import java.security.AccessControlException;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Queue;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
-
-import org.antlr.runtime.ClassicToken;
-import org.antlr.runtime.CommonToken;
-import org.antlr.runtime.Token;
-import org.antlr.runtime.tree.Tree;
-import org.antlr.runtime.tree.TreeWizard;
-import org.antlr.runtime.tree.TreeWizard.ContextVisitor;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.hive.common.BlobStorageUtils;
-import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
-import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.common.ObjectPair;
-import org.apache.hadoop.hive.common.StatsSetupConst;
-import org.apache.hadoop.hive.common.StatsSetupConst.StatDB;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.TableType;
-import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.Order;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
-import org.apache.hadoop.hive.ql.CompilationOpContext;
-import org.apache.hadoop.hive.ql.ErrorMsg;
-import org.apache.hadoop.hive.ql.QueryProperties;
-import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator;
-import org.apache.hadoop.hive.ql.exec.ArchiveUtils;
-import org.apache.hadoop.hive.ql.exec.ColumnInfo;
-import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
-import org.apache.hadoop.hive.ql.exec.FetchTask;
-import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
-import org.apache.hadoop.hive.ql.exec.FilterOperator;
-import org.apache.hadoop.hive.ql.exec.FunctionInfo;
-import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
-import org.apache.hadoop.hive.ql.exec.GroupByOperator;
-import org.apache.hadoop.hive.ql.exec.JoinOperator;
-import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.OperatorFactory;
-import org.apache.hadoop.hive.ql.exec.RecordReader;
-import org.apache.hadoop.hive.ql.exec.RecordWriter;
-import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
-import org.apache.hadoop.hive.ql.exec.RowSchema;
-import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
-import org.apache.hadoop.hive.ql.exec.SelectOperator;
-import org.apache.hadoop.hive.ql.exec.TableScanOperator;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.exec.TaskFactory;
-import org.apache.hadoop.hive.ql.exec.UnionOperator;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.hooks.ReadEntity;
-import org.apache.hadoop.hive.ql.hooks.WriteEntity;
-import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
-import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.io.AcidUtils.Operation;
-import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
-import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
-import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
-import org.apache.hadoop.hive.ql.io.NullRowsInputFormat;
-import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
-import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
-import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
-import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.metadata.DummyPartition;
-import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.HiveUtils;
-import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
-import org.apache.hadoop.hive.ql.metadata.Partition;
-import org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient;
-import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
-import org.apache.hadoop.hive.ql.optimizer.Optimizer;
-import org.apache.hadoop.hive.ql.optimizer.Transform;
-import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
-import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException.UnsupportedFeature;
-import org.apache.hadoop.hive.ql.optimizer.calcite.translator.HiveOpConverterPostProc;
-import org.apache.hadoop.hive.ql.optimizer.lineage.Generator;
-import org.apache.hadoop.hive.ql.optimizer.unionproc.UnionProcContext;
-import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.TableSpec.SpecType;
-import org.apache.hadoop.hive.ql.parse.CalcitePlanner.ASTSearcher;
-import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.OrderExpression;
-import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.OrderSpec;
-import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PTFInputSpec;
-import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PTFQueryInputSpec;
-import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PTFQueryInputType;
-import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitionExpression;
-import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitionSpec;
-import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitionedTableFunctionSpec;
-import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitioningSpec;
-import org.apache.hadoop.hive.ql.parse.QBSubQuery.SubQueryType;
-import org.apache.hadoop.hive.ql.parse.SubQueryUtils.ISubQueryJoinInfo;
-import org.apache.hadoop.hive.ql.parse.WindowingSpec.BoundarySpec;
-import org.apache.hadoop.hive.ql.parse.WindowingSpec.CurrentRowSpec;
-import org.apache.hadoop.hive.ql.parse.WindowingSpec.Direction;
-import org.apache.hadoop.hive.ql.parse.WindowingSpec.RangeBoundarySpec;
-import org.apache.hadoop.hive.ql.parse.WindowingSpec.ValueBoundarySpec;
-import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowExpressionSpec;
-import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFrameSpec;
-import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFunctionSpec;
-import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowSpec;
-import org.apache.hadoop.hive.ql.plan.AggregationDesc;
-import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
-import org.apache.hadoop.hive.ql.plan.CreateTableDesc;
-import org.apache.hadoop.hive.ql.plan.CreateTableLikeDesc;
-import org.apache.hadoop.hive.ql.plan.CreateViewDesc;
-import org.apache.hadoop.hive.ql.plan.DDLWork;
-import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
-import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeColumnListDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils;
-import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
-import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
-import org.apache.hadoop.hive.ql.plan.FilterDesc;
-import org.apache.hadoop.hive.ql.plan.FilterDesc.SampleDesc;
-import org.apache.hadoop.hive.ql.plan.ForwardDesc;
-import org.apache.hadoop.hive.ql.plan.GroupByDesc;
-import org.apache.hadoop.hive.ql.plan.HiveOperation;
-import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
-import org.apache.hadoop.hive.ql.plan.JoinDesc;
-import org.apache.hadoop.hive.ql.plan.LateralViewForwardDesc;
-import org.apache.hadoop.hive.ql.plan.LateralViewJoinDesc;
-import org.apache.hadoop.hive.ql.plan.LimitDesc;
-import org.apache.hadoop.hive.ql.plan.ListBucketingCtx;
-import org.apache.hadoop.hive.ql.plan.LoadFileDesc;
-import org.apache.hadoop.hive.ql.plan.LoadTableDesc;
-import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
-import org.apache.hadoop.hive.ql.plan.OperatorDesc;
-import org.apache.hadoop.hive.ql.plan.PTFDesc;
-import org.apache.hadoop.hive.ql.plan.PlanUtils;
-import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
-import org.apache.hadoop.hive.ql.plan.ScriptDesc;
-import org.apache.hadoop.hive.ql.plan.SelectDesc;
-import org.apache.hadoop.hive.ql.plan.TableDesc;
-import org.apache.hadoop.hive.ql.plan.TableScanDesc;
-import org.apache.hadoop.hive.ql.plan.UDTFDesc;
-import org.apache.hadoop.hive.ql.plan.UnionDesc;
-import org.apache.hadoop.hive.ql.plan.ptf.OrderExpressionDef;
-import org.apache.hadoop.hive.ql.plan.ptf.PTFExpressionDef;
-import org.apache.hadoop.hive.ql.plan.ptf.PartitionedTableFunctionDef;
-import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.ql.session.SessionState.ResourceType;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFHash;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
-import org.apache.hadoop.hive.ql.util.ResourceDownloader;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.Deserializer;
-import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
-import org.apache.hadoop.hive.serde2.NoOpFetchFormatter;
-import org.apache.hadoop.hive.serde2.NullStructSerDe;
-import org.apache.hadoop.hive.serde2.SerDeException;
-import org.apache.hadoop.hive.serde2.SerDeUtils;
-import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
-import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.StructField;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.hadoop.hive.serde2.thrift.ThriftJDBCBinarySerDe;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.hive.shims.HadoopShims;
-import org.apache.hadoop.hive.shims.Utils;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.OutputFormat;
-import org.apache.hadoop.security.UserGroupInformation;
-
-import com.google.common.collect.Sets;
-
-/**
- * Implementation of the semantic analyzer. It generates the query plan.
- * There are other specific semantic analyzers for some hive operations such as
- * DDLSemanticAnalyzer for ddl operations.
- */
-
-public class SemanticAnalyzer extends BaseSemanticAnalyzer {
-
-  public static final String DUMMY_DATABASE = "_dummy_database";
-  public static final String DUMMY_TABLE = "_dummy_table";
-  public static final String SUBQUERY_TAG_1 = "-subquery1";
-  public static final String SUBQUERY_TAG_2 = "-subquery2";
-
-  // Max characters when auto generating the column name with func name
-  private static final int AUTOGEN_COLALIAS_PRFX_MAXLENGTH = 20;
-
-  private static final String VALUES_TMP_TABLE_NAME_PREFIX = "Values__Tmp__Table__";
-
-  static final String MATERIALIZATION_MARKER = "$MATERIALIZATION";
-
-  private HashMap<TableScanOperator, ExprNodeDesc> opToPartPruner;
-  private HashMap<TableScanOperator, PrunedPartitionList> opToPartList;
-  protected HashMap<String, TableScanOperator> topOps;
-  protected LinkedHashMap<Operator<? extends OperatorDesc>, OpParseContext> opParseCtx;
-  private List<LoadTableDesc> loadTableWork;
-  private List<LoadFileDesc> loadFileWork;
-  private List<ColumnStatsAutoGatherContext> columnStatsAutoGatherContexts;
-  private final Map<JoinOperator, QBJoinTree> joinContext;
-  private final Map<SMBMapJoinOperator, QBJoinTree> smbMapJoinContext;
-  private final HashMap<TableScanOperator, Table> topToTable;
-  private final Map<FileSinkOperator, Table> fsopToTable;
-  private final List<ReduceSinkOperator> reduceSinkOperatorsAddedByEnforceBucketingSorting;
-  private final HashMap<TableScanOperator, Map<String, String>> topToTableProps;
-  private QB qb;
-  private ASTNode ast;
-  private int destTableId;
-  private UnionProcContext uCtx;
-  List<AbstractMapJoinOperator<? extends MapJoinDesc>> listMapJoinOpsNoReducer;
-  private HashMap<TableScanOperator, SampleDesc> opToSamplePruner;
-  private final Map<TableScanOperator, Map<String, ExprNodeDesc>> opToPartToSkewedPruner;
-  private Map<SelectOperator, Table> viewProjectToTableSchema;
-  /**
-   * a map for the split sampling, from alias to an instance of SplitSample
-   * that describes percentage and number.
-   */
-  private final HashMap<String, SplitSample> nameToSplitSample;
-  Map<GroupByOperator, Set<String>> groupOpToInputTables;
-  Map<String, PrunedPartitionList> prunedPartitions;
-  protected List<FieldSchema> resultSchema;
-  private CreateViewDesc createVwDesc;
-  private ArrayList<String> viewsExpanded;
-  private ASTNode viewSelect;
-  protected final UnparseTranslator unparseTranslator;
-  private final GlobalLimitCtx globalLimitCtx;
-
-  // prefix for column names auto generated by hive
-  private final String autogenColAliasPrfxLbl;
-  private final boolean autogenColAliasPrfxIncludeFuncName;
-
-  // Keep track of view alias to read entity corresponding to the view
-  // For eg: for a query like 'select * from V3', where V3 -> V2, V2 -> V1, V1 -> T
-  // keeps track of aliases for V3, V3:V2, V3:V2:V1.
-  // This is used when T is added as an input for the query, the parents of T is
-  // derived from the alias V3:V2:V1:T
-  private final Map<String, ReadEntity> viewAliasToInput;
-
-  //need merge isDirect flag to input even if the newInput does not have a parent
-  private boolean mergeIsDirect;
-
-  // flag for no scan during analyze ... compute statistics
-  protected boolean noscan;
-
-  //flag for partial scan during analyze ... compute statistics
-  protected boolean partialscan;
-
-  protected volatile boolean disableJoinMerge = false;
-
-  /*
-   * Capture the CTE definitions in a Query.
-   */
-  final Map<String, CTEClause> aliasToCTEs;
-
-  /*
-   * Used to check recursive CTE invocations. Similar to viewsExpanded
-   */
-  ArrayList<String> ctesExpanded;
-
-  /*
-   * Whether root tasks after materialized CTE linkage have been resolved
-   */
-  boolean rootTasksResolved;
-
-  protected TableMask tableMask;
-
-  CreateTableDesc tableDesc;
-
-  /** Not thread-safe. */
-  final ASTSearcher astSearcher = new ASTSearcher();
-
-  protected AnalyzeRewriteContext analyzeRewrite;
-
-  // A mapping from a tableName to a table object in metastore.
-  Map<String, Table> tabNameToTabObject;
-
-  // The tokens we should ignore when we are trying to do table masking.
-  private final Set<Integer> ignoredTokens = Sets.newHashSet(HiveParser.TOK_GROUPBY,
-      HiveParser.TOK_ORDERBY, HiveParser.TOK_WINDOWSPEC, HiveParser.TOK_CLUSTERBY,
-      HiveParser.TOK_DISTRIBUTEBY, HiveParser.TOK_SORTBY);
-
-  static class Phase1Ctx {
-    String dest;
-    int nextNum;
-  }
-
-  public SemanticAnalyzer(QueryState queryState) throws SemanticException {
-    super(queryState);
-    opToPartPruner = new HashMap<TableScanOperator, ExprNodeDesc>();
-    opToPartList = new HashMap<TableScanOperator, PrunedPartitionList>();
-    opToSamplePruner = new HashMap<TableScanOperator, SampleDesc>();
-    nameToSplitSample = new HashMap<String, SplitSample>();
-    // Must be deterministic order maps - see HIVE-8707
-    topOps = new LinkedHashMap<String, TableScanOperator>();
-    loadTableWork = new ArrayList<LoadTableDesc>();
-    loadFileWork = new ArrayList<LoadFileDesc>();
-    columnStatsAutoGatherContexts = new ArrayList<ColumnStatsAutoGatherContext>();
-    opParseCtx = new LinkedHashMap<Operator<? extends OperatorDesc>, OpParseContext>();
-    joinContext = new HashMap<JoinOperator, QBJoinTree>();
-    smbMapJoinContext = new HashMap<SMBMapJoinOperator, QBJoinTree>();
-    // Must be deterministic order map for consistent q-test output across Java versions
-    topToTable = new LinkedHashMap<TableScanOperator, Table>();
-    fsopToTable = new HashMap<FileSinkOperator, Table>();
-    reduceSinkOperatorsAddedByEnforceBucketingSorting = new ArrayList<ReduceSinkOperator>();
-    topToTableProps = new HashMap<TableScanOperator, Map<String, String>>();
-    destTableId = 1;
-    uCtx = null;
-    listMapJoinOpsNoReducer = new ArrayList<AbstractMapJoinOperator<? extends MapJoinDesc>>();
-    groupOpToInputTables = new HashMap<GroupByOperator, Set<String>>();
-    prunedPartitions = new HashMap<String, PrunedPartitionList>();
-    tabNameToTabObject = new HashMap<String, Table>();
-    unparseTranslator = new UnparseTranslator(conf);
-    autogenColAliasPrfxLbl = HiveConf.getVar(conf,
-        HiveConf.ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL);
-    autogenColAliasPrfxIncludeFuncName = HiveConf.getBoolVar(conf,
-        HiveConf.ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_INCLUDEFUNCNAME);
-    queryProperties = new QueryProperties();
-    opToPartToSkewedPruner = new HashMap<TableScanOperator, Map<String, ExprNodeDesc>>();
-    aliasToCTEs = new HashMap<String, CTEClause>();
-    globalLimitCtx = new GlobalLimitCtx();
-    viewAliasToInput = new HashMap<String, ReadEntity>();
-    mergeIsDirect = true;
-    noscan = partialscan = false;
-    tabNameToTabObject = new HashMap<>();
-  }
-
-  @Override
-  protected void reset(boolean clearPartsCache) {
-    super.reset(true);
-    if(clearPartsCache) {
-      prunedPartitions.clear();
-
-      //When init(true) combine with genResolvedParseTree, it will generate Resolved Parse tree from syntax tree
-      //ReadEntity created under these conditions should be all relevant to the syntax tree even the ones without parents
-      //set mergeIsDirect to true here.
-      mergeIsDirect = true;
-    } else {
-      mergeIsDirect = false;
-    }
-    tabNameToTabObject.clear();
-    loadTableWork.clear();
-    loadFileWork.clear();
-    columnStatsAutoGatherContexts.clear();
-    topOps.clear();
-    destTableId = 1;
-    idToTableNameMap.clear();
-    qb = null;
-    ast = null;
-    uCtx = null;
-    joinContext.clear();
-    smbMapJoinContext.clear();
-    opParseCtx.clear();
-    groupOpToInputTables.clear();
-    disableJoinMerge = false;
-    aliasToCTEs.clear();
-    topToTable.clear();
-    opToPartPruner.clear();
-    opToPartList.clear();
-    opToPartToSkewedPruner.clear();
-    opToSamplePruner.clear();
-    nameToSplitSample.clear();
-    fsopToTable.clear();
-    resultSchema = null;
-    createVwDesc = null;
-    viewsExpanded = null;
-    viewSelect = null;
-    ctesExpanded = null;
-    globalLimitCtx.disableOpt();
-    viewAliasToInput.clear();
-    reduceSinkOperatorsAddedByEnforceBucketingSorting.clear();
-    topToTableProps.clear();
-    listMapJoinOpsNoReducer.clear();
-    unparseTranslator.clear();
-    queryProperties.clear();
-    outputs.clear();
-  }
-
-  public void initParseCtx(ParseContext pctx) {
-    opToPartPruner = pctx.getOpToPartPruner();
-    opToPartList = pctx.getOpToPartList();
-    opToSamplePruner = pctx.getOpToSamplePruner();
-    topOps = pctx.getTopOps();
-    loadTableWork = pctx.getLoadTableWork();
-    loadFileWork = pctx.getLoadFileWork();
-    ctx = pctx.getContext();
-    destTableId = pctx.getDestTableId();
-    idToTableNameMap = pctx.getIdToTableNameMap();
-    uCtx = pctx.getUCtx();
-    listMapJoinOpsNoReducer = pctx.getListMapJoinOpsNoReducer();
-    prunedPartitions = pctx.getPrunedPartitions();
-    tabNameToTabObject = pctx.getTabNameToTabObject();
-    fetchTask = pctx.getFetchTask();
-    setLineageInfo(pctx.getLineageInfo());
-  }
-
-  public ParseContext getParseContext() {
-    // Make sure the basic query properties are initialized
-    copyInfoToQueryProperties(queryProperties);
-    return new ParseContext(queryState, opToPartPruner, opToPartList, topOps,
-        new HashSet<JoinOperator>(joinContext.keySet()),
-        new HashSet<SMBMapJoinOperator>(smbMapJoinContext.keySet()),
-        loadTableWork, loadFileWork, columnStatsAutoGatherContexts, ctx, idToTableNameMap, destTableId, uCtx,
-        listMapJoinOpsNoReducer, prunedPartitions, tabNameToTabObject,
-        opToSamplePruner, globalLimitCtx, nameToSplitSample, inputs, rootTasks,
-        opToPartToSkewedPruner, viewAliasToInput, reduceSinkOperatorsAddedByEnforceBucketingSorting,
-        analyzeRewrite, tableDesc, queryProperties, viewProjectToTableSchema, acidFileSinks);
-  }
-
-  public CompilationOpContext getOpContext() {
-    return ctx.getOpContext();
-  }
-
-  public void doPhase1QBExpr(ASTNode ast, QBExpr qbexpr, String id, String alias)
-      throws SemanticException {
-    doPhase1QBExpr(ast, qbexpr, id, alias, false);
-  }
-  @SuppressWarnings("nls")
-  public void doPhase1QBExpr(ASTNode ast, QBExpr qbexpr, String id, String alias, boolean insideView)
-      throws SemanticException {
-
-    assert (ast.getToken() != null);
-    switch (ast.getToken().getType()) {
-    case HiveParser.TOK_QUERY: {
-      QB qb = new QB(id, alias, true);
-      qb.setInsideView(insideView);
-      Phase1Ctx ctx_1 = initPhase1Ctx();
-      doPhase1(ast, qb, ctx_1, null);
-
-      qbexpr.setOpcode(QBExpr.Opcode.NULLOP);
-      qbexpr.setQB(qb);
-    }
-      break;
-    case HiveParser.TOK_UNIONALL: {
-      qbexpr.setOpcode(QBExpr.Opcode.UNION);
-      // query 1
-      assert (ast.getChild(0) != null);
-      QBExpr qbexpr1 = new QBExpr(alias + SUBQUERY_TAG_1);
-      doPhase1QBExpr((ASTNode) ast.getChild(0), qbexpr1, id + SUBQUERY_TAG_1,
-          alias + SUBQUERY_TAG_1, insideView);
-      qbexpr.setQBExpr1(qbexpr1);
-
-      // query 2
-      assert (ast.getChild(1) != null);
-      QBExpr qbexpr2 = new QBExpr(alias + SUBQUERY_TAG_2);
-      doPhase1QBExpr((ASTNode) ast.getChild(1), qbexpr2, id + SUBQUERY_TAG_2,
-          alias + SUBQUERY_TAG_2, insideView);
-      qbexpr.setQBExpr2(qbexpr2);
-    }
-      break;
-    }
-  }
-
-  private LinkedHashMap<String, ASTNode> doPhase1GetAggregationsFromSelect(
-      ASTNode selExpr, QB qb, String dest) throws SemanticException {
-
-    // Iterate over the selects search for aggregation Trees.
-    // Use String as keys to eliminate duplicate trees.
-    LinkedHashMap<String, ASTNode> aggregationTrees = new LinkedHashMap<String, ASTNode>();
-    List<ASTNode> wdwFns = new ArrayList<ASTNode>();
-    for (int i = 0; i < selExpr.getChildCount(); ++i) {
-      ASTNode function = (ASTNode) selExpr.getChild(i);
-      if (function.getType() == HiveParser.TOK_SELEXPR ||
-          function.getType() == HiveParser.TOK_SUBQUERY_EXPR) {
-        function = (ASTNode)function.getChild(0);
-      }
-      doPhase1GetAllAggregations(function, aggregationTrees, wdwFns);
-    }
-
-    // window based aggregations are handled differently
-    for (ASTNode wdwFn : wdwFns) {
-      WindowingSpec spec = qb.getWindowingSpec(dest);
-      if(spec == null) {
-        queryProperties.setHasWindowing(true);
-        spec = new WindowingSpec();
-        qb.addDestToWindowingSpec(dest, spec);
-      }
-      HashMap<String, ASTNode> wExprsInDest = qb.getParseInfo().getWindowingExprsForClause(dest);
-      int wColIdx = spec.getWindowExpressions() == null ? 0 : spec.getWindowExpressions().size();
-      WindowFunctionSpec wFnSpec = processWindowFunction(wdwFn,
-        (ASTNode)wdwFn.getChild(wdwFn.getChildCount()-1));
-      // If this is a duplicate invocation of a function; don't add to WindowingSpec.
-      if ( wExprsInDest != null &&
-          wExprsInDest.containsKey(wFnSpec.getExpression().toStringTree())) {
-        continue;
-      }
-      wFnSpec.setAlias(wFnSpec.getName() + "_window_" + wColIdx);
-      spec.addWindowFunction(wFnSpec);
-      qb.getParseInfo().addWindowingExprToClause(dest, wFnSpec.getExpression());
-    }
-
-    return aggregationTrees;
-  }
-
-  private void doPhase1GetColumnAliasesFromSelect(
-      ASTNode selectExpr, QBParseInfo qbp) {
-    for (int i = 0; i < selectExpr.getChildCount(); ++i) {
-      ASTNode selExpr = (ASTNode) selectExpr.getChild(i);
-      if ((selExpr.getToken().getType() == HiveParser.TOK_SELEXPR)
-          && (selExpr.getChildCount() == 2)) {
-        String columnAlias = unescapeIdentifier(selExpr.getChild(1).getText());
-        qbp.setExprToColumnAlias((ASTNode) selExpr.getChild(0), columnAlias);
-      }
-    }
-  }
-
-  /**
-   * DFS-scan the expressionTree to find all aggregation subtrees and put them
-   * in aggregations.
-   *
-   * @param expressionTree
-   * @param aggregations
-   *          the key to the HashTable is the toStringTree() representation of
-   *          the aggregation subtree.
-   * @throws SemanticException
-   */
-  private void doPhase1GetAllAggregations(ASTNode expressionTree,
-      HashMap<String, ASTNode> aggregations, List<ASTNode> wdwFns) throws SemanticException {
-    int exprTokenType = expressionTree.getToken().getType();
-    if (exprTokenType == HiveParser.TOK_FUNCTION
-        || exprTokenType == HiveParser.TOK_FUNCTIONDI
-        || exprTokenType == HiveParser.TOK_FUNCTIONSTAR) {
-      assert (expressionTree.getChildCount() != 0);
-      if (expressionTree.getChild(expressionTree.getChildCount()-1).getType()
-          == HiveParser.TOK_WINDOWSPEC) {
-        // If it is a windowing spec, we include it in the list
-        // Further, we will examine its children AST nodes to check whether
-        // there are aggregation functions within
-        wdwFns.add(expressionTree);
-        doPhase1GetAllAggregations((ASTNode) expressionTree.getChild(expressionTree.getChildCount()-1),
-                aggregations, wdwFns);
-        return;
-      }
-      if (expressionTree.getChild(0).getType() == HiveParser.Identifier) {
-        String functionName = unescapeIdentifier(expressionTree.getChild(0)
-            .getText());
-        // Validate the function name
-        if (FunctionRegistry.getFunctionInfo(functionName) == null) {
-          throw new SemanticException(ErrorMsg.INVALID_FUNCTION.getMsg(functionName));
-        }
-        if(FunctionRegistry.impliesOrder(functionName)) {
-          throw new SemanticException(ErrorMsg.MISSING_OVER_CLAUSE.getMsg(functionName));
-        }
-        if (FunctionRegistry.getGenericUDAFResolver(functionName) != null) {
-          if(containsLeadLagUDF(expressionTree)) {
-            throw new SemanticException(ErrorMsg.MISSING_OVER_CLAUSE.getMsg(functionName));
-          }
-          aggregations.put(expressionTree.toStringTree(), expressionTree);
-          FunctionInfo fi = FunctionRegistry.getFunctionInfo(functionName);
-          if (!fi.isNative()) {
-            unparseTranslator.addIdentifierTranslation((ASTNode) expressionTree
-                .getChild(0));
-          }
-          return;
-        }
-      }
-    }
-    for (int i = 0; i < expressionTree.getChildCount(); i++) {
-      doPhase1GetAllAggregations((ASTNode) expressionTree.getChild(i),
-          aggregations, wdwFns);
-    }
-  }
-
-  private List<ASTNode> doPhase1GetDistinctFuncExprs(
-      HashMap<String, ASTNode> aggregationTrees) throws SemanticException {
-    List<ASTNode> exprs = new ArrayList<ASTNode>();
-    for (Map.Entry<String, ASTNode> entry : aggregationTrees.entrySet()) {
-      ASTNode value = entry.getValue();
-      assert (value != null);
-      if (value.getToken().getType() == HiveParser.TOK_FUNCTIONDI) {
-        exprs.add(value);
-      }
-    }
-    return exprs;
-  }
-
-  public static String generateErrorMessage(ASTNode ast, String message) {
-    StringBuilder sb = new StringBuilder();
-    if (ast == null) {
-      sb.append(message).append(". Cannot tell the position of null AST.");
-      return sb.toString();
-    }
-    sb.append(ast.getLine());
-    sb.append(":");
-    sb.append(ast.getCharPositionInLine());
-    sb.append(" ");
-    sb.append(message);
-    sb.append(". Error encountered near token '");
-    sb.append(ErrorMsg.getText(ast));
-    sb.append("'");
-    return sb.toString();
-  }
-
-  ASTNode getAST() {
-    return this.ast;
-  }
-
-  protected void setAST(ASTNode newAST) {
-    this.ast = newAST;
-  }
-
-  /**
-   * Goes though the tabref tree and finds the alias for the table. Once found,
-   * it records the table name-> alias association in aliasToTabs. It also makes
-   * an association from the alias to the table AST in parse info.
-   *
-   * @return the alias of the table
-   */
-  private String processTable(QB qb, ASTNode tabref) throws SemanticException {
-    // For each table reference get the table name
-    // and the alias (if alias is not present, the table name
-    // is used as an alias)
-    int aliasIndex = 0;
-    int propsIndex = -1;
-    int tsampleIndex = -1;
-    int ssampleIndex = -1;
-    for (int index = 1; index < tabref.getChildCount(); index++) {
-      ASTNode ct = (ASTNode) tabref.getChild(index);
-      if (ct.getToken().getType() == HiveParser.TOK_TABLEBUCKETSAMPLE) {
-        tsampleIndex = index;
-      } else if (ct.getToken().getType() == HiveParser.TOK_TABLESPLITSAMPLE) {
-        ssampleIndex = index;
-      } else if (ct.getToken().getType() == HiveParser.TOK_TABLEPROPERTIES) {
-        propsIndex = index;
-      } else {
-        aliasIndex = index;
-      }
-    }
-
-    ASTNode tableTree = (ASTNode) (tabref.getChild(0));
-
-    String tabIdName = getUnescapedName(tableTree).toLowerCase();
-
-    String alias;
-    if (aliasIndex != 0) {
-      alias = unescapeIdentifier(tabref.getChild(aliasIndex).getText());
-    }
-    else {
-      alias = getUnescapedUnqualifiedTableName(tableTree);
-    }
-
-    if (propsIndex >= 0) {
-      Tree propsAST = tabref.getChild(propsIndex);
-      Map<String, String> props = DDLSemanticAnalyzer.getProps((ASTNode) propsAST.getChild(0));
-      // We get the information from Calcite.
-      if ("TRUE".equals(props.get("insideView"))) {
-        qb.getAliasInsideView().add(alias.toLowerCase());
-      }
-      qb.setTabProps(alias, props);
-    }
-
-    // If the alias is already there then we have a conflict
-    if (qb.exists(alias)) {
-      throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(tabref
-          .getChild(aliasIndex)));
-    }
-    if (tsampleIndex >= 0) {
-      ASTNode sampleClause = (ASTNode) tabref.getChild(tsampleIndex);
-      ArrayList<ASTNode> sampleCols = new ArrayList<ASTNode>();
-      if (sampleClause.getChildCount() > 2) {
-        for (int i = 2; i < sampleClause.getChildCount(); i++) {
-          sampleCols.add((ASTNode) sampleClause.getChild(i));
-        }
-      }
-      // TODO: For now only support sampling on up to two columns
-      // Need to change it to list of columns
-      if (sampleCols.size() > 2) {
-        throw new SemanticException(generateErrorMessage(
-            (ASTNode) tabref.getChild(0),
-            ErrorMsg.SAMPLE_RESTRICTION.getMsg()));
-      }
-      TableSample tabSample = new TableSample(
-              unescapeIdentifier(sampleClause.getChild(0).getText()),
-              unescapeIdentifier(sampleClause.getChild(1).getText()),
-              sampleCols);
-      qb.getParseInfo().setTabSample(alias, tabSample);
-      if (unparseTranslator.isEnabled()) {
-        for (ASTNode sampleCol : sampleCols) {
-          unparseTranslator.addIdentifierTranslation((ASTNode) sampleCol
-              .getChild(0));
-        }
-      }
-    } else if (ssampleIndex >= 0) {
-      ASTNode sampleClause = (ASTNode) tabref.getChild(ssampleIndex);
-
-      Tree type = sampleClause.getChild(0);
-      Tree numerator = sampleClause.getChild(1);
-      String value = unescapeIdentifier(numerator.getText());
-
-
-      SplitSample sample;
-      if (type.getType() == HiveParser.TOK_PERCENT) {
-        assertCombineInputFormat(numerator, "Percentage");
-        Double percent = Double.valueOf(value).doubleValue();
-        if (percent < 0  || percent > 100) {
-          throw new SemanticException(generateErrorMessage((ASTNode) numerator,
-              "Sampling percentage should be between 0 and 100"));
-        }
-        int seedNum = conf.getIntVar(ConfVars.HIVESAMPLERANDOMNUM);
-        sample = new SplitSample(percent, seedNum);
-      } else if (type.getType() == HiveParser.TOK_ROWCOUNT) {
-        sample = new SplitSample(Integer.parseInt(value));
-      } else {
-        assert type.getType() == HiveParser.TOK_LENGTH;
-        assertCombineInputFormat(numerator, "Total Length");
-        long length = Integer.parseInt(value.substring(0, value.length() - 1));
-        char last = value.charAt(value.length() - 1);
-        if (last == 'k' || last == 'K') {
-          length <<= 10;
-        } else if (last == 'm' || last == 'M') {
-          length <<= 20;
-        } else if (last == 'g' || last == 'G') {
-          length <<= 30;
-        }
-        int seedNum = conf.getIntVar(ConfVars.HIVESAMPLERANDOMNUM);
-        sample = new SplitSample(length, seedNum);
-      }
-      String alias_id = getAliasId(alias, qb);
-      nameToSplitSample.put(alias_id, sample);
-    }
-    // Insert this map into the stats
-    qb.setTabAlias(alias, tabIdName);
-    if (qb.isInsideView()) {
-      qb.getAliasInsideView().add(alias.toLowerCase());
-    }
-    qb.addAlias(alias);
-
-    qb.getParseInfo().setSrcForAlias(alias, tableTree);
-
-    // if alias to CTE contains the alias, we do not do the translation because
-    // cte is actually a subquery.
-    if (!this.aliasToCTEs.containsKey(alias)) {
-      unparseTranslator.addTableNameTranslation(tableTree, SessionState.get().getCurrentDatabase());
-      if (aliasIndex != 0) {
-        unparseTranslator.addIdentifierTranslation((ASTNode) tabref.getChild(aliasIndex));
-      }
-    }
-
-    return alias;
-  }
-
-  Map<String, SplitSample> getNameToSplitSampleMap() {
-    return this.nameToSplitSample;
-  }
-
-  /**
-   * Convert a string to Text format and write its bytes in the same way TextOutputFormat would do.
-   * This is needed to properly encode non-ascii characters.
-   */
-  private static void writeAsText(String text, FSDataOutputStream out) throws IOException {
-    Text to = new Text(text);
-    out.write(to.getBytes(), 0, to.getLength());
-  }
-
-  /**
-   * Generate a temp table out of a value clause
-   * See also {@link #preProcessForInsert(ASTNode, QB)}
-   */
-  private ASTNode genValuesTempTable(ASTNode originalFrom, QB qb) throws SemanticException {
-    Path dataDir = null;
-    if(!qb.getEncryptedTargetTablePaths().isEmpty()) {
-      //currently only Insert into T values(...) is supported thus only 1 values clause
-      //and only 1 target table are possible.  If/when support for
-      //select ... from values(...) is added an insert statement may have multiple
-      //encrypted target tables.
-      dataDir = ctx.getMRTmpPath(qb.getEncryptedTargetTablePaths().get(0).toUri());
-    }
-    // Pick a name for the table
-    SessionState ss = SessionState.get();
-    String tableName = VALUES_TMP_TABLE_NAME_PREFIX + ss.getNextValuesTempTableSuffix();
-
-    // Step 1, parse the values clause we were handed
-    List<? extends Node> fromChildren = originalFrom.getChildren();
-    // First child should be the virtual table ref
-    ASTNode virtualTableRef = (ASTNode)fromChildren.get(0);
-    assert virtualTableRef.getToken().getType() == HiveParser.TOK_VIRTUAL_TABREF :
-        "Expected first child of TOK_VIRTUAL_TABLE to be TOK_VIRTUAL_TABREF but was " +
-            virtualTableRef.getName();
-
-    List<? extends Node> virtualTableRefChildren = virtualTableRef.getChildren();
-    // First child of this should be the table name.  If it's anonymous,
-    // then we don't have a table name.
-    ASTNode tabName = (ASTNode)virtualTableRefChildren.get(0);
-    if (tabName.getToken().getType() != HiveParser.TOK_ANONYMOUS) {
-      // TODO, if you want to make select ... from (values(...) as foo(...) work,
-      // you need to parse this list of columns names and build it into the table
-      throw new SemanticException(ErrorMsg.VALUES_TABLE_CONSTRUCTOR_NOT_SUPPORTED.getMsg());
-    }
-
-    // The second child of the TOK_VIRTUAL_TABLE should be TOK_VALUES_TABLE
-    ASTNode valuesTable = (ASTNode)fromChildren.get(1);
-    assert valuesTable.getToken().getType() == HiveParser.TOK_VALUES_TABLE :
-        "Expected second child of TOK_VIRTUAL_TABLE to be TOK_VALUE_TABLE but was " +
-            valuesTable.getName();
-    // Each of the children of TOK_VALUES_TABLE will be a TOK_VALUE_ROW
-    List<? extends Node> valuesTableChildren = valuesTable.getChildren();
-
-    // Now that we're going to start reading through the rows, open a file to write the rows too
-    // If we leave this method before creating the temporary table we need to be sure to clean up
-    // this file.
-    Path tablePath = null;
-    FileSystem fs = null;
-    FSDataOutputStream out = null;
-    try {
-      if(dataDir == null) {
-        tablePath = Warehouse.getDnsPath(new Path(ss.getTempTableSpace(), tableName), conf);
-      }
-      else {
-        //if target table of insert is encrypted, make sure temporary table data is stored
-        //similarly encrypted
-        tablePath = Warehouse.getDnsPath(new Path(dataDir, tableName), conf);
-      }
-      fs = tablePath.getFileSystem(conf);
-      fs.mkdirs(tablePath);
-      Path dataFile = new Path(tablePath, "data_file");
-      out = fs.create(dataFile);
-      List<FieldSchema> fields = new ArrayList<FieldSchema>();
-
-      boolean firstRow = true;
-      for (Node n : valuesTableChildren) {
-        ASTNode valuesRow = (ASTNode) n;
-        assert valuesRow.getToken().getType() == HiveParser.TOK_VALUE_ROW :
-            "Expected child of TOK_VALUE_TABLE to be TOK_VALUE_ROW but was " + valuesRow.getName();
-        // Each of the children of this should be a literal
-        List<? extends Node> valuesRowChildren = valuesRow.getChildren();
-        boolean isFirst = true;
-        int nextColNum = 1;
-        for (Node n1 : valuesRowChildren) {
-          ASTNode value = (ASTNode) n1;
-          if (firstRow) {
-            fields.add(new FieldSchema("tmp_values_col" + nextColNum++, "string", ""));
-          }
-          if (isFirst) isFirst = false;
-          else writeAsText("\u0001", out);
-          writeAsText(unparseExprForValuesClause(value), out);
-        }
-        writeAsText("\n", out);
-        firstRow = false;
-      }
-
-      // Step 2, create a temp table, using the created file as the data
-      StorageFormat format = new StorageFormat(conf);
-      format.processStorageFormat("TextFile");
-      Table table = db.newTable(tableName);
-      table.setSerializationLib(format.getSerde());
-      table.setFields(fields);
-      table.setDataLocation(tablePath);
-      table.getTTable().setTemporary(true);
-      table.setStoredAsSubDirectories(false);
-      table.setInputFormatClass(format.getInputFormat());
-      table.setOutputFormatClass(format.getOutputFormat());
-      db.createTable(table, false);
-    } catch (Exception e) {
-      String errMsg = ErrorMsg.INSERT_CANNOT_CREATE_TEMP_FILE.getMsg() + e.getMessage();
-      LOG.error(errMsg);
-      // Try to delete the file
-      if (fs != null && tablePath != null) {
-        try {
-          fs.delete(tablePath, false);
-        } catch (IOException swallowIt) {}
-      }
-      throw new SemanticException(errMsg, e);
-    } finally {
-        IOUtils.closeStream(out);
-    }
-
-    // Step 3, return a new subtree with a from clause built around that temp table
-    // The form of the tree is TOK_TABREF->TOK_TABNAME->identifier(tablename)
-    Token t = new ClassicToken(HiveParser.TOK_TABREF);
-    ASTNode tabRef = new ASTNode(t);
-    t = new ClassicToken(HiveParser.TOK_TABNAME);
-    ASTNode tabNameNode = new ASTNode(t);
-    tabRef.addChild(tabNameNode);
-    t = new ClassicToken(HiveParser.Identifier, tableName);
-    ASTNode identifier = new ASTNode(t);
-    tabNameNode.addChild(identifier);
-    return tabRef;
-  }
-
-  // Take an expression in the values clause and turn it back into a string.  This is far from
-  // comprehensive.  At the moment it only supports:
-  // * literals (all types)
-  // * unary negatives
-  // * true/false
-  private String unparseExprForValuesClause(ASTNode expr) throws SemanticException {
-    switch (expr.getToken().getType()) {
-      case HiveParser.Number:
-        return expr.getText();
-
-      case HiveParser.StringLiteral:
-        return BaseSemanticAnalyzer.unescapeSQLString(expr.getText());
-
-      case HiveParser.KW_FALSE:
-        // UDFToBoolean casts any non-empty string to true, so set this to false
-        return "";
-
-      case HiveParser.KW_TRUE:
-        return "TRUE";
-
-      case HiveParser.MINUS:
-        return "-" + unparseExprForValuesClause((ASTNode)expr.getChildren().get(0));
-
-      case HiveParser.TOK_NULL:
-        // Hive's text input will translate this as a null
-        return "\\N";
-
-      default:
-        throw new SemanticException("Expression of type " + expr.getText() +
-            " not supported in insert/values");
-    }
-
-  }
-
-  private void assertCombineInputFormat(Tree numerator, String message) throws SemanticException {
-    String inputFormat = conf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez") ?
-      HiveConf.getVar(conf, HiveConf.ConfVars.HIVETEZINPUTFORMAT):
-      HiveConf.getVar(conf, HiveConf.ConfVars.HIVEINPUTFORMAT);
-    if (!inputFormat.equals(CombineHiveInputFormat.class.getName())) {
-      throw new SemanticException(generateErrorMessage((ASTNode) numerator,
-          message + " sampling is not supported in " + inputFormat));
-    }
-  }
-
-  private String processSubQuery(QB qb, ASTNode subq) throws SemanticException {
-
-    // This is a subquery and must have an alias
-    if (subq.getChildCount() != 2) {
-      throw new SemanticException(ErrorMsg.NO_SUBQUERY_ALIAS.getMsg(subq));
-    }
-    ASTNode subqref = (ASTNode) subq.getChild(0);
-    String alias = unescapeIdentifier(subq.getChild(1).getText());
-
-    // Recursively do the first phase of semantic analysis for the subquery
-    QBExpr qbexpr = new QBExpr(alias);
-
-    doPhase1QBExpr(subqref, qbexpr, qb.getId(), alias);
-
-    // If the alias is already there then we have a conflict
-    if (qb.exists(alias)) {
-      throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(subq
-          .getChild(1)));
-    }
-    // Insert this map into the stats
-    qb.setSubqAlias(alias, qbexpr);
-    qb.addAlias(alias);
-
-    unparseTranslator.addIdentifierTranslation((ASTNode) subq.getChild(1));
-
-    return alias;
-  }
-
-  /*
-   * Phase1: hold onto any CTE definitions in aliasToCTE.
-   * CTE definitions are global to the Query.
-   */
-  private void processCTE(QB qb, ASTNode ctes) throws SemanticException {
-
-    int numCTEs = ctes.getChildCount();
-
-    for(int i=0; i <numCTEs; i++) {
-      ASTNode cte = (ASTNode) ctes.getChild(i);
-      ASTNode cteQry = (ASTNode) cte.getChild(0);
-      String alias = unescapeIdentifier(cte.getChild(1).getText());
-
-      String qName = qb.getId() == null ? "" : qb.getId() + ":";
-      qName += alias.toLowerCase();
-
-      if ( aliasToCTEs.containsKey(qName)) {
-        throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(cte.getChild(1)));
-      }
-      aliasToCTEs.put(qName, new CTEClause(qName, cteQry));
-    }
-  }
-
-  /*
-   * We allow CTE definitions in views. So we can end up with a hierarchy of CTE definitions:
-   * - at the top level of a query statement
-   * - where a view is referenced.
-   * - views may refer to other views.
-   *
-   * The scoping rules we use are: to search for a CTE from the current QB outwards. In order to
-   * disambiguate between CTES are different levels we qualify(prefix) them with the id of the QB
-   * they appear in when adding them to the <code>aliasToCTEs</code> map.
-   *
-   */
-  private CTEClause findCTEFromName(QB qb, String cteName) {
-    StringBuilder qId = new StringBuilder();
-    if (qb.getId() != null) {
-      qId.append(qb.getId());
-    }
-
-    while (qId.length() > 0) {
-      String nm = qId + ":" + cteName;
-      CTEClause cte = aliasToCTEs.get(nm);
-      if (cte != null) {
-        return cte;
-      }
-      int lastIndex = qId.lastIndexOf(":");
-      lastIndex = lastIndex < 0 ? 0 : lastIndex;
-      qId.setLength(lastIndex);
-    }
-    return aliasToCTEs.get(cteName);
-  }
-
-  /*
-   * If a CTE is referenced in a QueryBlock:
-   * - add it as a SubQuery for now.
-   *   - SQ.alias is the alias used in QB. (if no alias is specified,
-   *     it used the CTE name. Works just like table references)
-   *   - Adding SQ done by:
-   *     - copying AST of CTE
-   *     - setting ASTOrigin on cloned AST.
-   *   - trigger phase 1 on new QBExpr.
-   *   - update QB data structs: remove this as a table reference, move it to a SQ invocation.
-   */
-  private void addCTEAsSubQuery(QB qb, String cteName, String cteAlias)
-      throws SemanticException {
-    cteAlias = cteAlias == null ? cteName : cteAlias;
-    CTEClause cte = findCTEFromName(qb, cteName);
-    ASTNode cteQryNode = cte.cteNode;
-    QBExpr cteQBExpr = new QBExpr(cteAlias);
-    doPhase1QBExpr(cteQryNode, cteQBExpr, qb.getId(), cteAlias);
-    qb.rewriteCTEToSubq(cteAlias, cteName, cteQBExpr);
-  }
-
-  private final CTEClause rootClause = new CTEClause(null, null);
-
-  @Override
-  public List<Task<? extends Serializable>> getAllRootTasks() {
-    if (!rootTasksResolved) {
-      rootTasks = toRealRootTasks(rootClause.asExecutionOrder());
-      rootTasksResolved = true;
-    }
-    return rootTasks;
-  }
-
-  @Override
-  public HashSet<ReadEntity> getAllInputs() {
-    HashSet<ReadEntity> readEntities = new HashSet<ReadEntity>(getInputs());
-    for (CTEClause cte : rootClause.asExecutionOrder()) {
-      if (cte.source != null) {
-        readEntities.addAll(cte.source.getInputs());
-      }
-    }
-    return readEntities;
-  }
-
-  @Override
-  public HashSet<WriteEntity> getAllOutputs() {
-    HashSet<WriteEntity> writeEntities = new HashSet<WriteEntity>(getOutputs());
-    for (CTEClause cte : rootClause.asExecutionOrder()) {
-      if (cte.source != null) {
-        writeEntities.addAll(cte.source.getOutputs());
-      }
-    }
-    return writeEntities;
-  }
-
-  class CTEClause {
-    CTEClause(String alias, ASTNode cteNode) {
-      this.alias = alias;
-      this.cteNode = cteNode;
-    }
-    String alias;
-    ASTNode cteNode;
-    boolean materialize;
-    int reference;
-    QBExpr qbExpr;
-    List<CTEClause> parents = new ArrayList<CTEClause>();
-
-    // materialized
-    Table table;
-    SemanticAnalyzer source;
-
-    List<Task<? extends Serializable>> getTasks() {
-      return source == null ? null : source.rootTasks;
-    }
-
-    List<CTEClause> asExecutionOrder() {
-      List<CTEClause> execution = new ArrayList<CTEClause>();
-      asExecutionOrder(new HashSet<CTEClause>(), execution);
-      return execution;
-    }
-
-    void asExecutionOrder(Set<CTEClause> visited, List<CTEClause> execution) {
-      for (CTEClause parent : parents) {
-        if (visited.add(parent)) {
-          parent.asExecutionOrder(visited, execution);
-        }
-      }
-      execution.add(this);
-    }
-
-    @Override
-    public String toString() {
-      return alias == null ? "<root>" : alias;
-    }
-  }
-
-  private List<Task<? extends Serializable>> toRealRootTasks(List<CTEClause> execution) {
-    List<Task<? extends Serializable>> cteRoots = new ArrayList<>();
-    List<Task<? extends Serializable>> cteLeafs = new ArrayList<>();
-    List<Task<? extends Serializable>> curTopRoots = null;
-    List<Task<? extends Serializable>> curBottomLeafs = null;
-    for (int i = 0; i < execution.size(); i++) {
-      CTEClause current = execution.get(i);
-      if (current.parents.isEmpty() && curTopRoots != null) {
-        cteRoots.addAll(curTopRoots);
-        cteLeafs.addAll(curBottomLeafs);
-        curTopRoots = curBottomLeafs = null;
-      }
-      List<Task<? extends Serializable>> curTasks = current.getTasks();
-      if (curTasks == null) {
-        continue;
-      }
-      if (curTopRoots == null) {
-        curTopRoots = curTasks;
-      }
-      if (curBottomLeafs != null) {
-        for (Task<?> topLeafTask : curBottomLeafs) {
-          for (Task<?> currentRootTask : curTasks) {
-            topLeafTask.addDependentTask(currentRootTask);
-          }
-        }
-      }
-      curBottomLeafs = Task.findLeafs(curTasks);
-    }
-    if (curTopRoots != null) {
-      cteRoots.addAll(curTopRoots);
-      cteLeafs.addAll(curBottomLeafs);
-    }
-
-    if (cteRoots.isEmpty()) {
-      return rootTasks;
-    }
-    for (Task<?> cteLeafTask : cteLeafs) {
-      for (Task<?> mainRootTask : rootTasks) {
-        cteLeafTask.addDependentTask(mainRootTask);
-      }
-    }
-    return cteRoots;
-  }
-
-  Table materializeCTE(String cteName, CTEClause cte) throws HiveException {
-
-    ASTNode createTable = new ASTNode(new ClassicToken(HiveParser.TOK_CREATETABLE));
-
-    ASTNode tableName = new ASTNode(new ClassicToken(HiveParser.TOK_TABNAME));
-    tableName.addChild(new ASTNode(new ClassicToken(HiveParser.Identifier, cteName)));
-
-    ASTNode temporary = new ASTNode(new ClassicToken(HiveParser.KW_TEMPORARY, MATERIALIZATION_MARKER));
-
-    createTable.addChild(tableName);
-    createTable.addChild(temporary);
-    createTable.addChild(cte.cteNode);
-
-    SemanticAnalyzer analyzer = new SemanticAnalyzer(queryState);
-    analyzer.initCtx(ctx);
-    analyzer.init(false);
-
-    // should share cte contexts
-    analyzer.aliasToCTEs.putAll(aliasToCTEs);
-
-    HiveOperation operation = queryState.getHiveOperation();
-    try {
-      analyzer.analyzeInternal(createTable);
-    } finally {
-      queryState.setCommandType(operation);
-    }
-
-    Table table = analyzer.tableDesc.toTable(conf);
-    Path location = table.getDataLocation();
-    try {
-      location.getFileSystem(conf).mkdirs(location);
-    } catch (IOException e) {
-      throw new HiveException(e);
-    }
-    table.setMaterializedTable(true);
-
-    LOG.info(cteName + " will be materialized into " + location);
-    cte.table = table;
-    cte.source = analyzer;
-
-    ctx.addMaterializedTable(cteName, table);
-
-    return table;
-  }
-
-
-  static boolean isJoinToken(ASTNode node) {
-    if ((node.getToken().getType() == HiveParser.TOK_JOIN)
-        || (node.getToken().getType() == HiveParser.TOK_CROSSJOIN)
-        || isOuterJoinToken(node)
-        || (node.getToken().getType() == HiveParser.TOK_LEFTSEMIJOIN)
-        || (node.getToken().getType() == HiveParser.TOK_UNIQUEJOIN)) {
-      return true;
-    }
-
-    return false;
-  }
-
-  static private boolean isOuterJoinToken(ASTNode node) {
-    return (node.getToken().getType() == HiveParser.TOK_LEFTOUTERJOIN)
-      || (node.getToken().getType() == HiveParser.TOK_RIGHTOUTERJOIN)
-      || (node.getToken().getType() == HiveParser.TOK_FULLOUTERJOIN);
-  }
-
-  /**
-   * Given the AST with TOK_JOIN as the root, get all the aliases for the tables
-   * or subqueries in the join.
-   *
-   * @param qb
-   * @param join
-   * @throws SemanticException
-   */
-  @SuppressWarnings("nls")
-  private void processJoin(QB qb, ASTNode join) throws SemanticException {
-    int numChildren = join.getChildCount();
-    if ((numChildren != 2) && (numChildren != 3)
-        && join.getToken().getType() != HiveParser.TOK_UNIQUEJOIN) {
-      throw new SemanticException(generateErrorMessage(join,
-          "Join with multiple children"));
-    }
-
-    queryProperties.incrementJoinCount(isOuterJoinToken(join));
-    for (int num = 0; num < numChildren; num++) {
-      ASTNode child = (ASTNode) join.getChild(num);
-      if (child.getToken().getType() == HiveParser.TOK_TABREF) {
-        processTable(qb, child);
-      } else if (child.getToken().getType() == HiveParser.TOK_SUBQUERY) {
-        processSubQuery(qb, child);
-      } else if (child.getToken().getType() == HiveParser.TOK_PTBLFUNCTION) {
-        queryProperties.setHasPTF(true);
-        processPTF(qb, child);
-        PTFInvocationSpec ptfInvocationSpec = qb.getPTFInvocationSpec(child);
-        String inputAlias = ptfInvocationSpec == null ? null :
-          ptfInvocationSpec.getFunction().getAlias();;
-        if ( inputAlias == null ) {
-          throw new SemanticException(generateErrorMessage(child,
-              "PTF invocation in a Join must have an alias"));
-        }
-
-      } else if (child.getToken().getType() == HiveParser.TOK_LATERAL_VIEW ||
-          child.getToken().getType() == HiveParser.TOK_LATERAL_VIEW_OUTER) {
-        // SELECT * FROM src1 LATERAL VIEW udtf() AS myTable JOIN src2 ...
-        // is not supported. Instead, the lateral view must be in a subquery
-        // SELECT * FROM (SELECT * FROM src1 LATERAL VIEW udtf() AS myTable) a
-        // JOIN src2 ...
-        throw new SemanticException(ErrorMsg.LATERAL_VIEW_WITH_JOIN
-            .getMsg(join));
-      } else if (isJoinToken(child)) {
-        processJoin(qb, child);
-      }
-    }
-  }
-
-  /**
-   * Given the AST with TOK_LATERAL_VIEW as the root, get the alias for the
-   * table or subquery in the lateral view and also make a mapping from the
-   * alias to all the lateral view AST's.
-   *
-   * @param qb
-   * @param lateralView
-   * @return the alias for the table/subquery
-   * @throws SemanticException
-   */
-
-  private String processLateralView(QB qb, ASTNode lateralView)
-      throws SemanticException {
-    int numChildren = lateralView.getChildCount();
-
-    assert (numChildren == 2);
-    ASTNode next = (ASTNode) lateralView.getChild(1);
-
-    String alias = null;
-
-    switch (next.getToken().getType()) {
-    case HiveParser.TOK_TABREF:
-      alias = processTable(qb, next);
-      break;
-    case HiveParser.TOK_SUBQUERY:
-      alias = processSubQuery(qb, next);
-      break;
-    case HiveParser.TOK_LATERAL_VIEW:
-    case HiveParser.TOK_LATERAL_VIEW_OUTER:
-      alias = processLateralView(qb, next);
-      break;
-    default:
-      throw new SemanticException(ErrorMsg.LATERAL_VIEW_INVALID_CHILD
-          .getMsg(lateralView));
-    }
-    alias = alias.toLowerCase();
-    qb.getParseInfo().addLateralViewForAlias(alias, lateralView);
-    qb.addAlias(alias);
-    return alias;
-  }
-
-  /**
-   * Phase 1: (including, but not limited to):
-   *
-   * 1. Gets all the aliases for all the tables / subqueries and makes the
-   * appropriate mapping in aliasToTabs, aliasToSubq 2. Gets the location of the
-   * destination and names the clause "inclause" + i 3. Creates a map from a
-   * string representation of an aggregation tree to the actual aggregation AST
-   * 4. Creates a mapping from the clause name to the select expression AST in
-   * destToSelExpr 5. Creates a mapping from a table alias to the lateral view
-   * AST's in aliasToLateralViews
-   *
-   * @param ast
-   * @param qb
-   * @param ctx_1
-   * @throws SemanticException
-   */
-  @SuppressWarnings({"fallthrough", "nls"})
-  public boolean doPhase1(ASTNode ast, QB qb, Phase1Ctx ctx_1, PlannerContext plannerCtx)
-      throws SemanticException {
-
-    boolean phase1Result = true;
-    QBParseInfo qbp = qb.getParseInfo();
-    boolean skipRecursion = false;
-
-    if (ast.getToken() != null) {
-      skipRecursion = true;
-      switch (ast.getToken().getType()) {
-      case HiveParser.TOK_SELECTDI:
-        qb.countSelDi();
-        // fall through
-      case HiveParser.TOK_SELECT:
-        qb.countSel();
-        qbp.setSelExprForClause(ctx_1.dest, ast);
-
-        int posn = 0;
-        if (((ASTNode) ast.getChild(0)).getToken().getType() == HiveParser.TOK_HINTLIST) {
-          qbp.setHints((ASTNode) ast.getChild(0));
-          posn++;
-        }
-
-        if ((ast.getChild(posn).getChild(0).getType() == HiveParser.TOK_TRANSFORM))
-          queryProperties.setUsesScript(true);
-
-        LinkedHashMap<String, ASTNode> aggregations = doPhase1GetAggregationsFromSelect(ast,
-            qb, ctx_1.dest);
-        doPhase1GetColumnAliasesFromSelect(ast, qbp);
-        qbp.setAggregationExprsForClause(ctx_1.dest, aggregations);
-        qbp.setDistinctFuncExprsForClause(ctx_1.dest,
-        doPhase1GetDistinctFuncExprs(aggregations));
-        break;
-
-      case HiveParser.TOK_WHERE:
-        qbp.setWhrExprForClause(ctx_1.dest, ast);
-        if (!SubQueryUtils.findSubQueries((ASTNode) ast.getChild(0)).isEmpty())
-            queryProperties.setFilterWithSubQuery(true);
-        break;
-
-      case HiveParser.TOK_INSERT_INTO:
-        String currentDatabase = SessionState.get().getCurrentDatabase();
-        String tab_name = getUnescapedName((ASTNode) ast.getChild(0).getChild(0), currentDatabase);
-        qbp.addInsertIntoTable(tab_name, ast);
-
-      case HiveParser.TOK_DESTINATION:
-        ctx_1.dest = "insclause-" + ctx_1.nextNum;
-        ctx_1.nextNum++;
-        boolean isTmpFileDest = false;
-        if (ast.getChildCount() > 0 && ast.getChild(0) instanceof ASTNode) {
-          ASTNode ch = (ASTNode) ast.getChild(0);
-          if (ch.getToken().getType() == HiveParser.TOK_DIR && ch.getChildCount() > 0
-              && ch.getChild(0) instanceof ASTNode) {
-            ch = (ASTNode) ch.getChild(0);
-            isTmpFileDest = ch.getToken().getType() == HiveParser.TOK_TMP_FILE;
-          } else {
-            if (ast.getToken().getType() == HiveParser.TOK_DESTINATION
-                && ast.getChild(0).getType() == HiveParser.TOK_TAB) {
-              String fullTableName = getUnescapedName((ASTNode) ast.getChild(0).getChild(0),
-                  SessionState.get().getCurrentDatabase());
-              qbp.getInsertOverwriteTables().put(fullTableName, ast);
-            }
-          }
-        }
-
-        // is there a insert in the subquery
-        if (qbp.getIsSubQ() && !isTmpFileDest) {
-          throw new SemanticException(ErrorMsg.NO_INSERT_INSUBQUERY.getMsg(ast));
-        }
-
-        if (plannerCtx != null) {
-          plannerCtx.setInsertToken(ast, isTmpFileDest);
-        }
-
-        qbp.setDestForClause(ctx_1.dest, (ASTNode) ast.getChild(0));
-        handleInsertStatementSpecPhase1(ast, qbp, ctx_1);
-        if (qbp.getClauseNamesForDest().size() > 1) {
-          queryProperties.setMultiDestQuery(true);
-        }
-        break;
-
-      case HiveParser.TOK_FROM:
-        int child_count = ast.getChildCount();
-        if (child_count != 1) {
-          throw new SemanticException(generateErrorMessage(ast,
-              "Multiple Children " + child_count));
-        }
-
-        // Check if this is a subquery / lateral view
-        ASTNode frm = (ASTNode) ast.getChild(0);
-        if (frm.getToken().getType() == HiveParser.TOK_TABREF) {
-          processTable(qb, frm);
-        } else if (frm.getToken().getType() == HiveParser.TOK_VIRTUAL_TABLE) {
-          // Create a temp table with the passed values in it then rewrite this portion of the
-          // tree to be from that table.
-          ASTNode newFrom = genValuesTempTable(frm, qb);
-          ast.setChild(0, newFrom);
-          processTable(qb, newFrom);
-        } else if (frm.getToken().getType() == HiveParser.TOK_SUBQUERY) {
-          processSubQuery(qb, frm);
-        } else if (frm.getToken().getType() == HiveParser.TOK_LATERAL_VIEW ||
-            frm.getToken().getType() == HiveParser.TOK_LATERAL_VIEW_OUTER) {
-          queryProperties.setHasLateralViews(true);
-          processLateralView(qb, frm);
-        } else if (isJoinToken(frm)) {
-          processJoin(qb, frm);
-          qbp.setJoinExpr(frm);
-        }else if(frm.getToken().getType() == HiveParser.TOK_PTBLFUNCTION){
-          queryProperties.setHasPTF(true);
-          processPTF(qb, frm);
-        }
-        break;
-
-      case HiveParser.TOK_CLUSTERBY:
-        // Get the clusterby aliases - these are aliased to the entries in the
-        // select list
-        queryProperties.setHasClusterBy(true);
-        qbp.setClusterByExprForClause(ctx_1.dest, ast);
-        break;
-
-      case HiveParser.TOK_DISTRIBUTEBY:
-        // Get the distribute by aliases - these are aliased to the entries in
-        // the
-        // select list
-        queryProperties.setHasDistributeBy(true);
-        qbp.setDistributeByExprForClause(ctx_1.dest, ast);
-        if (qbp.getClusterByForClause(ctx_1.dest) != null) {
-          throw new SemanticException(generateErrorMessage(ast,
-              ErrorMsg.CLUSTERBY_DISTRIBUTEBY_CONFLICT.getMsg()));
-        } else if (qbp.getOrderByForClause(ctx_1.dest) != null) {
-          throw new SemanticException(generateErrorMessage(ast,
-              ErrorMsg.ORDERBY_DISTRIBUTEBY_CONFLICT.getMsg()));
-        }
-        break;
-
-      case HiveParser.TOK_SORTBY:
-     // Get the sort by aliases - these are aliased to the entries in the
-        // select list
-        queryProperties.setHasSortBy(true);
-        qbp.setSortByExprForClause(ctx_1.dest, ast);
-        if (qbp.getClusterByForClause(ctx_1.dest) != null) {
-          throw new SemanticException(generateErrorMessage(ast,
-              ErrorMsg.CLUSTERBY_SORTBY_CONFLICT.getMsg()));
-        } else if (qbp.getOrderByForClause(ctx_1.dest) != null) {
-          throw new SemanticException(generateErrorMessage(ast,
-              ErrorMsg.ORDERBY_SORTBY_CONFLICT.getMsg()));
-        }
-
-        break;
-
-      case HiveParser.TOK_ORDERBY:
-        // Get the order by aliases - these are aliased to the entries in the
-        // select list
-        queryProperties.setHasOrderBy(true);
-        qbp.setOrderByExprForClause(ctx_1.dest, ast);
-        if (qbp.getClusterByForClause(ctx_1.dest) != null) {
-          throw new SemanticException(generateErrorMessage(ast,
-              ErrorMsg.CLUSTERBY_ORDERBY_CONFLICT.getMsg()));
-        }
-        break;
-
-      case HiveParser.TOK_GROUPBY:
-      case HiveParser.TOK_ROLLUP_GROUPBY:
-      case HiveParser.TOK_CUBE_GROUPBY:
-      case HiveParser.TOK_GROUPING_SETS:
-        // Get the groupby aliases - these are aliased to the entries in the
-        // select list
-        queryProperties.setHasGroupBy(true);
-        if (qbp.getJoinExpr() != null) {
-          queryProperties.setHasJoinFollowedByGroupBy(true);
-        }
-        if (qbp.getSelForClause(ctx_1.dest).getToken().getType() == HiveParser.TOK_SELECTDI) {
-          throw new SemanticException(generateErrorMessage(ast,
-              ErrorMsg.SELECT_DISTINCT_WITH_GROUPBY.getMsg()));
-        }
-        qbp.setGroupByExprForClause(ctx_1.dest, ast);
-        skipRecursion = true;
-
-        // Rollup and Cubes are syntactic sugar on top of grouping sets
-        if (ast.getToken().getType() == HiveParser.TOK_ROLLUP_GROUPBY) {
-          qbp.getDestRollups().add(ctx_1.dest);
-        } else if (ast.getToken().getType() == HiveParser.TOK_CUBE_GROUPBY) {
-          qbp.getDestCubes().add(ctx_1.dest);
-        } else if (ast.getToken().getType() == HiveParser.TOK_GROUPING_SETS) {
-          qbp.getDestGroupingSets().add(ctx_1.dest);
-        }
-        break;
-
-      case HiveParser.TOK_HAVING:
-        qbp.setHavingExprForClause(ctx_1.dest, ast);
-        qbp.addAggregationExprsForClause(ctx_1.dest,
-            doPhase1GetAggregationsFromSelect(ast, qb, ctx_1.dest));
-        break;
-
-      case HiveParser.KW_WINDOW:
-        if (!qb.hasWindowingSpec(ctx_1.dest) ) {
-          throw new SemanticException(generateErrorMessage(ast,
-              "Query has no Cluster/Distribute By; but has a Window definition"));
-        }
-        handleQueryWindowClauses(qb, ctx_1, ast);
-        break;
-
-      case HiveParser.TOK_LIMIT:
-        if (ast.getChildCount() == 2) {
-          qbp.setDestLimit(ctx_1.dest,
-              new Integer(ast.getChild(0).getText()),
-              new Integer(ast.getChild(1).getText()));
-        } else {
-          qbp.setDestLimit(ctx_1.dest, new Integer(0),
-              new Integer(ast.getChild(0).getText()));
-        }
-        break;
-
-      case HiveParser.TOK_ANALYZE:
-        // Case of analyze command
-
-        String table_name = getUnescapedName((ASTNode) ast.getChild(0).getChild(0)).toLowerCase();
-
-
-        qb.setTabAlias(table_name, table_name);
-        qb.addAlias(table_name);
-        qb.getParseInfo().setIsAnalyzeCommand(true);
-        qb.getParseInfo().setNoScanAnalyzeCommand(this.noscan);
-        qb.getParseInfo().setPartialScanAnalyzeCommand(this.partialscan);
-        // Allow analyze the whole table and dynamic partitions
-        HiveConf.setVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict");
-        HiveConf.setVar(conf, HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict");
-
-        break;
-
-      case HiveParser.TOK_UNIONALL:
-        if (!qbp.getIsSubQ()) {
-          // this shouldn't happen. The parser should have converted the union to be
-          // contained in a subquery. Just in case, we keep the error as a fallback.
-          throw new SemanticException(generateErrorMessage(ast,
-              ErrorMsg.UNION_NOTIN_SUBQ.getMsg()));
-        }
-        skipRecursion = false;
-        break;
-
-      case HiveParser.TOK_INSERT:
-        ASTNode destination = (ASTNode) ast.getChild(0);
-        Tree tab = destination.getChild(0);
-
-        // Proceed if AST contains partition & If Not Exists
-        if (destination.getChildCount() == 2 &&
-            tab.getChildCount() == 2 &&
-            destination.getChild(1).getType() == HiveParser.TOK_IFNOTEXISTS) {
-          String tableName = tab.getChild(0).getChild(0).getText();
-
-          Tree partitions = tab.getChild(1);
-          int childCount = partitions.getChildCount();
-          HashMap<String, String> partition = new HashMap<String, String>();
-          for (int i = 0; i < childCount; i++) {
-            String partitionName = partitions.getChild(i).getChild(0).getText();
-            Tree pvalue = partitions.getChild(i).getChild(1);
-            if (pvalue == null) {
-              break;
-            }
-            String partitionVal = stripQuotes(pvalue.getText());
-            partition.put(partitionName, partitionVal);
-          }
-          // if it is a dynamic partition throw the exception
-          if (childCount != partition.size()) {
-            throw new SemanticException(ErrorMsg.INSERT_INTO_DYNAMICPARTITION_IFNOTEXISTS
-                .getMsg(partition.toString()));
-          }
-          Table table = null;
-          try {
-            table = this.getTableObjectByName(tableName);
-          } catch (HiveException ex) {
-            throw new SemanticException(ex);
-          }
-          try {
-            Partition parMetaData = db.getPartition(table, partition, false);
-            // Check partition exists if it exists skip the overwrite
-            if (parMetaData != null) {
-              phase1Result = false;
-              skipRecursion = true;
-              LOG.info("Partition already exists so insert into overwrite " +
-                  "skipped for partition : " + parMetaData.toString());
-              break;
-            }
-          } catch (HiveException e) {
-            LOG.info("Error while getting metadata : ", e);
-          }
-          validatePartSpec(table, partition, (ASTNode)tab, conf, false);
-        }
-        skipRecursion = false;
-        break;
-      case HiveParser.TOK_LATERAL_VIEW:
-      case HiveParser.TOK_LATERAL_VIEW_OUTER:
-        // todo: nested LV
-        assert ast.getChildCount() == 1;
-        qb.getParseInfo().getDestToLateralView().put(ctx_1.dest, ast);
-        break;
-      case HiveParser.TOK_CTE:
-        processCTE(qb, ast);
-        break;
-      default:
-        skipRecursion = false;
-        break;
-      }
-    }
-
-    if (!skipRecursion) {
-      // Iterate over the rest of the children
-      int child_count = ast.getChildCount();
-      for (int child_pos = 0; child_pos < child_count && phase1Result; ++child_pos) {
-        // Recurse
-        phase1Result = phase1Result && doPhase1(
-            (ASTNode)ast.getChild(child_pos), qb, ctx_1, plannerCtx);
-      }
-    }
-    return phase1Result;
-  }
-
-  /**
-   * This is phase1 of supporting specifying schema in insert statement
-   * insert into foo(z,y) select a,b from bar;
-   * @see #handleInsertStatementSpec(java.util.List, String, RowResolver, RowResolver, QB, ASTNode)
-   * @throws SemanticException
-   */
-  private void handleInsertStatementSpecPhase1(ASTNode ast, QBParseInfo qbp, Phase1Ctx ctx_1) throws SemanticException {
-    ASTNode tabColName = (ASTNode)ast.getChild(1);
-    if(ast.getType() == HiveParser.TOK_INSERT_INTO && tabColName != null && tabColName.getType() == HiveParser.TOK_TABCOLNAME) {
-      //we have "insert into foo(a,b)..."; parser will enforce that 1+ columns are listed if TOK_TABCOLNAME is present
-      List<String> targetColNames = new ArrayList<String>();
-      for(Node col : tabColName.getChildren()) {
-        assert ((ASTNode)col).getType() == HiveParser.Identifier :
-          "expected token " + HiveParser.Identifier + " found " + ((ASTNode)col).getType();
-        targetColNames.add(((ASTNode)col).getText());
-      }
-      String fullTableName = getUnescapedName((ASTNode) ast.getChild(0).getChild(0),
-        SessionState.get().getCurrentDatabase());
-      qbp.setDestSchemaForClause(ctx_1.dest, targetColNames);
-      Set<String> targetColumns = new HashSet<String>();
-      targetColumns.addAll(targetColNames);
-      if(targetColNames.size() != targetColumns.size()) {
-        throw new SemanticException(generateErrorMessage(tabColName,
-          "Duplicate column name detected in " + fullTableName + " table schema specification"));
-      }
-      Table targetTable = null;
-      try {
-        targetTable = db.getTable(fullTableName, false);
-      }
-      catch (HiveException ex) {
-        LOG.error("Error processing HiveParser.TOK_DESTINATION: " + ex.getMessage(), ex);
-        throw new SemanticException(ex);
-      }
-      if(targetTable == null) {
-        throw new SemanticException(generateErrorMessage(ast,
-          "Unable to access metadata for table " + fullTableName));
-      }
-      for(FieldSchema f : targetTable.getCols()) {
-        //parser only allows foo(a,b), not foo(foo.a, foo.b)
-        targetColumns.remove(f.getName());
-      }
-      if(!targetColumns.isEmpty()) {//here we need to see if remaining columns are dynamic partition columns
-            /* We just checked the user specified schema columns among regular table column and found some which are not
-            'regular'.  Now check is they are dynamic partition columns
-              For dynamic partitioning,
-              Given "create table multipart(a int, b int) partitioned by (c int, d int);"
-              for "insert into multipart partition(c='1',d)(d,a) values(2,3);" we expect parse tree to look like this
-               (TOK_INSERT_INTO
-                (TOK_TAB
-                  (TOK_TABNAME multipart)
-                  (TOK_PARTSPEC
-                    (TOK_PARTVAL c '1')
-                    (TOK_PARTVAL d)
-                  )
-                )
-                (TOK_TABCOLNAME d a)
-               )*/
-        List<String> dynamicPartitionColumns = new ArrayList<String>();
-        if(ast.getChild(0) != null && ast.getChild(0).getType() == HiveParser.TOK_TAB) {
-          ASTNode tokTab = (ASTNode)ast.getChild(0);
-          ASTNode tokPartSpec = (ASTNode)tokTab.getFirstChildWithType(HiveParser.TOK_PARTSPEC);
-          if(tokPartSpec != null) {
-            for(Node n : tokPartSpec.getChildren()) {
-              ASTNode tokPartVal = null;
-              if(n instanceof ASTNode) {
-                tokPartVal = (ASTNode)n;
-              }
-              if(tokPartVal != null && tokPartVal.getType() == HiveParser.TOK_PARTVAL && tokPartVal.getChildCount() == 1) {
-                assert tokPartVal.getChild(0).getType() == HiveParser.Identifier :
-                  "Expected column name; found tokType=" + tokPartVal.getType();
-                dynamicPartitionColumns.add(tokPartVal.getChild(0).getText());
-              }
-            }
-          }
-        }
-        for(String colName : dynamicPartitionColumns) {
-          targetColumns.remove(colName);
-        }
-        if(!targetColumns.isEmpty()) {
-          //Found some columns in user specified schema which are neither regular not dynamic partition columns
-          throw new SemanticException(generateErrorMessage(tabColName,
-            "'" + (targetColumns.size() == 1 ? targetColumns.iterator().next() : targetColumns) +
-              "' in insert schema specification " + (targetColumns.size() == 1 ? "is" : "are") +
-              " not found among regular columns of " +
-              fullTableName + " nor dynamic partition columns."));
-        }
-      }
-    }
-  }
-
-  public void getMaterializationMetadata(QB qb) throws SemanticException {
-    try {
-      gatherCTEReferences(qb, rootClause);
-      int threshold = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_CTE_MATERIALIZE_THRESHOLD);      
-      for (CTEClause cte : Sets.newHashSet(aliasToCTEs.values())) {
-        if (threshold >= 0 && cte.reference >= threshold) {
-          cte.materialize = true;
-        }
-      }
-    } catch (HiveException e) {
-      // Has to use full name to make sure it does not conflict with
-      // org.apache.commons.lang.StringUtils
-      LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e));
-      if (e instanceof SemanticException) {
-        throw (SemanticException)e;
-      }
-      throw new SemanticException(e.getMessage(), e);
-    }
-  }
-
-  private void gatherCTEReferences(QBExpr qbexpr, CTEClause parent) throws HiveException {
-    if (qbexpr.getOpcode() == QBExpr.Opcode.NULLOP) {
-      gatherCTEReferences(qbexpr.getQB(), parent);
-    } else {
-      gatherCTEReferences(qbexpr.getQBExpr1(), parent);
-      gatherCTEReferences(qbexpr.getQBExpr2(), parent);
-    }
-  }
-
-  // TODO: check view references, too
-  private void gatherCTEReferences(QB qb, CTEClause current) throws HiveException {
-    for (String alias : qb.getTabAliases()) {
-      String tabName = qb.getTabNameForAlias(alias);
-      String cteName = tabName.toLowerCase();
-
-      CTEClause cte = findCTEFromName(qb, cteName);
-      if (cte != null) {
-        if (ctesExpanded.contains(cteName)) {
-          throw new SemanticException("Recursive cte " + cteName +
-                  " detected (cycle: " + StringUtils.join(ctesExpanded, " -> ") +
-                  " -> " + cteName + ").");
-        }
-        cte.reference++;
-        current.parents.add(cte);
-        if (cte.qbExpr != null) {
-          continue;
-        }
-        cte.qbExpr = new QBExpr(cteName);
-        doPhase1QBExpr(cte.cteNode, cte.qbExpr, qb.getId(), cteName);
-
-        ctesExpanded.add(cteName);
-        gatherCTEReferences(cte.qbExpr, cte);
-        ctesExpanded.remove(ctesExpanded.size() - 1);
-      }
-    }
-    for (String alias : qb.getSubqAliases()) {
-      gatherCTEReferences(qb.getSubqForAlias(alias), current);
-    }
-  }
-
-  public void getMetaData(QB qb) throws SemanticException {
-    getMetaData(qb, false);
-  }
-
-  public void getMetaData(QB qb, boolean enableMaterialization) throws SemanticException {
-    try {
-      if (enableMaterialization) {
-        getMaterializationMetadata(qb);
-      }
-      getMetaData(qb, null);
-    } catch (HiveException e) {
-      // Has to use full name to make sure it does not conflict with
-      // org.apache.commons.lang.StringUtils
-      LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e));
-      if (e instanceof SemanticException) {
-        throw (SemanticException)e;
-      }
-      throw new SemanticException(e.getMessage(), e);
-    }
-  }
-
-  private void getMetaData(QBExpr qbexpr, ReadEntity parentInput)
-          throws HiveException {
-    if (qbexpr.getOpcode() == QBExpr.Opcode.NULLOP) {
-      getMetaData(qbexpr.getQB(), parentInput);
-    } else {
-      getMetaData(qbexpr.getQBExpr1(), parentInput);
-      getMetaData(qbexpr.getQBExpr2(), parentInput);
-    }
-  }
-
-  @SuppressWarnings("nls")
-  private void getMetaData(QB qb, ReadEntity parentInput)
-          throws HiveException {
-    LOG.info("Get metadata for source tables");
-
-    // Go over the tables and populate the related structures.
-    // We have to materialize the table alias list since we might
-    // modify it in the middle for view rewrite.
-    List<String> tabAliases = new ArrayList<String>(qb.getTabAliases());
-
-    // Keep track of view alias to view name and read entity
-    // For eg: for a query like 'select * from V3', where V3 -> V2, V2 -> V1, V1 -> T
-    // keeps track of full view name and read entity corresponding to alias V3, V3:V2, V3:V2:V1.
-    // This is needed for tracking the dependencies for inputs, along with their parents.
-    Map<String, ObjectPair<String, ReadEntity>> aliasToViewInfo =
-        new HashMap<String, ObjectPair<String, ReadEntity>>();
-
-    /*
-     * used to capture view to SQ conversions. This is used to check for
-     * recursive CTE invocations.
-     */
-    Map<String, String> sqAliasToCTEName = new HashMap<String, String>();
-
-    for (String alias : tabAliases) {
-      String tabName = qb.getTabNameForAlias(alias);
-      String cteName = tabName.toLowerCase();
-
-      Table tab = db.getTable(tabName, false);
-      if (tab == null ||
-              tab.getDbName().equals(SessionState.get().getCurrentDatabase())) {
-        Table materializedTab = ctx.getMaterializedTable(cteName);
-        if (materializedTab == null) {
-          // we first look for this alias from CTE, and then from catalog.
-          CTEClause cte = findCTEFromName(qb, cteName);
-          if (cte != null) {
-            if (!cte.materialize) {
-              addCTEAsSubQuery(qb, cteName, alias);
-              sqAliasToCTEName.put(alias, cteName);
-              continue;
-            }
-            tab = materializeCTE(cteName, cte);
-          }
-        } else {
-          tab = materializedTab;
-        }
-      }
-
-      if (tab == null) {
-        ASTNode src = qb.getParseInfo().getSrcForAlias(alias);
-        if (null != src) {
-          throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(src));
-        } else {
-          throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(alias));
-        }
-      }
-
-      // Disallow INSERT INTO on bucketized tables
-      boolean isAcid = AcidUtils.isAcidTable(tab);
-      boolean isTableWrittenTo = qb.getParseInfo().isInsertIntoTable(tab.getDbName(), tab.getTableName());
-      if (isTableWrittenTo &&
-          tab.getNumBuckets() > 0 && !isAcid) {
-        throw new SemanticException(ErrorMsg.INSERT_INTO_BUCKETIZED_TABLE.
-            getMsg("Table: " + tabName));
-      }
-      // Disallow update and delete on non-acid tables
-      if ((updating() || deleting()) && !isAcid && isTableWrittenTo) {
-        //isTableWrittenTo: delete from acidTbl where a in (select id from nonAcidTable)
-        //so only assert this if we are actually writing to this table
-        // Whether we are using an acid compliant transaction manager has already been caught in
-        // UpdateDeleteSemanticAnalyzer, so if we are updating or deleting and getting nonAcid
-        // here, it means the table itself doesn't support it.
-        throw new SemanticException(ErrorMsg.ACID_OP_ON_NONACID_TABLE, tabName);
-      }
-
-     if (tab.isView()) {
-        if (qb.getParseInfo().isAnalyzeCommand()) {
-          throw new SemanticException(ErrorMsg.ANALYZE_VIEW.getMsg());
-        }
-        String fullViewName = tab.getDbName() + "." + tab.getTableName();
-        // Prevent view cycles
-        if (viewsExpanded.contains(fullViewName)) {
-          throw new SemanticException("Recursive view " + fullViewName +
-              " detected (cycle: " + StringUtils.join(viewsExpanded, " -> ") +
-              " -> " + fullViewName + ").");
-        }
-        replaceViewReferenceWithDefinition(qb, tab, tabName, alias);
-        // This is the last time we'll see the Table objects for views, so add it to the inputs
-        // now. isInsideView will tell if this view is embedded in another view.
-        ReadEntity viewInput = new ReadEntity(tab, parentInput, !qb.isInsideView());
-        viewInput = PlanUtils.addInput(inputs, viewInput);
-        aliasToViewInfo.put(alias, new ObjectPair<String, ReadEntity>(fullViewName, viewInput));
-        String aliasId = getAliasId(alias, qb);
-        if (aliasId != null) {
-          aliasId = aliasId.replace(SemanticAnalyzer.SUBQUERY_TAG_1, "")
-            .replace(SemanticAnalyzer.SUBQUERY_TAG_2, "");
-        }
-        viewAliasToInput.put(aliasId, viewInput);
-        continue;
-      }
-
-      if (!InputFormat.class.isAssignableFrom(tab.getInputFormatClass())) {
-        throw new SemanticException(generateErrorMessage(
-                qb.getParseInfo().getSrcForAlias(alias),
-                ErrorMsg.INVALID_INPUT_FORMAT_TYPE.getMsg()));
-      }
-
-      qb.getMetaData().setSrcForAlias(alias, tab);
-
-      if (qb.getParseInfo().isAnalyzeCommand()) {
-        // allow partial partition specification for nonscan since noscan is fast.
-        TableSpec ts = new TableSpec(db, conf, (ASTNode) ast.getChild(0), true, this.noscan);
-        if (ts.specType == SpecType.DYNAMIC_PARTITION) { // dynamic partitions
-          try {
-            ts.partitions = db.getPartitionsByNames(ts.tableHandle, ts.partSpec);
-          } catch (HiveException e) {
-            throw new SemanticException(generateErrorMessage(
-                    qb.getParseInfo().getSrcForAlias(alias),
-                    "Cannot get partitions for " + ts.partSpec), e);
-          }
-        }
-        // validate partial scan command
-        QBParseInfo qbpi = qb.getParseInfo();
-        if (qbpi.isPartialScanAnalyzeCommand()) {
-          Class<? extends InputFormat> inputFormatClass = null;
-          switch (ts.specType) {
-            case TABLE_ONLY:
-            case DYNAMIC_PARTITION:
-              inputFormatClass = ts.tableHandle.getInputFormatClass();
-              break;
-            case STATIC_PARTITION:
-              inputFormatClass = ts.partHandle.getInputFormatClass();
-              break;
-            default:
-              assert false;
-          }
-          // throw a HiveException for formats other than rcfile or orcfile.
-          if (!(inputFormatClass.equals(RCFileInputFormat.class) || inputFormatClass
-                  .equals(OrcInputFormat.class))) {
-            throw new SemanticException(ErrorMsg.ANALYZE_TABLE_PARTIALSCAN_NON_RCFILE.getMsg());
-          }
-        }
-
-        tab.setTableSpec(ts);
-        qb.getParseInfo().addTableSpec(alias, ts);
-      }
-
-      ReadEntity parentViewInfo = PlanUtils.getParentViewInfo(getAliasId(alias, qb), viewAliasToInput);
-      PlanUtils.addInput(inputs,
-              new ReadEntity(tab, parentViewInfo, parentViewInfo == null),mergeIsDirect);
-    }
-
-    LOG.info("Get metadata for subqueries");
-    // Go over the subqueries and getMetaData for these
-    for (String alias : qb.getSubqAliases()) {
-      boolean wasView = aliasToViewInfo.containsKey(alias);
-      boolean wasCTE = sqAliasToCTEName.containsKey(alias);
-      ReadEntity newParentInput = null;
-      if (wasView) {
-        viewsExpanded.add(aliasToViewInfo.get(alias).getFirst());
-        newParentInput = aliasToViewInfo.get(alias).getSecond();
-      } else if (wasCTE) {
-        ctesExpanded.add(sqAliasToCTEName.get(alias));
-      }
-      QBExpr qbexpr = qb.getSubqForAlias(alias);
-      getMetaData(qbexpr, newParentInput);
-      if (wasView) {
-        viewsExpanded.remove(viewsExpanded.size() - 1);
-      } else if (wasCTE) {
-        ctesExpanded.remove(ctesExpanded.size() - 1);
-      }
-    }
-
-    RowFormatParams rowFormatParams = new RowFormatParams();
-    StorageFormat storageFormat = new StorageFormat(conf);
-
-    LOG.info("Get metadata for destination tables");
-    // Go over all the destination structures and populate the related
-    // metadata
-    QBParseInfo qbp = qb.getParseInfo();
-
-    for (String name : qbp.getClauseNamesForDest()) {
-      ASTNode ast = qbp.getDestForClause(name);
-      switch (ast.getToken().getType()) {
-        case HiveParser.TOK_TAB: {
-          TableSpec ts = new TableSpec(db, conf, ast);
-          if (ts.tableHandle.isView()) {
-            throw new SemanticException(ErrorMsg.DML_AGAINST_VIEW.getMsg());
-          }
-
-          Class<?> outputFormatClass = ts.tableHandle.getOutputFormatClass();
-          if (!ts.tableHandle.isNonNative() &&
-              !HiveOutputFormat.class.isAssignableFrom(outputFormatClass)) {
-            throw new SemanticException(ErrorMsg.INVALID_OUTPUT_FORMAT_TYPE
-                .getMsg(ast, "The class is " + outputFormatClass.toString()));
-          }
-
-          // TableSpec ts is got from the query (user specified),
-          // which means the user didn't specify partitions in their query,
-          // but whether the table itself is partitioned is not know.
-          if (ts.specType != SpecType.STATIC_PARTITION) {
-            // This is a table or dynamic partition
-            qb.getMetaData().setDestForAlias(name, ts.tableHandle);
-            // has dynamic as well as static partitions
-            if (ts.partSpec != null && ts.partSpec.size() > 0) {
-              qb.getMetaData().setPartSpecForAlias(name, ts.partSpec);
-            }
-          } else {
-            // This is a partition
-            qb.getMetaData().setDestForAlias(name, ts.partHandle);
-          }
-          if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVESTATSAUTOGATHER)) {
-            // Add the table spec for the destination table.
-            qb.getParseInfo().addTableSpec(ts.tableName.toLowerCase(), ts);
-          }
-          break;
-        }
-
-        case HiveParser.TOK_DIR: {
-          // This is a dfs file
-          String fname = stripQuotes(ast.getChild(0).getText());
-          if ((!qb.getParseInfo().getIsSubQ())
-              && (((ASTNode) ast.getChild(0)).getToken().getType() == HiveParser.TOK_TMP_FILE)) {
-
-            if (qb.isCTAS()) {
-              qb.setIsQuery(false);
-              ctx.setResDir(null);
-              ctx.setResFile(null);
-
-              // allocate

<TRUNCATED>

[20/31] hive git commit: HIVE-14681: Remove AssertionError for non zero return codes in QTestUtil cleanup (Siddharth Seth reviewed by Hari Subramaniyan)

Posted by se...@apache.org.
HIVE-14681: Remove AssertionError for non zero return codes in QTestUtil cleanup (Siddharth Seth reviewed by Hari Subramaniyan)


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

Branch: refs/heads/hive-14535
Commit: 518789df22310b2ff95e92d0ee7cf7636daa70b3
Parents: e09f3c7
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Mon Sep 12 12:09:45 2016 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Mon Sep 12 12:09:45 2016 -0700

----------------------------------------------------------------------
 .../util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/518789df/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 4d4a929..f456dfb 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
@@ -860,7 +860,8 @@ public class QTestUtil {
       SessionState.get().getConf().setBoolean("hive.test.shutdown.phase", true);
       int result = cliDriver.processLine(cleanupCommands);
       if (result != 0) {
-        Assert.fail("Failed during cleanup processLine with code=" + result);
+        LOG.error("Failed during cleanup processLine with code={}. Ignoring", result);
+        // TODO Convert this to an Assert.fail once HIVE-14682 is fixed
       }
       SessionState.get().getConf().setBoolean("hive.test.shutdown.phase", false);
     } else {


[12/31] hive git commit: HIVE-14542 - VirtualColumn::equals() should use object equality (Eugene Koifman, reviewed by Gopal V)

Posted by se...@apache.org.
HIVE-14542 - VirtualColumn::equals() should use object equality (Eugene Koifman, reviewed by Gopal V)


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

Branch: refs/heads/hive-14535
Commit: 0755348d781ab96a260db2a4fc6a4eabb8986d73
Parents: 943a361
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Thu Sep 8 17:45:26 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Thu Sep 8 17:45:26 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/MapOperator.java | 110 ++++++++++---------
 .../hadoop/hive/ql/metadata/VirtualColumn.java  |  41 ++-----
 2 files changed, 67 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0755348d/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
index f3eed75..4bdd3c9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
@@ -542,61 +542,67 @@ public class MapOperator extends AbstractMapOperator {
       vcValues = new Object[vcs.size()];
     }
     for (int i = 0; i < vcs.size(); i++) {
-      VirtualColumn vc = vcs.get(i);
-      if (vc.equals(VirtualColumn.FILENAME)) {
-        if (ctx.inputFileChanged()) {
-          vcValues[i] = new Text(ctx.getCurrentInputPath().toString());
-        }
-      } else if (vc.equals(VirtualColumn.BLOCKOFFSET)) {
-        long current = ctx.getIoCxt().getCurrentBlockStart();
-        LongWritable old = (LongWritable) vcValues[i];
-        if (old == null) {
-          old = new LongWritable(current);
-          vcValues[i] = old;
-          continue;
-        }
-        if (current != old.get()) {
-          old.set(current);
-        }
-      } else if (vc.equals(VirtualColumn.ROWOFFSET)) {
-        long current = ctx.getIoCxt().getCurrentRow();
-        LongWritable old = (LongWritable) vcValues[i];
-        if (old == null) {
-          old = new LongWritable(current);
-          vcValues[i] = old;
-          continue;
-        }
-        if (current != old.get()) {
-          old.set(current);
-        }
-      } else if (vc.equals(VirtualColumn.RAWDATASIZE)) {
-        long current = 0L;
-        SerDeStats stats = deserializer.getSerDeStats();
-        if(stats != null) {
-          current = stats.getRawDataSize();
-        }
-        LongWritable old = (LongWritable) vcValues[i];
-        if (old == null) {
-          old = new LongWritable(current);
-          vcValues[i] = old;
-          continue;
-        }
-        if (current != old.get()) {
-          old.set(current);
-        }
-      }
-      else if(vc.equals(VirtualColumn.ROWID)) {
-        if(ctx.getIoCxt().getRecordIdentifier() == null) {
-          vcValues[i] = null;
+      switch(vcs.get(i)) {
+        case FILENAME :
+          if (ctx.inputFileChanged()) {
+            vcValues[i] = new Text(ctx.getCurrentInputPath().toString());
+          }
+          break;
+        case BLOCKOFFSET: {
+          long current = ctx.getIoCxt().getCurrentBlockStart();
+          LongWritable old = (LongWritable) vcValues[i];
+          if (old == null) {
+            old = new LongWritable(current);
+            vcValues[i] = old;
+            continue;
+          }
+          if (current != old.get()) {
+            old.set(current);
+          }
         }
-        else {
-          if(vcValues[i] == null) {
-            vcValues[i] = new Object[RecordIdentifier.Field.values().length];
+        break;
+        case ROWOFFSET: {
+          long current = ctx.getIoCxt().getCurrentRow();
+          LongWritable old = (LongWritable) vcValues[i];
+          if (old == null) {
+            old = new LongWritable(current);
+            vcValues[i] = old;
+            continue;
+          }
+          if (current != old.get()) {
+            old.set(current);
           }
-          RecordIdentifier.StructInfo.toArray(ctx.getIoCxt().getRecordIdentifier(), (Object[])vcValues[i]);
-          ctx.getIoCxt().setRecordIdentifier(null);//so we don't accidentally cache the value; shouldn't
-          //happen since IO layer either knows how to produce ROW__ID or not - but to be safe
         }
+        break;
+        case RAWDATASIZE:
+          long current = 0L;
+          SerDeStats stats = deserializer.getSerDeStats();
+          if(stats != null) {
+            current = stats.getRawDataSize();
+          }
+          LongWritable old = (LongWritable) vcValues[i];
+          if (old == null) {
+            old = new LongWritable(current);
+            vcValues[i] = old;
+            continue;
+          }
+          if (current != old.get()) {
+            old.set(current);
+          }
+          break;
+        case ROWID:
+          if(ctx.getIoCxt().getRecordIdentifier() == null) {
+            vcValues[i] = null;
+          }
+          else {
+            if(vcValues[i] == null) {
+              vcValues[i] = new Object[RecordIdentifier.Field.values().length];
+            }
+            RecordIdentifier.StructInfo.toArray(ctx.getIoCxt().getRecordIdentifier(), (Object[])vcValues[i]);
+            ctx.getIoCxt().setRecordIdentifier(null);//so we don't accidentally cache the value; shouldn't
+            //happen since IO layer either knows how to produce ROW__ID or not - but to be safe
+          }
+	  break;
       }
     }
     return vcValues;

http://git-wip-us.apache.org/repos/asf/hive/blob/0755348d/ql/src/java/org/apache/hadoop/hive/ql/metadata/VirtualColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/VirtualColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/VirtualColumn.java
index 3549143..044d64c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/VirtualColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/VirtualColumn.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.metadata;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -41,19 +40,16 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 
 @InterfaceAudience.Private
-public class VirtualColumn implements Serializable {
+public enum VirtualColumn {
+  FILENAME("INPUT__FILE__NAME", TypeInfoFactory.stringTypeInfo),
+  BLOCKOFFSET("BLOCK__OFFSET__INSIDE__FILE", TypeInfoFactory.longTypeInfo),
+  ROWOFFSET("ROW__OFFSET__INSIDE__BLOCK", TypeInfoFactory.longTypeInfo),
 
-  private static final long serialVersionUID = 1L;
-
-  public static final VirtualColumn FILENAME = new VirtualColumn("INPUT__FILE__NAME", (PrimitiveTypeInfo)TypeInfoFactory.stringTypeInfo);
-  public static final VirtualColumn BLOCKOFFSET = new VirtualColumn("BLOCK__OFFSET__INSIDE__FILE", (PrimitiveTypeInfo)TypeInfoFactory.longTypeInfo);
-  public static final VirtualColumn ROWOFFSET = new VirtualColumn("ROW__OFFSET__INSIDE__BLOCK", (PrimitiveTypeInfo)TypeInfoFactory.longTypeInfo);
-
-  public static final VirtualColumn RAWDATASIZE = new VirtualColumn("RAW__DATA__SIZE", (PrimitiveTypeInfo)TypeInfoFactory.longTypeInfo);
+  RAWDATASIZE("RAW__DATA__SIZE", TypeInfoFactory.longTypeInfo),
   /**
    * {@link org.apache.hadoop.hive.ql.io.RecordIdentifier} 
    */
-  public static final VirtualColumn ROWID = new VirtualColumn("ROW__ID", RecordIdentifier.StructInfo.typeInfo, true, RecordIdentifier.StructInfo.oi);
+  ROWID("ROW__ID", RecordIdentifier.StructInfo.typeInfo, true, RecordIdentifier.StructInfo.oi),
 
   /**
    * GROUPINGID is used with GROUP BY GROUPINGS SETS, ROLLUP and CUBE.
@@ -62,8 +58,7 @@ public class VirtualColumn implements Serializable {
    * set if that column has been aggregated in that row. Otherwise the
    * value is "0".  Returns the decimal representation of the bit vector.
    */
-  public static final VirtualColumn GROUPINGID =
-      new VirtualColumn("GROUPING__ID", (PrimitiveTypeInfo) TypeInfoFactory.intTypeInfo);
+  GROUPINGID("GROUPING__ID", TypeInfoFactory.intTypeInfo);
 
   public static ImmutableSet<String> VIRTUAL_COLUMN_NAMES =
       ImmutableSet.of(FILENAME.getName(), BLOCKOFFSET.getName(), ROWOFFSET.getName(),
@@ -74,12 +69,12 @@ public class VirtualColumn implements Serializable {
   private final boolean isHidden;
   private final ObjectInspector oi;
 
-  private VirtualColumn(String name, PrimitiveTypeInfo typeInfo) {
+  VirtualColumn(String name, PrimitiveTypeInfo typeInfo) {
     this(name, typeInfo, true, 
       PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(typeInfo));
   }
 
-  private VirtualColumn(String name, TypeInfo typeInfo, boolean isHidden, ObjectInspector oi) {
+  VirtualColumn(String name, TypeInfo typeInfo, boolean isHidden, ObjectInspector oi) {
     this.name = name;
     this.typeInfo = typeInfo;
     this.isHidden = isHidden;
@@ -124,24 +119,6 @@ public class VirtualColumn implements Serializable {
     return oi;
   }
 
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if(!(o instanceof VirtualColumn)) {
-      return false;
-    }
-    VirtualColumn c = (VirtualColumn) o;
-    return this.name.equals(c.name)
-        && this.typeInfo.getTypeName().equals(c.getTypeInfo().getTypeName());
-  }
-  @Override
-  public int hashCode() {
-    int c = 19;
-    c = 31 * name.hashCode() + c;
-    return  31 * typeInfo.getTypeName().hashCode() + c;
-  }
   public static Collection<String> removeVirtualColumns(final Collection<String> columns) {
     Iterables.removeAll(columns, VIRTUAL_COLUMN_NAMES);
     return columns;


[27/31] hive git commit: HIVE-14644 : use metastore information on the read path appropriately (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 d228a73..17a8fa3 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1178,6 +1178,11 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    * @return \metastore\HeartbeatWriteIdResult
    */
   public function heartbeat_write_id(\metastore\HeartbeatWriteIdRequest $req);
+  /**
+   * @param \metastore\GetValidWriteIdsRequest $req
+   * @return \metastore\GetValidWriteIdsResult
+   */
+  public function get_valid_write_ids(\metastore\GetValidWriteIdsRequest $req);
 }
 
 class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metastore\ThriftHiveMetastoreIf {
@@ -9809,6 +9814,57 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("heartbeat_write_id failed: unknown result");
   }
 
+  public function get_valid_write_ids(\metastore\GetValidWriteIdsRequest $req)
+  {
+    $this->send_get_valid_write_ids($req);
+    return $this->recv_get_valid_write_ids();
+  }
+
+  public function send_get_valid_write_ids(\metastore\GetValidWriteIdsRequest $req)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_valid_write_ids_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_, 'get_valid_write_ids', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_valid_write_ids', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_valid_write_ids()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_valid_write_ids_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_get_valid_write_ids_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    throw new \Exception("get_valid_write_ids failed: unknown result");
+  }
+
 }
 
 // HELPER FUNCTIONS AND STRUCTURES
@@ -10960,14 +11016,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size569 = 0;
-            $_etype572 = 0;
-            $xfer += $input->readListBegin($_etype572, $_size569);
-            for ($_i573 = 0; $_i573 < $_size569; ++$_i573)
+            $_size576 = 0;
+            $_etype579 = 0;
+            $xfer += $input->readListBegin($_etype579, $_size576);
+            for ($_i580 = 0; $_i580 < $_size576; ++$_i580)
             {
-              $elem574 = null;
-              $xfer += $input->readString($elem574);
-              $this->success []= $elem574;
+              $elem581 = null;
+              $xfer += $input->readString($elem581);
+              $this->success []= $elem581;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11003,9 +11059,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter575)
+          foreach ($this->success as $iter582)
           {
-            $xfer += $output->writeString($iter575);
+            $xfer += $output->writeString($iter582);
           }
         }
         $output->writeListEnd();
@@ -11136,14 +11192,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size576 = 0;
-            $_etype579 = 0;
-            $xfer += $input->readListBegin($_etype579, $_size576);
-            for ($_i580 = 0; $_i580 < $_size576; ++$_i580)
+            $_size583 = 0;
+            $_etype586 = 0;
+            $xfer += $input->readListBegin($_etype586, $_size583);
+            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
             {
-              $elem581 = null;
-              $xfer += $input->readString($elem581);
-              $this->success []= $elem581;
+              $elem588 = null;
+              $xfer += $input->readString($elem588);
+              $this->success []= $elem588;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11179,9 +11235,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter582)
+          foreach ($this->success as $iter589)
           {
-            $xfer += $output->writeString($iter582);
+            $xfer += $output->writeString($iter589);
           }
         }
         $output->writeListEnd();
@@ -12182,18 +12238,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size583 = 0;
-            $_ktype584 = 0;
-            $_vtype585 = 0;
-            $xfer += $input->readMapBegin($_ktype584, $_vtype585, $_size583);
-            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
+            $_size590 = 0;
+            $_ktype591 = 0;
+            $_vtype592 = 0;
+            $xfer += $input->readMapBegin($_ktype591, $_vtype592, $_size590);
+            for ($_i594 = 0; $_i594 < $_size590; ++$_i594)
             {
-              $key588 = '';
-              $val589 = new \metastore\Type();
-              $xfer += $input->readString($key588);
-              $val589 = new \metastore\Type();
-              $xfer += $val589->read($input);
-              $this->success[$key588] = $val589;
+              $key595 = '';
+              $val596 = new \metastore\Type();
+              $xfer += $input->readString($key595);
+              $val596 = new \metastore\Type();
+              $xfer += $val596->read($input);
+              $this->success[$key595] = $val596;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -12229,10 +12285,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter590 => $viter591)
+          foreach ($this->success as $kiter597 => $viter598)
           {
-            $xfer += $output->writeString($kiter590);
-            $xfer += $viter591->write($output);
+            $xfer += $output->writeString($kiter597);
+            $xfer += $viter598->write($output);
           }
         }
         $output->writeMapEnd();
@@ -12436,15 +12492,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size592 = 0;
-            $_etype595 = 0;
-            $xfer += $input->readListBegin($_etype595, $_size592);
-            for ($_i596 = 0; $_i596 < $_size592; ++$_i596)
+            $_size599 = 0;
+            $_etype602 = 0;
+            $xfer += $input->readListBegin($_etype602, $_size599);
+            for ($_i603 = 0; $_i603 < $_size599; ++$_i603)
             {
-              $elem597 = null;
-              $elem597 = new \metastore\FieldSchema();
-              $xfer += $elem597->read($input);
-              $this->success []= $elem597;
+              $elem604 = null;
+              $elem604 = new \metastore\FieldSchema();
+              $xfer += $elem604->read($input);
+              $this->success []= $elem604;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12496,9 +12552,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter598)
+          foreach ($this->success as $iter605)
           {
-            $xfer += $iter598->write($output);
+            $xfer += $iter605->write($output);
           }
         }
         $output->writeListEnd();
@@ -12740,15 +12796,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size599 = 0;
-            $_etype602 = 0;
-            $xfer += $input->readListBegin($_etype602, $_size599);
-            for ($_i603 = 0; $_i603 < $_size599; ++$_i603)
+            $_size606 = 0;
+            $_etype609 = 0;
+            $xfer += $input->readListBegin($_etype609, $_size606);
+            for ($_i610 = 0; $_i610 < $_size606; ++$_i610)
             {
-              $elem604 = null;
-              $elem604 = new \metastore\FieldSchema();
-              $xfer += $elem604->read($input);
-              $this->success []= $elem604;
+              $elem611 = null;
+              $elem611 = new \metastore\FieldSchema();
+              $xfer += $elem611->read($input);
+              $this->success []= $elem611;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12800,9 +12856,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter605)
+          foreach ($this->success as $iter612)
           {
-            $xfer += $iter605->write($output);
+            $xfer += $iter612->write($output);
           }
         }
         $output->writeListEnd();
@@ -13016,15 +13072,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size606 = 0;
-            $_etype609 = 0;
-            $xfer += $input->readListBegin($_etype609, $_size606);
-            for ($_i610 = 0; $_i610 < $_size606; ++$_i610)
+            $_size613 = 0;
+            $_etype616 = 0;
+            $xfer += $input->readListBegin($_etype616, $_size613);
+            for ($_i617 = 0; $_i617 < $_size613; ++$_i617)
             {
-              $elem611 = null;
-              $elem611 = new \metastore\FieldSchema();
-              $xfer += $elem611->read($input);
-              $this->success []= $elem611;
+              $elem618 = null;
+              $elem618 = new \metastore\FieldSchema();
+              $xfer += $elem618->read($input);
+              $this->success []= $elem618;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13076,9 +13132,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter612)
+          foreach ($this->success as $iter619)
           {
-            $xfer += $iter612->write($output);
+            $xfer += $iter619->write($output);
           }
         }
         $output->writeListEnd();
@@ -13320,15 +13376,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size613 = 0;
-            $_etype616 = 0;
-            $xfer += $input->readListBegin($_etype616, $_size613);
-            for ($_i617 = 0; $_i617 < $_size613; ++$_i617)
+            $_size620 = 0;
+            $_etype623 = 0;
+            $xfer += $input->readListBegin($_etype623, $_size620);
+            for ($_i624 = 0; $_i624 < $_size620; ++$_i624)
             {
-              $elem618 = null;
-              $elem618 = new \metastore\FieldSchema();
-              $xfer += $elem618->read($input);
-              $this->success []= $elem618;
+              $elem625 = null;
+              $elem625 = new \metastore\FieldSchema();
+              $xfer += $elem625->read($input);
+              $this->success []= $elem625;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13380,9 +13436,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter619)
+          foreach ($this->success as $iter626)
           {
-            $xfer += $iter619->write($output);
+            $xfer += $iter626->write($output);
           }
         }
         $output->writeListEnd();
@@ -13990,15 +14046,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size620 = 0;
-            $_etype623 = 0;
-            $xfer += $input->readListBegin($_etype623, $_size620);
-            for ($_i624 = 0; $_i624 < $_size620; ++$_i624)
+            $_size627 = 0;
+            $_etype630 = 0;
+            $xfer += $input->readListBegin($_etype630, $_size627);
+            for ($_i631 = 0; $_i631 < $_size627; ++$_i631)
             {
-              $elem625 = null;
-              $elem625 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem625->read($input);
-              $this->primaryKeys []= $elem625;
+              $elem632 = null;
+              $elem632 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem632->read($input);
+              $this->primaryKeys []= $elem632;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14008,15 +14064,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size626 = 0;
-            $_etype629 = 0;
-            $xfer += $input->readListBegin($_etype629, $_size626);
-            for ($_i630 = 0; $_i630 < $_size626; ++$_i630)
+            $_size633 = 0;
+            $_etype636 = 0;
+            $xfer += $input->readListBegin($_etype636, $_size633);
+            for ($_i637 = 0; $_i637 < $_size633; ++$_i637)
             {
-              $elem631 = null;
-              $elem631 = new \metastore\SQLForeignKey();
-              $xfer += $elem631->read($input);
-              $this->foreignKeys []= $elem631;
+              $elem638 = null;
+              $elem638 = new \metastore\SQLForeignKey();
+              $xfer += $elem638->read($input);
+              $this->foreignKeys []= $elem638;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14052,9 +14108,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter632)
+          foreach ($this->primaryKeys as $iter639)
           {
-            $xfer += $iter632->write($output);
+            $xfer += $iter639->write($output);
           }
         }
         $output->writeListEnd();
@@ -14069,9 +14125,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter633)
+          foreach ($this->foreignKeys as $iter640)
           {
-            $xfer += $iter633->write($output);
+            $xfer += $iter640->write($output);
           }
         }
         $output->writeListEnd();
@@ -15417,14 +15473,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size634 = 0;
-            $_etype637 = 0;
-            $xfer += $input->readListBegin($_etype637, $_size634);
-            for ($_i638 = 0; $_i638 < $_size634; ++$_i638)
+            $_size641 = 0;
+            $_etype644 = 0;
+            $xfer += $input->readListBegin($_etype644, $_size641);
+            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
             {
-              $elem639 = null;
-              $xfer += $input->readString($elem639);
-              $this->success []= $elem639;
+              $elem646 = null;
+              $xfer += $input->readString($elem646);
+              $this->success []= $elem646;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15460,9 +15516,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter640)
+          foreach ($this->success as $iter647)
           {
-            $xfer += $output->writeString($iter640);
+            $xfer += $output->writeString($iter647);
           }
         }
         $output->writeListEnd();
@@ -15567,14 +15623,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size641 = 0;
-            $_etype644 = 0;
-            $xfer += $input->readListBegin($_etype644, $_size641);
-            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
+            $_size648 = 0;
+            $_etype651 = 0;
+            $xfer += $input->readListBegin($_etype651, $_size648);
+            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
             {
-              $elem646 = null;
-              $xfer += $input->readString($elem646);
-              $this->tbl_types []= $elem646;
+              $elem653 = null;
+              $xfer += $input->readString($elem653);
+              $this->tbl_types []= $elem653;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15612,9 +15668,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter647)
+          foreach ($this->tbl_types as $iter654)
           {
-            $xfer += $output->writeString($iter647);
+            $xfer += $output->writeString($iter654);
           }
         }
         $output->writeListEnd();
@@ -15691,15 +15747,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size648 = 0;
-            $_etype651 = 0;
-            $xfer += $input->readListBegin($_etype651, $_size648);
-            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
+            $_size655 = 0;
+            $_etype658 = 0;
+            $xfer += $input->readListBegin($_etype658, $_size655);
+            for ($_i659 = 0; $_i659 < $_size655; ++$_i659)
             {
-              $elem653 = null;
-              $elem653 = new \metastore\TableMeta();
-              $xfer += $elem653->read($input);
-              $this->success []= $elem653;
+              $elem660 = null;
+              $elem660 = new \metastore\TableMeta();
+              $xfer += $elem660->read($input);
+              $this->success []= $elem660;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15735,9 +15791,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter654)
+          foreach ($this->success as $iter661)
           {
-            $xfer += $iter654->write($output);
+            $xfer += $iter661->write($output);
           }
         }
         $output->writeListEnd();
@@ -15893,14 +15949,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size655 = 0;
-            $_etype658 = 0;
-            $xfer += $input->readListBegin($_etype658, $_size655);
-            for ($_i659 = 0; $_i659 < $_size655; ++$_i659)
+            $_size662 = 0;
+            $_etype665 = 0;
+            $xfer += $input->readListBegin($_etype665, $_size662);
+            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
             {
-              $elem660 = null;
-              $xfer += $input->readString($elem660);
-              $this->success []= $elem660;
+              $elem667 = null;
+              $xfer += $input->readString($elem667);
+              $this->success []= $elem667;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15936,9 +15992,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter661)
+          foreach ($this->success as $iter668)
           {
-            $xfer += $output->writeString($iter661);
+            $xfer += $output->writeString($iter668);
           }
         }
         $output->writeListEnd();
@@ -16253,14 +16309,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size662 = 0;
-            $_etype665 = 0;
-            $xfer += $input->readListBegin($_etype665, $_size662);
-            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
+            $_size669 = 0;
+            $_etype672 = 0;
+            $xfer += $input->readListBegin($_etype672, $_size669);
+            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
             {
-              $elem667 = null;
-              $xfer += $input->readString($elem667);
-              $this->tbl_names []= $elem667;
+              $elem674 = null;
+              $xfer += $input->readString($elem674);
+              $this->tbl_names []= $elem674;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16293,9 +16349,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter668)
+          foreach ($this->tbl_names as $iter675)
           {
-            $xfer += $output->writeString($iter668);
+            $xfer += $output->writeString($iter675);
           }
         }
         $output->writeListEnd();
@@ -16396,15 +16452,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size669 = 0;
-            $_etype672 = 0;
-            $xfer += $input->readListBegin($_etype672, $_size669);
-            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
+            $_size676 = 0;
+            $_etype679 = 0;
+            $xfer += $input->readListBegin($_etype679, $_size676);
+            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
             {
-              $elem674 = null;
-              $elem674 = new \metastore\Table();
-              $xfer += $elem674->read($input);
-              $this->success []= $elem674;
+              $elem681 = null;
+              $elem681 = new \metastore\Table();
+              $xfer += $elem681->read($input);
+              $this->success []= $elem681;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16456,9 +16512,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter675)
+          foreach ($this->success as $iter682)
           {
-            $xfer += $iter675->write($output);
+            $xfer += $iter682->write($output);
           }
         }
         $output->writeListEnd();
@@ -16694,14 +16750,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size676 = 0;
-            $_etype679 = 0;
-            $xfer += $input->readListBegin($_etype679, $_size676);
-            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
+            $_size683 = 0;
+            $_etype686 = 0;
+            $xfer += $input->readListBegin($_etype686, $_size683);
+            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
             {
-              $elem681 = null;
-              $xfer += $input->readString($elem681);
-              $this->success []= $elem681;
+              $elem688 = null;
+              $xfer += $input->readString($elem688);
+              $this->success []= $elem688;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16753,9 +16809,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter682)
+          foreach ($this->success as $iter689)
           {
-            $xfer += $output->writeString($iter682);
+            $xfer += $output->writeString($iter689);
           }
         }
         $output->writeListEnd();
@@ -18068,15 +18124,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size683 = 0;
-            $_etype686 = 0;
-            $xfer += $input->readListBegin($_etype686, $_size683);
-            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
+            $_size690 = 0;
+            $_etype693 = 0;
+            $xfer += $input->readListBegin($_etype693, $_size690);
+            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
             {
-              $elem688 = null;
-              $elem688 = new \metastore\Partition();
-              $xfer += $elem688->read($input);
-              $this->new_parts []= $elem688;
+              $elem695 = null;
+              $elem695 = new \metastore\Partition();
+              $xfer += $elem695->read($input);
+              $this->new_parts []= $elem695;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18104,9 +18160,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter689)
+          foreach ($this->new_parts as $iter696)
           {
-            $xfer += $iter689->write($output);
+            $xfer += $iter696->write($output);
           }
         }
         $output->writeListEnd();
@@ -18321,15 +18377,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size690 = 0;
-            $_etype693 = 0;
-            $xfer += $input->readListBegin($_etype693, $_size690);
-            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
+            $_size697 = 0;
+            $_etype700 = 0;
+            $xfer += $input->readListBegin($_etype700, $_size697);
+            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
             {
-              $elem695 = null;
-              $elem695 = new \metastore\PartitionSpec();
-              $xfer += $elem695->read($input);
-              $this->new_parts []= $elem695;
+              $elem702 = null;
+              $elem702 = new \metastore\PartitionSpec();
+              $xfer += $elem702->read($input);
+              $this->new_parts []= $elem702;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18357,9 +18413,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter696)
+          foreach ($this->new_parts as $iter703)
           {
-            $xfer += $iter696->write($output);
+            $xfer += $iter703->write($output);
           }
         }
         $output->writeListEnd();
@@ -18609,14 +18665,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size697 = 0;
-            $_etype700 = 0;
-            $xfer += $input->readListBegin($_etype700, $_size697);
-            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
+            $_size704 = 0;
+            $_etype707 = 0;
+            $xfer += $input->readListBegin($_etype707, $_size704);
+            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
             {
-              $elem702 = null;
-              $xfer += $input->readString($elem702);
-              $this->part_vals []= $elem702;
+              $elem709 = null;
+              $xfer += $input->readString($elem709);
+              $this->part_vals []= $elem709;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18654,9 +18710,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter703)
+          foreach ($this->part_vals as $iter710)
           {
-            $xfer += $output->writeString($iter703);
+            $xfer += $output->writeString($iter710);
           }
         }
         $output->writeListEnd();
@@ -19158,14 +19214,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size704 = 0;
-            $_etype707 = 0;
-            $xfer += $input->readListBegin($_etype707, $_size704);
-            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
+            $_size711 = 0;
+            $_etype714 = 0;
+            $xfer += $input->readListBegin($_etype714, $_size711);
+            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
             {
-              $elem709 = null;
-              $xfer += $input->readString($elem709);
-              $this->part_vals []= $elem709;
+              $elem716 = null;
+              $xfer += $input->readString($elem716);
+              $this->part_vals []= $elem716;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19211,9 +19267,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter710)
+          foreach ($this->part_vals as $iter717)
           {
-            $xfer += $output->writeString($iter710);
+            $xfer += $output->writeString($iter717);
           }
         }
         $output->writeListEnd();
@@ -20067,14 +20123,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size711 = 0;
-            $_etype714 = 0;
-            $xfer += $input->readListBegin($_etype714, $_size711);
-            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
+            $_size718 = 0;
+            $_etype721 = 0;
+            $xfer += $input->readListBegin($_etype721, $_size718);
+            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
             {
-              $elem716 = null;
-              $xfer += $input->readString($elem716);
-              $this->part_vals []= $elem716;
+              $elem723 = null;
+              $xfer += $input->readString($elem723);
+              $this->part_vals []= $elem723;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20119,9 +20175,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter717)
+          foreach ($this->part_vals as $iter724)
           {
-            $xfer += $output->writeString($iter717);
+            $xfer += $output->writeString($iter724);
           }
         }
         $output->writeListEnd();
@@ -20374,14 +20430,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size718 = 0;
-            $_etype721 = 0;
-            $xfer += $input->readListBegin($_etype721, $_size718);
-            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
+            $_size725 = 0;
+            $_etype728 = 0;
+            $xfer += $input->readListBegin($_etype728, $_size725);
+            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
             {
-              $elem723 = null;
-              $xfer += $input->readString($elem723);
-              $this->part_vals []= $elem723;
+              $elem730 = null;
+              $xfer += $input->readString($elem730);
+              $this->part_vals []= $elem730;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20434,9 +20490,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter724)
+          foreach ($this->part_vals as $iter731)
           {
-            $xfer += $output->writeString($iter724);
+            $xfer += $output->writeString($iter731);
           }
         }
         $output->writeListEnd();
@@ -21450,14 +21506,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size725 = 0;
-            $_etype728 = 0;
-            $xfer += $input->readListBegin($_etype728, $_size725);
-            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
+            $_size732 = 0;
+            $_etype735 = 0;
+            $xfer += $input->readListBegin($_etype735, $_size732);
+            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
             {
-              $elem730 = null;
-              $xfer += $input->readString($elem730);
-              $this->part_vals []= $elem730;
+              $elem737 = null;
+              $xfer += $input->readString($elem737);
+              $this->part_vals []= $elem737;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21495,9 +21551,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter731)
+          foreach ($this->part_vals as $iter738)
           {
-            $xfer += $output->writeString($iter731);
+            $xfer += $output->writeString($iter738);
           }
         }
         $output->writeListEnd();
@@ -21739,17 +21795,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size732 = 0;
-            $_ktype733 = 0;
-            $_vtype734 = 0;
-            $xfer += $input->readMapBegin($_ktype733, $_vtype734, $_size732);
-            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
+            $_size739 = 0;
+            $_ktype740 = 0;
+            $_vtype741 = 0;
+            $xfer += $input->readMapBegin($_ktype740, $_vtype741, $_size739);
+            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
             {
-              $key737 = '';
-              $val738 = '';
-              $xfer += $input->readString($key737);
-              $xfer += $input->readString($val738);
-              $this->partitionSpecs[$key737] = $val738;
+              $key744 = '';
+              $val745 = '';
+              $xfer += $input->readString($key744);
+              $xfer += $input->readString($val745);
+              $this->partitionSpecs[$key744] = $val745;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -21805,10 +21861,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter739 => $viter740)
+          foreach ($this->partitionSpecs as $kiter746 => $viter747)
           {
-            $xfer += $output->writeString($kiter739);
-            $xfer += $output->writeString($viter740);
+            $xfer += $output->writeString($kiter746);
+            $xfer += $output->writeString($viter747);
           }
         }
         $output->writeMapEnd();
@@ -22120,17 +22176,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size741 = 0;
-            $_ktype742 = 0;
-            $_vtype743 = 0;
-            $xfer += $input->readMapBegin($_ktype742, $_vtype743, $_size741);
-            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
+            $_size748 = 0;
+            $_ktype749 = 0;
+            $_vtype750 = 0;
+            $xfer += $input->readMapBegin($_ktype749, $_vtype750, $_size748);
+            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
             {
-              $key746 = '';
-              $val747 = '';
-              $xfer += $input->readString($key746);
-              $xfer += $input->readString($val747);
-              $this->partitionSpecs[$key746] = $val747;
+              $key753 = '';
+              $val754 = '';
+              $xfer += $input->readString($key753);
+              $xfer += $input->readString($val754);
+              $this->partitionSpecs[$key753] = $val754;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22186,10 +22242,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter748 => $viter749)
+          foreach ($this->partitionSpecs as $kiter755 => $viter756)
           {
-            $xfer += $output->writeString($kiter748);
-            $xfer += $output->writeString($viter749);
+            $xfer += $output->writeString($kiter755);
+            $xfer += $output->writeString($viter756);
           }
         }
         $output->writeMapEnd();
@@ -22322,15 +22378,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size750 = 0;
-            $_etype753 = 0;
-            $xfer += $input->readListBegin($_etype753, $_size750);
-            for ($_i754 = 0; $_i754 < $_size750; ++$_i754)
+            $_size757 = 0;
+            $_etype760 = 0;
+            $xfer += $input->readListBegin($_etype760, $_size757);
+            for ($_i761 = 0; $_i761 < $_size757; ++$_i761)
             {
-              $elem755 = null;
-              $elem755 = new \metastore\Partition();
-              $xfer += $elem755->read($input);
-              $this->success []= $elem755;
+              $elem762 = null;
+              $elem762 = new \metastore\Partition();
+              $xfer += $elem762->read($input);
+              $this->success []= $elem762;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22390,9 +22446,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter756)
+          foreach ($this->success as $iter763)
           {
-            $xfer += $iter756->write($output);
+            $xfer += $iter763->write($output);
           }
         }
         $output->writeListEnd();
@@ -22538,14 +22594,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size757 = 0;
-            $_etype760 = 0;
-            $xfer += $input->readListBegin($_etype760, $_size757);
-            for ($_i761 = 0; $_i761 < $_size757; ++$_i761)
+            $_size764 = 0;
+            $_etype767 = 0;
+            $xfer += $input->readListBegin($_etype767, $_size764);
+            for ($_i768 = 0; $_i768 < $_size764; ++$_i768)
             {
-              $elem762 = null;
-              $xfer += $input->readString($elem762);
-              $this->part_vals []= $elem762;
+              $elem769 = null;
+              $xfer += $input->readString($elem769);
+              $this->part_vals []= $elem769;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22562,14 +22618,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size763 = 0;
-            $_etype766 = 0;
-            $xfer += $input->readListBegin($_etype766, $_size763);
-            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
+            $_size770 = 0;
+            $_etype773 = 0;
+            $xfer += $input->readListBegin($_etype773, $_size770);
+            for ($_i774 = 0; $_i774 < $_size770; ++$_i774)
             {
-              $elem768 = null;
-              $xfer += $input->readString($elem768);
-              $this->group_names []= $elem768;
+              $elem775 = null;
+              $xfer += $input->readString($elem775);
+              $this->group_names []= $elem775;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22607,9 +22663,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter769)
+          foreach ($this->part_vals as $iter776)
           {
-            $xfer += $output->writeString($iter769);
+            $xfer += $output->writeString($iter776);
           }
         }
         $output->writeListEnd();
@@ -22629,9 +22685,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter770)
+          foreach ($this->group_names as $iter777)
           {
-            $xfer += $output->writeString($iter770);
+            $xfer += $output->writeString($iter777);
           }
         }
         $output->writeListEnd();
@@ -23222,15 +23278,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size771 = 0;
-            $_etype774 = 0;
-            $xfer += $input->readListBegin($_etype774, $_size771);
-            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
+            $_size778 = 0;
+            $_etype781 = 0;
+            $xfer += $input->readListBegin($_etype781, $_size778);
+            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
             {
-              $elem776 = null;
-              $elem776 = new \metastore\Partition();
-              $xfer += $elem776->read($input);
-              $this->success []= $elem776;
+              $elem783 = null;
+              $elem783 = new \metastore\Partition();
+              $xfer += $elem783->read($input);
+              $this->success []= $elem783;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23274,9 +23330,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter777)
+          foreach ($this->success as $iter784)
           {
-            $xfer += $iter777->write($output);
+            $xfer += $iter784->write($output);
           }
         }
         $output->writeListEnd();
@@ -23422,14 +23478,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size778 = 0;
-            $_etype781 = 0;
-            $xfer += $input->readListBegin($_etype781, $_size778);
-            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
+            $_size785 = 0;
+            $_etype788 = 0;
+            $xfer += $input->readListBegin($_etype788, $_size785);
+            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
             {
-              $elem783 = null;
-              $xfer += $input->readString($elem783);
-              $this->group_names []= $elem783;
+              $elem790 = null;
+              $xfer += $input->readString($elem790);
+              $this->group_names []= $elem790;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23477,9 +23533,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter784)
+          foreach ($this->group_names as $iter791)
           {
-            $xfer += $output->writeString($iter784);
+            $xfer += $output->writeString($iter791);
           }
         }
         $output->writeListEnd();
@@ -23568,15 +23624,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size785 = 0;
-            $_etype788 = 0;
-            $xfer += $input->readListBegin($_etype788, $_size785);
-            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
+            $_size792 = 0;
+            $_etype795 = 0;
+            $xfer += $input->readListBegin($_etype795, $_size792);
+            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
             {
-              $elem790 = null;
-              $elem790 = new \metastore\Partition();
-              $xfer += $elem790->read($input);
-              $this->success []= $elem790;
+              $elem797 = null;
+              $elem797 = new \metastore\Partition();
+              $xfer += $elem797->read($input);
+              $this->success []= $elem797;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23620,9 +23676,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter791)
+          foreach ($this->success as $iter798)
           {
-            $xfer += $iter791->write($output);
+            $xfer += $iter798->write($output);
           }
         }
         $output->writeListEnd();
@@ -23842,15 +23898,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size792 = 0;
-            $_etype795 = 0;
-            $xfer += $input->readListBegin($_etype795, $_size792);
-            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
+            $_size799 = 0;
+            $_etype802 = 0;
+            $xfer += $input->readListBegin($_etype802, $_size799);
+            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
             {
-              $elem797 = null;
-              $elem797 = new \metastore\PartitionSpec();
-              $xfer += $elem797->read($input);
-              $this->success []= $elem797;
+              $elem804 = null;
+              $elem804 = new \metastore\PartitionSpec();
+              $xfer += $elem804->read($input);
+              $this->success []= $elem804;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23894,9 +23950,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter798)
+          foreach ($this->success as $iter805)
           {
-            $xfer += $iter798->write($output);
+            $xfer += $iter805->write($output);
           }
         }
         $output->writeListEnd();
@@ -24103,14 +24159,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size799 = 0;
-            $_etype802 = 0;
-            $xfer += $input->readListBegin($_etype802, $_size799);
-            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
+            $_size806 = 0;
+            $_etype809 = 0;
+            $xfer += $input->readListBegin($_etype809, $_size806);
+            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
             {
-              $elem804 = null;
-              $xfer += $input->readString($elem804);
-              $this->success []= $elem804;
+              $elem811 = null;
+              $xfer += $input->readString($elem811);
+              $this->success []= $elem811;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24146,9 +24202,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter805)
+          foreach ($this->success as $iter812)
           {
-            $xfer += $output->writeString($iter805);
+            $xfer += $output->writeString($iter812);
           }
         }
         $output->writeListEnd();
@@ -24264,14 +24320,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size806 = 0;
-            $_etype809 = 0;
-            $xfer += $input->readListBegin($_etype809, $_size806);
-            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
+            $_size813 = 0;
+            $_etype816 = 0;
+            $xfer += $input->readListBegin($_etype816, $_size813);
+            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
             {
-              $elem811 = null;
-              $xfer += $input->readString($elem811);
-              $this->part_vals []= $elem811;
+              $elem818 = null;
+              $xfer += $input->readString($elem818);
+              $this->part_vals []= $elem818;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24316,9 +24372,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter812)
+          foreach ($this->part_vals as $iter819)
           {
-            $xfer += $output->writeString($iter812);
+            $xfer += $output->writeString($iter819);
           }
         }
         $output->writeListEnd();
@@ -24412,15 +24468,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size813 = 0;
-            $_etype816 = 0;
-            $xfer += $input->readListBegin($_etype816, $_size813);
-            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
+            $_size820 = 0;
+            $_etype823 = 0;
+            $xfer += $input->readListBegin($_etype823, $_size820);
+            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
             {
-              $elem818 = null;
-              $elem818 = new \metastore\Partition();
-              $xfer += $elem818->read($input);
-              $this->success []= $elem818;
+              $elem825 = null;
+              $elem825 = new \metastore\Partition();
+              $xfer += $elem825->read($input);
+              $this->success []= $elem825;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24464,9 +24520,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter819)
+          foreach ($this->success as $iter826)
           {
-            $xfer += $iter819->write($output);
+            $xfer += $iter826->write($output);
           }
         }
         $output->writeListEnd();
@@ -24613,14 +24669,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size820 = 0;
-            $_etype823 = 0;
-            $xfer += $input->readListBegin($_etype823, $_size820);
-            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
+            $_size827 = 0;
+            $_etype830 = 0;
+            $xfer += $input->readListBegin($_etype830, $_size827);
+            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
             {
-              $elem825 = null;
-              $xfer += $input->readString($elem825);
-              $this->part_vals []= $elem825;
+              $elem832 = null;
+              $xfer += $input->readString($elem832);
+              $this->part_vals []= $elem832;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24644,14 +24700,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size826 = 0;
-            $_etype829 = 0;
-            $xfer += $input->readListBegin($_etype829, $_size826);
-            for ($_i830 = 0; $_i830 < $_size826; ++$_i830)
+            $_size833 = 0;
+            $_etype836 = 0;
+            $xfer += $input->readListBegin($_etype836, $_size833);
+            for ($_i837 = 0; $_i837 < $_size833; ++$_i837)
             {
-              $elem831 = null;
-              $xfer += $input->readString($elem831);
-              $this->group_names []= $elem831;
+              $elem838 = null;
+              $xfer += $input->readString($elem838);
+              $this->group_names []= $elem838;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24689,9 +24745,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter832)
+          foreach ($this->part_vals as $iter839)
           {
-            $xfer += $output->writeString($iter832);
+            $xfer += $output->writeString($iter839);
           }
         }
         $output->writeListEnd();
@@ -24716,9 +24772,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter833)
+          foreach ($this->group_names as $iter840)
           {
-            $xfer += $output->writeString($iter833);
+            $xfer += $output->writeString($iter840);
           }
         }
         $output->writeListEnd();
@@ -24807,15 +24863,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size834 = 0;
-            $_etype837 = 0;
-            $xfer += $input->readListBegin($_etype837, $_size834);
-            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
+            $_size841 = 0;
+            $_etype844 = 0;
+            $xfer += $input->readListBegin($_etype844, $_size841);
+            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
             {
-              $elem839 = null;
-              $elem839 = new \metastore\Partition();
-              $xfer += $elem839->read($input);
-              $this->success []= $elem839;
+              $elem846 = null;
+              $elem846 = new \metastore\Partition();
+              $xfer += $elem846->read($input);
+              $this->success []= $elem846;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24859,9 +24915,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter840)
+          foreach ($this->success as $iter847)
           {
-            $xfer += $iter840->write($output);
+            $xfer += $iter847->write($output);
           }
         }
         $output->writeListEnd();
@@ -24982,14 +25038,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size841 = 0;
-            $_etype844 = 0;
-            $xfer += $input->readListBegin($_etype844, $_size841);
-            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
+            $_size848 = 0;
+            $_etype851 = 0;
+            $xfer += $input->readListBegin($_etype851, $_size848);
+            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
             {
-              $elem846 = null;
-              $xfer += $input->readString($elem846);
-              $this->part_vals []= $elem846;
+              $elem853 = null;
+              $xfer += $input->readString($elem853);
+              $this->part_vals []= $elem853;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25034,9 +25090,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter847)
+          foreach ($this->part_vals as $iter854)
           {
-            $xfer += $output->writeString($iter847);
+            $xfer += $output->writeString($iter854);
           }
         }
         $output->writeListEnd();
@@ -25129,14 +25185,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size848 = 0;
-            $_etype851 = 0;
-            $xfer += $input->readListBegin($_etype851, $_size848);
-            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
+            $_size855 = 0;
+            $_etype858 = 0;
+            $xfer += $input->readListBegin($_etype858, $_size855);
+            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
             {
-              $elem853 = null;
-              $xfer += $input->readString($elem853);
-              $this->success []= $elem853;
+              $elem860 = null;
+              $xfer += $input->readString($elem860);
+              $this->success []= $elem860;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25180,9 +25236,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter854)
+          foreach ($this->success as $iter861)
           {
-            $xfer += $output->writeString($iter854);
+            $xfer += $output->writeString($iter861);
           }
         }
         $output->writeListEnd();
@@ -25425,15 +25481,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size855 = 0;
-            $_etype858 = 0;
-            $xfer += $input->readListBegin($_etype858, $_size855);
-            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
+            $_size862 = 0;
+            $_etype865 = 0;
+            $xfer += $input->readListBegin($_etype865, $_size862);
+            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
             {
-              $elem860 = null;
-              $elem860 = new \metastore\Partition();
-              $xfer += $elem860->read($input);
-              $this->success []= $elem860;
+              $elem867 = null;
+              $elem867 = new \metastore\Partition();
+              $xfer += $elem867->read($input);
+              $this->success []= $elem867;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25477,9 +25533,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter861)
+          foreach ($this->success as $iter868)
           {
-            $xfer += $iter861->write($output);
+            $xfer += $iter868->write($output);
           }
         }
         $output->writeListEnd();
@@ -25722,15 +25778,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size862 = 0;
-            $_etype865 = 0;
-            $xfer += $input->readListBegin($_etype865, $_size862);
-            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
+            $_size869 = 0;
+            $_etype872 = 0;
+            $xfer += $input->readListBegin($_etype872, $_size869);
+            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
             {
-              $elem867 = null;
-              $elem867 = new \metastore\PartitionSpec();
-              $xfer += $elem867->read($input);
-              $this->success []= $elem867;
+              $elem874 = null;
+              $elem874 = new \metastore\PartitionSpec();
+              $xfer += $elem874->read($input);
+              $this->success []= $elem874;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25774,9 +25830,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter868)
+          foreach ($this->success as $iter875)
           {
-            $xfer += $iter868->write($output);
+            $xfer += $iter875->write($output);
           }
         }
         $output->writeListEnd();
@@ -26342,14 +26398,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size869 = 0;
-            $_etype872 = 0;
-            $xfer += $input->readListBegin($_etype872, $_size869);
-            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
+            $_size876 = 0;
+            $_etype879 = 0;
+            $xfer += $input->readListBegin($_etype879, $_size876);
+            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
             {
-              $elem874 = null;
-              $xfer += $input->readString($elem874);
-              $this->names []= $elem874;
+              $elem881 = null;
+              $xfer += $input->readString($elem881);
+              $this->names []= $elem881;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26387,9 +26443,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter875)
+          foreach ($this->names as $iter882)
           {
-            $xfer += $output->writeString($iter875);
+            $xfer += $output->writeString($iter882);
           }
         }
         $output->writeListEnd();
@@ -26478,15 +26534,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size876 = 0;
-            $_etype879 = 0;
-            $xfer += $input->readListBegin($_etype879, $_size876);
-            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
+            $_size883 = 0;
+            $_etype886 = 0;
+            $xfer += $input->readListBegin($_etype886, $_size883);
+            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
             {
-              $elem881 = null;
-              $elem881 = new \metastore\Partition();
-              $xfer += $elem881->read($input);
-              $this->success []= $elem881;
+              $elem888 = null;
+              $elem888 = new \metastore\Partition();
+              $xfer += $elem888->read($input);
+              $this->success []= $elem888;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26530,9 +26586,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter882)
+          foreach ($this->success as $iter889)
           {
-            $xfer += $iter882->write($output);
+            $xfer += $iter889->write($output);
           }
         }
         $output->writeListEnd();
@@ -26871,15 +26927,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size883 = 0;
-            $_etype886 = 0;
-            $xfer += $input->readListBegin($_etype886, $_size883);
-            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
+            $_size890 = 0;
+            $_etype893 = 0;
+            $xfer += $input->readListBegin($_etype893, $_size890);
+            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
             {
-              $elem888 = null;
-              $elem888 = new \metastore\Partition();
-              $xfer += $elem888->read($input);
-              $this->new_parts []= $elem888;
+              $elem895 = null;
+              $elem895 = new \metastore\Partition();
+              $xfer += $elem895->read($input);
+              $this->new_parts []= $elem895;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26917,9 +26973,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter889)
+          foreach ($this->new_parts as $iter896)
           {
-            $xfer += $iter889->write($output);
+            $xfer += $iter896->write($output);
           }
         }
         $output->writeListEnd();
@@ -27134,15 +27190,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size890 = 0;
-            $_etype893 = 0;
-            $xfer += $input->readListBegin($_etype893, $_size890);
-            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
+            $_size897 = 0;
+            $_etype900 = 0;
+            $xfer += $input->readListBegin($_etype900, $_size897);
+            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
             {
-              $elem895 = null;
-              $elem895 = new \metastore\Partition();
-              $xfer += $elem895->read($input);
-              $this->new_parts []= $elem895;
+              $elem902 = null;
+              $elem902 = new \metastore\Partition();
+              $xfer += $elem902->read($input);
+              $this->new_parts []= $elem902;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27188,9 +27244,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter896)
+          foreach ($this->new_parts as $iter903)
           {
-            $xfer += $iter896->write($output);
+            $xfer += $iter903->write($output);
           }
         }
         $output->writeListEnd();
@@ -27668,14 +27724,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size897 = 0;
-            $_etype900 = 0;
-            $xfer += $input->readListBegin($_etype900, $_size897);
-            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
+            $_size904 = 0;
+            $_etype907 = 0;
+            $xfer += $input->readListBegin($_etype907, $_size904);
+            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
             {
-              $elem902 = null;
-              $xfer += $input->readString($elem902);
-              $this->part_vals []= $elem902;
+              $elem909 = null;
+              $xfer += $input->readString($elem909);
+              $this->part_vals []= $elem909;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27721,9 +27777,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter903)
+          foreach ($this->part_vals as $iter910)
           {
-            $xfer += $output->writeString($iter903);
+            $xfer += $output->writeString($iter910);
           }
         }
         $output->writeListEnd();
@@ -27908,14 +27964,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size904 = 0;
-            $_etype907 = 0;
-            $xfer += $input->readListBegin($_etype907, $_size904);
-            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
+            $_size911 = 0;
+            $_etype914 = 0;
+            $xfer += $input->readListBegin($_etype914, $_size911);
+            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
             {
-              $elem909 = null;
-              $xfer += $input->readString($elem909);
-              $this->part_vals []= $elem909;
+              $elem916 = null;
+              $xfer += $input->readString($elem916);
+              $this->part_vals []= $elem916;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27950,9 +28006,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter910)
+          foreach ($this->part_vals as $iter917)
           {
-            $xfer += $output->writeString($iter910);
+            $xfer += $output->writeString($iter917);
           }
         }
         $output->writeListEnd();
@@ -28406,14 +28462,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size911 = 0;
-            $_etype914 = 0;
-            $xfer += $input->readListBegin($_etype914, $_size911);
-            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
+            $_size918 = 0;
+            $_etype921 = 0;
+            $xfer += $input->readListBegin($_etype921, $_size918);
+            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
             {
-              $elem916 = null;
-              $xfer += $input->readString($elem916);
-              $this->success []= $elem916;
+              $elem923 = null;
+              $xfer += $input->readString($elem923);
+              $this->success []= $elem923;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28449,9 +28505,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter917)
+          foreach ($this->success as $iter924)
           {
-            $xfer += $output->writeString($iter917);
+            $xfer += $output->writeString($iter924);
           }
         }
         $output->writeListEnd();
@@ -28611,17 +28667,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size918 = 0;
-            $_ktype919 = 0;
-            $_vtype920 = 0;
-            $xfer += $input->readMapBegin($_ktype919, $_vtype920, $_size918);
-            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
+            $_size925 = 0;
+            $_ktype926 = 0;
+            $_vtype927 = 0;
+            $xfer += $input->readMapBegin($_ktype926, $_vtype927, $_size925);
+            for ($_i929 = 0; $_i929 < $_size925; ++$_i929)
             {
-              $key923 = '';
-              $val924 = '';
-              $xfer += $input->readString($key923);
-              $xfer += $input->readString($val924);
-              $this->success[$key923] = $val924;
+              $key930 = '';
+              $val931 = '';
+              $xfer += $input->readString($key930);
+              $xfer += $input->readString($val931);
+              $this->success[$key930] = $val931;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28657,10 +28713,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter925 => $viter926)
+          foreach ($this->success as $kiter932 => $viter933)
           {
-            $xfer += $output->writeString($kiter925);
-            $xfer += $output->writeString($viter926);
+            $xfer += $output->writeString($kiter932);
+            $xfer += $output->writeString($viter933);
           }
         }
         $output->writeMapEnd();
@@ -28780,17 +28836,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size927 = 0;
-            $_ktype928 = 0;
-            $_vtype929 = 0;
-            $xfer += $input->readMapBegin($_ktype928, $_vtype929, $_size927);
-            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
+            $_size934 = 0;
+            $_ktype935 = 0;
+            $_vtype936 = 0;
+            $xfer += $input->readMapBegin($_ktype935, $_vtype936, $_size934);
+            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
             {
-              $key932 = '';
-              $val933 = '';
-              $xfer += $input->readString($key932);
-              $xfer += $input->readString($val933);
-              $this->part_vals[$key932] = $val933;
+              $key939 = '';
+              $val940 = '';
+              $xfer += $input->readString($key939);
+              $xfer += $input->readString($val940);
+              $this->part_vals[$key939] = $val940;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28835,10 +28891,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter934 => $viter935)
+          foreach ($this->part_vals as $kiter941 => $viter942)
           {
-            $xfer += $output->writeString($kiter934);
-            $xfer += $output->writeString($viter935);
+            $xfer += $output->writeString($kiter941);
+            $xfer += $output->writeString($viter942);
           }
         }
         $output->writeMapEnd();
@@ -29160,17 +29216,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size936 = 0;
-            $_ktype937 = 0;
-            $_vtype938 = 0;
-            $xfer += $input->readMapBegin($_ktype937, $_vtype938, $_size936);
-            for ($_i940 = 0; $_i940 < $_size936; ++$_i940)
+            $_size943 = 0;
+            $_ktype944 = 0;
+            $_vtype945 = 0;
+            $xfer += $input->readMapBegin($_ktype944, $_vtype945, $_size943);
+            for ($_i947 = 0; $_i947 < $_size943; ++$_i947)
             {
-              $key941 = '';
-              $val942 = '';
-              $xfer += $input->readString($key941);
-              $xfer += $input->readString($val942);
-              $this->part_vals[$key941] = $val942;
+              $key948 = '';
+              $val949 = '';
+              $xfer += $input->readString($key948);
+              $xfer += $input->readString($val949);
+              $this->part_vals[$key948] = $val949;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29215,10 +29271,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter943 => $viter944)
+          foreach ($this->part_vals as $kiter950 => $viter951)
           {
-            $xfer += $output->writeString($kiter943);
-            $xfer += $output->writeString($viter944);
+            $xfer += $output->writeString($kiter950);
+            $xfer += $output->writeString($viter951);
           }
         }
         $output->writeMapEnd();
@@ -30692,15 +30748,15 @@ class ThriftHiveMetastore_get_indexes_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size945 = 0;
-            $_etype948 = 0;
-            $xfer += $input->readListBegin($_etype948, $_size945);
-            for ($_i949 = 0; $_i949 < $_size945; ++$_i949)
+            $_size952 = 0;
+            $_etype955 = 0;
+            $xfer += $input->readListBegin($_etype955, $_size952);
+            for ($_i956 = 0; $_i956 < $_size952; ++$_i956)
             {
-              $elem950 = null;
-              $elem950 = new \metastore\Index();
-              $xfer += $elem950->read($input);
-              $this->success []= $elem950;
+              $elem957 = null;
+              $elem957 = new \metastore\Index();
+              $xfer += $elem957->read($input);
+              $this->success []= $elem957;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30744,9 +30800,9 @@ class ThriftHiveMetastore_get_indexes_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter951)
+          foreach ($this->success as $iter958)
           {
-            $xfer += $iter951->write($output);
+            $xfer += $iter958->write($output);
           }
         }
         $output->writeListEnd();
@@ -30953,14 +31009,14 @@ class ThriftHiveMetastore_get_index_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size952 = 0;
-            $_etype955 = 0;
-            $xfer += $input->readListBegin($_etype955, $_size952);
-            for ($_i956 = 0; $_i956 < $_size952; ++$_i956)
+            $_size959 = 0;
+            $_etype962 = 0;
+            $xfer += $input->readListBegin($_etype962, $_size959);
+            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
             {
-              $elem957 = null;
-              $xfer += $input->readString($elem957);
-              $this->success []= $elem957;
+              $elem964 = null;
+              $xfer += $input->readString($elem964);
+              $this->success []= $elem964;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30996,9 +31052,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter958)
+          foreach ($this->success as $iter965)
           {
-            $xfer += $output->writeString($iter958);
+            $xfer += $output->writeString($iter965);
           }
         }
         $output->writeListEnd();
@@ -34892,14 +34948,14 @@ class ThriftHiveMetastore_get_functions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size959 = 0;
-            $_etype962 = 0;
-            $xfer += $input->readListBegin($_etype962, $_size959);
-            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
+            $_size966 = 0;
+            $_etype969 = 0;
+            $xfer += $input->readListBegin($_etype969, $_size966);
+            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
             {
-              $elem964 = null;
-              $xfer += $input->readString($elem964);
-              $this->success []= $elem964;
+              $elem971 = null;
+              $xfer += $input->readString($elem971);
+              $this->success []= $elem971;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34935,9 +34991,9 @@ class ThriftHiveMetastore_get_functions_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter965)
+          foreach ($this->success as $iter972)
           {
-            $xfer += $output->writeString($iter965);
+            $xfer += $output->writeString($iter972);
           }
         }
         $output->writeListEnd();
@@ -35806,14 +35862,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size966 = 0;
-            $_etype969 = 0;
-            $xfer += $input->readListBegin($_etype969, $_size966);
-            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
+            $_size973 = 0;
+            $_etype976 = 0;
+            $xfer += $input->readListBegin($_etype976, $_size973);
+            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
             {
-              $elem971 = null;
-              $xfer += $input->readString($elem971);
-              $this->success []= $elem971;
+              $elem978 = null;
+              $xfer += $input->readString($elem978);
+              $this->success []= $elem978;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35849,9 +35905,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter972)
+          foreach ($this->success as $iter979)
           {
-            $xfer += $output->writeString($iter972);
+            $xfer += $output->writeString($iter979);
           }
         }
         $output->writeListEnd();
@@ -36542,15 +36598,15 @@ class ThriftHiveMetastore_list_roles_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size973 = 0;
-            $_etype976 = 0;
-            $xfer += $input->readListBegin($_etype976, $_size973);
-            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
+            $_size980 = 0;
+            $_etype983 = 0;
+            $xfer += $input->readListBegin($_etype983, $_size980);
+            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
             {
-              $elem978 = null;
-              $elem978 = new \metastore\Role();
-              $xfer += $elem978->read($input);
-              $this->success []= $elem978;
+              $elem985 = null;
+              $elem985 = new \metastore\Role();
+              $xfer += $elem985->read($input);
+              $this->success []= $elem985;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36586,9 +36642,9 @@ class ThriftHiveMetastore_list_roles_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter979)
+          foreach ($this->success as $iter986)
           {
-            $xfer += $iter979->write($output);
+            $xfer += $iter986->write($output);
           }
         }
         $output->writeListEnd();
@@ -37250,14 +37306,14 @@ class ThriftHiveMetastore_get_privilege_set_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size980 = 0;
-            $_etype983 = 0;
-            $xfer += $input->readListBegin($_etype983, $_size980);
-            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
+            $_size987 = 0;
+            $_etype990 = 0;
+            $xfer += $input->readListBegin($_etype990, $_size987);
+            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
             {
-              $elem985 = null;
-              $xfer += $input->readString($elem985);
-              $this->group_names []= $elem985;
+              $elem992 = null;
+              $xfer += $input->readString($elem992);
+              $this->group_names []= $elem992;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37298,9 +37354,9 @@ class ThriftHiveMetastore_get_privilege_set_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter986)
+          foreach ($this->group_names as $iter993)
           {
-            $xfer += $output->writeString($iter986);
+            $xfer += $output->writeString($iter993);
           }
         }
         $output->writeListEnd();
@@ -37608,15 +37664,15 @@ class ThriftHiveMetastore_list_privileges_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size987 = 0;
-            $_etype990 = 0;
-            $xfer += $input->readListBegin($_etype990, $_size987);
-            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
+            $_size994 = 0;
+            $_etype997 = 0;
+            $xfer += $input->readListBegin($_etype997, $_size994);
+            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
             {
-              $elem992 = null;
-              $elem992 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem992->read($input);
-              $this->success []= $elem992;
+              $elem999 = null;
+              $elem999 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem999->read($input);
+              $this->success []= $elem999;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37652,9 +37708,9 @@ class ThriftHiveMetastore_list_privileges_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter993)
+          foreach ($this->success as $iter1000)
           {
-            $xfer += $iter993->write($output);
+            $xfer += $iter1000->write($output);
           }
         }
         $output->writeListEnd();
@@ -38286,14 +38342,14 @@ class ThriftHiveMetastore_set_ugi_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size994 = 0;
-            $_etype997 = 0;
-            $xfer += $input->readListBegin($_etype997, $_size994);
-            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
+            $_size1001 = 0;
+            $_etype1004 = 0;
+            $xfer += $input->readListBegin($_etype1004, $_size1001);
+            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
             {
-              $elem999 = null;
-              $xfer += $input->readString($elem999);
-              $this->group_names []= $elem999;
+              $elem1006 = null;
+              $xfer += $input->readString($elem1006);
+              $this->group_names []= $elem1006;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -38326,9 +38382,9 @@ class ThriftHiveMetastore_set_ugi_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1000)
+          foreach ($this->group_names as $iter1007)
           {
-            $xfer += $output->writeString($iter1000);
+            $xfer += $output->writeString($iter1007);
           }
         }
         $output->writeListEnd();
@@ -38404,14 +38460,14 @@ class ThriftHiveMetastore_set_ugi_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1001 = 0;
-            $_etype1004 = 0;
-            $xfer += $input->readListBegin($_etype1004, $_size1001);
-            for 

<TRUNCATED>

[18/31] hive git commit: HIVE-14715: Hive throws NumberFormatException with query with Null value (Yongzhi Chen, reviewed by Aihua Xu)

Posted by se...@apache.org.
HIVE-14715: Hive throws NumberFormatException with query with Null value (Yongzhi Chen, 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/d4ffa5d3
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d4ffa5d3
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d4ffa5d3

Branch: refs/heads/hive-14535
Commit: d4ffa5d3d8c272e33754335aa5174ed5fe092f87
Parents: bc75e46
Author: Yongzhi Chen <yc...@apache.org>
Authored: Wed Sep 7 21:30:01 2016 -0400
Committer: Yongzhi Chen <yc...@apache.org>
Committed: Mon Sep 12 09:06:57 2016 -0400

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  3 +-
 .../queries/clientpositive/groupby_nullvalues.q | 29 +++++++
 .../clientpositive/groupby_nullvalues.q.out     | 83 ++++++++++++++++++++
 3 files changed, 114 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d4ffa5d3/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 e4a37f4..b071ddf 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
@@ -5128,6 +5128,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         colExprMap);
 
     int keyLength = reduceKeys.size();
+    int numOfColsRmedFromkey = grpByExprs.size() - keyLength;
 
     // add a key for reduce sink
     if (groupingSetsPresent) {
@@ -5157,7 +5158,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           reduceSinkOutputRowResolver, outputValueColumnNames, reduceValues, colExprMap);
     } else {
       // Put partial aggregation results in reduceValues
-      int inputField = reduceKeys.size();
+      int inputField = reduceKeys.size() + numOfColsRmedFromkey;
 
       for (Map.Entry<String, ASTNode> entry : aggregationTrees.entrySet()) {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d4ffa5d3/ql/src/test/queries/clientpositive/groupby_nullvalues.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/groupby_nullvalues.q b/ql/src/test/queries/clientpositive/groupby_nullvalues.q
new file mode 100644
index 0000000..85ccb38
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/groupby_nullvalues.q
@@ -0,0 +1,29 @@
+set hive.cbo.enable=false;
+drop table if exists paqtest;
+CREATE TABLE paqtest(
+c1 int,
+s1 string,
+s2 string,
+bn1 bigint)
+ROW FORMAT SERDE
+'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+STORED AS INPUTFORMAT
+'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+OUTPUTFORMAT
+'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat';
+
+insert into paqtest values (58, '', 'ABC', 0);
+
+SELECT
+'PM' AS cy,
+c1,
+NULL AS iused,
+NULL AS itp,
+s2,
+NULL AS cvg,
+NULL AS acavg,
+sum(bn1) AS cca
+FROM paqtest
+WHERE (s1 IS NULL OR length(s1) = 0)
+GROUP BY 'Pricing mismatch', c1, NULL, NULL, s2, NULL, NULL;
+drop table paqtest;

http://git-wip-us.apache.org/repos/asf/hive/blob/d4ffa5d3/ql/src/test/results/clientpositive/groupby_nullvalues.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_nullvalues.q.out b/ql/src/test/results/clientpositive/groupby_nullvalues.q.out
new file mode 100644
index 0000000..aae2a12
--- /dev/null
+++ b/ql/src/test/results/clientpositive/groupby_nullvalues.q.out
@@ -0,0 +1,83 @@
+PREHOOK: query: drop table if exists paqtest
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists paqtest
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE paqtest(
+c1 int,
+s1 string,
+s2 string,
+bn1 bigint)
+ROW FORMAT SERDE
+'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+STORED AS INPUTFORMAT
+'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+OUTPUTFORMAT
+'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@paqtest
+POSTHOOK: query: CREATE TABLE paqtest(
+c1 int,
+s1 string,
+s2 string,
+bn1 bigint)
+ROW FORMAT SERDE
+'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+STORED AS INPUTFORMAT
+'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+OUTPUTFORMAT
+'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@paqtest
+PREHOOK: query: insert into paqtest values (58, '', 'ABC', 0)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@paqtest
+POSTHOOK: query: insert into paqtest values (58, '', 'ABC', 0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@paqtest
+POSTHOOK: Lineage: paqtest.bn1 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: paqtest.c1 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: paqtest.s1 SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: paqtest.s2 SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+PREHOOK: query: SELECT
+'PM' AS cy,
+c1,
+NULL AS iused,
+NULL AS itp,
+s2,
+NULL AS cvg,
+NULL AS acavg,
+sum(bn1) AS cca
+FROM paqtest
+WHERE (s1 IS NULL OR length(s1) = 0)
+GROUP BY 'Pricing mismatch', c1, NULL, NULL, s2, NULL, NULL
+PREHOOK: type: QUERY
+PREHOOK: Input: default@paqtest
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT
+'PM' AS cy,
+c1,
+NULL AS iused,
+NULL AS itp,
+s2,
+NULL AS cvg,
+NULL AS acavg,
+sum(bn1) AS cca
+FROM paqtest
+WHERE (s1 IS NULL OR length(s1) = 0)
+GROUP BY 'Pricing mismatch', c1, NULL, NULL, s2, NULL, NULL
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@paqtest
+#### A masked pattern was here ####
+PM	58	NULL	NULL	ABC	NULL	NULL	0
+PREHOOK: query: drop table paqtest
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@paqtest
+PREHOOK: Output: default@paqtest
+POSTHOOK: query: drop table paqtest
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@paqtest
+POSTHOOK: Output: default@paqtest


[11/31] hive git commit: HIVE-14626: Support Trash in Truncate Table (Chaoyu Tang, reviewed by Sergio Pena)

Posted by se...@apache.org.
HIVE-14626: Support Trash in Truncate Table (Chaoyu Tang, 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/943a361d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/943a361d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/943a361d

Branch: refs/heads/hive-14535
Commit: 943a361da621ebe587c613c811231c606c66e515
Parents: 4c543b1
Author: ctang <ct...@cloudera.com>
Authored: Thu Sep 8 14:09:15 2016 -0400
Committer: ctang <ct...@cloudera.com>
Committed: Thu Sep 8 14:09:15 2016 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  33 ++++--
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   2 +-
 .../clientpositive/encryption_drop_partition.q  |   7 ++
 .../clientpositive/encryption_drop_table.q      |  23 ++++
 .../encrypted/encryption_drop_partition.q.out   |  33 ++++++
 .../encrypted/encryption_drop_table.q.out       | 110 +++++++++++++++++++
 6 files changed, 198 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/943a361d/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 66cbdd3..569c19e 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
@@ -225,6 +225,8 @@ import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.hive.shims.HadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.tools.HadoopArchives;
@@ -4269,15 +4271,28 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       // this is not transactional
       for (Path location : getLocations(db, table, partSpec)) {
         FileSystem fs = location.getFileSystem(conf);
-        HdfsUtils.HadoopFileStatus status = new HdfsUtils.HadoopFileStatus(conf, fs, location);
-        FileStatus targetStatus = fs.getFileStatus(location);
-        String targetGroup = targetStatus == null ? null : targetStatus.getGroup();
-        fs.delete(location, true);
-        fs.mkdirs(location);
-        try {
-          HdfsUtils.setFullFileStatus(conf, status, targetGroup, fs, location, false);
-        } catch (Exception e) {
-          LOG.warn("Error setting permissions of " + location, e);
+        HadoopShims.HdfsEncryptionShim shim
+          = ShimLoader.getHadoopShims().createHdfsEncryptionShim(fs, conf);
+        if (!shim.isPathEncrypted(location)) {
+          HdfsUtils.HadoopFileStatus status = new HdfsUtils.HadoopFileStatus(conf, fs, location);
+          FileStatus targetStatus = fs.getFileStatus(location);
+          String targetGroup = targetStatus == null ? null : targetStatus.getGroup();
+          FileUtils.moveToTrash(fs, location, conf);
+          fs.mkdirs(location);
+          try {
+            HdfsUtils.setFullFileStatus(conf, status, targetGroup, fs, location, false);
+          } catch (Exception e) {
+            LOG.warn("Error setting permissions of " + location, e);
+          }
+        } else {
+          FileStatus[] statuses = fs.listStatus(location, FileUtils.HIDDEN_FILES_PATH_FILTER);
+          if (statuses == null || statuses.length == 0) {
+            continue;
+          }
+          boolean success = Hive.trashFiles(fs, statuses, conf);
+          if (!success) {
+            throw new HiveException("Error in deleting the contents of " + location.toString());
+          }
         }
       }
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/943a361d/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 5f53aef..da46854 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
@@ -3365,7 +3365,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
    * @return true if deletion successful
    * @throws IOException
    */
-  private boolean trashFiles(final FileSystem fs, final FileStatus[] statuses, final Configuration conf)
+  public static boolean trashFiles(final FileSystem fs, final FileStatus[] statuses, final Configuration conf)
       throws IOException {
     boolean result = true;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/943a361d/ql/src/test/queries/clientpositive/encryption_drop_partition.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/encryption_drop_partition.q b/ql/src/test/queries/clientpositive/encryption_drop_partition.q
index 57dfabd..d968459 100644
--- a/ql/src/test/queries/clientpositive/encryption_drop_partition.q
+++ b/ql/src/test/queries/clientpositive/encryption_drop_partition.q
@@ -25,4 +25,11 @@ ALTER TABLE encrypted_table_dp DROP PARTITION (p='2014-09-23');
 SELECT * FROM encrypted_table_dp;
 ALTER TABLE encrypted_table_dp DROP PARTITION (p='2014-09-23') PURGE;
 SELECT * FROM encrypted_table_dp;
+
+TRUNCATE TABLE encrypted_table_dp PARTITION (p='2014-09-24');
+SHOW PARTITIONS encrypted_table_dp;
+SELECT * FROM encrypted_table_dp;
+
+ALTER TABLE encrypted_table_dp DROP PARTITION (p='2014-09-24');
+DROP TABLE encrypted_table_dp;
 DROP TABLE encrypted_table_dp PURGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/943a361d/ql/src/test/queries/clientpositive/encryption_drop_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/encryption_drop_table.q b/ql/src/test/queries/clientpositive/encryption_drop_table.q
index 2ae3c69..76be118 100644
--- a/ql/src/test/queries/clientpositive/encryption_drop_table.q
+++ b/ql/src/test/queries/clientpositive/encryption_drop_table.q
@@ -5,6 +5,8 @@
 set hive.cli.errors.ignore=true;
 
 DROP TABLE IF EXISTS encrypted_table PURGE;
+DROP TABLE IF EXISTS encrypted_ext_table PURGE;
+
 CREATE TABLE encrypted_table (key INT, value STRING) LOCATION '${hiveconf:hive.metastore.warehouse.dir}/default/encrypted_table';
 CRYPTO CREATE_KEY --keyName key_128 --bitLength 128;
 CRYPTO CREATE_ZONE --keyName key_128 --path ${hiveconf:hive.metastore.warehouse.dir}/default/encrypted_table;
@@ -22,4 +24,25 @@ SHOW TABLES;
 
 DROP TABLE default.encrypted_table PURGE;
 SHOW TABLES;
+
+DROP TABLE IF EXISTS encrypted_table1 PURGE;
+CREATE TABLE encrypted_table1 (key INT, value STRING) LOCATION '${hiveconf:hive.metastore.warehouse.dir}/default/encrypted_table1';
+CRYPTO CREATE_ZONE --keyName key_128 --path ${hiveconf:hive.metastore.warehouse.dir}/default/encrypted_table1;
+INSERT OVERWRITE TABLE encrypted_table1 SELECT * FROM src;
+
+SELECT COUNT(*) FROM encrypted_table1;
+TRUNCATE TABLE encrypted_table1;
+SELECT COUNT(*) FROM encrypted_table1;
+
+INSERT OVERWRITE TABLE encrypted_table1 SELECT * FROM src;
+DROP TABLE default.encrypted_table1;
+SHOW TABLES;
+
+TRUNCATE TABLE encrypted_table1;
+DROP TABLE default.encrypted_table1;
+SHOW TABLES;
+
+DROP TABLE default.encrypted_table1 PURGE;
+SHOW TABLES;
+
 CRYPTO DELETE_KEY --keyName key_128;

http://git-wip-us.apache.org/repos/asf/hive/blob/943a361d/ql/src/test/results/clientpositive/encrypted/encryption_drop_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_drop_partition.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_drop_partition.q.out
index 2643006..c2f6b03 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_drop_partition.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_drop_partition.q.out
@@ -147,6 +147,39 @@ POSTHOOK: Input: default@encrypted_table_dp
 POSTHOOK: Input: default@encrypted_table_dp@p=2014-09-24
 #### A PARTIAL masked pattern was here #### data/warehouse/default/encrypted_table_dp/.hive-staging
 2	bar	2014-09-24
+PREHOOK: query: TRUNCATE TABLE encrypted_table_dp PARTITION (p='2014-09-24')
+PREHOOK: type: TRUNCATETABLE
+PREHOOK: Output: default@encrypted_table_dp@p=2014-09-24
+POSTHOOK: query: TRUNCATE TABLE encrypted_table_dp PARTITION (p='2014-09-24')
+POSTHOOK: type: TRUNCATETABLE
+POSTHOOK: Output: default@encrypted_table_dp@p=2014-09-24
+PREHOOK: query: SHOW PARTITIONS encrypted_table_dp
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@encrypted_table_dp
+POSTHOOK: query: SHOW PARTITIONS encrypted_table_dp
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@encrypted_table_dp
+p=2014-09-24
+PREHOOK: query: SELECT * FROM encrypted_table_dp
+PREHOOK: type: QUERY
+PREHOOK: Input: default@encrypted_table_dp
+PREHOOK: Input: default@encrypted_table_dp@p=2014-09-24
+#### A PARTIAL masked pattern was here #### data/warehouse/default/encrypted_table_dp/.hive-staging
+POSTHOOK: query: SELECT * FROM encrypted_table_dp
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@encrypted_table_dp
+POSTHOOK: Input: default@encrypted_table_dp@p=2014-09-24
+#### A PARTIAL masked pattern was here #### data/warehouse/default/encrypted_table_dp/.hive-staging
+PREHOOK: query: ALTER TABLE encrypted_table_dp DROP PARTITION (p='2014-09-24')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@encrypted_table_dp
+PREHOOK: Output: default@encrypted_table_dp@p=2014-09-24
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to drop default.encrypted_table_dp.[2014-09-24] because it is in an encryption zone and trash is enabled.  Use PURGE option to skip trash.
+PREHOOK: query: DROP TABLE encrypted_table_dp
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@encrypted_table_dp
+PREHOOK: Output: default@encrypted_table_dp
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:Unable to drop default.encrypted_table_dp because it is in an encryption zone and trash is enabled.  Use PURGE option to skip trash.)
 PREHOOK: query: DROP TABLE encrypted_table_dp PURGE
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@encrypted_table_dp

http://git-wip-us.apache.org/repos/asf/hive/blob/943a361d/ql/src/test/results/clientpositive/encrypted/encryption_drop_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_drop_table.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_drop_table.q.out
index c5007ee..a3f41d8 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_drop_table.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_drop_table.q.out
@@ -2,6 +2,10 @@ PREHOOK: query: DROP TABLE IF EXISTS encrypted_table PURGE
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: DROP TABLE IF EXISTS encrypted_table PURGE
 POSTHOOK: type: DROPTABLE
+PREHOOK: query: DROP TABLE IF EXISTS encrypted_ext_table PURGE
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE IF EXISTS encrypted_ext_table PURGE
+POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
 #### A masked pattern was here ####
@@ -87,3 +91,109 @@ POSTHOOK: query: SHOW TABLES
 POSTHOOK: type: SHOWTABLES
 POSTHOOK: Input: database:default
 src
+PREHOOK: query: DROP TABLE IF EXISTS encrypted_table1 PURGE
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE IF EXISTS encrypted_table1 PURGE
+POSTHOOK: type: DROPTABLE
+#### A masked pattern was here ####
+PREHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+PREHOOK: Output: database:default
+PREHOOK: Output: default@encrypted_table1
+#### A masked pattern was here ####
+POSTHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@encrypted_table1
+Encryption zone created: '/build/ql/test/data/warehouse/default/encrypted_table1' using key: 'key_128'
+PREHOOK: query: INSERT OVERWRITE TABLE encrypted_table1 SELECT * FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@encrypted_table1
+POSTHOOK: query: INSERT OVERWRITE TABLE encrypted_table1 SELECT * FROM src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@encrypted_table1
+POSTHOOK: Lineage: encrypted_table1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: encrypted_table1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: SELECT COUNT(*) FROM encrypted_table1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@encrypted_table1
+#### A PARTIAL masked pattern was here #### data/warehouse/default/encrypted_table1/.hive-staging
+POSTHOOK: query: SELECT COUNT(*) FROM encrypted_table1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@encrypted_table1
+#### A PARTIAL masked pattern was here #### data/warehouse/default/encrypted_table1/.hive-staging
+500
+PREHOOK: query: TRUNCATE TABLE encrypted_table1
+PREHOOK: type: TRUNCATETABLE
+PREHOOK: Output: default@encrypted_table1
+POSTHOOK: query: TRUNCATE TABLE encrypted_table1
+POSTHOOK: type: TRUNCATETABLE
+POSTHOOK: Output: default@encrypted_table1
+PREHOOK: query: SELECT COUNT(*) FROM encrypted_table1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@encrypted_table1
+#### A PARTIAL masked pattern was here #### data/warehouse/default/encrypted_table1/.hive-staging
+POSTHOOK: query: SELECT COUNT(*) FROM encrypted_table1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@encrypted_table1
+#### A PARTIAL masked pattern was here #### data/warehouse/default/encrypted_table1/.hive-staging
+0
+PREHOOK: query: INSERT OVERWRITE TABLE encrypted_table1 SELECT * FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@encrypted_table1
+POSTHOOK: query: INSERT OVERWRITE TABLE encrypted_table1 SELECT * FROM src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@encrypted_table1
+POSTHOOK: Lineage: encrypted_table1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: encrypted_table1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: DROP TABLE default.encrypted_table1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@encrypted_table1
+PREHOOK: Output: default@encrypted_table1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:Unable to drop default.encrypted_table1 because it is in an encryption zone and trash is enabled.  Use PURGE option to skip trash.)
+PREHOOK: query: SHOW TABLES
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:default
+POSTHOOK: query: SHOW TABLES
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:default
+encrypted_table1
+src
+PREHOOK: query: TRUNCATE TABLE encrypted_table1
+PREHOOK: type: TRUNCATETABLE
+PREHOOK: Output: default@encrypted_table1
+POSTHOOK: query: TRUNCATE TABLE encrypted_table1
+POSTHOOK: type: TRUNCATETABLE
+POSTHOOK: Output: default@encrypted_table1
+PREHOOK: query: DROP TABLE default.encrypted_table1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@encrypted_table1
+PREHOOK: Output: default@encrypted_table1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:Unable to drop default.encrypted_table1 because it is in an encryption zone and trash is enabled.  Use PURGE option to skip trash.)
+PREHOOK: query: SHOW TABLES
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:default
+POSTHOOK: query: SHOW TABLES
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:default
+encrypted_table1
+src
+PREHOOK: query: DROP TABLE default.encrypted_table1 PURGE
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@encrypted_table1
+PREHOOK: Output: default@encrypted_table1
+POSTHOOK: query: DROP TABLE default.encrypted_table1 PURGE
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@encrypted_table1
+POSTHOOK: Output: default@encrypted_table1
+PREHOOK: query: SHOW TABLES
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:default
+POSTHOOK: query: SHOW TABLES
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:default
+src


[25/31] hive git commit: HIVE-14644 : use metastore information on the read path appropriately (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 1377648..22c0cc6 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1221,6 +1221,13 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_valid_write_ids(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
 
 class Client(fb303.FacebookService.Client, Iface):
   """
@@ -6710,6 +6717,37 @@ class Client(fb303.FacebookService.Client, Iface):
       return result.success
     raise TApplicationException(TApplicationException.MISSING_RESULT, "heartbeat_write_id failed: unknown result")
 
+  def get_valid_write_ids(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_get_valid_write_ids(req)
+    return self.recv_get_valid_write_ids()
+
+  def send_get_valid_write_ids(self, req):
+    self._oprot.writeMessageBegin('get_valid_write_ids', TMessageType.CALL, self._seqid)
+    args = get_valid_write_ids_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_valid_write_ids(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_valid_write_ids_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_valid_write_ids failed: unknown result")
+
 
 class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
   def __init__(self, handler):
@@ -6866,6 +6904,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_next_write_id"] = Processor.process_get_next_write_id
     self._processMap["finalize_write_id"] = Processor.process_finalize_write_id
     self._processMap["heartbeat_write_id"] = Processor.process_heartbeat_write_id
+    self._processMap["get_valid_write_ids"] = Processor.process_get_valid_write_ids
 
   def process(self, iprot, oprot):
     (name, type, seqid) = iprot.readMessageBegin()
@@ -10586,6 +10625,25 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_valid_write_ids(self, seqid, iprot, oprot):
+    args = get_valid_write_ids_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_valid_write_ids_result()
+    try:
+      result.success = self._handler.get_valid_write_ids(args.req)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_valid_write_ids", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
 
 # HELPER FUNCTIONS AND STRUCTURES
 
@@ -11472,10 +11530,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype569, _size566) = iprot.readListBegin()
-          for _i570 in xrange(_size566):
-            _elem571 = iprot.readString()
-            self.success.append(_elem571)
+          (_etype576, _size573) = iprot.readListBegin()
+          for _i577 in xrange(_size573):
+            _elem578 = iprot.readString()
+            self.success.append(_elem578)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11498,8 +11556,8 @@ class get_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter572 in self.success:
-        oprot.writeString(iter572)
+      for iter579 in self.success:
+        oprot.writeString(iter579)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11604,10 +11662,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype576, _size573) = iprot.readListBegin()
-          for _i577 in xrange(_size573):
-            _elem578 = iprot.readString()
-            self.success.append(_elem578)
+          (_etype583, _size580) = iprot.readListBegin()
+          for _i584 in xrange(_size580):
+            _elem585 = iprot.readString()
+            self.success.append(_elem585)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11630,8 +11688,8 @@ class get_all_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter579 in self.success:
-        oprot.writeString(iter579)
+      for iter586 in self.success:
+        oprot.writeString(iter586)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12401,12 +12459,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype581, _vtype582, _size580 ) = iprot.readMapBegin()
-          for _i584 in xrange(_size580):
-            _key585 = iprot.readString()
-            _val586 = Type()
-            _val586.read(iprot)
-            self.success[_key585] = _val586
+          (_ktype588, _vtype589, _size587 ) = iprot.readMapBegin()
+          for _i591 in xrange(_size587):
+            _key592 = iprot.readString()
+            _val593 = Type()
+            _val593.read(iprot)
+            self.success[_key592] = _val593
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -12429,9 +12487,9 @@ class get_type_all_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter587,viter588 in self.success.items():
-        oprot.writeString(kiter587)
-        viter588.write(oprot)
+      for kiter594,viter595 in self.success.items():
+        oprot.writeString(kiter594)
+        viter595.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -12574,11 +12632,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype592, _size589) = iprot.readListBegin()
-          for _i593 in xrange(_size589):
-            _elem594 = FieldSchema()
-            _elem594.read(iprot)
-            self.success.append(_elem594)
+          (_etype599, _size596) = iprot.readListBegin()
+          for _i600 in xrange(_size596):
+            _elem601 = FieldSchema()
+            _elem601.read(iprot)
+            self.success.append(_elem601)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12613,8 +12671,8 @@ class get_fields_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter595 in self.success:
-        iter595.write(oprot)
+      for iter602 in self.success:
+        iter602.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12781,199 +12839,6 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype599, _size596) = iprot.readListBegin()
-          for _i600 in xrange(_size596):
-            _elem601 = FieldSchema()
-            _elem601.read(iprot)
-            self.success.append(_elem601)
-          iprot.readListEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 1:
-        if ftype == TType.STRUCT:
-          self.o1 = MetaException()
-          self.o1.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.STRUCT:
-          self.o2 = UnknownTableException()
-          self.o2.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.STRUCT:
-          self.o3 = UnknownDBException()
-          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('get_fields_with_environment_context_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.LIST, 0)
-      oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter602 in self.success:
-        iter602.write(oprot)
-      oprot.writeListEnd()
-      oprot.writeFieldEnd()
-    if self.o1 is not None:
-      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
-      self.o1.write(oprot)
-      oprot.writeFieldEnd()
-    if self.o2 is not None:
-      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
-      self.o2.write(oprot)
-      oprot.writeFieldEnd()
-    if self.o3 is not None:
-      oprot.writeFieldBegin('o3', TType.STRUCT, 3)
-      self.o3.write(oprot)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.success)
-    value = (value * 31) ^ hash(self.o1)
-    value = (value * 31) ^ hash(self.o2)
-    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 get_schema_args:
-  """
-  Attributes:
-   - db_name
-   - table_name
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.STRING, 'db_name', None, None, ), # 1
-    (2, TType.STRING, 'table_name', None, None, ), # 2
-  )
-
-  def __init__(self, db_name=None, table_name=None,):
-    self.db_name = db_name
-    self.table_name = table_name
-
-  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.db_name = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.STRING:
-          self.table_name = 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('get_schema_args')
-    if self.db_name is not None:
-      oprot.writeFieldBegin('db_name', TType.STRING, 1)
-      oprot.writeString(self.db_name)
-      oprot.writeFieldEnd()
-    if self.table_name is not None:
-      oprot.writeFieldBegin('table_name', TType.STRING, 2)
-      oprot.writeString(self.table_name)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.db_name)
-    value = (value * 31) ^ hash(self.table_name)
-    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 get_schema_result:
-  """
-  Attributes:
-   - success
-   - o1
-   - o2
-   - o3
-  """
-
-  thrift_spec = (
-    (0, TType.LIST, 'success', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 0
-    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
-    (2, TType.STRUCT, 'o2', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 2
-    (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3
-  )
-
-  def __init__(self, success=None, o1=None, o2=None, o3=None,):
-    self.success = success
-    self.o1 = o1
-    self.o2 = o2
-    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 == 0:
-        if ftype == TType.LIST:
-          self.success = []
           (_etype606, _size603) = iprot.readListBegin()
           for _i607 in xrange(_size603):
             _elem608 = FieldSchema()
@@ -13009,7 +12874,7 @@ class get_schema_result:
     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('get_schema_result')
+    oprot.writeStructBegin('get_fields_with_environment_context_result')
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
@@ -13055,6 +12920,199 @@ class get_schema_result:
   def __ne__(self, other):
     return not (self == other)
 
+class get_schema_args:
+  """
+  Attributes:
+   - db_name
+   - table_name
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'db_name', None, None, ), # 1
+    (2, TType.STRING, 'table_name', None, None, ), # 2
+  )
+
+  def __init__(self, db_name=None, table_name=None,):
+    self.db_name = db_name
+    self.table_name = table_name
+
+  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.db_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.table_name = 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('get_schema_args')
+    if self.db_name is not None:
+      oprot.writeFieldBegin('db_name', TType.STRING, 1)
+      oprot.writeString(self.db_name)
+      oprot.writeFieldEnd()
+    if self.table_name is not None:
+      oprot.writeFieldBegin('table_name', TType.STRING, 2)
+      oprot.writeString(self.table_name)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.table_name)
+    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 get_schema_result:
+  """
+  Attributes:
+   - success
+   - o1
+   - o2
+   - o3
+  """
+
+  thrift_spec = (
+    (0, TType.LIST, 'success', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3
+  )
+
+  def __init__(self, success=None, o1=None, o2=None, o3=None,):
+    self.success = success
+    self.o1 = o1
+    self.o2 = o2
+    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 == 0:
+        if ftype == TType.LIST:
+          self.success = []
+          (_etype613, _size610) = iprot.readListBegin()
+          for _i614 in xrange(_size610):
+            _elem615 = FieldSchema()
+            _elem615.read(iprot)
+            self.success.append(_elem615)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.o2 = UnknownTableException()
+          self.o2.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.o3 = UnknownDBException()
+          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('get_schema_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.LIST, 0)
+      oprot.writeListBegin(TType.STRUCT, len(self.success))
+      for iter616 in self.success:
+        iter616.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o2 is not None:
+      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
+      self.o2.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o3 is not None:
+      oprot.writeFieldBegin('o3', TType.STRUCT, 3)
+      self.o3.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    value = (value * 31) ^ hash(self.o2)
+    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 get_schema_with_environment_context_args:
   """
   Attributes:
@@ -13181,11 +13239,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype613, _size610) = iprot.readListBegin()
-          for _i614 in xrange(_size610):
-            _elem615 = FieldSchema()
-            _elem615.read(iprot)
-            self.success.append(_elem615)
+          (_etype620, _size617) = iprot.readListBegin()
+          for _i621 in xrange(_size617):
+            _elem622 = FieldSchema()
+            _elem622.read(iprot)
+            self.success.append(_elem622)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13220,8 +13278,8 @@ class get_schema_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter616 in self.success:
-        iter616.write(oprot)
+      for iter623 in self.success:
+        iter623.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13662,22 +13720,22 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype620, _size617) = iprot.readListBegin()
-          for _i621 in xrange(_size617):
-            _elem622 = SQLPrimaryKey()
-            _elem622.read(iprot)
-            self.primaryKeys.append(_elem622)
+          (_etype627, _size624) = iprot.readListBegin()
+          for _i628 in xrange(_size624):
+            _elem629 = SQLPrimaryKey()
+            _elem629.read(iprot)
+            self.primaryKeys.append(_elem629)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype626, _size623) = iprot.readListBegin()
-          for _i627 in xrange(_size623):
-            _elem628 = SQLForeignKey()
-            _elem628.read(iprot)
-            self.foreignKeys.append(_elem628)
+          (_etype633, _size630) = iprot.readListBegin()
+          for _i634 in xrange(_size630):
+            _elem635 = SQLForeignKey()
+            _elem635.read(iprot)
+            self.foreignKeys.append(_elem635)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13698,15 +13756,15 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter629 in self.primaryKeys:
-        iter629.write(oprot)
+      for iter636 in self.primaryKeys:
+        iter636.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter630 in self.foreignKeys:
-        iter630.write(oprot)
+      for iter637 in self.foreignKeys:
+        iter637.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14742,10 +14800,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype634, _size631) = iprot.readListBegin()
-          for _i635 in xrange(_size631):
-            _elem636 = iprot.readString()
-            self.success.append(_elem636)
+          (_etype641, _size638) = iprot.readListBegin()
+          for _i642 in xrange(_size638):
+            _elem643 = iprot.readString()
+            self.success.append(_elem643)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14768,8 +14826,8 @@ class get_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter637 in self.success:
-        oprot.writeString(iter637)
+      for iter644 in self.success:
+        oprot.writeString(iter644)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14842,10 +14900,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype641, _size638) = iprot.readListBegin()
-          for _i642 in xrange(_size638):
-            _elem643 = iprot.readString()
-            self.tbl_types.append(_elem643)
+          (_etype648, _size645) = iprot.readListBegin()
+          for _i649 in xrange(_size645):
+            _elem650 = iprot.readString()
+            self.tbl_types.append(_elem650)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14870,8 +14928,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter644 in self.tbl_types:
-        oprot.writeString(iter644)
+      for iter651 in self.tbl_types:
+        oprot.writeString(iter651)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14927,11 +14985,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype648, _size645) = iprot.readListBegin()
-          for _i649 in xrange(_size645):
-            _elem650 = TableMeta()
-            _elem650.read(iprot)
-            self.success.append(_elem650)
+          (_etype655, _size652) = iprot.readListBegin()
+          for _i656 in xrange(_size652):
+            _elem657 = TableMeta()
+            _elem657.read(iprot)
+            self.success.append(_elem657)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14954,8 +15012,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter651 in self.success:
-        iter651.write(oprot)
+      for iter658 in self.success:
+        iter658.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15079,10 +15137,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype655, _size652) = iprot.readListBegin()
-          for _i656 in xrange(_size652):
-            _elem657 = iprot.readString()
-            self.success.append(_elem657)
+          (_etype662, _size659) = iprot.readListBegin()
+          for _i663 in xrange(_size659):
+            _elem664 = iprot.readString()
+            self.success.append(_elem664)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15105,8 +15163,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter658 in self.success:
-        oprot.writeString(iter658)
+      for iter665 in self.success:
+        oprot.writeString(iter665)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15342,10 +15400,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype662, _size659) = iprot.readListBegin()
-          for _i663 in xrange(_size659):
-            _elem664 = iprot.readString()
-            self.tbl_names.append(_elem664)
+          (_etype669, _size666) = iprot.readListBegin()
+          for _i670 in xrange(_size666):
+            _elem671 = iprot.readString()
+            self.tbl_names.append(_elem671)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15366,8 +15424,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter665 in self.tbl_names:
-        oprot.writeString(iter665)
+      for iter672 in self.tbl_names:
+        oprot.writeString(iter672)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15428,11 +15486,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype669, _size666) = iprot.readListBegin()
-          for _i670 in xrange(_size666):
-            _elem671 = Table()
-            _elem671.read(iprot)
-            self.success.append(_elem671)
+          (_etype676, _size673) = iprot.readListBegin()
+          for _i677 in xrange(_size673):
+            _elem678 = Table()
+            _elem678.read(iprot)
+            self.success.append(_elem678)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15467,8 +15525,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter672 in self.success:
-        iter672.write(oprot)
+      for iter679 in self.success:
+        iter679.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15634,10 +15692,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype676, _size673) = iprot.readListBegin()
-          for _i677 in xrange(_size673):
-            _elem678 = iprot.readString()
-            self.success.append(_elem678)
+          (_etype683, _size680) = iprot.readListBegin()
+          for _i684 in xrange(_size680):
+            _elem685 = iprot.readString()
+            self.success.append(_elem685)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15672,8 +15730,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter679 in self.success:
-        oprot.writeString(iter679)
+      for iter686 in self.success:
+        oprot.writeString(iter686)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16643,11 +16701,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype683, _size680) = iprot.readListBegin()
-          for _i684 in xrange(_size680):
-            _elem685 = Partition()
-            _elem685.read(iprot)
-            self.new_parts.append(_elem685)
+          (_etype690, _size687) = iprot.readListBegin()
+          for _i691 in xrange(_size687):
+            _elem692 = Partition()
+            _elem692.read(iprot)
+            self.new_parts.append(_elem692)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16664,8 +16722,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter686 in self.new_parts:
-        iter686.write(oprot)
+      for iter693 in self.new_parts:
+        iter693.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16823,11 +16881,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype690, _size687) = iprot.readListBegin()
-          for _i691 in xrange(_size687):
-            _elem692 = PartitionSpec()
-            _elem692.read(iprot)
-            self.new_parts.append(_elem692)
+          (_etype697, _size694) = iprot.readListBegin()
+          for _i698 in xrange(_size694):
+            _elem699 = PartitionSpec()
+            _elem699.read(iprot)
+            self.new_parts.append(_elem699)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16844,8 +16902,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter693 in self.new_parts:
-        iter693.write(oprot)
+      for iter700 in self.new_parts:
+        iter700.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17019,10 +17077,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype697, _size694) = iprot.readListBegin()
-          for _i698 in xrange(_size694):
-            _elem699 = iprot.readString()
-            self.part_vals.append(_elem699)
+          (_etype704, _size701) = iprot.readListBegin()
+          for _i705 in xrange(_size701):
+            _elem706 = iprot.readString()
+            self.part_vals.append(_elem706)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17047,8 +17105,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter700 in self.part_vals:
-        oprot.writeString(iter700)
+      for iter707 in self.part_vals:
+        oprot.writeString(iter707)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17401,10 +17459,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype704, _size701) = iprot.readListBegin()
-          for _i705 in xrange(_size701):
-            _elem706 = iprot.readString()
-            self.part_vals.append(_elem706)
+          (_etype711, _size708) = iprot.readListBegin()
+          for _i712 in xrange(_size708):
+            _elem713 = iprot.readString()
+            self.part_vals.append(_elem713)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17435,8 +17493,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter707 in self.part_vals:
-        oprot.writeString(iter707)
+      for iter714 in self.part_vals:
+        oprot.writeString(iter714)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -18031,10 +18089,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype711, _size708) = iprot.readListBegin()
-          for _i712 in xrange(_size708):
-            _elem713 = iprot.readString()
-            self.part_vals.append(_elem713)
+          (_etype718, _size715) = iprot.readListBegin()
+          for _i719 in xrange(_size715):
+            _elem720 = iprot.readString()
+            self.part_vals.append(_elem720)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18064,8 +18122,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter714 in self.part_vals:
-        oprot.writeString(iter714)
+      for iter721 in self.part_vals:
+        oprot.writeString(iter721)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -18238,10 +18296,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype718, _size715) = iprot.readListBegin()
-          for _i719 in xrange(_size715):
-            _elem720 = iprot.readString()
-            self.part_vals.append(_elem720)
+          (_etype725, _size722) = iprot.readListBegin()
+          for _i726 in xrange(_size722):
+            _elem727 = iprot.readString()
+            self.part_vals.append(_elem727)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18277,8 +18335,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter721 in self.part_vals:
-        oprot.writeString(iter721)
+      for iter728 in self.part_vals:
+        oprot.writeString(iter728)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -19015,10 +19073,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype725, _size722) = iprot.readListBegin()
-          for _i726 in xrange(_size722):
-            _elem727 = iprot.readString()
-            self.part_vals.append(_elem727)
+          (_etype732, _size729) = iprot.readListBegin()
+          for _i733 in xrange(_size729):
+            _elem734 = iprot.readString()
+            self.part_vals.append(_elem734)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19043,8 +19101,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter728 in self.part_vals:
-        oprot.writeString(iter728)
+      for iter735 in self.part_vals:
+        oprot.writeString(iter735)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19203,11 +19261,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype730, _vtype731, _size729 ) = iprot.readMapBegin()
-          for _i733 in xrange(_size729):
-            _key734 = iprot.readString()
-            _val735 = iprot.readString()
-            self.partitionSpecs[_key734] = _val735
+          (_ktype737, _vtype738, _size736 ) = iprot.readMapBegin()
+          for _i740 in xrange(_size736):
+            _key741 = iprot.readString()
+            _val742 = iprot.readString()
+            self.partitionSpecs[_key741] = _val742
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19244,9 +19302,9 @@ class exchange_partition_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter736,viter737 in self.partitionSpecs.items():
-        oprot.writeString(kiter736)
-        oprot.writeString(viter737)
+      for kiter743,viter744 in self.partitionSpecs.items():
+        oprot.writeString(kiter743)
+        oprot.writeString(viter744)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -19451,11 +19509,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype739, _vtype740, _size738 ) = iprot.readMapBegin()
-          for _i742 in xrange(_size738):
-            _key743 = iprot.readString()
-            _val744 = iprot.readString()
-            self.partitionSpecs[_key743] = _val744
+          (_ktype746, _vtype747, _size745 ) = iprot.readMapBegin()
+          for _i749 in xrange(_size745):
+            _key750 = iprot.readString()
+            _val751 = iprot.readString()
+            self.partitionSpecs[_key750] = _val751
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19492,9 +19550,9 @@ class exchange_partitions_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter745,viter746 in self.partitionSpecs.items():
-        oprot.writeString(kiter745)
-        oprot.writeString(viter746)
+      for kiter752,viter753 in self.partitionSpecs.items():
+        oprot.writeString(kiter752)
+        oprot.writeString(viter753)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -19577,11 +19635,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype750, _size747) = iprot.readListBegin()
-          for _i751 in xrange(_size747):
-            _elem752 = Partition()
-            _elem752.read(iprot)
-            self.success.append(_elem752)
+          (_etype757, _size754) = iprot.readListBegin()
+          for _i758 in xrange(_size754):
+            _elem759 = Partition()
+            _elem759.read(iprot)
+            self.success.append(_elem759)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19622,8 +19680,8 @@ class exchange_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter753 in self.success:
-        iter753.write(oprot)
+      for iter760 in self.success:
+        iter760.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19717,10 +19775,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype757, _size754) = iprot.readListBegin()
-          for _i758 in xrange(_size754):
-            _elem759 = iprot.readString()
-            self.part_vals.append(_elem759)
+          (_etype764, _size761) = iprot.readListBegin()
+          for _i765 in xrange(_size761):
+            _elem766 = iprot.readString()
+            self.part_vals.append(_elem766)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19732,10 +19790,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype763, _size760) = iprot.readListBegin()
-          for _i764 in xrange(_size760):
-            _elem765 = iprot.readString()
-            self.group_names.append(_elem765)
+          (_etype770, _size767) = iprot.readListBegin()
+          for _i771 in xrange(_size767):
+            _elem772 = iprot.readString()
+            self.group_names.append(_elem772)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19760,8 +19818,8 @@ class get_partition_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter766 in self.part_vals:
-        oprot.writeString(iter766)
+      for iter773 in self.part_vals:
+        oprot.writeString(iter773)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -19771,8 +19829,8 @@ class get_partition_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter767 in self.group_names:
-        oprot.writeString(iter767)
+      for iter774 in self.group_names:
+        oprot.writeString(iter774)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20201,11 +20259,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype771, _size768) = iprot.readListBegin()
-          for _i772 in xrange(_size768):
-            _elem773 = Partition()
-            _elem773.read(iprot)
-            self.success.append(_elem773)
+          (_etype778, _size775) = iprot.readListBegin()
+          for _i779 in xrange(_size775):
+            _elem780 = Partition()
+            _elem780.read(iprot)
+            self.success.append(_elem780)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20234,8 +20292,8 @@ class get_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter774 in self.success:
-        iter774.write(oprot)
+      for iter781 in self.success:
+        iter781.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20329,10 +20387,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype778, _size775) = iprot.readListBegin()
-          for _i779 in xrange(_size775):
-            _elem780 = iprot.readString()
-            self.group_names.append(_elem780)
+          (_etype785, _size782) = iprot.readListBegin()
+          for _i786 in xrange(_size782):
+            _elem787 = iprot.readString()
+            self.group_names.append(_elem787)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20365,8 +20423,8 @@ class get_partitions_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter781 in self.group_names:
-        oprot.writeString(iter781)
+      for iter788 in self.group_names:
+        oprot.writeString(iter788)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20427,11 +20485,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype785, _size782) = iprot.readListBegin()
-          for _i786 in xrange(_size782):
-            _elem787 = Partition()
-            _elem787.read(iprot)
-            self.success.append(_elem787)
+          (_etype792, _size789) = iprot.readListBegin()
+          for _i793 in xrange(_size789):
+            _elem794 = Partition()
+            _elem794.read(iprot)
+            self.success.append(_elem794)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20460,8 +20518,8 @@ class get_partitions_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter788 in self.success:
-        iter788.write(oprot)
+      for iter795 in self.success:
+        iter795.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20619,11 +20677,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype792, _size789) = iprot.readListBegin()
-          for _i793 in xrange(_size789):
-            _elem794 = PartitionSpec()
-            _elem794.read(iprot)
-            self.success.append(_elem794)
+          (_etype799, _size796) = iprot.readListBegin()
+          for _i800 in xrange(_size796):
+            _elem801 = PartitionSpec()
+            _elem801.read(iprot)
+            self.success.append(_elem801)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20652,8 +20710,8 @@ class get_partitions_pspec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter795 in self.success:
-        iter795.write(oprot)
+      for iter802 in self.success:
+        iter802.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20808,10 +20866,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype799, _size796) = iprot.readListBegin()
-          for _i800 in xrange(_size796):
-            _elem801 = iprot.readString()
-            self.success.append(_elem801)
+          (_etype806, _size803) = iprot.readListBegin()
+          for _i807 in xrange(_size803):
+            _elem808 = iprot.readString()
+            self.success.append(_elem808)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20834,8 +20892,8 @@ class get_partition_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter802 in self.success:
-        oprot.writeString(iter802)
+      for iter809 in self.success:
+        oprot.writeString(iter809)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -20911,10 +20969,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype806, _size803) = iprot.readListBegin()
-          for _i807 in xrange(_size803):
-            _elem808 = iprot.readString()
-            self.part_vals.append(_elem808)
+          (_etype813, _size810) = iprot.readListBegin()
+          for _i814 in xrange(_size810):
+            _elem815 = iprot.readString()
+            self.part_vals.append(_elem815)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20944,8 +21002,8 @@ class get_partitions_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter809 in self.part_vals:
-        oprot.writeString(iter809)
+      for iter816 in self.part_vals:
+        oprot.writeString(iter816)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21009,11 +21067,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype813, _size810) = iprot.readListBegin()
-          for _i814 in xrange(_size810):
-            _elem815 = Partition()
-            _elem815.read(iprot)
-            self.success.append(_elem815)
+          (_etype820, _size817) = iprot.readListBegin()
+          for _i821 in xrange(_size817):
+            _elem822 = Partition()
+            _elem822.read(iprot)
+            self.success.append(_elem822)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21042,8 +21100,8 @@ class get_partitions_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter816 in self.success:
-        iter816.write(oprot)
+      for iter823 in self.success:
+        iter823.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21130,10 +21188,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype820, _size817) = iprot.readListBegin()
-          for _i821 in xrange(_size817):
-            _elem822 = iprot.readString()
-            self.part_vals.append(_elem822)
+          (_etype827, _size824) = iprot.readListBegin()
+          for _i828 in xrange(_size824):
+            _elem829 = iprot.readString()
+            self.part_vals.append(_elem829)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21150,10 +21208,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype826, _size823) = iprot.readListBegin()
-          for _i827 in xrange(_size823):
-            _elem828 = iprot.readString()
-            self.group_names.append(_elem828)
+          (_etype833, _size830) = iprot.readListBegin()
+          for _i834 in xrange(_size830):
+            _elem835 = iprot.readString()
+            self.group_names.append(_elem835)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21178,8 +21236,8 @@ class get_partitions_ps_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter829 in self.part_vals:
-        oprot.writeString(iter829)
+      for iter836 in self.part_vals:
+        oprot.writeString(iter836)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21193,8 +21251,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter830 in self.group_names:
-        oprot.writeString(iter830)
+      for iter837 in self.group_names:
+        oprot.writeString(iter837)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21256,11 +21314,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype834, _size831) = iprot.readListBegin()
-          for _i835 in xrange(_size831):
-            _elem836 = Partition()
-            _elem836.read(iprot)
-            self.success.append(_elem836)
+          (_etype841, _size838) = iprot.readListBegin()
+          for _i842 in xrange(_size838):
+            _elem843 = Partition()
+            _elem843.read(iprot)
+            self.success.append(_elem843)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21289,8 +21347,8 @@ class get_partitions_ps_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter837 in self.success:
-        iter837.write(oprot)
+      for iter844 in self.success:
+        iter844.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21371,10 +21429,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype841, _size838) = iprot.readListBegin()
-          for _i842 in xrange(_size838):
-            _elem843 = iprot.readString()
-            self.part_vals.append(_elem843)
+          (_etype848, _size845) = iprot.readListBegin()
+          for _i849 in xrange(_size845):
+            _elem850 = iprot.readString()
+            self.part_vals.append(_elem850)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21404,8 +21462,8 @@ class get_partition_names_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter844 in self.part_vals:
-        oprot.writeString(iter844)
+      for iter851 in self.part_vals:
+        oprot.writeString(iter851)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21469,10 +21527,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype848, _size845) = iprot.readListBegin()
-          for _i849 in xrange(_size845):
-            _elem850 = iprot.readString()
-            self.success.append(_elem850)
+          (_etype855, _size852) = iprot.readListBegin()
+          for _i856 in xrange(_size852):
+            _elem857 = iprot.readString()
+            self.success.append(_elem857)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21501,8 +21559,8 @@ class get_partition_names_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter851 in self.success:
-        oprot.writeString(iter851)
+      for iter858 in self.success:
+        oprot.writeString(iter858)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21673,11 +21731,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype855, _size852) = iprot.readListBegin()
-          for _i856 in xrange(_size852):
-            _elem857 = Partition()
-            _elem857.read(iprot)
-            self.success.append(_elem857)
+          (_etype862, _size859) = iprot.readListBegin()
+          for _i863 in xrange(_size859):
+            _elem864 = Partition()
+            _elem864.read(iprot)
+            self.success.append(_elem864)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21706,8 +21764,8 @@ class get_partitions_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter858 in self.success:
-        iter858.write(oprot)
+      for iter865 in self.success:
+        iter865.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21878,11 +21936,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype862, _size859) = iprot.readListBegin()
-          for _i863 in xrange(_size859):
-            _elem864 = PartitionSpec()
-            _elem864.read(iprot)
-            self.success.append(_elem864)
+          (_etype869, _size866) = iprot.readListBegin()
+          for _i870 in xrange(_size866):
+            _elem871 = PartitionSpec()
+            _elem871.read(iprot)
+            self.success.append(_elem871)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21911,8 +21969,8 @@ class get_part_specs_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter865 in self.success:
-        iter865.write(oprot)
+      for iter872 in self.success:
+        iter872.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22332,10 +22390,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype869, _size866) = iprot.readListBegin()
-          for _i870 in xrange(_size866):
-            _elem871 = iprot.readString()
-            self.names.append(_elem871)
+          (_etype876, _size873) = iprot.readListBegin()
+          for _i877 in xrange(_size873):
+            _elem878 = iprot.readString()
+            self.names.append(_elem878)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22360,8 +22418,8 @@ class get_partitions_by_names_args:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter872 in self.names:
-        oprot.writeString(iter872)
+      for iter879 in self.names:
+        oprot.writeString(iter879)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22420,11 +22478,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype876, _size873) = iprot.readListBegin()
-          for _i877 in xrange(_size873):
-            _elem878 = Partition()
-            _elem878.read(iprot)
-            self.success.append(_elem878)
+          (_etype883, _size880) = iprot.readListBegin()
+          for _i884 in xrange(_size880):
+            _elem885 = Partition()
+            _elem885.read(iprot)
+            self.success.append(_elem885)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22453,8 +22511,8 @@ class get_partitions_by_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter879 in self.success:
-        iter879.write(oprot)
+      for iter886 in self.success:
+        iter886.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22704,11 +22762,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype883, _size880) = iprot.readListBegin()
-          for _i884 in xrange(_size880):
-            _elem885 = Partition()
-            _elem885.read(iprot)
-            self.new_parts.append(_elem885)
+          (_etype890, _size887) = iprot.readListBegin()
+          for _i891 in xrange(_size887):
+            _elem892 = Partition()
+            _elem892.read(iprot)
+            self.new_parts.append(_elem892)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22733,8 +22791,8 @@ class alter_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter886 in self.new_parts:
-        iter886.write(oprot)
+      for iter893 in self.new_parts:
+        iter893.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22887,11 +22945,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype890, _size887) = iprot.readListBegin()
-          for _i891 in xrange(_size887):
-            _elem892 = Partition()
-            _elem892.read(iprot)
-            self.new_parts.append(_elem892)
+          (_etype897, _size894) = iprot.readListBegin()
+          for _i898 in xrange(_size894):
+            _elem899 = Partition()
+            _elem899.read(iprot)
+            self.new_parts.append(_elem899)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22922,8 +22980,8 @@ class alter_partitions_with_environment_context_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter893 in self.new_parts:
-        iter893.write(oprot)
+      for iter900 in self.new_parts:
+        iter900.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -23267,10 +23325,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype897, _size894) = iprot.readListBegin()
-          for _i898 in xrange(_size894):
-            _elem899 = iprot.readString()
-            self.part_vals.append(_elem899)
+          (_etype904, _size901) = iprot.readListBegin()
+          for _i905 in xrange(_size901):
+            _elem906 = iprot.readString()
+            self.part_vals.append(_elem906)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23301,8 +23359,8 @@ class rename_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter900 in self.part_vals:
-        oprot.writeString(iter900)
+      for iter907 in self.part_vals:
+        oprot.writeString(iter907)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -23444,10 +23502,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype904, _size901) = iprot.readListBegin()
-          for _i905 in xrange(_size901):
-            _elem906 = iprot.readString()
-            self.part_vals.append(_elem906)
+          (_etype911, _size908) = iprot.readListBegin()
+          for _i912 in xrange(_size908):
+            _elem913 = iprot.readString()
+            self.part_vals.append(_elem913)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23469,8 +23527,8 @@ class partition_name_has_valid_characters_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter907 in self.part_vals:
-        oprot.writeString(iter907)
+      for iter914 in self.part_vals:
+        oprot.writeString(iter914)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -23828,10 +23886,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype911, _size908) = iprot.readListBegin()
-          for _i912 in xrange(_size908):
-            _elem913 = iprot.readString()
-            self.success.append(_elem913)
+          (_etype918, _size915) = iprot.readListBegin()
+          for _i919 in xrange(_size915):
+            _elem920 = iprot.readString()
+            self.success.append(_elem920)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23854,8 +23912,8 @@ class partition_name_to_vals_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter914 in self.success:
-        oprot.writeString(iter914)
+      for iter921 in self.success:
+        oprot.writeString(iter921)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23979,11 +24037,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype916, _vtype917, _size915 ) = iprot.readMapBegin()
-          for _i919 in xrange(_size915):
-            _key920 = iprot.readString()
-            _val921 = iprot.readString()
-            self.success[_key920] = _val921
+          (_ktype923, _vtype924, _size922 ) = iprot.readMapBegin()
+          for _i926 in xrange(_size922):
+            _key927 = iprot.readString()
+            _val928 = iprot.readString()
+            self.success[_key927] = _val928
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24006,9 +24064,9 @@ class partition_name_to_spec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter922,viter923 in self.success.items():
-        oprot.writeString(kiter922)
-        oprot.writeString(viter923)
+      for kiter929,viter930 in self.success.items():
+        oprot.writeString(kiter929)
+        oprot.writeString(viter930)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24084,11 +24142,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype925, _vtype926, _size924 ) = iprot.readMapBegin()
-          for _i928 in xrange(_size924):
-            _key929 = iprot.readString()
-            _val930 = iprot.readString()
-            self.part_vals[_key929] = _val930
+          (_ktype932, _vtype933, _size931 ) = iprot.readMapBegin()
+          for _i935 in xrange(_size931):
+            _key936 = iprot.readString()
+            _val937 = iprot.readString()
+            self.part_vals[_key936] = _val937
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24118,9 +24176,9 @@ class markPartitionForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter931,viter932 in self.part_vals.items():
-        oprot.writeString(kiter931)
-        oprot.writeString(viter932)
+      for kiter938,viter939 in self.part_vals.items():
+        oprot.writeString(kiter938)
+        oprot.writeString(viter939)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -24334,11 +24392,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype934, _vtype935, _size933 ) = iprot.readMapBegin()
-          for _i937 in xrange(_size933):
-            _key938 = iprot.readString()
-            _val939 = iprot.readString()
-            self.part_vals[_key938] = _val939
+          (_ktype941, _vtype942, _size940 ) = iprot.readMapBegin()
+          for _i944 in xrange(_size940):
+            _key945 = iprot.readString()
+            _val946 = iprot.readString()
+            self.part_vals[_key945] = _val946
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24368,9 +24426,9 @@ class isPartitionMarkedForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter940,viter941 in self.part_vals.items():
-        oprot.writeString(kiter940)
-        oprot.writeString(viter941)
+      for kiter947,viter948 in self.part_vals.items():
+        oprot.writeString(kiter947)
+        oprot.writeString(viter948)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -25425,11 +25483,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype945, _size942) = iprot.readListBegin()
-          for _i946 in xrange(_size942):
-            _elem947 = Index()
-            _elem947.read(iprot)
-            self.success.append(_elem947)
+          (_etype952, _size949) = iprot.readListBegin()
+          for _i953 in xrange(_size949):
+            _elem954 = Index()
+            _elem954.read(iprot)
+            self.success.append(_elem954)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25458,8 +25516,8 @@ class get_indexes_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter948 in self.success:
-        iter948.write(oprot)
+      for iter955 in self.success:
+        iter955.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25614,10 +25672,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype952, _size949) = iprot.readListBegin()
-          for _i953 in xrange(_size949):
-            _elem954 = iprot.readString()
-            self.success.append(_elem954)
+          (_etype959, _size956) = iprot.readListBegin()
+          for _i960 in xrange(_size956):
+            _elem961 = iprot.readString()
+            self.success.append(_elem961)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25640,8 +25698,8 @@ class get_index_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter955 in self.success:
-        oprot.writeString(iter955)
+      for iter962 in self.success:
+        oprot.writeString(iter962)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -28507,10 +28565,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype959, _size956) = iprot.readListBegin()
-          for _i960 in xrange(_size956):
-            _elem961 = iprot.readString()
-            self.success.append(_elem961)
+          (_etype966, _size963) = iprot.readListBegin()
+          for _i967 in xrange(_size963):
+            _elem968 = iprot.readString()
+            self.success.append(_elem968)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28533,8 +28591,8 @@ class get_functions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter962 in self.success:
-        oprot.writeString(iter962)
+      for iter969 in self.success:
+        oprot.writeString(iter969)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29222,10 +29280,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype966, _size963) = iprot.readListBegin()
-          for _i967 in xrange(_size963):
-            _elem968 = iprot.readString()
-            self.success.append(_elem968)
+          (_etype973, _size970) = iprot.readListBegin()
+          for _i974 in xrange(_size970):
+            _elem975 = iprot.readString()
+            self.success.append(_elem975)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29248,8 +29306,8 @@ class get_role_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter969 in self.success:
-        oprot.writeString(iter969)
+      for iter976 in self.success:
+        oprot.writeString(iter976)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29763,11 +29821,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype973, _size970) = iprot.readListBegin()
-          for _i974 in xrange(_size970):
-            _elem975 = Role()
-            _elem975.read(iprot)
-            self.success.append(_elem975)
+          (_etype980, _size977) = iprot.readListBegin()
+          for _i981 in xrange(_size977):
+            _elem982 = Role()
+            _elem982.read(iprot)
+            self.success.append(_elem982)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29790,8 +29848,8 @@ class list_roles_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter976 in self.success:
-        iter976.write(oprot)
+      for iter983 in self.success:
+        iter983.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30300,10 +30358,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype980, _size977) = iprot.readListBegin()
-          for _i981 in xrange(_size977):
-            _elem982 = iprot.readString()
-            self.group_names.append(_elem982)
+          (_etype987, _size984) = iprot.readListBegin()
+          for _i988 in xrange(_size984):
+            _elem989 = iprot.readString()
+            self.group_names.append(_elem989)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30328,8 +30386,8 @@ class get_privilege_set_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter983 in self.group_names:
-        oprot.writeString(iter983)
+      for iter990 in self.group_names:
+        oprot.writeString(iter990)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -30556,11 +30614,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype987, _size984) = iprot.readListBegin()
-          for _i988 in xrange(_size984):
-            _elem989 = HiveObjectPrivilege()
-            _elem989.read(iprot)
-            self.success.append(_elem989)
+          (_etype994, _size991) = iprot.readListBegin()
+          for _i995 in xrange(_size991):
+            _elem996 = HiveObjectPrivilege()
+            _elem996.read(iprot)
+            self.success.append(_elem996)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30583,8 +30641,8 @@ class list_privileges_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter990 in self.success:
-        iter990.write(oprot)
+      for iter997 in self.success:
+        iter997.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -31082,10 +31140,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype994, _size991) = iprot.readListBegin()
-          for _i995 in xrange(_size991):
-            _elem996 = iprot.readString()
-            self.group_names.append(_elem996)
+          (_etype1001, _size998) = iprot.readListBegin()
+          for _i1002 in xrange(_size998):
+            _elem1003 = iprot.readString()
+            self.group_names.append(_elem1003)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31106,8 +31164,8 @@ class set_ugi_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter997 in self.group_names:
-        oprot.writeString(iter997)
+      for iter1004 in self.group_names:
+        oprot.writeString(iter1004)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -31162,10 +31220,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1001, _size998) = iprot.readListBegin()
-          for _i1002 in xrange(_size998):
-            _elem1003 = iprot.readString()
-            self.success.append(_elem1003)
+          (_etype1008, _size1005) = iprot.readListBegin()
+          for _i1009 in xrange(_size1005):
+            _elem1010 = iprot.readString()
+            self.success.append(_elem1010)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31188,8 +31246,8 @@ class set_ugi_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1004 in self.success:
-        oprot.writeString(iter1004)
+      for iter1011 in self.success:
+        oprot.writeString(iter1011)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -32121,10 +32179,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1008, _size1005) = iprot.readListBegin()
-          for _i1009 in xrange(_size1005):
-            _elem1010 = iprot.readString()
-            self.success.append(_elem1010)
+          (_etype1015, _size1012) = iprot.readListBegin()
+          for _i1016 in xrange(_size1012):
+            _elem1017 = iprot.readString()
+            self.success.append(_elem1017)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -32141,8 +32199,8 @@ class get_all_token_identifiers_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1011 in self.success:
-        oprot.writeString(iter1011)
+      for iter1018 in self.success:
+        oprot.writeString(iter1018)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -32669,10 +32727,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1015, _size1012) = iprot.readListBegin()
-          for _i1016 in xrange(_size1012):
-            _elem1017 = iprot.readString()
-            self.success.append(_elem1017)
+          (_etype1022, _size1019) = iprot.readListBegin()
+          for _i1023 in xrange(_size1019):
+            _elem1024 = iprot.readString()
+            self.success.append(_elem1024)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -32689,8 +32747,8 @@ class get_master_keys_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1018 in self.success:
-        oprot.writeString(iter1018)
+      for iter1025 in self.success:
+        oprot.writeString(iter1025)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -36280,3 +36338,134 @@ class heartbeat_write_id_result:
 
   def __ne__(self, other):
     return not (self == other)
+
+class get_valid_write_ids_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (GetValidWriteIdsRequest, GetValidWriteIdsRequest.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 = GetValidWriteIdsRequest()
+          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('get_valid_write_ids_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 get_valid_write_ids_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (GetValidWriteIdsResult, GetValidWriteIdsResult.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  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 == 0:
+        if ftype == TType.STRUCT:
+          self.success = GetValidWriteIdsResult()
+          self.success.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('get_valid_write_ids_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    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)


[21/31] hive git commit: HIVE-14727 : llap-server may case file descriptor leak in BuddyAllocator class (Yechao Chen, reviewed by Sergey Shelukhin, Gopal V)

Posted by se...@apache.org.
HIVE-14727 : llap-server may case file descriptor leak in BuddyAllocator class (Yechao Chen, reviewed by Sergey Shelukhin, Gopal V)


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

Branch: refs/heads/hive-14535
Commit: 49f103c7f86f12256a91be85804c58ed544f2735
Parents: 518789d
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Sep 12 12:47:09 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Sep 12 12:47:09 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/llap/cache/BuddyAllocator.java    | 18 ++++++++++++------
 1 file changed, 12 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/49f103c7/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 0c53779..d9d407d 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
@@ -35,6 +35,7 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.io.encoded.MemoryBuffer;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -335,23 +336,28 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
 
   private ByteBuffer preallocate(int arenaSize) {
     if (isMapped) {
+      RandomAccessFile rwf = null;
+      File rf = null;
       Preconditions.checkArgument(isDirect, "All memory mapped allocations have to be direct buffers");
       try {
-        File rf = File.createTempFile("arena-", ".cache", cacheDir.toFile());
-        RandomAccessFile rwf = new RandomAccessFile(rf, "rw");
+        rf = File.createTempFile("arena-", ".cache", cacheDir.toFile());
+        rwf = new RandomAccessFile(rf, "rw");
         rwf.setLength(arenaSize); // truncate (TODO: posix_fallocate?)
         // Use RW, not PRIVATE because the copy-on-write is irrelevant for a deleted file
         // see discussion in YARN-5551 for the memory accounting discussion
         ByteBuffer rwbuf = rwf.getChannel().map(MapMode.READ_WRITE, 0, arenaSize);
-        // A mapping, once established, is not dependent upon the file channel that was used to
-        // create it. delete file and hold onto the map
-        rwf.close();
-        rf.delete();
         return rwbuf;
       } catch (IOException ioe) {
         LlapIoImpl.LOG.warn("Failed trying to allocate memory mapped arena", ioe);
         // fail similarly when memory allocations fail
         throw new OutOfMemoryError("Failed trying to allocate memory mapped arena: " + ioe.getMessage());
+      } finally {
+        // A mapping, once established, is not dependent upon the file channel that was used to
+        // create it. delete file and hold onto the map
+        IOUtils.closeQuietly(rwf);
+        if (rf != null) {
+          rf.delete();
+        }
       }
     }
     return isDirect ? ByteBuffer.allocateDirect(arenaSize) : ByteBuffer.allocate(arenaSize);


[10/31] hive git commit: HIVE-14716: Duplicate pom.xml entries for mockito (Vihang Karajgaonkar, reviewed by Sergio Pena)

Posted by se...@apache.org.
HIVE-14716: Duplicate pom.xml entries for mockito (Vihang Karajgaonkar, 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/4c543b17
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4c543b17
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4c543b17

Branch: refs/heads/hive-14535
Commit: 4c543b17d15763590d5d1619aeff8411c7aa6c73
Parents: 58d1bef
Author: Vihang Karajgaonkar <vi...@cloudera.com>
Authored: Thu Sep 8 09:50:33 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Thu Sep 8 09:50:33 2016 -0500

----------------------------------------------------------------------
 beeline/pom.xml | 10 ----------
 1 file changed, 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4c543b17/beeline/pom.xml
----------------------------------------------------------------------
diff --git a/beeline/pom.xml b/beeline/pom.xml
index d03f770..5503add 100644
--- a/beeline/pom.xml
+++ b/beeline/pom.xml
@@ -124,21 +124,11 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>postgresql</groupId>
       <artifactId>postgresql</artifactId>
       <version>9.1-901.jdbc4</version>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
   </dependencies>
 
   <profiles>


[30/31] hive git commit: HIVE-14644 : use metastore information on the read path appropriately (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/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 5ed3912..0b6fe91 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 _size793;
-            ::apache::thrift::protocol::TType _etype796;
-            xfer += iprot->readListBegin(_etype796, _size793);
-            this->success.resize(_size793);
-            uint32_t _i797;
-            for (_i797 = 0; _i797 < _size793; ++_i797)
+            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)
             {
-              xfer += iprot->readString(this->success[_i797]);
+              xfer += iprot->readString(this->success[_i807]);
             }
             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 _iter798;
-      for (_iter798 = this->success.begin(); _iter798 != this->success.end(); ++_iter798)
+      std::vector<std::string> ::const_iterator _iter808;
+      for (_iter808 = this->success.begin(); _iter808 != this->success.end(); ++_iter808)
       {
-        xfer += oprot->writeString((*_iter798));
+        xfer += oprot->writeString((*_iter808));
       }
       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 _size799;
-            ::apache::thrift::protocol::TType _etype802;
-            xfer += iprot->readListBegin(_etype802, _size799);
-            (*(this->success)).resize(_size799);
-            uint32_t _i803;
-            for (_i803 = 0; _i803 < _size799; ++_i803)
+            uint32_t _size809;
+            ::apache::thrift::protocol::TType _etype812;
+            xfer += iprot->readListBegin(_etype812, _size809);
+            (*(this->success)).resize(_size809);
+            uint32_t _i813;
+            for (_i813 = 0; _i813 < _size809; ++_i813)
             {
-              xfer += iprot->readString((*(this->success))[_i803]);
+              xfer += iprot->readString((*(this->success))[_i813]);
             }
             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 _size804;
-            ::apache::thrift::protocol::TType _etype807;
-            xfer += iprot->readListBegin(_etype807, _size804);
-            this->success.resize(_size804);
-            uint32_t _i808;
-            for (_i808 = 0; _i808 < _size804; ++_i808)
+            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)
             {
-              xfer += iprot->readString(this->success[_i808]);
+              xfer += iprot->readString(this->success[_i818]);
             }
             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 _iter809;
-      for (_iter809 = this->success.begin(); _iter809 != this->success.end(); ++_iter809)
+      std::vector<std::string> ::const_iterator _iter819;
+      for (_iter819 = this->success.begin(); _iter819 != this->success.end(); ++_iter819)
       {
-        xfer += oprot->writeString((*_iter809));
+        xfer += oprot->writeString((*_iter819));
       }
       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 _size810;
-            ::apache::thrift::protocol::TType _etype813;
-            xfer += iprot->readListBegin(_etype813, _size810);
-            (*(this->success)).resize(_size810);
-            uint32_t _i814;
-            for (_i814 = 0; _i814 < _size810; ++_i814)
+            uint32_t _size820;
+            ::apache::thrift::protocol::TType _etype823;
+            xfer += iprot->readListBegin(_etype823, _size820);
+            (*(this->success)).resize(_size820);
+            uint32_t _i824;
+            for (_i824 = 0; _i824 < _size820; ++_i824)
             {
-              xfer += iprot->readString((*(this->success))[_i814]);
+              xfer += iprot->readString((*(this->success))[_i824]);
             }
             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 _size815;
-            ::apache::thrift::protocol::TType _ktype816;
-            ::apache::thrift::protocol::TType _vtype817;
-            xfer += iprot->readMapBegin(_ktype816, _vtype817, _size815);
-            uint32_t _i819;
-            for (_i819 = 0; _i819 < _size815; ++_i819)
+            uint32_t _size825;
+            ::apache::thrift::protocol::TType _ktype826;
+            ::apache::thrift::protocol::TType _vtype827;
+            xfer += iprot->readMapBegin(_ktype826, _vtype827, _size825);
+            uint32_t _i829;
+            for (_i829 = 0; _i829 < _size825; ++_i829)
             {
-              std::string _key820;
-              xfer += iprot->readString(_key820);
-              Type& _val821 = this->success[_key820];
-              xfer += _val821.read(iprot);
+              std::string _key830;
+              xfer += iprot->readString(_key830);
+              Type& _val831 = this->success[_key830];
+              xfer += _val831.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 _iter822;
-      for (_iter822 = this->success.begin(); _iter822 != this->success.end(); ++_iter822)
+      std::map<std::string, Type> ::const_iterator _iter832;
+      for (_iter832 = this->success.begin(); _iter832 != this->success.end(); ++_iter832)
       {
-        xfer += oprot->writeString(_iter822->first);
-        xfer += _iter822->second.write(oprot);
+        xfer += oprot->writeString(_iter832->first);
+        xfer += _iter832->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 _size823;
-            ::apache::thrift::protocol::TType _ktype824;
-            ::apache::thrift::protocol::TType _vtype825;
-            xfer += iprot->readMapBegin(_ktype824, _vtype825, _size823);
-            uint32_t _i827;
-            for (_i827 = 0; _i827 < _size823; ++_i827)
+            uint32_t _size833;
+            ::apache::thrift::protocol::TType _ktype834;
+            ::apache::thrift::protocol::TType _vtype835;
+            xfer += iprot->readMapBegin(_ktype834, _vtype835, _size833);
+            uint32_t _i837;
+            for (_i837 = 0; _i837 < _size833; ++_i837)
             {
-              std::string _key828;
-              xfer += iprot->readString(_key828);
-              Type& _val829 = (*(this->success))[_key828];
-              xfer += _val829.read(iprot);
+              std::string _key838;
+              xfer += iprot->readString(_key838);
+              Type& _val839 = (*(this->success))[_key838];
+              xfer += _val839.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 _size830;
-            ::apache::thrift::protocol::TType _etype833;
-            xfer += iprot->readListBegin(_etype833, _size830);
-            this->success.resize(_size830);
-            uint32_t _i834;
-            for (_i834 = 0; _i834 < _size830; ++_i834)
+            uint32_t _size840;
+            ::apache::thrift::protocol::TType _etype843;
+            xfer += iprot->readListBegin(_etype843, _size840);
+            this->success.resize(_size840);
+            uint32_t _i844;
+            for (_i844 = 0; _i844 < _size840; ++_i844)
             {
-              xfer += this->success[_i834].read(iprot);
+              xfer += this->success[_i844].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 _iter835;
-      for (_iter835 = this->success.begin(); _iter835 != this->success.end(); ++_iter835)
+      std::vector<FieldSchema> ::const_iterator _iter845;
+      for (_iter845 = this->success.begin(); _iter845 != this->success.end(); ++_iter845)
       {
-        xfer += (*_iter835).write(oprot);
+        xfer += (*_iter845).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 _size836;
-            ::apache::thrift::protocol::TType _etype839;
-            xfer += iprot->readListBegin(_etype839, _size836);
-            (*(this->success)).resize(_size836);
-            uint32_t _i840;
-            for (_i840 = 0; _i840 < _size836; ++_i840)
+            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 += (*(this->success))[_i840].read(iprot);
+              xfer += (*(this->success))[_i850].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 _size841;
-            ::apache::thrift::protocol::TType _etype844;
-            xfer += iprot->readListBegin(_etype844, _size841);
-            this->success.resize(_size841);
-            uint32_t _i845;
-            for (_i845 = 0; _i845 < _size841; ++_i845)
+            uint32_t _size851;
+            ::apache::thrift::protocol::TType _etype854;
+            xfer += iprot->readListBegin(_etype854, _size851);
+            this->success.resize(_size851);
+            uint32_t _i855;
+            for (_i855 = 0; _i855 < _size851; ++_i855)
             {
-              xfer += this->success[_i845].read(iprot);
+              xfer += this->success[_i855].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 _iter846;
-      for (_iter846 = this->success.begin(); _iter846 != this->success.end(); ++_iter846)
+      std::vector<FieldSchema> ::const_iterator _iter856;
+      for (_iter856 = this->success.begin(); _iter856 != this->success.end(); ++_iter856)
       {
-        xfer += (*_iter846).write(oprot);
+        xfer += (*_iter856).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 _size847;
-            ::apache::thrift::protocol::TType _etype850;
-            xfer += iprot->readListBegin(_etype850, _size847);
-            (*(this->success)).resize(_size847);
-            uint32_t _i851;
-            for (_i851 = 0; _i851 < _size847; ++_i851)
+            uint32_t _size857;
+            ::apache::thrift::protocol::TType _etype860;
+            xfer += iprot->readListBegin(_etype860, _size857);
+            (*(this->success)).resize(_size857);
+            uint32_t _i861;
+            for (_i861 = 0; _i861 < _size857; ++_i861)
             {
-              xfer += (*(this->success))[_i851].read(iprot);
+              xfer += (*(this->success))[_i861].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 _size852;
-            ::apache::thrift::protocol::TType _etype855;
-            xfer += iprot->readListBegin(_etype855, _size852);
-            this->success.resize(_size852);
-            uint32_t _i856;
-            for (_i856 = 0; _i856 < _size852; ++_i856)
+            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)
             {
-              xfer += this->success[_i856].read(iprot);
+              xfer += this->success[_i866].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 _iter857;
-      for (_iter857 = this->success.begin(); _iter857 != this->success.end(); ++_iter857)
+      std::vector<FieldSchema> ::const_iterator _iter867;
+      for (_iter867 = this->success.begin(); _iter867 != this->success.end(); ++_iter867)
       {
-        xfer += (*_iter857).write(oprot);
+        xfer += (*_iter867).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 _size858;
-            ::apache::thrift::protocol::TType _etype861;
-            xfer += iprot->readListBegin(_etype861, _size858);
-            (*(this->success)).resize(_size858);
-            uint32_t _i862;
-            for (_i862 = 0; _i862 < _size858; ++_i862)
+            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)
             {
-              xfer += (*(this->success))[_i862].read(iprot);
+              xfer += (*(this->success))[_i872].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 _size863;
-            ::apache::thrift::protocol::TType _etype866;
-            xfer += iprot->readListBegin(_etype866, _size863);
-            this->success.resize(_size863);
-            uint32_t _i867;
-            for (_i867 = 0; _i867 < _size863; ++_i867)
+            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)
             {
-              xfer += this->success[_i867].read(iprot);
+              xfer += this->success[_i877].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 _iter868;
-      for (_iter868 = this->success.begin(); _iter868 != this->success.end(); ++_iter868)
+      std::vector<FieldSchema> ::const_iterator _iter878;
+      for (_iter878 = this->success.begin(); _iter878 != this->success.end(); ++_iter878)
       {
-        xfer += (*_iter868).write(oprot);
+        xfer += (*_iter878).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 _size869;
-            ::apache::thrift::protocol::TType _etype872;
-            xfer += iprot->readListBegin(_etype872, _size869);
-            (*(this->success)).resize(_size869);
-            uint32_t _i873;
-            for (_i873 = 0; _i873 < _size869; ++_i873)
+            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)
             {
-              xfer += (*(this->success))[_i873].read(iprot);
+              xfer += (*(this->success))[_i883].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 _size874;
-            ::apache::thrift::protocol::TType _etype877;
-            xfer += iprot->readListBegin(_etype877, _size874);
-            this->primaryKeys.resize(_size874);
-            uint32_t _i878;
-            for (_i878 = 0; _i878 < _size874; ++_i878)
+            uint32_t _size884;
+            ::apache::thrift::protocol::TType _etype887;
+            xfer += iprot->readListBegin(_etype887, _size884);
+            this->primaryKeys.resize(_size884);
+            uint32_t _i888;
+            for (_i888 = 0; _i888 < _size884; ++_i888)
             {
-              xfer += this->primaryKeys[_i878].read(iprot);
+              xfer += this->primaryKeys[_i888].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 _size879;
-            ::apache::thrift::protocol::TType _etype882;
-            xfer += iprot->readListBegin(_etype882, _size879);
-            this->foreignKeys.resize(_size879);
-            uint32_t _i883;
-            for (_i883 = 0; _i883 < _size879; ++_i883)
+            uint32_t _size889;
+            ::apache::thrift::protocol::TType _etype892;
+            xfer += iprot->readListBegin(_etype892, _size889);
+            this->foreignKeys.resize(_size889);
+            uint32_t _i893;
+            for (_i893 = 0; _i893 < _size889; ++_i893)
             {
-              xfer += this->foreignKeys[_i883].read(iprot);
+              xfer += this->foreignKeys[_i893].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 _iter884;
-    for (_iter884 = this->primaryKeys.begin(); _iter884 != this->primaryKeys.end(); ++_iter884)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter894;
+    for (_iter894 = this->primaryKeys.begin(); _iter894 != this->primaryKeys.end(); ++_iter894)
     {
-      xfer += (*_iter884).write(oprot);
+      xfer += (*_iter894).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 _iter885;
-    for (_iter885 = this->foreignKeys.begin(); _iter885 != this->foreignKeys.end(); ++_iter885)
+    std::vector<SQLForeignKey> ::const_iterator _iter895;
+    for (_iter895 = this->foreignKeys.begin(); _iter895 != this->foreignKeys.end(); ++_iter895)
     {
-      xfer += (*_iter885).write(oprot);
+      xfer += (*_iter895).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 _iter886;
-    for (_iter886 = (*(this->primaryKeys)).begin(); _iter886 != (*(this->primaryKeys)).end(); ++_iter886)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter896;
+    for (_iter896 = (*(this->primaryKeys)).begin(); _iter896 != (*(this->primaryKeys)).end(); ++_iter896)
     {
-      xfer += (*_iter886).write(oprot);
+      xfer += (*_iter896).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 _iter887;
-    for (_iter887 = (*(this->foreignKeys)).begin(); _iter887 != (*(this->foreignKeys)).end(); ++_iter887)
+    std::vector<SQLForeignKey> ::const_iterator _iter897;
+    for (_iter897 = (*(this->foreignKeys)).begin(); _iter897 != (*(this->foreignKeys)).end(); ++_iter897)
     {
-      xfer += (*_iter887).write(oprot);
+      xfer += (*_iter897).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -6055,14 +6055,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size888;
-            ::apache::thrift::protocol::TType _etype891;
-            xfer += iprot->readListBegin(_etype891, _size888);
-            this->success.resize(_size888);
-            uint32_t _i892;
-            for (_i892 = 0; _i892 < _size888; ++_i892)
+            uint32_t _size898;
+            ::apache::thrift::protocol::TType _etype901;
+            xfer += iprot->readListBegin(_etype901, _size898);
+            this->success.resize(_size898);
+            uint32_t _i902;
+            for (_i902 = 0; _i902 < _size898; ++_i902)
             {
-              xfer += iprot->readString(this->success[_i892]);
+              xfer += iprot->readString(this->success[_i902]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6101,10 +6101,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 _iter893;
-      for (_iter893 = this->success.begin(); _iter893 != this->success.end(); ++_iter893)
+      std::vector<std::string> ::const_iterator _iter903;
+      for (_iter903 = this->success.begin(); _iter903 != this->success.end(); ++_iter903)
       {
-        xfer += oprot->writeString((*_iter893));
+        xfer += oprot->writeString((*_iter903));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6149,14 +6149,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size894;
-            ::apache::thrift::protocol::TType _etype897;
-            xfer += iprot->readListBegin(_etype897, _size894);
-            (*(this->success)).resize(_size894);
-            uint32_t _i898;
-            for (_i898 = 0; _i898 < _size894; ++_i898)
+            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))[_i898]);
+              xfer += iprot->readString((*(this->success))[_i908]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6231,14 +6231,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 _size899;
-            ::apache::thrift::protocol::TType _etype902;
-            xfer += iprot->readListBegin(_etype902, _size899);
-            this->tbl_types.resize(_size899);
-            uint32_t _i903;
-            for (_i903 = 0; _i903 < _size899; ++_i903)
+            uint32_t _size909;
+            ::apache::thrift::protocol::TType _etype912;
+            xfer += iprot->readListBegin(_etype912, _size909);
+            this->tbl_types.resize(_size909);
+            uint32_t _i913;
+            for (_i913 = 0; _i913 < _size909; ++_i913)
             {
-              xfer += iprot->readString(this->tbl_types[_i903]);
+              xfer += iprot->readString(this->tbl_types[_i913]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6275,10 +6275,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 _iter904;
-    for (_iter904 = this->tbl_types.begin(); _iter904 != this->tbl_types.end(); ++_iter904)
+    std::vector<std::string> ::const_iterator _iter914;
+    for (_iter914 = this->tbl_types.begin(); _iter914 != this->tbl_types.end(); ++_iter914)
     {
-      xfer += oprot->writeString((*_iter904));
+      xfer += oprot->writeString((*_iter914));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6310,10 +6310,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 _iter905;
-    for (_iter905 = (*(this->tbl_types)).begin(); _iter905 != (*(this->tbl_types)).end(); ++_iter905)
+    std::vector<std::string> ::const_iterator _iter915;
+    for (_iter915 = (*(this->tbl_types)).begin(); _iter915 != (*(this->tbl_types)).end(); ++_iter915)
     {
-      xfer += oprot->writeString((*_iter905));
+      xfer += oprot->writeString((*_iter915));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6354,14 +6354,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size906;
-            ::apache::thrift::protocol::TType _etype909;
-            xfer += iprot->readListBegin(_etype909, _size906);
-            this->success.resize(_size906);
-            uint32_t _i910;
-            for (_i910 = 0; _i910 < _size906; ++_i910)
+            uint32_t _size916;
+            ::apache::thrift::protocol::TType _etype919;
+            xfer += iprot->readListBegin(_etype919, _size916);
+            this->success.resize(_size916);
+            uint32_t _i920;
+            for (_i920 = 0; _i920 < _size916; ++_i920)
             {
-              xfer += this->success[_i910].read(iprot);
+              xfer += this->success[_i920].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6400,10 +6400,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 _iter911;
-      for (_iter911 = this->success.begin(); _iter911 != this->success.end(); ++_iter911)
+      std::vector<TableMeta> ::const_iterator _iter921;
+      for (_iter921 = this->success.begin(); _iter921 != this->success.end(); ++_iter921)
       {
-        xfer += (*_iter911).write(oprot);
+        xfer += (*_iter921).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6448,14 +6448,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size912;
-            ::apache::thrift::protocol::TType _etype915;
-            xfer += iprot->readListBegin(_etype915, _size912);
-            (*(this->success)).resize(_size912);
-            uint32_t _i916;
-            for (_i916 = 0; _i916 < _size912; ++_i916)
+            uint32_t _size922;
+            ::apache::thrift::protocol::TType _etype925;
+            xfer += iprot->readListBegin(_etype925, _size922);
+            (*(this->success)).resize(_size922);
+            uint32_t _i926;
+            for (_i926 = 0; _i926 < _size922; ++_i926)
             {
-              xfer += (*(this->success))[_i916].read(iprot);
+              xfer += (*(this->success))[_i926].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6593,14 +6593,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size917;
-            ::apache::thrift::protocol::TType _etype920;
-            xfer += iprot->readListBegin(_etype920, _size917);
-            this->success.resize(_size917);
-            uint32_t _i921;
-            for (_i921 = 0; _i921 < _size917; ++_i921)
+            uint32_t _size927;
+            ::apache::thrift::protocol::TType _etype930;
+            xfer += iprot->readListBegin(_etype930, _size927);
+            this->success.resize(_size927);
+            uint32_t _i931;
+            for (_i931 = 0; _i931 < _size927; ++_i931)
             {
-              xfer += iprot->readString(this->success[_i921]);
+              xfer += iprot->readString(this->success[_i931]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6639,10 +6639,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 _iter922;
-      for (_iter922 = this->success.begin(); _iter922 != this->success.end(); ++_iter922)
+      std::vector<std::string> ::const_iterator _iter932;
+      for (_iter932 = this->success.begin(); _iter932 != this->success.end(); ++_iter932)
       {
-        xfer += oprot->writeString((*_iter922));
+        xfer += oprot->writeString((*_iter932));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6687,14 +6687,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size923;
-            ::apache::thrift::protocol::TType _etype926;
-            xfer += iprot->readListBegin(_etype926, _size923);
-            (*(this->success)).resize(_size923);
-            uint32_t _i927;
-            for (_i927 = 0; _i927 < _size923; ++_i927)
+            uint32_t _size933;
+            ::apache::thrift::protocol::TType _etype936;
+            xfer += iprot->readListBegin(_etype936, _size933);
+            (*(this->success)).resize(_size933);
+            uint32_t _i937;
+            for (_i937 = 0; _i937 < _size933; ++_i937)
             {
-              xfer += iprot->readString((*(this->success))[_i927]);
+              xfer += iprot->readString((*(this->success))[_i937]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7004,14 +7004,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 _size928;
-            ::apache::thrift::protocol::TType _etype931;
-            xfer += iprot->readListBegin(_etype931, _size928);
-            this->tbl_names.resize(_size928);
-            uint32_t _i932;
-            for (_i932 = 0; _i932 < _size928; ++_i932)
+            uint32_t _size938;
+            ::apache::thrift::protocol::TType _etype941;
+            xfer += iprot->readListBegin(_etype941, _size938);
+            this->tbl_names.resize(_size938);
+            uint32_t _i942;
+            for (_i942 = 0; _i942 < _size938; ++_i942)
             {
-              xfer += iprot->readString(this->tbl_names[_i932]);
+              xfer += iprot->readString(this->tbl_names[_i942]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7044,10 +7044,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 _iter933;
-    for (_iter933 = this->tbl_names.begin(); _iter933 != this->tbl_names.end(); ++_iter933)
+    std::vector<std::string> ::const_iterator _iter943;
+    for (_iter943 = this->tbl_names.begin(); _iter943 != this->tbl_names.end(); ++_iter943)
     {
-      xfer += oprot->writeString((*_iter933));
+      xfer += oprot->writeString((*_iter943));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7075,10 +7075,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 _iter934;
-    for (_iter934 = (*(this->tbl_names)).begin(); _iter934 != (*(this->tbl_names)).end(); ++_iter934)
+    std::vector<std::string> ::const_iterator _iter944;
+    for (_iter944 = (*(this->tbl_names)).begin(); _iter944 != (*(this->tbl_names)).end(); ++_iter944)
     {
-      xfer += oprot->writeString((*_iter934));
+      xfer += oprot->writeString((*_iter944));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7119,14 +7119,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 _size935;
-            ::apache::thrift::protocol::TType _etype938;
-            xfer += iprot->readListBegin(_etype938, _size935);
-            this->success.resize(_size935);
-            uint32_t _i939;
-            for (_i939 = 0; _i939 < _size935; ++_i939)
+            uint32_t _size945;
+            ::apache::thrift::protocol::TType _etype948;
+            xfer += iprot->readListBegin(_etype948, _size945);
+            this->success.resize(_size945);
+            uint32_t _i949;
+            for (_i949 = 0; _i949 < _size945; ++_i949)
             {
-              xfer += this->success[_i939].read(iprot);
+              xfer += this->success[_i949].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7181,10 +7181,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 _iter940;
-      for (_iter940 = this->success.begin(); _iter940 != this->success.end(); ++_iter940)
+      std::vector<Table> ::const_iterator _iter950;
+      for (_iter950 = this->success.begin(); _iter950 != this->success.end(); ++_iter950)
       {
-        xfer += (*_iter940).write(oprot);
+        xfer += (*_iter950).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7237,14 +7237,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 _size941;
-            ::apache::thrift::protocol::TType _etype944;
-            xfer += iprot->readListBegin(_etype944, _size941);
-            (*(this->success)).resize(_size941);
-            uint32_t _i945;
-            for (_i945 = 0; _i945 < _size941; ++_i945)
+            uint32_t _size951;
+            ::apache::thrift::protocol::TType _etype954;
+            xfer += iprot->readListBegin(_etype954, _size951);
+            (*(this->success)).resize(_size951);
+            uint32_t _i955;
+            for (_i955 = 0; _i955 < _size951; ++_i955)
             {
-              xfer += (*(this->success))[_i945].read(iprot);
+              xfer += (*(this->success))[_i955].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7430,14 +7430,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 _size946;
-            ::apache::thrift::protocol::TType _etype949;
-            xfer += iprot->readListBegin(_etype949, _size946);
-            this->success.resize(_size946);
-            uint32_t _i950;
-            for (_i950 = 0; _i950 < _size946; ++_i950)
+            uint32_t _size956;
+            ::apache::thrift::protocol::TType _etype959;
+            xfer += iprot->readListBegin(_etype959, _size956);
+            this->success.resize(_size956);
+            uint32_t _i960;
+            for (_i960 = 0; _i960 < _size956; ++_i960)
             {
-              xfer += iprot->readString(this->success[_i950]);
+              xfer += iprot->readString(this->success[_i960]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7492,10 +7492,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 _iter951;
-      for (_iter951 = this->success.begin(); _iter951 != this->success.end(); ++_iter951)
+      std::vector<std::string> ::const_iterator _iter961;
+      for (_iter961 = this->success.begin(); _iter961 != this->success.end(); ++_iter961)
       {
-        xfer += oprot->writeString((*_iter951));
+        xfer += oprot->writeString((*_iter961));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7548,14 +7548,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 _size952;
-            ::apache::thrift::protocol::TType _etype955;
-            xfer += iprot->readListBegin(_etype955, _size952);
-            (*(this->success)).resize(_size952);
-            uint32_t _i956;
-            for (_i956 = 0; _i956 < _size952; ++_i956)
+            uint32_t _size962;
+            ::apache::thrift::protocol::TType _etype965;
+            xfer += iprot->readListBegin(_etype965, _size962);
+            (*(this->success)).resize(_size962);
+            uint32_t _i966;
+            for (_i966 = 0; _i966 < _size962; ++_i966)
             {
-              xfer += iprot->readString((*(this->success))[_i956]);
+              xfer += iprot->readString((*(this->success))[_i966]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8889,14 +8889,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size957;
-            ::apache::thrift::protocol::TType _etype960;
-            xfer += iprot->readListBegin(_etype960, _size957);
-            this->new_parts.resize(_size957);
-            uint32_t _i961;
-            for (_i961 = 0; _i961 < _size957; ++_i961)
+            uint32_t _size967;
+            ::apache::thrift::protocol::TType _etype970;
+            xfer += iprot->readListBegin(_etype970, _size967);
+            this->new_parts.resize(_size967);
+            uint32_t _i971;
+            for (_i971 = 0; _i971 < _size967; ++_i971)
             {
-              xfer += this->new_parts[_i961].read(iprot);
+              xfer += this->new_parts[_i971].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8925,10 +8925,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 _iter962;
-    for (_iter962 = this->new_parts.begin(); _iter962 != this->new_parts.end(); ++_iter962)
+    std::vector<Partition> ::const_iterator _iter972;
+    for (_iter972 = this->new_parts.begin(); _iter972 != this->new_parts.end(); ++_iter972)
     {
-      xfer += (*_iter962).write(oprot);
+      xfer += (*_iter972).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8952,10 +8952,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 _iter963;
-    for (_iter963 = (*(this->new_parts)).begin(); _iter963 != (*(this->new_parts)).end(); ++_iter963)
+    std::vector<Partition> ::const_iterator _iter973;
+    for (_iter973 = (*(this->new_parts)).begin(); _iter973 != (*(this->new_parts)).end(); ++_iter973)
     {
-      xfer += (*_iter963).write(oprot);
+      xfer += (*_iter973).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9164,14 +9164,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 _size964;
-            ::apache::thrift::protocol::TType _etype967;
-            xfer += iprot->readListBegin(_etype967, _size964);
-            this->new_parts.resize(_size964);
-            uint32_t _i968;
-            for (_i968 = 0; _i968 < _size964; ++_i968)
+            uint32_t _size974;
+            ::apache::thrift::protocol::TType _etype977;
+            xfer += iprot->readListBegin(_etype977, _size974);
+            this->new_parts.resize(_size974);
+            uint32_t _i978;
+            for (_i978 = 0; _i978 < _size974; ++_i978)
             {
-              xfer += this->new_parts[_i968].read(iprot);
+              xfer += this->new_parts[_i978].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9200,10 +9200,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 _iter969;
-    for (_iter969 = this->new_parts.begin(); _iter969 != this->new_parts.end(); ++_iter969)
+    std::vector<PartitionSpec> ::const_iterator _iter979;
+    for (_iter979 = this->new_parts.begin(); _iter979 != this->new_parts.end(); ++_iter979)
     {
-      xfer += (*_iter969).write(oprot);
+      xfer += (*_iter979).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9227,10 +9227,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 _iter970;
-    for (_iter970 = (*(this->new_parts)).begin(); _iter970 != (*(this->new_parts)).end(); ++_iter970)
+    std::vector<PartitionSpec> ::const_iterator _iter980;
+    for (_iter980 = (*(this->new_parts)).begin(); _iter980 != (*(this->new_parts)).end(); ++_iter980)
     {
-      xfer += (*_iter970).write(oprot);
+      xfer += (*_iter980).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9455,14 +9455,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size971;
-            ::apache::thrift::protocol::TType _etype974;
-            xfer += iprot->readListBegin(_etype974, _size971);
-            this->part_vals.resize(_size971);
-            uint32_t _i975;
-            for (_i975 = 0; _i975 < _size971; ++_i975)
+            uint32_t _size981;
+            ::apache::thrift::protocol::TType _etype984;
+            xfer += iprot->readListBegin(_etype984, _size981);
+            this->part_vals.resize(_size981);
+            uint32_t _i985;
+            for (_i985 = 0; _i985 < _size981; ++_i985)
             {
-              xfer += iprot->readString(this->part_vals[_i975]);
+              xfer += iprot->readString(this->part_vals[_i985]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9499,10 +9499,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 _iter976;
-    for (_iter976 = this->part_vals.begin(); _iter976 != this->part_vals.end(); ++_iter976)
+    std::vector<std::string> ::const_iterator _iter986;
+    for (_iter986 = this->part_vals.begin(); _iter986 != this->part_vals.end(); ++_iter986)
     {
-      xfer += oprot->writeString((*_iter976));
+      xfer += oprot->writeString((*_iter986));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9534,10 +9534,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 _iter977;
-    for (_iter977 = (*(this->part_vals)).begin(); _iter977 != (*(this->part_vals)).end(); ++_iter977)
+    std::vector<std::string> ::const_iterator _iter987;
+    for (_iter987 = (*(this->part_vals)).begin(); _iter987 != (*(this->part_vals)).end(); ++_iter987)
     {
-      xfer += oprot->writeString((*_iter977));
+      xfer += oprot->writeString((*_iter987));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10009,14 +10009,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size978;
-            ::apache::thrift::protocol::TType _etype981;
-            xfer += iprot->readListBegin(_etype981, _size978);
-            this->part_vals.resize(_size978);
-            uint32_t _i982;
-            for (_i982 = 0; _i982 < _size978; ++_i982)
+            uint32_t _size988;
+            ::apache::thrift::protocol::TType _etype991;
+            xfer += iprot->readListBegin(_etype991, _size988);
+            this->part_vals.resize(_size988);
+            uint32_t _i992;
+            for (_i992 = 0; _i992 < _size988; ++_i992)
             {
-              xfer += iprot->readString(this->part_vals[_i982]);
+              xfer += iprot->readString(this->part_vals[_i992]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10061,10 +10061,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 _iter983;
-    for (_iter983 = this->part_vals.begin(); _iter983 != this->part_vals.end(); ++_iter983)
+    std::vector<std::string> ::const_iterator _iter993;
+    for (_iter993 = this->part_vals.begin(); _iter993 != this->part_vals.end(); ++_iter993)
     {
-      xfer += oprot->writeString((*_iter983));
+      xfer += oprot->writeString((*_iter993));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10100,10 +10100,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 _iter984;
-    for (_iter984 = (*(this->part_vals)).begin(); _iter984 != (*(this->part_vals)).end(); ++_iter984)
+    std::vector<std::string> ::const_iterator _iter994;
+    for (_iter994 = (*(this->part_vals)).begin(); _iter994 != (*(this->part_vals)).end(); ++_iter994)
     {
-      xfer += oprot->writeString((*_iter984));
+      xfer += oprot->writeString((*_iter994));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10906,14 +10906,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size985;
-            ::apache::thrift::protocol::TType _etype988;
-            xfer += iprot->readListBegin(_etype988, _size985);
-            this->part_vals.resize(_size985);
-            uint32_t _i989;
-            for (_i989 = 0; _i989 < _size985; ++_i989)
+            uint32_t _size995;
+            ::apache::thrift::protocol::TType _etype998;
+            xfer += iprot->readListBegin(_etype998, _size995);
+            this->part_vals.resize(_size995);
+            uint32_t _i999;
+            for (_i999 = 0; _i999 < _size995; ++_i999)
             {
-              xfer += iprot->readString(this->part_vals[_i989]);
+              xfer += iprot->readString(this->part_vals[_i999]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10958,10 +10958,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 _iter990;
-    for (_iter990 = this->part_vals.begin(); _iter990 != this->part_vals.end(); ++_iter990)
+    std::vector<std::string> ::const_iterator _iter1000;
+    for (_iter1000 = this->part_vals.begin(); _iter1000 != this->part_vals.end(); ++_iter1000)
     {
-      xfer += oprot->writeString((*_iter990));
+      xfer += oprot->writeString((*_iter1000));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10997,10 +10997,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 _iter991;
-    for (_iter991 = (*(this->part_vals)).begin(); _iter991 != (*(this->part_vals)).end(); ++_iter991)
+    std::vector<std::string> ::const_iterator _iter1001;
+    for (_iter1001 = (*(this->part_vals)).begin(); _iter1001 != (*(this->part_vals)).end(); ++_iter1001)
     {
-      xfer += oprot->writeString((*_iter991));
+      xfer += oprot->writeString((*_iter1001));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11209,14 +11209,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size992;
-            ::apache::thrift::protocol::TType _etype995;
-            xfer += iprot->readListBegin(_etype995, _size992);
-            this->part_vals.resize(_size992);
-            uint32_t _i996;
-            for (_i996 = 0; _i996 < _size992; ++_i996)
+            uint32_t _size1002;
+            ::apache::thrift::protocol::TType _etype1005;
+            xfer += iprot->readListBegin(_etype1005, _size1002);
+            this->part_vals.resize(_size1002);
+            uint32_t _i1006;
+            for (_i1006 = 0; _i1006 < _size1002; ++_i1006)
             {
-              xfer += iprot->readString(this->part_vals[_i996]);
+              xfer += iprot->readString(this->part_vals[_i1006]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11269,10 +11269,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 _iter997;
-    for (_iter997 = this->part_vals.begin(); _iter997 != this->part_vals.end(); ++_iter997)
+    std::vector<std::string> ::const_iterator _iter1007;
+    for (_iter1007 = this->part_vals.begin(); _iter1007 != this->part_vals.end(); ++_iter1007)
     {
-      xfer += oprot->writeString((*_iter997));
+      xfer += oprot->writeString((*_iter1007));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11312,10 +11312,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 _iter998;
-    for (_iter998 = (*(this->part_vals)).begin(); _iter998 != (*(this->part_vals)).end(); ++_iter998)
+    std::vector<std::string> ::const_iterator _iter1008;
+    for (_iter1008 = (*(this->part_vals)).begin(); _iter1008 != (*(this->part_vals)).end(); ++_iter1008)
     {
-      xfer += oprot->writeString((*_iter998));
+      xfer += oprot->writeString((*_iter1008));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12321,14 +12321,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size999;
-            ::apache::thrift::protocol::TType _etype1002;
-            xfer += iprot->readListBegin(_etype1002, _size999);
-            this->part_vals.resize(_size999);
-            uint32_t _i1003;
-            for (_i1003 = 0; _i1003 < _size999; ++_i1003)
+            uint32_t _size1009;
+            ::apache::thrift::protocol::TType _etype1012;
+            xfer += iprot->readListBegin(_etype1012, _size1009);
+            this->part_vals.resize(_size1009);
+            uint32_t _i1013;
+            for (_i1013 = 0; _i1013 < _size1009; ++_i1013)
             {
-              xfer += iprot->readString(this->part_vals[_i1003]);
+              xfer += iprot->readString(this->part_vals[_i1013]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12365,10 +12365,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 _iter1004;
-    for (_iter1004 = this->part_vals.begin(); _iter1004 != this->part_vals.end(); ++_iter1004)
+    std::vector<std::string> ::const_iterator _iter1014;
+    for (_iter1014 = this->part_vals.begin(); _iter1014 != this->part_vals.end(); ++_iter1014)
     {
-      xfer += oprot->writeString((*_iter1004));
+      xfer += oprot->writeString((*_iter1014));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12400,10 +12400,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 _iter1005;
-    for (_iter1005 = (*(this->part_vals)).begin(); _iter1005 != (*(this->part_vals)).end(); ++_iter1005)
+    std::vector<std::string> ::const_iterator _iter1015;
+    for (_iter1015 = (*(this->part_vals)).begin(); _iter1015 != (*(this->part_vals)).end(); ++_iter1015)
     {
-      xfer += oprot->writeString((*_iter1005));
+      xfer += oprot->writeString((*_iter1015));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12592,17 +12592,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1006;
-            ::apache::thrift::protocol::TType _ktype1007;
-            ::apache::thrift::protocol::TType _vtype1008;
-            xfer += iprot->readMapBegin(_ktype1007, _vtype1008, _size1006);
-            uint32_t _i1010;
-            for (_i1010 = 0; _i1010 < _size1006; ++_i1010)
+            uint32_t _size1016;
+            ::apache::thrift::protocol::TType _ktype1017;
+            ::apache::thrift::protocol::TType _vtype1018;
+            xfer += iprot->readMapBegin(_ktype1017, _vtype1018, _size1016);
+            uint32_t _i1020;
+            for (_i1020 = 0; _i1020 < _size1016; ++_i1020)
             {
-              std::string _key1011;
-              xfer += iprot->readString(_key1011);
-              std::string& _val1012 = this->partitionSpecs[_key1011];
-              xfer += iprot->readString(_val1012);
+              std::string _key1021;
+              xfer += iprot->readString(_key1021);
+              std::string& _val1022 = this->partitionSpecs[_key1021];
+              xfer += iprot->readString(_val1022);
             }
             xfer += iprot->readMapEnd();
           }
@@ -12663,11 +12663,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 _iter1013;
-    for (_iter1013 = this->partitionSpecs.begin(); _iter1013 != this->partitionSpecs.end(); ++_iter1013)
+    std::map<std::string, std::string> ::const_iterator _iter1023;
+    for (_iter1023 = this->partitionSpecs.begin(); _iter1023 != this->partitionSpecs.end(); ++_iter1023)
     {
-      xfer += oprot->writeString(_iter1013->first);
-      xfer += oprot->writeString(_iter1013->second);
+      xfer += oprot->writeString(_iter1023->first);
+      xfer += oprot->writeString(_iter1023->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12707,11 +12707,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 _iter1014;
-    for (_iter1014 = (*(this->partitionSpecs)).begin(); _iter1014 != (*(this->partitionSpecs)).end(); ++_iter1014)
+    std::map<std::string, std::string> ::const_iterator _iter1024;
+    for (_iter1024 = (*(this->partitionSpecs)).begin(); _iter1024 != (*(this->partitionSpecs)).end(); ++_iter1024)
     {
-      xfer += oprot->writeString(_iter1014->first);
-      xfer += oprot->writeString(_iter1014->second);
+      xfer += oprot->writeString(_iter1024->first);
+      xfer += oprot->writeString(_iter1024->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12956,17 +12956,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1015;
-            ::apache::thrift::protocol::TType _ktype1016;
-            ::apache::thrift::protocol::TType _vtype1017;
-            xfer += iprot->readMapBegin(_ktype1016, _vtype1017, _size1015);
-            uint32_t _i1019;
-            for (_i1019 = 0; _i1019 < _size1015; ++_i1019)
+            uint32_t _size1025;
+            ::apache::thrift::protocol::TType _ktype1026;
+            ::apache::thrift::protocol::TType _vtype1027;
+            xfer += iprot->readMapBegin(_ktype1026, _vtype1027, _size1025);
+            uint32_t _i1029;
+            for (_i1029 = 0; _i1029 < _size1025; ++_i1029)
             {
-              std::string _key1020;
-              xfer += iprot->readString(_key1020);
-              std::string& _val1021 = this->partitionSpecs[_key1020];
-              xfer += iprot->readString(_val1021);
+              std::string _key1030;
+              xfer += iprot->readString(_key1030);
+              std::string& _val1031 = this->partitionSpecs[_key1030];
+              xfer += iprot->readString(_val1031);
             }
             xfer += iprot->readMapEnd();
           }
@@ -13027,11 +13027,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 _iter1022;
-    for (_iter1022 = this->partitionSpecs.begin(); _iter1022 != this->partitionSpecs.end(); ++_iter1022)
+    std::map<std::string, std::string> ::const_iterator _iter1032;
+    for (_iter1032 = this->partitionSpecs.begin(); _iter1032 != this->partitionSpecs.end(); ++_iter1032)
     {
-      xfer += oprot->writeString(_iter1022->first);
-      xfer += oprot->writeString(_iter1022->second);
+      xfer += oprot->writeString(_iter1032->first);
+      xfer += oprot->writeString(_iter1032->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13071,11 +13071,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 _iter1023;
-    for (_iter1023 = (*(this->partitionSpecs)).begin(); _iter1023 != (*(this->partitionSpecs)).end(); ++_iter1023)
+    std::map<std::string, std::string> ::const_iterator _iter1033;
+    for (_iter1033 = (*(this->partitionSpecs)).begin(); _iter1033 != (*(this->partitionSpecs)).end(); ++_iter1033)
     {
-      xfer += oprot->writeString(_iter1023->first);
-      xfer += oprot->writeString(_iter1023->second);
+      xfer += oprot->writeString(_iter1033->first);
+      xfer += oprot->writeString(_iter1033->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13132,14 +13132,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1024;
-            ::apache::thrift::protocol::TType _etype1027;
-            xfer += iprot->readListBegin(_etype1027, _size1024);
-            this->success.resize(_size1024);
-            uint32_t _i1028;
-            for (_i1028 = 0; _i1028 < _size1024; ++_i1028)
+            uint32_t _size1034;
+            ::apache::thrift::protocol::TType _etype1037;
+            xfer += iprot->readListBegin(_etype1037, _size1034);
+            this->success.resize(_size1034);
+            uint32_t _i1038;
+            for (_i1038 = 0; _i1038 < _size1034; ++_i1038)
             {
-              xfer += this->success[_i1028].read(iprot);
+              xfer += this->success[_i1038].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13202,10 +13202,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 _iter1029;
-      for (_iter1029 = this->success.begin(); _iter1029 != this->success.end(); ++_iter1029)
+      std::vector<Partition> ::const_iterator _iter1039;
+      for (_iter1039 = this->success.begin(); _iter1039 != this->success.end(); ++_iter1039)
       {
-        xfer += (*_iter1029).write(oprot);
+        xfer += (*_iter1039).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13262,14 +13262,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1030;
-            ::apache::thrift::protocol::TType _etype1033;
-            xfer += iprot->readListBegin(_etype1033, _size1030);
-            (*(this->success)).resize(_size1030);
-            uint32_t _i1034;
-            for (_i1034 = 0; _i1034 < _size1030; ++_i1034)
+            uint32_t _size1040;
+            ::apache::thrift::protocol::TType _etype1043;
+            xfer += iprot->readListBegin(_etype1043, _size1040);
+            (*(this->success)).resize(_size1040);
+            uint32_t _i1044;
+            for (_i1044 = 0; _i1044 < _size1040; ++_i1044)
             {
-              xfer += (*(this->success))[_i1034].read(iprot);
+              xfer += (*(this->success))[_i1044].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13368,14 +13368,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 _size1035;
-            ::apache::thrift::protocol::TType _etype1038;
-            xfer += iprot->readListBegin(_etype1038, _size1035);
-            this->part_vals.resize(_size1035);
-            uint32_t _i1039;
-            for (_i1039 = 0; _i1039 < _size1035; ++_i1039)
+            uint32_t _size1045;
+            ::apache::thrift::protocol::TType _etype1048;
+            xfer += iprot->readListBegin(_etype1048, _size1045);
+            this->part_vals.resize(_size1045);
+            uint32_t _i1049;
+            for (_i1049 = 0; _i1049 < _size1045; ++_i1049)
             {
-              xfer += iprot->readString(this->part_vals[_i1039]);
+              xfer += iprot->readString(this->part_vals[_i1049]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13396,14 +13396,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 _size1040;
-            ::apache::thrift::protocol::TType _etype1043;
-            xfer += iprot->readListBegin(_etype1043, _size1040);
-            this->group_names.resize(_size1040);
-            uint32_t _i1044;
-            for (_i1044 = 0; _i1044 < _size1040; ++_i1044)
+            uint32_t _size1050;
+            ::apache::thrift::protocol::TType _etype1053;
+            xfer += iprot->readListBegin(_etype1053, _size1050);
+            this->group_names.resize(_size1050);
+            uint32_t _i1054;
+            for (_i1054 = 0; _i1054 < _size1050; ++_i1054)
             {
-              xfer += iprot->readString(this->group_names[_i1044]);
+              xfer += iprot->readString(this->group_names[_i1054]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13440,10 +13440,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 _iter1045;
-    for (_iter1045 = this->part_vals.begin(); _iter1045 != this->part_vals.end(); ++_iter1045)
+    std::vector<std::string> ::const_iterator _iter1055;
+    for (_iter1055 = this->part_vals.begin(); _iter1055 != this->part_vals.end(); ++_iter1055)
     {
-      xfer += oprot->writeString((*_iter1045));
+      xfer += oprot->writeString((*_iter1055));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13456,10 +13456,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 _iter1046;
-    for (_iter1046 = this->group_names.begin(); _iter1046 != this->group_names.end(); ++_iter1046)
+    std::vector<std::string> ::const_iterator _iter1056;
+    for (_iter1056 = this->group_names.begin(); _iter1056 != this->group_names.end(); ++_iter1056)
     {
-      xfer += oprot->writeString((*_iter1046));
+      xfer += oprot->writeString((*_iter1056));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13491,10 +13491,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 _iter1047;
-    for (_iter1047 = (*(this->part_vals)).begin(); _iter1047 != (*(this->part_vals)).end(); ++_iter1047)
+    std::vector<std::string> ::const_iterator _iter1057;
+    for (_iter1057 = (*(this->part_vals)).begin(); _iter1057 != (*(this->part_vals)).end(); ++_iter1057)
     {
-      xfer += oprot->writeString((*_iter1047));
+      xfer += oprot->writeString((*_iter1057));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13507,10 +13507,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 _iter1048;
-    for (_iter1048 = (*(this->group_names)).begin(); _iter1048 != (*(this->group_names)).end(); ++_iter1048)
+    std::vector<std::string> ::const_iterator _iter1058;
+    for (_iter1058 = (*(this->group_names)).begin(); _iter1058 != (*(this->group_names)).end(); ++_iter1058)
     {
-      xfer += oprot->writeString((*_iter1048));
+      xfer += oprot->writeString((*_iter1058));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14069,14 +14069,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1049;
-            ::apache::thrift::protocol::TType _etype1052;
-            xfer += iprot->readListBegin(_etype1052, _size1049);
-            this->success.resize(_size1049);
-            uint32_t _i1053;
-            for (_i1053 = 0; _i1053 < _size1049; ++_i1053)
+            uint32_t _size1059;
+            ::apache::thrift::protocol::TType _etype1062;
+            xfer += iprot->readListBegin(_etype1062, _size1059);
+            this->success.resize(_size1059);
+            uint32_t _i1063;
+            for (_i1063 = 0; _i1063 < _size1059; ++_i1063)
             {
-              xfer += this->success[_i1053].read(iprot);
+              xfer += this->success[_i1063].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14123,10 +14123,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 _iter1054;
-      for (_iter1054 = this->success.begin(); _iter1054 != this->success.end(); ++_iter1054)
+      std::vector<Partition> ::const_iterator _iter1064;
+      for (_iter1064 = this->success.begin(); _iter1064 != this->success.end(); ++_iter1064)
       {
-        xfer += (*_iter1054).write(oprot);
+        xfer += (*_iter1064).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14175,14 +14175,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1055;
-            ::apache::thrift::protocol::TType _etype1058;
-            xfer += iprot->readListBegin(_etype1058, _size1055);
-            (*(this->success)).resize(_size1055);
-            uint32_t _i1059;
-            for (_i1059 = 0; _i1059 < _size1055; ++_i1059)
+            uint32_t _size1065;
+            ::apache::thrift::protocol::TType _etype1068;
+            xfer += iprot->readListBegin(_etype1068, _size1065);
+            (*(this->success)).resize(_size1065);
+            uint32_t _i1069;
+            for (_i1069 = 0; _i1069 < _size1065; ++_i1069)
             {
-              xfer += (*(this->success))[_i1059].read(iprot);
+              xfer += (*(this->success))[_i1069].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14281,14 +14281,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 _size1060;
-            ::apache::thrift::protocol::TType _etype1063;
-            xfer += iprot->readListBegin(_etype1063, _size1060);
-            this->group_names.resize(_size1060);
-            uint32_t _i1064;
-            for (_i1064 = 0; _i1064 < _size1060; ++_i1064)
+            uint32_t _size1070;
+            ::apache::thrift::protocol::TType _etype1073;
+            xfer += iprot->readListBegin(_etype1073, _size1070);
+            this->group_names.resize(_size1070);
+            uint32_t _i1074;
+            for (_i1074 = 0; _i1074 < _size1070; ++_i1074)
             {
-              xfer += iprot->readString(this->group_names[_i1064]);
+              xfer += iprot->readString(this->group_names[_i1074]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14333,10 +14333,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 _iter1065;
-    for (_iter1065 = this->group_names.begin(); _iter1065 != this->group_names.end(); ++_iter1065)
+    std::vector<std::string> ::const_iterator _iter1075;
+    for (_iter1075 = this->group_names.begin(); _iter1075 != this->group_names.end(); ++_iter1075)
     {
-      xfer += oprot->writeString((*_iter1065));
+      xfer += oprot->writeString((*_iter1075));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14376,10 +14376,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 _iter1066;
-    for (_iter1066 = (*(this->group_names)).begin(); _iter1066 != (*(this->group_names)).end(); ++_iter1066)
+    std::vector<std::string> ::const_iterator _iter1076;
+    for (_iter1076 = (*(this->group_names)).begin(); _iter1076 != (*(this->group_names)).end(); ++_iter1076)
     {
-      xfer += oprot->writeString((*_iter1066));
+      xfer += oprot->writeString((*_iter1076));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14420,14 +14420,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1067;
-            ::apache::thrift::protocol::TType _etype1070;
-            xfer += iprot->readListBegin(_etype1070, _size1067);
-            this->success.resize(_size1067);
-            uint32_t _i1071;
-            for (_i1071 = 0; _i1071 < _size1067; ++_i1071)
+            uint32_t _size1077;
+            ::apache::thrift::protocol::TType _etype1080;
+            xfer += iprot->readListBegin(_etype1080, _size1077);
+            this->success.resize(_size1077);
+            uint32_t _i1081;
+            for (_i1081 = 0; _i1081 < _size1077; ++_i1081)
             {
-              xfer += this->success[_i1071].read(iprot);
+              xfer += this->success[_i1081].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14474,10 +14474,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     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 _iter1072;
-      for (_iter1072 = this->success.begin(); _iter1072 != this->success.end(); ++_iter1072)
+      std::vector<Partition> ::const_iterator _iter1082;
+      for (_iter1082 = this->success.begin(); _iter1082 != this->success.end(); ++_iter1082)
       {
-        xfer += (*_iter1072).write(oprot);
+        xfer += (*_iter1082).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14526,14 +14526,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1073;
-            ::apache::thrift::protocol::TType _etype1076;
-            xfer += iprot->readListBegin(_etype1076, _size1073);
-            (*(this->success)).resize(_size1073);
-            uint32_t _i1077;
-            for (_i1077 = 0; _i1077 < _size1073; ++_i1077)
+            uint32_t _size1083;
+            ::apache::thrift::protocol::TType _etype1086;
+            xfer += iprot->readListBegin(_etype1086, _size1083);
+            (*(this->success)).resize(_size1083);
+            uint32_t _i1087;
+            for (_i1087 = 0; _i1087 < _size1083; ++_i1087)
             {
-              xfer += (*(this->success))[_i1077].read(iprot);
+              xfer += (*(this->success))[_i1087].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14711,14 +14711,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1078;
-            ::apache::thrift::protocol::TType _etype1081;
-            xfer += iprot->readListBegin(_etype1081, _size1078);
-            this->success.resize(_size1078);
-            uint32_t _i1082;
-            for (_i1082 = 0; _i1082 < _size1078; ++_i1082)
+            uint32_t _size1088;
+            ::apache::thrift::protocol::TType _etype1091;
+            xfer += iprot->readListBegin(_etype1091, _size1088);
+            this->success.resize(_size1088);
+            uint32_t _i1092;
+            for (_i1092 = 0; _i1092 < _size1088; ++_i1092)
             {
-              xfer += this->success[_i1082].read(iprot);
+              xfer += this->success[_i1092].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14765,10 +14765,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_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<PartitionSpec> ::const_iterator _iter1083;
-      for (_iter1083 = this->success.begin(); _iter1083 != this->success.end(); ++_iter1083)
+      std::vector<PartitionSpec> ::const_iterator _iter1093;
+      for (_iter1093 = this->success.begin(); _iter1093 != this->success.end(); ++_iter1093)
       {
-        xfer += (*_iter1083).write(oprot);
+        xfer += (*_iter1093).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14817,14 +14817,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1084;
-            ::apache::thrift::protocol::TType _etype1087;
-            xfer += iprot->readListBegin(_etype1087, _size1084);
-            (*(this->success)).resize(_size1084);
-            uint32_t _i1088;
-            for (_i1088 = 0; _i1088 < _size1084; ++_i1088)
+            uint32_t _size1094;
+            ::apache::thrift::protocol::TType _etype1097;
+            xfer += iprot->readListBegin(_etype1097, _size1094);
+            (*(this->success)).resize(_size1094);
+            uint32_t _i1098;
+            for (_i1098 = 0; _i1098 < _size1094; ++_i1098)
             {
-              xfer += (*(this->success))[_i1088].read(iprot);
+              xfer += (*(this->success))[_i1098].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15002,14 +15002,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1089;
-            ::apache::thrift::protocol::TType _etype1092;
-            xfer += iprot->readListBegin(_etype1092, _size1089);
-            this->success.resize(_size1089);
-            uint32_t _i1093;
-            for (_i1093 = 0; _i1093 < _size1089; ++_i1093)
+            uint32_t _size1099;
+            ::apache::thrift::protocol::TType _etype1102;
+            xfer += iprot->readListBegin(_etype1102, _size1099);
+            this->success.resize(_size1099);
+            uint32_t _i1103;
+            for (_i1103 = 0; _i1103 < _size1099; ++_i1103)
       

<TRUNCATED>

[17/31] hive git commit: HIVE-14728: Redundant orig files (Rui reviewed by Pengcheng)

Posted by se...@apache.org.
HIVE-14728: Redundant orig files (Rui reviewed by Pengcheng)


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

Branch: refs/heads/hive-14535
Commit: bc75e46adabd3e8b4738da985291900631b8509d
Parents: 407cfe1
Author: Rui Li <sh...@cn.ibm.com>
Authored: Sat Sep 10 14:33:15 2016 +0800
Committer: Rui Li <sh...@cn.ibm.com>
Committed: Sat Sep 10 14:33:15 2016 +0800

----------------------------------------------------------------------
 .../resources/testconfiguration.properties.orig |  1377 --
 .../org/apache/hadoop/hive/ql/Context.java.orig |   829 --
 .../hive/ql/parse/SemanticAnalyzer.java.orig    | 13038 -----------------
 3 files changed, 15244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bc75e46a/itests/src/test/resources/testconfiguration.properties.orig
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties.orig b/itests/src/test/resources/testconfiguration.properties.orig
deleted file mode 100644
index a920ca9..0000000
--- a/itests/src/test/resources/testconfiguration.properties.orig
+++ /dev/null
@@ -1,1377 +0,0 @@
-# NOTE: files should be listed in alphabetical order
-minimr.query.files=auto_sortmerge_join_16.q,\
-  bucket4.q,\
-  bucket5.q,\
-  bucket6.q,\
-  bucket_many.q,\
-  bucket_num_reducers.q,\
-  bucket_num_reducers2.q,\
-  bucketizedhiveinputformat.q,\
-  bucketmapjoin6.q,\
-  bucketmapjoin7.q,\
-  disable_merge_for_bucketing.q,\
-  empty_dir_in_table.q,\
-  exchgpartition2lel.q,\
-  external_table_with_space_in_location_path.q,\
-  file_with_header_footer.q,\
-  groupby2.q,\
-  import_exported_table.q,\
-  index_bitmap3.q,\
-  index_bitmap_auto.q,\
-  infer_bucket_sort_bucketed_table.q,\
-  infer_bucket_sort_dyn_part.q,\
-  infer_bucket_sort_map_operators.q,\
-  infer_bucket_sort_merge.q,\
-  infer_bucket_sort_num_buckets.q,\
-  infer_bucket_sort_reducers_power_two.q,\
-  input16_cc.q,\
-  insert_dir_distcp.q,\
-  join1.q,\
-  join_acid_non_acid.q,\
-  leftsemijoin_mr.q,\
-  list_bucket_dml_10.q,\
-  load_fs2.q,\
-  load_hdfs_file_with_space_in_the_name.q,\
-  non_native_window_udf.q, \
-  parallel_orderby.q,\
-  quotedid_smb.q,\
-  reduce_deduplicate.q,\
-  remote_script.q,\
-  root_dir_external_table.q,\
-  schemeAuthority.q,\
-  schemeAuthority2.q,\
-  scriptfile1.q,\
-  scriptfile1_win.q,\
-  skewjoin_onesideskew.q,\
-  table_nonprintable.q,\
-  temp_table_external.q,\
-  truncate_column_buckets.q,\
-  uber_reduce.q,\
-  udf_using.q
-
-# These tests are disabled for minimr
-#  ql_rewrite_gbtoidx.q,\
-#  ql_rewrite_gbtoidx_cbo_1.q,\
-#  ql_rewrite_gbtoidx_cbo_2.q,\
-#  smb_mapjoin_8.q,\
-
-
-# Tests that are not enabled for CLI Driver
-disabled.query.files=ql_rewrite_gbtoidx.q,\
-  ql_rewrite_gbtoidx_cbo_1.q,\
-  ql_rewrite_gbtoidx_cbo_2.q,\
-  rcfile_merge1.q,\
-  smb_mapjoin_8.q
-
-# NOTE: Add tests to minitez only if it is very
-# specific to tez and cannot be added to minillap.
-minitez.query.files.shared=delete_orig_table.q,\
-  orc_merge12.q,\
-  orc_vectorization_ppd.q,\
-  unionDistinct_2.q,\
-  update_orig_table.q,\
-  vector_join_part_col_char.q,\
-  vector_non_string_partition.q,\
-  vectorization_div0.q,\
-  vectorization_limit.q
-
-# NOTE: Add tests to minitez only if it is very
-# specific to tez and cannot be added to minillap.
-minitez.query.files=explainuser_3.q,\
-  explainanalyze_1.q,\
-  explainanalyze_2.q,\
-  explainanalyze_3.q,\
-  explainanalyze_4.q,\
-  explainanalyze_5.q,\
-  hybridgrace_hashjoin_1.q,\
-  hybridgrace_hashjoin_2.q,\
-  partition_column_names_with_leading_and_trailing_spaces.q,\
-  stats_filemetadata.q,\
-  tez_union_with_udf.q
-
-minillap.shared.query.files=acid_globallimit.q,\
-  alter_merge_2_orc.q,\
-  alter_merge_orc.q,\
-  alter_merge_stats_orc.q,\
-  auto_join0.q,\
-  auto_join1.q,\
-  auto_join21.q,\
-  auto_join29.q,\
-  auto_join30.q,\
-  auto_join_filters.q,\
-  auto_join_nulls.q,\
-  auto_sortmerge_join_1.q,\
-  auto_sortmerge_join_10.q,\
-  auto_sortmerge_join_11.q,\
-  auto_sortmerge_join_12.q,\
-  auto_sortmerge_join_13.q,\
-  auto_sortmerge_join_14.q,\
-  auto_sortmerge_join_15.q,\
-  auto_sortmerge_join_16.q,\
-  auto_sortmerge_join_2.q,\
-  auto_sortmerge_join_3.q,\
-  auto_sortmerge_join_4.q,\
-  auto_sortmerge_join_5.q,\
-  auto_sortmerge_join_6.q,\
-  auto_sortmerge_join_7.q,\
-  auto_sortmerge_join_8.q,\
-  auto_sortmerge_join_9.q,\
-  bucket2.q,\
-  bucket3.q,\
-  bucket4.q,\
-  bucket_map_join_tez1.q,\
-  bucket_map_join_tez2.q,\
-  cbo_gby.q,\
-  cbo_gby_empty.q,\
-  cbo_join.q,\
-  cbo_limit.q,\
-  cbo_semijoin.q,\
-  cbo_simple_select.q,\
-  cbo_stats.q,\
-  cbo_subq_exists.q,\
-  cbo_subq_in.q,\
-  cbo_subq_not_in.q,\
-  cbo_udf_udaf.q,\
-  cbo_union.q,\
-  cbo_views.q,\
-  cbo_windowing.q,\
-  column_names_with_leading_and_trailing_spaces.q,\
-  constprog_dpp.q,\
-  constprog_semijoin.q,\
-  correlationoptimizer1.q,\
-  count.q,\
-  create_merge_compressed.q,\
-  cross_join.q,\
-  cross_product_check_1.q,\
-  cross_product_check_2.q,\
-  ctas.q,\
-  cte_1.q,\
-  cte_2.q,\
-  cte_3.q,\
-  cte_4.q,\
-  cte_5.q,\
-  cte_mat_1.q,\
-  cte_mat_2.q,\
-  cte_mat_3.q,\
-  cte_mat_4.q,\
-  cte_mat_5.q,\
-  custom_input_output_format.q,\
-  deleteAnalyze.q,\
-  delete_all_non_partitioned.q,\
-  delete_all_partitioned.q,\
-  delete_tmp_table.q,\
-  delete_where_no_match.q,\
-  delete_where_non_partitioned.q,\
-  delete_where_partitioned.q,\
-  delete_whole_partition.q,\
-  disable_merge_for_bucketing.q,\
-  dynamic_partition_pruning.q,\
-  dynamic_partition_pruning_2.q,\
-  dynpart_sort_opt_vectorization.q,\
-  dynpart_sort_optimization.q,\
-  dynpart_sort_optimization2.q,\
-  empty_join.q,\
-  enforce_order.q,\
-  filter_join_breaktask.q,\
-  filter_join_breaktask2.q,\
-  groupby1.q,\
-  groupby2.q,\
-  groupby3.q,\
-  having.q,\
-  identity_project_remove_skip.q,\
-  insert1.q,\
-  insert_into1.q,\
-  insert_into2.q,\
-  insert_orig_table.q,\
-  insert_update_delete.q,\
-  insert_values_dynamic_partitioned.q,\
-  insert_values_non_partitioned.q,\
-  insert_values_orig_table.,\
-  insert_values_partitioned.q,\
-  insert_values_tmp_table.q,\
-  join0.q,\
-  join1.q,\
-  join_nullsafe.q,\
-  leftsemijoin.q,\
-  limit_pushdown.q,\
-  llap_nullscan.q,\
-  llapdecider.q,\
-  load_dyn_part1.q,\
-  load_dyn_part2.q,\
-  load_dyn_part3.q,\
-  lvj_mapjoin.q,\
-  mapjoin2.q,\
-  mapjoin_decimal.q,\
-  mapjoin_mapjoin.q,\
-  mapreduce1.q,\
-  mapreduce2.q,\
-  merge1.q,\
-  merge2.q,\
-  mergejoin.q,\
-  metadata_only_queries.q,\
-  metadata_only_queries_with_filters.q,\
-  metadataonly1.q,\
-  mrr.q,\
-  nonmr_fetch_threshold.q,\
-  optimize_nullscan.q,\
-  orc_analyze.q,\
-  orc_merge1.q,\
-  orc_merge10.q,\
-  orc_merge11.q,\
-  orc_merge2.q,\
-  orc_merge3.q,\
-  orc_merge4.q,\
-  orc_merge5.q,\
-  orc_merge6.q,\
-  orc_merge7.q,\
-  orc_merge8.q,\
-  orc_merge9.q,\
-  orc_merge_diff_fs.q,\
-  orc_merge_incompat1.q,\
-  orc_merge_incompat2.q,\
-  orc_merge_incompat3.q,\
-  orc_ppd_basic.q,\
-  orc_ppd_schema_evol_1a.q,\
-  orc_ppd_schema_evol_1b.q,\
-  orc_ppd_schema_evol_2a.q,\
-  orc_ppd_schema_evol_2b.q,\
-  orc_ppd_schema_evol_3a.q,\
-  order_null.q,\
-  parallel.q,\
-  ptf.q,\
-  ptf_matchpath.q,\
-  ptf_streaming.q,\
-  sample1.q,\
-  script_env_var1.q,\
-  script_env_var2.q,\
-  script_pipe.q,\
-  scriptfile1.q,\
-  selectDistinctStar.q,\
-  select_dummy_source.q,\
-  skewjoin.q,\
-  stats_noscan_1.q,\
-  stats_only_null.q,\
-  subquery_exists.q,\
-  subquery_in.q,\
-  temp_table.q,\
-  tez_bmj_schema_evolution.q,\
-  tez_dml.q,\
-  tez_dynpart_hashjoin_1.q,\
-  tez_dynpart_hashjoin_2.q,\
-  tez_fsstat.q,\
-  tez_insert_overwrite_local_directory_1.q,\
-  tez_join.q,\
-  tez_join_hash.q,\
-  tez_join_result_complex.q,\
-  tez_join_tests.q,\
-  tez_joins_explain.q,\
-  tez_multi_union.q,\
-  tez_schema_evolution.q,\
-  tez_self_join.q,\
-  tez_smb_1.q,\
-  tez_smb_main.q,\
-  tez_union.q,\
-  tez_union2.q,\
-  tez_union_decimal.q,\
-  tez_union_dynamic_partition.q,\
-  tez_union_group_by.q,\
-  tez_union_multiinsert.q,\
-  tez_union_view.q,\
-  tez_vector_dynpart_hashjoin_1.q,\
-  tez_vector_dynpart_hashjoin_2.q,\
-  transform1.q,\
-  transform2.q,\
-  transform_ppr1.q,\
-  transform_ppr2.q,\
-  union2.q,\
-  union3.q,\
-  union4.q,\
-  union5.q,\
-  union6.q,\
-  union7.q,\
-  union8.q,\
-  union9.q,\
-  unionDistinct_1.q,\
-  union_fast_stats.q,\
-  union_stats.q,\
-  union_type_chk.q,\
-  update_after_multiple_inserts.q,\
-  update_all_non_partitioned.q,\
-  update_all_partitioned.q,\
-  update_all_types.q,\
-  update_tmp_table.q,\
-  update_two_cols.q,\
-  update_where_no_match.q,\
-  update_where_non_partitioned.q,\
-  update_where_partitioned.q,\
-  vector_acid3.q,\
-  vector_aggregate_9.q,\
-  vector_aggregate_without_gby.q,\
-  vector_auto_smb_mapjoin_14.q,\
-  vector_between_columns.q,\
-  vector_between_in.q,\
-  vector_binary_join_groupby.q,\
-  vector_bround.q,\
-  vector_bucket.q,\
-  vector_cast_constant.q,\
-  vector_char_2.q,\
-  vector_char_4.q,\
-  vector_char_cast.q,\
-  vector_char_mapjoin1.q,\
-  vector_char_simple.q,\
-  vector_coalesce.q,\
-  vector_coalesce_2.q,\
-  vector_complex_all.q,\
-  vector_complex_join.q,\
-  vector_count.q,\
-  vector_count_distinct.q,\
-  vector_data_types.q,\
-  vector_date_1.q,\
-  vector_decimal_1.q,\
-  vector_decimal_10_0.q,\
-  vector_decimal_2.q,\
-  vector_decimal_3.q,\
-  vector_decimal_4.q,\
-  vector_decimal_5.q,\
-  vector_decimal_6.q,\
-  vector_decimal_aggregate.q,\
-  vector_decimal_cast.q,\
-  vector_decimal_expressions.q,\
-  vector_decimal_mapjoin.q,\
-  vector_decimal_math_funcs.q,\
-  vector_decimal_precision.q,\
-  vector_decimal_round.q,\
-  vector_decimal_round_2.q,\
-  vector_decimal_trailing.q,\
-  vector_decimal_udf.q,\
-  vector_decimal_udf2.q,\
-  vector_distinct_2.q,\
-  vector_elt.q,\
-  vector_groupby4.q,\
-  vector_groupby6.q,\
-  vector_groupby_3.q,\
-  vector_groupby_mapjoin.q,\
-  vector_groupby_reduce.q,\
-  vector_grouping_sets.q,\
-  vector_if_expr.q,\
-  vector_include_no_sel.q,\
-  vector_inner_join.q,\
-  vector_interval_1.q,\
-  vector_interval_2.q,\
-  vector_interval_arithmetic.q,\
-  vector_interval_mapjoin.q,\
-  vector_join30.q,\
-  vector_join_filters.q,\
-  vector_join_nulls.q,\
-  vector_left_outer_join.q,\
-  vector_left_outer_join2.q,\
-  vector_leftsemi_mapjoin.q,\
-  vector_mapjoin_reduce.q,\
-  vector_mr_diff_schema_alias.q,\
-  vector_multi_insert.q,\
-  vector_null_projection.q,\
-  vector_nullsafe_join.q,\
-  vector_nvl.q,\
-  vector_orderby_5.q,\
-  vector_outer_join0.q,\
-  vector_outer_join1.q,\
-  vector_outer_join2.q,\
-  vector_outer_join3.q,\
-  vector_outer_join4.q,\
-  vector_outer_join5.q,\
-  vector_outer_join6.q,\
-  vector_partition_diff_num_cols.q,\
-  vector_partitioned_date_time.q,\
-  vector_reduce1.q,\
-  vector_reduce2.q,\
-  vector_reduce3.q,\
-  vector_reduce_groupby_decimal.q,\
-  vector_string_concat.q,\
-  vector_struct_in.q,\
-  vector_varchar_4.q,\
-  vector_varchar_mapjoin1.q,\
-  vector_varchar_simple.q,\
-  vector_when_case_null.q,\
-  vectorization_0.q,\
-  vectorization_1.q,\
-  vectorization_10.q,\
-  vectorization_11.q,\
-  vectorization_12.q,\
-  vectorization_13.q,\
-  vectorization_14.q,\
-  vectorization_15.q,\
-  vectorization_16.q,\
-  vectorization_17.q,\
-  vectorization_2.q,\
-  vectorization_3.q,\
-  vectorization_4.q,\
-  vectorization_5.q,\
-  vectorization_6.q,\
-  vectorization_7.q,\
-  vectorization_8.q,\
-  vectorization_9.q,\
-  vectorization_decimal_date.q,\
-  vectorization_nested_udf.q,\
-  vectorization_not.q,\
-  vectorization_part.q,\
-  vectorization_part_project.q,\
-  vectorization_part_varchar.q,\
-  vectorization_pushdown.q,\
-  vectorization_short_regress.q,\
-  vectorized_bucketmapjoin1.q,\
-  vectorized_case.q,\
-  vectorized_casts.q,\
-  vectorized_context.q,\
-  vectorized_date_funcs.q,\
-  vectorized_distinct_gby.q,\
-  vectorized_dynamic_partition_pruning.q,\
-  vectorized_mapjoin.q,\
-  vectorized_math_funcs.q,\
-  vectorized_nested_mapjoin.q,\
-  vectorized_parquet.q,\
-  vectorized_parquet_types.q,\
-  vectorized_ptf.q,\
-  vectorized_rcfile_columnar.q,\
-  vectorized_shufflejoin.q,\
-  vectorized_string_funcs.q,\
-  vectorized_timestamp.q,\
-  vectorized_timestamp_funcs.q,\
-  vectorized_timestamp_ints_casts.q
-
-minillap.query.files=acid_bucket_pruning.q,\
-  acid_vectorization_missing_cols.q,\
-  bucket_map_join_tez1.q,\
-  bucket_map_join_tez2.q,\
-  bucketpruning1.q,\
-  constprog_dpp.q,\
-  dynamic_partition_pruning.q,\
-  dynamic_partition_pruning_2.q,\
-  explainuser_1.q,\
-  explainuser_2.q,\
-  explainuser_4.q,\
-  hybridgrace_hashjoin_1.q,\
-  hybridgrace_hashjoin_2.q,\
-  llap_nullscan.q,\
-  llap_udf.q,\
-  llapdecider.q,\
-  lvj_mapjoin.q,\
-  mapjoin_decimal.q,\
-  mergejoin_3way.q,\
-  mrr.q,\
-  orc_llap.q,\
-  orc_llap_counters.q,\
-  orc_llap_counters1.q,\
-  orc_llap_nonvector.q,\
-  orc_ppd_basic.q,\
-  schema_evol_orc_acid_part.q,\
-  schema_evol_orc_acid_part_update.q,\
-  schema_evol_orc_acid_table.q,\
-  schema_evol_orc_acid_table_update.q,\
-  schema_evol_orc_acidvec_part.q,\
-  schema_evol_orc_acidvec_part_update.q,\
-  schema_evol_orc_acidvec_table.q,\
-  schema_evol_orc_acidvec_table_update.q,\
-  schema_evol_orc_nonvec_part.q,\
-  schema_evol_orc_nonvec_part_all_complex.q,\
-  schema_evol_orc_nonvec_part_all_primitive.q,\
-  schema_evol_orc_nonvec_table.q,\
-  schema_evol_orc_vec_part.q,\
-  schema_evol_orc_vec_part_all_complex.q,\
-  schema_evol_orc_vec_part_all_primitive.q,\
-  schema_evol_orc_vec_table.q,\
-  schema_evol_stats.q,\
-  schema_evol_text_nonvec_part.q,\
-  schema_evol_text_nonvec_part_all_complex.q,\
-  schema_evol_text_nonvec_part_all_primitive.q,\
-  schema_evol_text_nonvec_table.q,\
-  schema_evol_text_vec_part.q,\
-  schema_evol_text_vec_part_all_complex.q,\
-  schema_evol_text_vec_part_all_primitive.q,\
-  schema_evol_text_vec_table.q,\
-  schema_evol_text_vecrow_part.q,\
-  schema_evol_text_vecrow_part_all_complex.q,\
-  schema_evol_text_vecrow_part_all_primitive.q,\
-  schema_evol_text_vecrow_table.q,\
-  smb_cache.q,\
-  tez_aggr_part_stats.q,\
-  tez_bmj_schema_evolution.q,\
-  tez_dml.q,\
-  tez_dynpart_hashjoin_1.q,\
-  tez_dynpart_hashjoin_2.q,\
-  tez_dynpart_hashjoin_3.q,\
-  tez_fsstat.q,\
-  tez_insert_overwrite_local_directory_1.q,\
-  tez_join.q,\
-  tez_join_result_complex.q,\
-  tez_join_tests.q,\
-  tez_joins_explain.q,\
-  tez_multi_union.q,\
-  tez_schema_evolution.q,\
-  tez_self_join.q,\
-  tez_smb_1.q,\
-  tez_smb_empty.q,\
-  tez_smb_main.q,\
-  tez_union.q,\
-  tez_union2.q,\
-  tez_union_decimal.q,\
-  tez_union_dynamic_partition.q,\
-  tez_union_group_by.q,\
-  tez_union_multiinsert.q,\
-  tez_union_view.q,\
-  tez_vector_dynpart_hashjoin_1.q,\
-  tez_vector_dynpart_hashjoin_2.q,\
-  vectorized_dynamic_partition_pruning.q,\
-  windowing_gby.q
-
-encrypted.query.files=encryption_join_unencrypted_tbl.q,\
-  encryption_insert_partition_static.q,\
-  encryption_insert_partition_dynamic.q,\
-  encryption_join_with_different_encryption_keys.q,\
-  encryption_select_read_only_encrypted_tbl.q,\
-  encryption_select_read_only_unencrypted_tbl.q,\
-  encryption_load_data_to_encrypted_tables.q, \
-  encryption_unencrypted_nonhdfs_external_tables.q \
-  encryption_move_tbl.q \
-  encryption_drop_table.q \
-  encryption_insert_values.q \
-  encryption_drop_view.q \
-  encryption_drop_partition.q \
-  encryption_with_trash.q \
-  encryption_ctas.q
-
-beeline.positive.exclude=add_part_exist.q,\
-  alter1.q,\
-  alter2.q,\
-  alter4.q,\
-  alter5.q,\
-  alter_rename_partition.q,\
-  alter_rename_partition_authorization.q,\
-  archive.q,\
-  archive_corrupt.q,\
-  archive_mr_1806.q,\
-  archive_multi.q,\
-  archive_multi_mr_1806.q,\
-  authorization_1.q,\
-  authorization_2.q,\
-  authorization_4.q,\
-  authorization_5.q,\
-  authorization_6.q,\
-  authorization_7.q,\
-  ba_table1.q,\
-  ba_table2.q,\
-  ba_table3.q,\
-  ba_table_udfs.q,\
-  binary_table_bincolserde.q,\
-  binary_table_colserde.q,\
-  cluster.q,\
-  columnarserde_create_shortcut.q,\
-  combine2.q,\
-  constant_prop.q,\
-  create_nested_type.q,\
-  create_or_replace_view.q,\
-  create_struct_table.q,\
-  create_union_table.q,\
-  database.q,\
-  database_location.q,\
-  database_properties.q,\
-  describe_database_json.q,\
-  drop_database_removes_partition_dirs.q,\
-  escape1.q,\
-  escape2.q,\
-  exim_00_nonpart_empty.q,\
-  exim_01_nonpart.q,\
-  exim_02_00_part_empty.q,\
-  exim_02_part.q,\
-  exim_03_nonpart_over_compat.q,\
-  exim_04_all_part.q,\
-  exim_04_evolved_parts.q,\
-  exim_05_some_part.q,\
-  exim_06_one_part.q,\
-  exim_07_all_part_over_nonoverlap.q,\
-  exim_08_nonpart_rename.q,\
-  exim_09_part_spec_nonoverlap.q,\
-  exim_10_external_managed.q,\
-  exim_11_managed_external.q,\
-  exim_12_external_location.q,\
-  exim_13_managed_location.q,\
-  exim_14_managed_location_over_existing.q,\
-  exim_15_external_part.q,\
-  exim_16_part_external.q,\
-  exim_17_part_managed.q,\
-  exim_18_part_external.q,\
-  exim_19_00_part_external_location.q,\
-  exim_19_part_external_location.q,\
-  exim_20_part_managed_location.q,\
-  exim_21_export_authsuccess.q,\
-  exim_22_import_exist_authsuccess.q,\
-  exim_23_import_part_authsuccess.q,\
-  exim_24_import_nonexist_authsuccess.q,\
-  global_limit.q,\
-  groupby_complex_types.q,\
-  groupby_complex_types_multi_single_reducer.q,\
-  index_auth.q,\
-  index_auto.q,\
-  index_auto_empty.q,\
-  index_bitmap.q,\
-  index_bitmap1.q,\
-  index_bitmap2.q,\
-  index_bitmap3.q,\
-  index_bitmap_auto.q,\
-  index_bitmap_rc.q,\
-  index_compact.q,\
-  index_compact_1.q,\
-  index_compact_2.q,\
-  index_compact_3.q,\
-  index_stale_partitioned.q,\
-  init_file.q,\
-  input16.q,\
-  input16_cc.q,\
-  input46.q,\
-  input_columnarserde.q,\
-  input_dynamicserde.q,\
-  input_lazyserde.q,\
-  input_testxpath3.q,\
-  input_testxpath4.q,\
-  insert2_overwrite_partitions.q,\
-  insertexternal1.q,\
-  join_thrift.q,\
-  lateral_view.q,\
-  load_binary_data.q,\
-  load_exist_part_authsuccess.q,\
-  load_nonpart_authsuccess.q,\
-  load_part_authsuccess.q,\
-  loadpart_err.q,\
-  lock1.q,\
-  lock2.q,\
-  lock3.q,\
-  lock4.q,\
-  merge_dynamic_partition.q,\
-  multi_insert.q,\
-  multi_insert_move_tasks_share_dependencies.q,\
-  null_column.q,\
-  ppd_clusterby.q,\
-  query_with_semi.q,\
-  rename_column.q,\
-  sample6.q,\
-  sample_islocalmode_hook.q,\
-  set_processor_namespaces.q,\
-  show_tables.q,\
-  source.q,\
-  split_sample.q,\
-  str_to_map.q,\
-  transform1.q,\
-  udaf_collect_set.q,\
-  udaf_context_ngrams.q,\
-  udaf_histogram_numeric.q,\
-  udaf_ngrams.q,\
-  udaf_percentile_approx.q,\
-  udf_array.q,\
-  udf_bitmap_and.q,\
-  udf_bitmap_or.q,\
-  udf_explode.q,\
-  udf_format_number.q,\
-  udf_map.q,\
-  udf_map_keys.q,\
-  udf_map_values.q,\
-  udf_mask.q,\
-  udf_mask_first_n.q,\
-  udf_mask_hash.q,\
-  udf_mask_last_n.q,\
-  udf_mask_show_first_n.q,\
-  udf_mask_show_last_n.q,\
-  udf_max.q,\
-  udf_min.q,\
-  udf_named_struct.q,\
-  udf_percentile.q,\
-  udf_printf.q,\
-  udf_sentences.q,\
-  udf_sort_array.q,\
-  udf_split.q,\
-  udf_struct.q,\
-  udf_substr.q,\
-  udf_translate.q,\
-  udf_union.q,\
-  udf_xpath.q,\
-  udtf_stack.q,\
-  view.q,\
-  virtual_column.q
-
-minimr.query.negative.files=cluster_tasklog_retrieval.q,\
-  file_with_header_footer_negative.q,\
-  local_mapred_error_cache.q,\
-  mapreduce_stack_trace.q,\
-  mapreduce_stack_trace_hadoop20.q,\
-  mapreduce_stack_trace_turnoff.q,\
-  mapreduce_stack_trace_turnoff_hadoop20.q,\
-  minimr_broken_pipe.q,\
-  table_nonprintable_negative.q,\
-  udf_local_resource.q
-
-# tests are sorted use: perl -pe 's@\\\s*\n@ @g' testconfiguration.properties \
-# | awk -F= '/spark.query.files/{print $2}' | perl -pe 's@.q *, *@\n@g' \
-# | egrep -v '^ *$' |  sort -V | uniq | perl -pe 's@\n@.q, \\\n@g' | perl -pe 's@^@  @g'
-spark.query.files=add_part_multiple.q, \
-  alter_merge_orc.q, \
-  alter_merge_stats_orc.q, \
-  annotate_stats_join.q, \
-  auto_join0.q, \
-  auto_join1.q, \
-  auto_join10.q, \
-  auto_join11.q, \
-  auto_join12.q, \
-  auto_join13.q, \
-  auto_join14.q, \
-  auto_join15.q, \
-  auto_join16.q, \
-  auto_join17.q, \
-  auto_join18.q, \
-  auto_join18_multi_distinct.q, \
-  auto_join19.q, \
-  auto_join2.q, \
-  auto_join20.q, \
-  auto_join21.q, \
-  auto_join22.q, \
-  auto_join23.q, \
-  auto_join24.q, \
-  auto_join26.q, \
-  auto_join27.q, \
-  auto_join28.q, \
-  auto_join29.q, \
-  auto_join3.q, \
-  auto_join30.q, \
-  auto_join31.q, \
-  auto_join4.q, \
-  auto_join5.q, \
-  auto_join6.q, \
-  auto_join7.q, \
-  auto_join8.q, \
-  auto_join9.q, \
-  auto_join_filters.q, \
-  auto_join_nulls.q, \
-  auto_join_reordering_values.q, \
-  auto_join_stats.q, \
-  auto_join_stats2.q, \
-  auto_join_without_localtask.q, \
-  auto_smb_mapjoin_14.q, \
-  auto_sortmerge_join_1.q, \
-  auto_sortmerge_join_10.q, \
-  auto_sortmerge_join_12.q, \
-  auto_sortmerge_join_13.q, \
-  auto_sortmerge_join_14.q, \
-  auto_sortmerge_join_15.q, \
-  auto_sortmerge_join_16.q, \
-  auto_sortmerge_join_3.q, \
-  auto_sortmerge_join_4.q, \
-  auto_sortmerge_join_5.q, \
-  auto_sortmerge_join_6.q, \
-  auto_sortmerge_join_7.q, \
-  auto_sortmerge_join_8.q, \
-  auto_sortmerge_join_9.q, \
-  avro_compression_enabled_native.q, \
-  avro_decimal_native.q, \
-  avro_joins.q, \
-  avro_joins_native.q, \
-  bucket2.q, \
-  bucket3.q, \
-  bucket4.q, \
-  bucket_map_join_1.q, \
-  bucket_map_join_2.q, \
-  bucket_map_join_spark1.q, \
-  bucket_map_join_spark2.q, \
-  bucket_map_join_spark3.q, \
-  bucket_map_join_spark4.q, \
-  bucket_map_join_tez1.q, \
-  bucket_map_join_tez2.q, \
-  bucketmapjoin1.q, \
-  bucketmapjoin10.q, \
-  bucketmapjoin11.q, \
-  bucketmapjoin12.q, \
-  bucketmapjoin13.q, \
-  bucketmapjoin2.q, \
-  bucketmapjoin3.q, \
-  bucketmapjoin4.q, \
-  bucketmapjoin5.q, \
-  bucketmapjoin7.q, \
-  bucketmapjoin8.q, \
-  bucketmapjoin9.q, \
-  bucketmapjoin_negative.q, \
-  bucketmapjoin_negative2.q, \
-  bucketmapjoin_negative3.q, \
-  bucketsortoptimize_insert_2.q, \
-  bucketsortoptimize_insert_4.q, \
-  bucketsortoptimize_insert_6.q, \
-  bucketsortoptimize_insert_7.q, \
-  bucketsortoptimize_insert_8.q, \
-  cbo_gby.q, \
-  cbo_gby_empty.q, \
-  cbo_limit.q, \
-  cbo_semijoin.q, \
-  cbo_simple_select.q, \
-  cbo_stats.q, \
-  cbo_subq_in.q, \
-  cbo_subq_not_in.q, \
-  cbo_udf_udaf.q, \
-  cbo_union.q, \
-  column_access_stats.q, \
-  count.q, \
-  create_merge_compressed.q, \
-  cross_join.q, \
-  cross_product_check_1.q, \
-  cross_product_check_2.q, \
-  ctas.q, \
-  custom_input_output_format.q, \
-  date_join1.q, \
-  date_udf.q, \
-  decimal_1_1.q, \
-  decimal_join.q, \
-  disable_merge_for_bucketing.q, \
-  dynamic_rdd_cache.q, \
-  enforce_order.q, \
-  escape_clusterby1.q, \
-  escape_distributeby1.q, \
-  escape_orderby1.q, \
-  escape_sortby1.q, \
-  filter_join_breaktask.q, \
-  filter_join_breaktask2.q, \
-  groupby1.q, \
-  groupby10.q, \
-  groupby11.q, \
-  groupby1_map.q, \
-  groupby1_map_nomap.q, \
-  groupby1_map_skew.q, \
-  groupby1_noskew.q, \
-  groupby2.q, \
-  groupby2_map.q, \
-  groupby2_map_multi_distinct.q, \
-  groupby2_map_skew.q, \
-  groupby2_noskew.q, \
-  groupby2_noskew_multi_distinct.q, \
-  groupby3.q, \
-  groupby3_map.q, \
-  groupby3_map_multi_distinct.q, \
-  groupby3_map_skew.q, \
-  groupby3_noskew.q, \
-  groupby3_noskew_multi_distinct.q, \
-  groupby4.q, \
-  groupby4_map.q, \
-  groupby4_map_skew.q, \
-  groupby4_noskew.q, \
-  groupby5.q, \
-  groupby5_map.q, \
-  groupby5_map_skew.q, \
-  groupby5_noskew.q, \
-  groupby6.q, \
-  groupby6_map.q, \
-  groupby6_map_skew.q, \
-  groupby6_noskew.q, \
-  groupby7.q, \
-  groupby7_map.q, \
-  groupby7_map_multi_single_reducer.q, \
-  groupby7_map_skew.q, \
-  groupby7_noskew.q, \
-  groupby7_noskew_multi_single_reducer.q, \
-  groupby8.q, \
-  groupby8_map.q, \
-  groupby8_map_skew.q, \
-  groupby8_noskew.q, \
-  groupby9.q, \
-  groupby_bigdata.q, \
-  groupby_complex_types.q, \
-  groupby_complex_types_multi_single_reducer.q, \
-  groupby_cube1.q, \
-  groupby_grouping_id2.q, \
-  groupby_map_ppr.q, \
-  groupby_map_ppr_multi_distinct.q, \
-  groupby_multi_insert_common_distinct.q, \
-  groupby_multi_single_reducer.q, \
-  groupby_multi_single_reducer2.q, \
-  groupby_multi_single_reducer3.q, \
-  groupby_position.q, \
-  groupby_ppr.q, \
-  groupby_ppr_multi_distinct.q, \
-  groupby_resolution.q, \
-  groupby_rollup1.q, \
-  groupby_sort_1_23.q, \
-  groupby_sort_skew_1.q, \
-  groupby_sort_skew_1_23.q, \
-  qroupby_limit_extrastep.q, \
-  having.q, \
-  identity_project_remove_skip.q, \
-  index_auto_self_join.q, \
-  innerjoin.q, \
-  input12.q, \
-  input13.q, \
-  input14.q, \
-  input17.q, \
-  input18.q, \
-  input1_limit.q, \
-  input_part2.q, \
-  insert_into1.q, \
-  insert_into2.q, \
-  insert_into3.q, \
-  join0.q, \
-  join1.q, \
-  join10.q, \
-  join11.q, \
-  join12.q, \
-  join13.q, \
-  join14.q, \
-  join15.q, \
-  join16.q, \
-  join17.q, \
-  join18.q, \
-  join18_multi_distinct.q, \
-  join19.q, \
-  join2.q, \
-  join20.q, \
-  join21.q, \
-  join22.q, \
-  join23.q, \
-  join24.q, \
-  join25.q, \
-  join26.q, \
-  join27.q, \
-  join28.q, \
-  join29.q, \
-  join3.q, \
-  join30.q, \
-  join31.q, \
-  join32.q, \
-  join32_lessSize.q, \
-  join33.q, \
-  join34.q, \
-  join35.q, \
-  join36.q, \
-  join37.q, \
-  join38.q, \
-  join39.q, \
-  join4.q, \
-  join41.q, \
-  join5.q, \
-  join6.q, \
-  join7.q, \
-  join8.q, \
-  join9.q, \
-  join_1to1.q, \
-  join_alt_syntax.q, \
-  join_array.q, \
-  join_casesensitive.q, \
-  join_cond_pushdown_1.q, \
-  join_cond_pushdown_2.q, \
-  join_cond_pushdown_3.q, \
-  join_cond_pushdown_4.q, \
-  join_cond_pushdown_unqual1.q, \
-  join_cond_pushdown_unqual2.q, \
-  join_cond_pushdown_unqual3.q, \
-  join_cond_pushdown_unqual4.q, \
-  join_filters_overlap.q, \
-  join_hive_626.q, \
-  join_literals.q, \
-  join_map_ppr.q, \
-  join_merge_multi_expressions.q, \
-  join_merging.q, \
-  join_nullsafe.q, \
-  join_rc.q, \
-  join_reorder.q, \
-  join_reorder2.q, \
-  join_reorder3.q, \
-  join_reorder4.q, \
-  join_star.q, \
-  join_thrift.q, \
-  join_vc.q, \
-  join_view.q, \
-  lateral_view_explode2.q, \
-  leftsemijoin.q, \
-  leftsemijoin_mr.q, \
-  limit_partition_metadataonly.q, \
-  limit_pushdown.q, \
-  list_bucket_dml_2.q, \
-  load_dyn_part1.q, \
-  load_dyn_part10.q, \
-  load_dyn_part11.q, \
-  load_dyn_part12.q, \
-  load_dyn_part13.q, \
-  load_dyn_part14.q, \
-  load_dyn_part15.q, \
-  load_dyn_part2.q, \
-  load_dyn_part3.q, \
-  load_dyn_part4.q, \
-  load_dyn_part5.q, \
-  load_dyn_part6.q, \
-  load_dyn_part7.q, \
-  load_dyn_part8.q, \
-  load_dyn_part9.q, \
-  louter_join_ppr.q, \
-  mapjoin1.q, \
-  mapjoin_addjar.q, \
-  mapjoin_decimal.q, \
-  mapjoin_distinct.q, \
-  mapjoin_filter_on_outerjoin.q, \
-  mapjoin_mapjoin.q, \
-  mapjoin_memcheck.q, \
-  mapjoin_subquery.q, \
-  mapjoin_subquery2.q, \
-  mapjoin_test_outer.q, \
-  mapreduce1.q, \
-  mapreduce2.q, \
-  merge1.q, \
-  merge2.q, \
-  mergejoins.q, \
-  mergejoins_mixed.q, \
-  metadata_only_queries.q, \
-  metadata_only_queries_with_filters.q, \
-  multi_insert.q, \
-  multi_insert_gby.q, \
-  multi_insert_gby2.q, \
-  multi_insert_gby3.q, \
-  multi_insert_lateral_view.q, \
-  multi_insert_mixed.q, \
-  multi_insert_move_tasks_share_dependencies.q, \
-  multi_insert_with_join.q, \
-  multi_join_union.q, \
-  multi_join_union_src.q, \
-  multigroupby_singlemr.q, \
-  nullgroup.q, \
-  nullgroup2.q, \
-  nullgroup4.q, \
-  nullgroup4_multi_distinct.q, \
-  optimize_nullscan.q, \
-  order.q, \
-  order2.q, \
-  outer_join_ppr.q, \
-  parallel.q, \
-  parallel_join0.q, \
-  parallel_join1.q, \
-  parquet_join.q, \
-  pcr.q, \
-  ppd_gby_join.q, \
-  ppd_join.q, \
-  ppd_join2.q, \
-  ppd_join3.q, \
-  ppd_join5.q, \
-  ppd_join_filter.q, \
-  ppd_multi_insert.q, \
-  ppd_outer_join1.q, \
-  ppd_outer_join2.q, \
-  ppd_outer_join3.q, \
-  ppd_outer_join4.q, \
-  ppd_outer_join5.q, \
-  ppd_transform.q, \
-  ptf.q, \
-  ptf_decimal.q, \
-  ptf_general_queries.q, \
-  ptf_matchpath.q, \
-  ptf_rcfile.q, \
-  ptf_register_tblfn.q, \
-  ptf_seqfile.q, \
-  ptf_streaming.q, \
-  rcfile_bigdata.q, \
-  reduce_deduplicate_exclude_join.q, \
-  router_join_ppr.q, \
-  runtime_skewjoin_mapjoin_spark.q, \
-  sample1.q, \
-  sample10.q, \
-  sample2.q, \
-  sample3.q, \
-  sample4.q, \
-  sample5.q, \
-  sample6.q, \
-  sample7.q, \
-  sample8.q, \
-  sample9.q, \
-  script_env_var1.q, \
-  script_env_var2.q, \
-  script_pipe.q, \
-  scriptfile1.q, \
-  semijoin.q, \
-  skewjoin.q, \
-  skewjoin_noskew.q, \
-  skewjoin_union_remove_1.q, \
-  skewjoin_union_remove_2.q, \
-  skewjoinopt1.q, \
-  skewjoinopt10.q, \
-  skewjoinopt11.q, \
-  skewjoinopt12.q, \
-  skewjoinopt13.q, \
-  skewjoinopt14.q, \
-  skewjoinopt15.q, \
-  skewjoinopt16.q, \
-  skewjoinopt17.q, \
-  skewjoinopt18.q, \
-  skewjoinopt19.q, \
-  skewjoinopt2.q, \
-  skewjoinopt20.q, \
-  skewjoinopt3.q, \
-  skewjoinopt4.q, \
-  skewjoinopt5.q, \
-  skewjoinopt6.q, \
-  skewjoinopt7.q, \
-  skewjoinopt8.q, \
-  skewjoinopt9.q, \
-  smb_mapjoin_1.q, \
-  smb_mapjoin_10.q, \
-  smb_mapjoin_11.q, \
-  smb_mapjoin_12.q, \
-  smb_mapjoin_13.q, \
-  smb_mapjoin_14.q, \
-  smb_mapjoin_15.q, \
-  smb_mapjoin_16.q, \
-  smb_mapjoin_17.q, \
-  smb_mapjoin_18.q, \
-  smb_mapjoin_19.q, \
-  smb_mapjoin_2.q, \
-  smb_mapjoin_20.q, \
-  smb_mapjoin_21.q, \
-  smb_mapjoin_22.q, \
-  smb_mapjoin_25.q, \
-  smb_mapjoin_3.q, \
-  smb_mapjoin_4.q, \
-  smb_mapjoin_5.q, \
-  smb_mapjoin_6.q, \
-  smb_mapjoin_7.q, \
-  smb_mapjoin_8.q, \
-  smb_mapjoin_9.q, \
-  sort.q, \
-  stats0.q, \
-  stats1.q, \
-  stats10.q, \
-  stats12.q, \
-  stats13.q, \
-  stats14.q, \
-  stats15.q, \
-  stats16.q, \
-  stats18.q, \
-  stats2.q, \
-  stats3.q, \
-  stats5.q, \
-  stats6.q, \
-  stats7.q, \
-  stats8.q, \
-  stats9.q, \
-  stats_noscan_1.q, \
-  stats_noscan_2.q, \
-  stats_only_null.q, \
-  stats_partscan_1_23.q, \
-  statsfs.q, \
-  subquery_exists.q, \
-  subquery_in.q, \
-  subquery_multiinsert.q, \
-  table_access_keys_stats.q, \
-  temp_table.q, \
-  temp_table_gb1.q, \
-  temp_table_join1.q, \
-  tez_join_tests.q, \
-  tez_joins_explain.q, \
-  timestamp_1.q, \
-  timestamp_2.q, \
-  timestamp_3.q, \
-  timestamp_comparison.q, \
-  timestamp_lazy.q, \
-  timestamp_null.q, \
-  timestamp_udf.q, \
-  transform2.q, \
-  transform_ppr1.q, \
-  transform_ppr2.q, \
-  udaf_collect_set.q, \
-  udf_example_add.q, \
-  udf_in_file.q, \
-  udf_max.q, \
-  udf_min.q, \
-  udf_percentile.q, \
-  union.q, \
-  union10.q, \
-  union11.q, \
-  union12.q, \
-  union13.q, \
-  union14.q, \
-  union15.q, \
-  union16.q, \
-  union17.q, \
-  union18.q, \
-  union19.q, \
-  union2.q, \
-  union20.q, \
-  union21.q, \
-  union22.q, \
-  union23.q, \
-  union24.q, \
-  union25.q, \
-  union26.q, \
-  union27.q, \
-  union28.q, \
-  union29.q, \
-  union3.q, \
-  union30.q, \
-  union31.q, \
-  union32.q, \
-  union33.q, \
-  union34.q, \
-  union4.q, \
-  union5.q, \
-  union6.q, \
-  union7.q, \
-  union8.q, \
-  union9.q, \
-  union_date.q, \
-  union_date_trim.q, \
-  union_lateralview.q, \
-  union_null.q, \
-  union_ppr.q, \
-  union_remove_1.q, \
-  union_remove_10.q, \
-  union_remove_11.q, \
-  union_remove_12.q, \
-  union_remove_13.q, \
-  union_remove_14.q, \
-  union_remove_15.q, \
-  union_remove_16.q, \
-  union_remove_17.q, \
-  union_remove_18.q, \
-  union_remove_19.q, \
-  union_remove_2.q, \
-  union_remove_20.q, \
-  union_remove_21.q, \
-  union_remove_22.q, \
-  union_remove_23.q, \
-  union_remove_24.q, \
-  union_remove_25.q, \
-  union_remove_3.q, \
-  union_remove_4.q, \
-  union_remove_5.q, \
-  union_remove_6.q, \
-  union_remove_6_subq.q, \
-  union_remove_7.q, \
-  union_remove_8.q, \
-  union_remove_9.q, \
-  union_script.q, \
-  union_top_level.q, \
-  union_view.q, \
-  uniquejoin.q, \
-  varchar_join1.q, \
-  vector_between_in.q, \
-  vector_cast_constant.q, \
-  vector_char_4.q, \
-  vector_count_distinct.q, \
-  vector_data_types.q, \
-  vector_decimal_aggregate.q, \
-  vector_decimal_mapjoin.q, \
-  vector_distinct_2.q, \
-  vector_elt.q, \
-  vector_groupby_3.q, \
-  vector_left_outer_join.q, \
-  vector_mapjoin_reduce.q, \
-  vector_orderby_5.q, \
-  vector_string_concat.q, \
-  vector_varchar_4.q, \
-  vectorization_0.q, \
-  vectorization_1.q, \
-  vectorization_10.q, \
-  vectorization_11.q, \
-  vectorization_12.q, \
-  vectorization_13.q, \
-  vectorization_14.q, \
-  vectorization_15.q, \
-  vectorization_16.q, \
-  vectorization_17.q, \
-  vectorization_2.q, \
-  vectorization_3.q, \
-  vectorization_4.q, \
-  vectorization_5.q, \
-  vectorization_6.q, \
-  vectorization_9.q, \
-  vectorization_decimal_date.q, \
-  vectorization_div0.q, \
-  vectorization_nested_udf.q, \
-  vectorization_not.q, \
-  vectorization_part.q, \
-  vectorization_part_project.q, \
-  vectorization_pushdown.q, \
-  vectorization_short_regress.q, \
-  vectorized_case.q, \
-  vectorized_mapjoin.q, \
-  vectorized_math_funcs.q, \
-  vectorized_nested_mapjoin.q, \
-  vectorized_ptf.q, \
-  vectorized_rcfile_columnar.q, \
-  vectorized_shufflejoin.q, \
-  vectorized_string_funcs.q, \
-  vectorized_timestamp_funcs.q, \
-  windowing.q
-
-# Unlike "spark.query.files" above, these tests only run
-# under Spark engine.
-spark.only.query.files=spark_dynamic_partition_pruning.q,\
-  spark_dynamic_partition_pruning_2.q,\
-  spark_vectorized_dynamic_partition_pruning.q
-
-miniSparkOnYarn.query.files=auto_sortmerge_join_16.q,\
-  bucket4.q,\
-  bucket5.q,\
-  bucket6.q,\
-  bucketizedhiveinputformat.q,\
-  bucketmapjoin6.q,\
-  bucketmapjoin7.q,\
-  constprog_partitioner.q,\
-  constprog_semijoin.q,\
-  disable_merge_for_bucketing.q,\
-  empty_dir_in_table.q,\
-  external_table_with_space_in_location_path.q,\
-  file_with_header_footer.q,\
-  gen_udf_example_add10.q,\
-  import_exported_table.q,\
-  index_bitmap3.q,\
-  index_bitmap_auto.q,\
-  infer_bucket_sort_bucketed_table.q,\
-  infer_bucket_sort_map_operators.q,\
-  infer_bucket_sort_merge.q,\
-  infer_bucket_sort_num_buckets.q,\
-  infer_bucket_sort_reducers_power_two.q,\
-  input16_cc.q,\
-  insert_overwrite_directory2.q,\
-  leftsemijoin_mr.q,\
-  list_bucket_dml_10.q,\
-  load_fs2.q,\
-  load_hdfs_file_with_space_in_the_name.q,\
-  orc_merge1.q,\
-  orc_merge2.q,\
-  orc_merge3.q,\
-  orc_merge4.q,\
-  orc_merge5.q,\
-  orc_merge6.q,\
-  orc_merge7.q,\
-  orc_merge8.q,\
-  orc_merge9.q,\
-  orc_merge_diff_fs.q,\
-  orc_merge_incompat1.q,\
-  orc_merge_incompat2.q,\
-  parallel_orderby.q,\
-  quotedid_smb.q,\
-  reduce_deduplicate.q,\
-  remote_script.q,\
-  root_dir_external_table.q,\
-  schemeAuthority.q,\
-  schemeAuthority2.q,\
-  scriptfile1.q,\
-  scriptfile1_win.q,\
-  temp_table_external.q,\
-  truncate_column_buckets.q,\
-  uber_reduce.q,\
-  vector_inner_join.q,\
-  vector_outer_join0.q,\
-  vector_outer_join1.q,\
-  vector_outer_join2.q,\
-  vector_outer_join3.q,\
-  vector_outer_join4.q,\
-  vector_outer_join5.q
-
-# These tests are removed from miniSparkOnYarn.query.files
-#  ql_rewrite_gbtoidx.q,\
-#  ql_rewrite_gbtoidx_cbo_1.q,\
-#  smb_mapjoin_8.q,\
-
-
-spark.query.negative.files=groupby2_map_skew_multi_distinct.q,\
-  groupby2_multi_distinct.q,\
-  groupby3_map_skew_multi_distinct.q,\
-  groupby3_multi_distinct.q,\
-  groupby_grouping_sets7.q

http://git-wip-us.apache.org/repos/asf/hive/blob/bc75e46a/ql/src/java/org/apache/hadoop/hive/ql/Context.java.orig
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java.orig b/ql/src/java/org/apache/hadoop/hive/ql/Context.java.orig
deleted file mode 100644
index 4667f68..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java.orig
+++ /dev/null
@@ -1,829 +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;
-
-import java.io.DataInput;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URI;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.antlr.runtime.TokenRewriteStream;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.common.BlobStorageUtils;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.exec.TaskRunner;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.hooks.WriteEntity;
-import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager.Heartbeater;
-import org.apache.hadoop.hive.ql.lockmgr.HiveLock;
-import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj;
-import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
-import org.apache.hadoop.hive.ql.lockmgr.LockException;
-import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.hive.ql.plan.LoadTableDesc;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Context for Semantic Analyzers. Usage: not reusable - construct a new one for
- * each query should call clear() at end of use to remove temporary folders
- */
-public class Context {
-  private boolean isHDFSCleanup;
-  private Path resFile;
-  private Path resDir;
-  private FileSystem resFs;
-  private static final Logger LOG = LoggerFactory.getLogger("hive.ql.Context");
-  private Path[] resDirPaths;
-  private int resDirFilesNum;
-  boolean initialized;
-  String originalTracker = null;
-  private final CompilationOpContext opContext;
-  private final Map<String, ContentSummary> pathToCS = new ConcurrentHashMap<String, ContentSummary>();
-
-  // scratch path to use for all non-local (ie. hdfs) file system tmp folders
-  private final Path nonLocalScratchPath;
-
-  // scratch directory to use for local file system tmp folders
-  private final String localScratchDir;
-
-  // the permission to scratch directory (local and hdfs)
-  private final String scratchDirPermission;
-
-  // Keeps track of scratch directories created for different scheme/authority
-  private final Map<String, Path> fsScratchDirs = new HashMap<String, Path>();
-
-  private final Configuration conf;
-  protected int pathid = 10000;
-  protected boolean explain = false;
-  protected String cboInfo;
-  protected boolean cboSucceeded;
-  protected boolean explainLogical = false;
-  protected String cmd = "";
-  // number of previous attempts
-  protected int tryCount = 0;
-  private TokenRewriteStream tokenRewriteStream;
-
-  private String executionId;
-
-  // List of Locks for this query
-  protected List<HiveLock> hiveLocks;
-
-  // Transaction manager for this query
-  protected HiveTxnManager hiveTxnManager;
-
-  // Used to track what type of acid operation (insert, update, or delete) we are doing.  Useful
-  // since we want to change where bucket columns are accessed in some operators and
-  // optimizations when doing updates and deletes.
-  private AcidUtils.Operation acidOperation = AcidUtils.Operation.NOT_ACID;
-
-  private boolean needLockMgr;
-
-  private AtomicInteger sequencer = new AtomicInteger();
-
-  private final Map<String, Table> cteTables = new HashMap<String, Table>();
-
-  // Keep track of the mapping from load table desc to the output and the lock
-  private final Map<LoadTableDesc, WriteEntity> loadTableOutputMap =
-      new HashMap<LoadTableDesc, WriteEntity>();
-  private final Map<WriteEntity, List<HiveLockObj>> outputLockObjects =
-      new HashMap<WriteEntity, List<HiveLockObj>>();
-
-  private final String stagingDir;
-
-  private Heartbeater heartbeater;
-  
-  private boolean skipTableMasking;
-
-  public Context(Configuration conf) throws IOException {
-    this(conf, generateExecutionId());
-  }
-
-  /**
-   * Create a Context with a given executionId.  ExecutionId, together with
-   * user name and conf, will determine the temporary directory locations.
-   */
-  public Context(Configuration conf, String executionId)  {
-    this.conf = conf;
-    this.executionId = executionId;
-
-    // local & non-local tmp location is configurable. however it is the same across
-    // all external file systems
-    nonLocalScratchPath = new Path(SessionState.getHDFSSessionPath(conf), executionId);
-    localScratchDir = new Path(SessionState.getLocalSessionPath(conf), executionId).toUri().getPath();
-    scratchDirPermission = HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIRPERMISSION);
-    stagingDir = HiveConf.getVar(conf, HiveConf.ConfVars.STAGINGDIR);
-    opContext = new CompilationOpContext();
-  }
-
-
-  public Map<LoadTableDesc, WriteEntity> getLoadTableOutputMap() {
-    return loadTableOutputMap;
-  }
-
-  public Map<WriteEntity, List<HiveLockObj>> getOutputLockObjects() {
-    return outputLockObjects;
-  }
-
-  /**
-   * Set the context on whether the current query is an explain query.
-   * @param value true if the query is an explain query, false if not
-   */
-  public void setExplain(boolean value) {
-    explain = value;
-  }
-
-  /**
-   * Find whether the current query is an explain query
-   * @return true if the query is an explain query, false if not
-   */
-  public boolean getExplain() {
-    return explain;
-  }
-
-  /**
-   * Find whether the current query is a logical explain query
-   */
-  public boolean getExplainLogical() {
-    return explainLogical;
-  }
-
-  /**
-   * Set the context on whether the current query is a logical
-   * explain query.
-   */
-  public void setExplainLogical(boolean explainLogical) {
-    this.explainLogical = explainLogical;
-  }
-
-  /**
-   * Set the original query command.
-   * @param cmd the original query command string
-   */
-  public void setCmd(String cmd) {
-    this.cmd = cmd;
-  }
-
-  /**
-   * Find the original query command.
-   * @return the original query command string
-   */
-  public String getCmd () {
-    return cmd;
-  }
-
-  /**
-   * Gets a temporary staging directory related to a path.
-   * If a path already contains a staging directory, then returns the current directory; otherwise
-   * create the directory if needed.
-   *
-   * @param inputPath URI of the temporary directory
-   * @param mkdir Create the directory if True.
-   * @return A temporary path.
-   */
-  private Path getStagingDir(Path inputPath, boolean mkdir) {
-    final URI inputPathUri = inputPath.toUri();
-    final String inputPathName = inputPathUri.getPath();
-    final String fileSystem = inputPathUri.getScheme() + ":" + inputPathUri.getAuthority();
-    final FileSystem fs;
-
-    try {
-      fs = inputPath.getFileSystem(conf);
-    } catch (IOException e) {
-      throw new IllegalStateException("Error getting FileSystem for " + inputPath + ": "+ e, e);
-    }
-
-    String stagingPathName;
-    if (inputPathName.indexOf(stagingDir) == -1) {
-      stagingPathName = new Path(inputPathName, stagingDir).toString();
-    } else {
-      stagingPathName = inputPathName.substring(0, inputPathName.indexOf(stagingDir) + stagingDir.length());
-    }
-
-    final String key = fileSystem + "-" + stagingPathName + "-" + TaskRunner.getTaskRunnerID();
-
-    Path dir = fsScratchDirs.get(key);
-    if (dir == null) {
-      // Append task specific info to stagingPathName, instead of creating a sub-directory.
-      // This way we don't have to worry about deleting the stagingPathName separately at
-      // end of query execution.
-      dir = fs.makeQualified(new Path(stagingPathName + "_" + this.executionId + "-" + TaskRunner.getTaskRunnerID()));
-
-      LOG.debug("Created staging dir = " + dir + " for path = " + inputPath);
-
-      if (mkdir) {
-        try {
-          boolean inheritPerms = HiveConf.getBoolVar(conf,
-              HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
-          if (!FileUtils.mkdir(fs, dir, inheritPerms, conf)) {
-            throw new IllegalStateException("Cannot create staging directory  '" + dir.toString() + "'");
-          }
-
-          if (isHDFSCleanup) {
-            fs.deleteOnExit(dir);
-          }
-        } catch (IOException e) {
-          throw new RuntimeException("Cannot create staging directory '" + dir.toString() + "': " + e.getMessage(), e);
-        }
-      }
-
-      fsScratchDirs.put(key, dir);
-    }
-
-    return dir;
-  }
-
-  /**
-   * Get a tmp directory on specified URI
-   *
-   * @param scheme Scheme of the target FS
-   * @param authority Authority of the target FS
-   * @param mkdir create the directory if true
-   * @param scratchDir path of tmp directory
-   */
-  private Path getScratchDir(String scheme, String authority,
-      boolean mkdir, String scratchDir) {
-
-    String fileSystem =  scheme + ":" + authority;
-    Path dir = fsScratchDirs.get(fileSystem + "-" + TaskRunner.getTaskRunnerID());
-
-    if (dir == null) {
-      Path dirPath = new Path(scheme, authority,
-          scratchDir + "-" + TaskRunner.getTaskRunnerID());
-      if (mkdir) {
-        try {
-          FileSystem fs = dirPath.getFileSystem(conf);
-          dirPath = new Path(fs.makeQualified(dirPath).toString());
-          FsPermission fsPermission = new FsPermission(scratchDirPermission);
-
-          if (!fs.mkdirs(dirPath, fsPermission)) {
-            throw new RuntimeException("Cannot make directory: "
-                + dirPath.toString());
-          }
-          if (isHDFSCleanup) {
-            fs.deleteOnExit(dirPath);
-          }
-        } catch (IOException e) {
-          throw new RuntimeException (e);
-        }
-      }
-      dir = dirPath;
-      fsScratchDirs.put(fileSystem + "-" + TaskRunner.getTaskRunnerID(), dir);
-
-    }
-
-    return dir;
-  }
-
-
-  /**
-   * Create a local scratch directory on demand and return it.
-   */
-  public Path getLocalScratchDir(boolean mkdir) {
-    try {
-      FileSystem fs = FileSystem.getLocal(conf);
-      URI uri = fs.getUri();
-      return getScratchDir(uri.getScheme(), uri.getAuthority(),
-          mkdir, localScratchDir);
-    } catch (IOException e) {
-      throw new RuntimeException (e);
-    }
-  }
-
-
-  /**
-   * Create a map-reduce scratch directory on demand and return it.
-   *
-   */
-  public Path getMRScratchDir() {
-
-    // if we are executing entirely on the client side - then
-    // just (re)use the local scratch directory
-    if(isLocalOnlyExecutionMode()) {
-      return getLocalScratchDir(!explain);
-    }
-
-    try {
-      Path dir = FileUtils.makeQualified(nonLocalScratchPath, conf);
-      URI uri = dir.toUri();
-
-      Path newScratchDir = getScratchDir(uri.getScheme(), uri.getAuthority(),
-          !explain, uri.getPath());
-      LOG.info("New scratch dir is " + newScratchDir);
-      return newScratchDir;
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    } catch (IllegalArgumentException e) {
-      throw new RuntimeException("Error while making MR scratch "
-          + "directory - check filesystem config (" + e.getCause() + ")", e);
-    }
-  }
-
-  /**
-   * Create a temporary directory depending of the path specified.
-   * - If path is an Object store filesystem, then use the default MR scratch directory (HDFS)
-   * - If path is on HDFS, then create a staging directory inside the path
-   *
-   * @param path Path used to verify the Filesystem to use for temporary directory
-   * @return A path to the new temporary directory
-     */
-  public Path getTempDirForPath(Path path) {
-    boolean isLocal = isPathLocal(path);
-    if ((BlobStorageUtils.isBlobStoragePath(conf, path) && !BlobStorageUtils.isBlobStorageAsScratchDir(conf))
-        || isLocal) {
-      // For better write performance, we use HDFS for temporary data when object store is used.
-      // Note that the scratch directory configuration variable must use HDFS or any other non-blobstorage system
-      // to take advantage of this performance.
-      return getMRTmpPath();
-    } else {
-      return getExtTmpPathRelTo(path);
-    }
-  }
-
-  /*
-   * Checks if the path is for the local filesystem or not
-   */
-  private boolean isPathLocal(Path path) {
-    boolean isLocal = false;
-    if (path != null) {
-      String scheme = path.toUri().getScheme();
-      if (scheme != null) {
-        isLocal = scheme.equals(Utilities.HADOOP_LOCAL_FS_SCHEME);
-      }
-    }
-    return isLocal;
-  }
-
-  private Path getExternalScratchDir(URI extURI) {
-    return getStagingDir(new Path(extURI.getScheme(), extURI.getAuthority(), extURI.getPath()), !explain);
-  }
-
-  /**
-   * Remove any created scratch directories.
-   */
-  public void removeScratchDir() {
-    for (Map.Entry<String, Path> entry : fsScratchDirs.entrySet()) {
-      try {
-        Path p = entry.getValue();
-        FileSystem fs = p.getFileSystem(conf);
-        fs.delete(p, true);
-        fs.cancelDeleteOnExit(p);
-      } catch (Exception e) {
-        LOG.warn("Error Removing Scratch: "
-            + StringUtils.stringifyException(e));
-      }
-    }
-    fsScratchDirs.clear();
-  }
-
-  /**
-   * Remove any created directories for CTEs.
-   */
-  public void removeMaterializedCTEs() {
-    // clean CTE tables
-    for (Table materializedTable : cteTables.values()) {
-      Path location = materializedTable.getDataLocation();
-      try {
-        FileSystem fs = location.getFileSystem(conf);
-        boolean status = fs.delete(location, true);
-        LOG.info("Removed " + location + " for materialized "
-            + materializedTable.getTableName() + ", status=" + status);
-      } catch (IOException e) {
-        // ignore
-        LOG.warn("Error removing " + location + " for materialized " + materializedTable.getTableName() +
-                ": " + StringUtils.stringifyException(e));
-      }
-    }
-    cteTables.clear();
-  }
-
-  private String nextPathId() {
-    return Integer.toString(pathid++);
-  }
-
-
-  private static final String MR_PREFIX = "-mr-";
-  private static final String EXT_PREFIX = "-ext-";
-  private static final String LOCAL_PREFIX = "-local-";
-
-  /**
-   * Check if path is for intermediate data
-   * @return true if a uri is a temporary uri for map-reduce intermediate data,
-   *         false otherwise
-   */
-  public boolean isMRTmpFileURI(String uriStr) {
-    return (uriStr.indexOf(executionId) != -1) &&
-        (uriStr.indexOf(MR_PREFIX) != -1);
-  }
-
-  public Path getMRTmpPath(URI uri) {
-    return new Path(getStagingDir(new Path(uri), !explain), MR_PREFIX + nextPathId());
-  }
-
-  /**
-   * Get a path to store map-reduce intermediate data in.
-   *
-   * @return next available path for map-red intermediate data
-   */
-  public Path getMRTmpPath() {
-    return new Path(getMRScratchDir(), MR_PREFIX +
-        nextPathId());
-  }
-
-  /**
-   * Get a tmp path on local host to store intermediate data.
-   *
-   * @return next available tmp path on local fs
-   */
-  public Path getLocalTmpPath() {
-    return new Path(getLocalScratchDir(true), LOCAL_PREFIX + nextPathId());
-  }
-
-  /**
-   * Get a path to store tmp data destined for external Path.
-   *
-   * @param path external Path to which the tmp data has to be eventually moved
-   * @return next available tmp path on the file system corresponding extURI
-   */
-  public Path getExternalTmpPath(Path path) {
-    URI extURI = path.toUri();
-    if (extURI.getScheme().equals("viewfs")) {
-      // if we are on viewfs we don't want to use /tmp as tmp dir since rename from /tmp/..
-      // to final /user/hive/warehouse/ will fail later, so instead pick tmp dir
-      // on same namespace as tbl dir.
-      return getExtTmpPathRelTo(path.getParent());
-    }
-    return new Path(getExternalScratchDir(extURI), EXT_PREFIX +
-        nextPathId());
-  }
-
-  /**
-   * This is similar to getExternalTmpPath() with difference being this method returns temp path
-   * within passed in uri, whereas getExternalTmpPath() ignores passed in path and returns temp
-   * path within /tmp
-   */
-  public Path getExtTmpPathRelTo(Path path) {
-    return new Path(getStagingDir(path, !explain), EXT_PREFIX + nextPathId());
-  }
-
-  /**
-   * @return the resFile
-   */
-  public Path getResFile() {
-    return resFile;
-  }
-
-  /**
-   * @param resFile
-   *          the resFile to set
-   */
-  public void setResFile(Path resFile) {
-    this.resFile = resFile;
-    resDir = null;
-    resDirPaths = null;
-    resDirFilesNum = 0;
-  }
-
-  /**
-   * @return the resDir
-   */
-  public Path getResDir() {
-    return resDir;
-  }
-
-  /**
-   * @param resDir
-   *          the resDir to set
-   */
-  public void setResDir(Path resDir) {
-    this.resDir = resDir;
-    resFile = null;
-
-    resDirFilesNum = 0;
-    resDirPaths = null;
-  }
-
-  public void clear() throws IOException {
-    if (resDir != null) {
-      try {
-        FileSystem fs = resDir.getFileSystem(conf);
-        fs.delete(resDir, true);
-      } catch (IOException e) {
-        LOG.info("Context clear error: " + StringUtils.stringifyException(e));
-      }
-    }
-
-    if (resFile != null) {
-      try {
-        FileSystem fs = resFile.getFileSystem(conf);
-        fs.delete(resFile, false);
-      } catch (IOException e) {
-        LOG.info("Context clear error: " + StringUtils.stringifyException(e));
-      }
-    }
-    removeMaterializedCTEs();
-    removeScratchDir();
-    originalTracker = null;
-    setNeedLockMgr(false);
-  }
-
-  public DataInput getStream() {
-    try {
-      if (!initialized) {
-        initialized = true;
-        if ((resFile == null) && (resDir == null)) {
-          return null;
-        }
-
-        if (resFile != null) {
-          return resFile.getFileSystem(conf).open(resFile);
-        }
-
-        resFs = resDir.getFileSystem(conf);
-        FileStatus status = resFs.getFileStatus(resDir);
-        assert status.isDir();
-        FileStatus[] resDirFS = resFs.globStatus(new Path(resDir + "/*"), FileUtils.HIDDEN_FILES_PATH_FILTER);
-        resDirPaths = new Path[resDirFS.length];
-        int pos = 0;
-        for (FileStatus resFS : resDirFS) {
-          if (!resFS.isDir()) {
-            resDirPaths[pos++] = resFS.getPath();
-          }
-        }
-        if (pos == 0) {
-          return null;
-        }
-
-        return resFs.open(resDirPaths[resDirFilesNum++]);
-      } else {
-        return getNextStream();
-      }
-    } catch (FileNotFoundException e) {
-      LOG.info("getStream error: " + StringUtils.stringifyException(e));
-      return null;
-    } catch (IOException e) {
-      LOG.info("getStream error: " + StringUtils.stringifyException(e));
-      return null;
-    }
-  }
-
-  private DataInput getNextStream() {
-    try {
-      if (resDir != null && resDirFilesNum < resDirPaths.length
-          && (resDirPaths[resDirFilesNum] != null)) {
-        return resFs.open(resDirPaths[resDirFilesNum++]);
-      }
-    } catch (FileNotFoundException e) {
-      LOG.info("getNextStream error: " + StringUtils.stringifyException(e));
-      return null;
-    } catch (IOException e) {
-      LOG.info("getNextStream error: " + StringUtils.stringifyException(e));
-      return null;
-    }
-
-    return null;
-  }
-
-  public void resetStream() {
-    if (initialized) {
-      resDirFilesNum = 0;
-      initialized = false;
-    }
-  }
-
-  /**
-   * Little abbreviation for StringUtils.
-   */
-  private static boolean strEquals(String str1, String str2) {
-    return org.apache.commons.lang.StringUtils.equals(str1, str2);
-  }
-
-  /**
-   * Set the token rewrite stream being used to parse the current top-level SQL
-   * statement. Note that this should <b>not</b> be used for other parsing
-   * activities; for example, when we encounter a reference to a view, we switch
-   * to a new stream for parsing the stored view definition from the catalog,
-   * but we don't clobber the top-level stream in the context.
-   *
-   * @param tokenRewriteStream
-   *          the stream being used
-   */
-  public void setTokenRewriteStream(TokenRewriteStream tokenRewriteStream) {
-    assert (this.tokenRewriteStream == null);
-    this.tokenRewriteStream = tokenRewriteStream;
-  }
-
-  /**
-   * @return the token rewrite stream being used to parse the current top-level
-   *         SQL statement, or null if it isn't available (e.g. for parser
-   *         tests)
-   */
-  public TokenRewriteStream getTokenRewriteStream() {
-    return tokenRewriteStream;
-  }
-
-  /**
-   * Generate a unique executionId.  An executionId, together with user name and
-   * the configuration, will determine the temporary locations of all intermediate
-   * files.
-   *
-   * In the future, users can use the executionId to resume a query.
-   */
-  public static String generateExecutionId() {
-    Random rand = new Random();
-    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss_SSS");
-    String executionId = "hive_" + format.format(new Date()) + "_"
-        + Math.abs(rand.nextLong());
-    return executionId;
-  }
-
-  /**
-   * Does Hive wants to run tasks entirely on the local machine
-   * (where the query is being compiled)?
-   *
-   * Today this translates into running hadoop jobs locally
-   */
-  public boolean isLocalOnlyExecutionMode() {
-    // Always allow spark to run in a cluster mode. Without this, depending on
-    // user's local hadoop settings, true may be returned, which causes plan to be
-    // stored in local path.
-    if (HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) {
-      return false;
-    }
-
-    return ShimLoader.getHadoopShims().isLocalMode(conf);
-  }
-
-  public List<HiveLock> getHiveLocks() {
-    return hiveLocks;
-  }
-
-  public void setHiveLocks(List<HiveLock> hiveLocks) {
-    this.hiveLocks = hiveLocks;
-  }
-
-  public HiveTxnManager getHiveTxnManager() {
-    return hiveTxnManager;
-  }
-
-  public void setHiveTxnManager(HiveTxnManager txnMgr) {
-    hiveTxnManager = txnMgr;
-  }
-
-  public void setOriginalTracker(String originalTracker) {
-    this.originalTracker = originalTracker;
-  }
-
-  public void restoreOriginalTracker() {
-    if (originalTracker != null) {
-      ShimLoader.getHadoopShims().setJobLauncherRpcAddress(conf, originalTracker);
-      originalTracker = null;
-    }
-  }
-
-  public void addCS(String path, ContentSummary cs) {
-    pathToCS.put(path, cs);
-  }
-
-  public ContentSummary getCS(Path path) {
-    return getCS(path.toString());
-  }
-
-  public ContentSummary getCS(String path) {
-    return pathToCS.get(path);
-  }
-
-  public Map<String, ContentSummary> getPathToCS() {
-    return pathToCS;
-  }
-
-  public Configuration getConf() {
-    return conf;
-  }
-
-  /**
-   * @return the isHDFSCleanup
-   */
-  public boolean isHDFSCleanup() {
-    return isHDFSCleanup;
-  }
-
-  /**
-   * @param isHDFSCleanup the isHDFSCleanup to set
-   */
-  public void setHDFSCleanup(boolean isHDFSCleanup) {
-    this.isHDFSCleanup = isHDFSCleanup;
-  }
-
-  public boolean isNeedLockMgr() {
-    return needLockMgr;
-  }
-
-  public void setNeedLockMgr(boolean needLockMgr) {
-    this.needLockMgr = needLockMgr;
-  }
-
-  public int getTryCount() {
-    return tryCount;
-  }
-
-  public void setTryCount(int tryCount) {
-    this.tryCount = tryCount;
-  }
-
-  public void setAcidOperation(AcidUtils.Operation op) {
-    acidOperation = op;
-  }
-
-  public AcidUtils.Operation getAcidOperation() {
-    return acidOperation;
-  }
-
-  public String getCboInfo() {
-    return cboInfo;
-  }
-
-  public void setCboInfo(String cboInfo) {
-    this.cboInfo = cboInfo;
-  }
-
-  public boolean isCboSucceeded() {
-    return cboSucceeded;
-  }
-
-  public void setCboSucceeded(boolean cboSucceeded) {
-    this.cboSucceeded = cboSucceeded;
-  }
-
-  public Table getMaterializedTable(String cteName) {
-    return cteTables.get(cteName);
-  }
-
-  public void addMaterializedTable(String cteName, Table table) {
-    cteTables.put(cteName, table);
-  }
-
-  public AtomicInteger getSequencer() {
-    return sequencer;
-  }
-
-  public CompilationOpContext getOpContext() {
-    return opContext;
-  }
-
-  public Heartbeater getHeartbeater() {
-    return heartbeater;
-  }
-
-  public void setHeartbeater(Heartbeater heartbeater) {
-    this.heartbeater = heartbeater;
-  }
-
-  public void checkHeartbeaterLockException() throws LockException {
-    if (getHeartbeater() != null && getHeartbeater().getLockException() != null) {
-      throw getHeartbeater().getLockException();
-    }
-  }
-
-  public boolean isSkipTableMasking() {
-    return skipTableMasking;
-  }
-
-  public void setSkipTableMasking(boolean skipTableMasking) {
-    this.skipTableMasking = skipTableMasking;
-  }
-}


[31/31] hive git commit: HIVE-14644 : use metastore information on the read path appropriately (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14644 : use metastore information on the read path appropriately (Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: 3e481b4719f49a5c15ce1d745b05392f5d0f2627
Parents: 815e069
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Sep 12 13:20:59 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Sep 12 13:20:59 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/common/ValidWriteIds.java       |  158 +
 itests/pom.xml                                  |   28 -
 itests/qtest-spark/pom.xml                      |   54 +-
 metastore/if/hive_metastore.thrift              |   12 +
 .../upgrade/derby/037-HIVE-14637.derby.sql      |    4 +-
 .../upgrade/derby/hive-schema-2.2.0.derby.sql   |    2 +-
 .../upgrade/mssql/022-HIVE-14637.mssql.sql      |    4 +-
 .../upgrade/mssql/hive-schema-2.2.0.mssql.sql   |    4 +-
 .../upgrade/mysql/037-HIVE-14637.mysql.sql      |    4 +-
 .../upgrade/mysql/hive-schema-2.2.0.mysql.sql   |    4 +-
 .../upgrade/oracle/037-HIVE-14637.oracle.sql    |    4 +-
 .../upgrade/oracle/hive-schema-2.2.0.oracle.sql |    4 +-
 .../postgres/036-HIVE-14637.postgres.sql        |    4 +-
 .../postgres/hive-schema-2.2.0.postgres.sql     |    4 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2403 +++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  126 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  533 ++-
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  115 +
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../metastore/api/GetValidWriteIdsRequest.java  |  490 ++
 .../metastore/api/GetValidWriteIdsResult.java   |  740 +++
 .../hive/metastore/api/ThriftHiveMetastore.java | 4242 +++++++++++-------
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1448 +++---
 .../src/gen/thrift/gen-php/metastore/Types.php  |  288 +-
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       | 1379 +++---
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  212 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   44 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   54 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |  167 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    6 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    3 +
 .../hive/metastore/MetaStoreDirectSql.java      |    9 +-
 .../hadoop/hive/metastore/ObjectStore.java      |  128 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |   11 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   27 +-
 metastore/src/model/package.jdo                 |   16 +-
 .../DummyRawStoreControlledCommit.java          |   12 +
 .../DummyRawStoreForJdoConnection.java          |   13 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   38 +
 .../hadoop/hive/ql/exec/FetchOperator.java      |    5 +
 .../apache/hadoop/hive/ql/exec/FetchTask.java   |    5 +
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |   49 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |   14 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |   52 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |    4 -
 .../hadoop/hive/ql/io/HiveInputFormat.java      |   59 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   94 +-
 .../hive/ql/optimizer/GenMapRedUtils.java       |    2 +-
 .../physical/AnnotateRunTimeStatsOptimizer.java |    3 +-
 .../hadoop/hive/ql/parse/TaskCompiler.java      |    2 +-
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |    2 +
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |   14 -
 .../apache/hadoop/hive/ql/plan/TableDesc.java   |    3 +-
 ql/src/test/queries/clientpositive/mm_current.q |    3 +-
 .../clientpositive/llap/mm_current.q.out        |    8 +-
 57 files changed, 8788 insertions(+), 4373 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/common/src/java/org/apache/hadoop/hive/common/ValidWriteIds.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/ValidWriteIds.java b/common/src/java/org/apache/hadoop/hive/common/ValidWriteIds.java
new file mode 100644
index 0000000..b25a72d
--- /dev/null
+++ b/common/src/java/org/apache/hadoop/hive/common/ValidWriteIds.java
@@ -0,0 +1,158 @@
+/**
+ * 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.common;
+
+import java.util.Arrays;
+import java.util.HashSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ValidWriteIds {
+  public static final ValidWriteIds NO_WRITE_IDS = new ValidWriteIds(-1, -1, false, null);
+
+  private static final String MM_PREFIX = "mm";
+
+  private final static Logger LOG = LoggerFactory.getLogger(ValidWriteIds.class);
+
+  private static final String VALID_WRITEIDS_PREFIX = "hive.valid.write.ids.";
+  private final long lowWatermark, highWatermark;
+  private final boolean areIdsValid;
+  private final HashSet<Long> ids;
+  private String source = null;
+
+  public ValidWriteIds(
+      long lowWatermark, long highWatermark, boolean areIdsValid, HashSet<Long> ids) {
+    this.lowWatermark = lowWatermark;
+    this.highWatermark = highWatermark;
+    this.areIdsValid = areIdsValid;
+    this.ids = ids;
+  }
+
+  public static ValidWriteIds createFromConf(Configuration conf, String dbName, String tblName) {
+    return createFromConf(conf, dbName + "." + tblName);
+  }
+
+  public static ValidWriteIds createFromConf(Configuration conf, String fullTblName) {
+    String idStr = conf.get(createConfKey(fullTblName), null);
+    if (idStr == null || idStr.isEmpty()) return null;
+    return new ValidWriteIds(idStr);
+  }
+
+  private static String createConfKey(String dbName, String tblName) {
+    return createConfKey(dbName + "." + tblName);
+  }
+
+  private static String createConfKey(String fullName) {
+    return VALID_WRITEIDS_PREFIX + fullName;
+  }
+
+  private ValidWriteIds(String src) {
+    // TODO: lifted from ACID config implementation... optimize if needed? e.g. ranges, base64
+    String[] values = src.split(":");
+    highWatermark = Long.parseLong(values[0]);
+    lowWatermark = Long.parseLong(values[1]);
+    if (values.length > 2) {
+      areIdsValid = Long.parseLong(values[2]) > 0;
+      ids = new HashSet<Long>();
+      for(int i = 3; i < values.length; ++i) {
+        ids.add(Long.parseLong(values[i]));
+      }
+    } else {
+      areIdsValid = false;
+      ids = null;
+    }
+  }
+
+  public void addToConf(Configuration conf, String dbName, String tblName) {
+    if (source == null) {
+      source = toString();
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Setting " + createConfKey(dbName, tblName) + " => " + source);
+    }
+    conf.set(createConfKey(dbName, tblName), source);
+  }
+
+  public String toString() {
+    // TODO: lifted from ACID config implementation... optimize if needed? e.g. ranges, base64
+    StringBuilder buf = new StringBuilder();
+    buf.append(highWatermark);
+    buf.append(':');
+    buf.append(lowWatermark);
+    if (ids != null) {
+      buf.append(':');
+      buf.append(areIdsValid ? 1 : 0);
+      for (long id : ids) {
+        buf.append(':');
+        buf.append(id);
+      }
+    }
+    return buf.toString();
+  }
+
+  public boolean isValid(long writeId) {
+    if (writeId < 0) throw new RuntimeException("Incorrect write ID " + writeId);
+    if (writeId <= lowWatermark) return true;
+    if (writeId >= highWatermark) return false;
+    return ids != null && (areIdsValid == ids.contains(writeId));
+  }
+
+  public boolean isValidInput(Path file) {
+    String fileName = file.getName();
+    String[] parts = fileName.split("_", 3);
+    if (parts.length < 2 || !MM_PREFIX.equals(parts[0])) {
+      LOG.info("Ignoring unknown file for a MM table: " + file
+          + " (" + Arrays.toString(parts) + ")");
+      return false;
+    }
+    long writeId = -1;
+    try {
+      writeId = Long.parseLong(parts[1]);
+    } catch (NumberFormatException ex) {
+      LOG.info("Ignoring unknown file for a MM table: " + file
+          + "; parsing " + parts[1] + " got " + ex.getMessage());
+      return false;
+    }
+    return isValid(writeId);
+  }
+
+  public static String getMmFilePrefix(long mmWriteId) {
+    return MM_PREFIX + "_" + mmWriteId;
+  }
+
+
+  public static class IdPathFilter implements PathFilter {
+    private final String prefix, tmpPrefix;
+    private final boolean isMatch;
+    public IdPathFilter(long writeId, boolean isMatch) {
+      this.prefix = ValidWriteIds.getMmFilePrefix(writeId);
+      this.tmpPrefix = "_tmp." + prefix;
+      this.isMatch = isMatch;
+    }
+
+    @Override
+    public boolean accept(Path path) {
+      String name = path.getName();
+      return isMatch == (name.startsWith(prefix) || name.startsWith(tmpPrefix));
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/itests/pom.xml
----------------------------------------------------------------------
diff --git a/itests/pom.xml b/itests/pom.xml
index a452db3..8b5b6b7 100644
--- a/itests/pom.xml
+++ b/itests/pom.xml
@@ -72,34 +72,6 @@
                   <target>
                     <echo file="target/download.sh">
                       set -x
-                      /bin/pwd
-                      BASE_DIR=./target
-                      HIVE_ROOT=$BASE_DIR/../../../
-                      DOWNLOAD_DIR=./../thirdparty
-                      download() {
-                        url=$1;
-                        finalName=$2
-                        tarName=$(basename $url)
-                        rm -rf $BASE_DIR/$finalName
-                        if [[ ! -f $DOWNLOAD_DIR/$tarName ]]
-                        then
-                         curl -Sso $DOWNLOAD_DIR/$tarName $url
-                        else
-                          local md5File="$tarName".md5sum
-                          curl -Sso $DOWNLOAD_DIR/$md5File "$url".md5sum
-                          cd $DOWNLOAD_DIR
-                          if ! md5sum -c $md5File; then
-                            curl -Sso $DOWNLOAD_DIR/$tarName $url || return 1
-                          fi
-
-                          cd -
-                        fi
-                        tar -zxf $DOWNLOAD_DIR/$tarName -C $BASE_DIR
-                        mv $BASE_DIR/spark-${spark.version}-bin-hadoop2-without-hive $BASE_DIR/$finalName
-                      }
-                      mkdir -p $DOWNLOAD_DIR
-                      download "http://d3jw87u4immizc.cloudfront.net/spark-tarball/spark-${spark.version}-bin-hadoop2-without-hive.tgz" "spark"
-                      cp -f $HIVE_ROOT/data/conf/spark/log4j2.properties $BASE_DIR/spark/conf/
                     </echo>
                   </target>
                 </configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/itests/qtest-spark/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest-spark/pom.xml b/itests/qtest-spark/pom.xml
index 1e6c3a2..07282fb 100644
--- a/itests/qtest-spark/pom.xml
+++ b/itests/qtest-spark/pom.xml
@@ -348,6 +348,38 @@
         </executions>
       </plugin>
       <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>${maven.build-helper.plugin.version}</version>
+        <executions>
+          <execution>
+            <id>add-test-sources</id>
+            <phase>generate-test-sources</phase>
+            <goals>
+              <goal>add-test-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>target/generated-test-sources/java</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+<profiles>
+  <profile>
+    <id>spark-test</id>
+    <activation>
+      <property>
+        <name>!skipSparkTests</name>
+      </property>
+    </activation>
+  <build>
+    <plugins>
+      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-antrun-plugin</artifactId>
         <executions>
@@ -388,26 +420,8 @@
           </execution>
         </executions>
       </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>build-helper-maven-plugin</artifactId>
-        <version>${maven.build-helper.plugin.version}</version>
-        <executions>
-          <execution>
-            <id>add-test-sources</id>
-            <phase>generate-test-sources</phase>
-            <goals>
-              <goal>add-test-source</goal>
-            </goals>
-            <configuration>
-              <sources>
-                <source>target/generated-test-sources/java</source>
-              </sources>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
     </plugins>
   </build>
-
+</profile>
+</profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 95eee27..0101eab 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -918,6 +918,17 @@ struct HeartbeatWriteIdRequest {
 struct HeartbeatWriteIdResult {
 }
 
+struct GetValidWriteIdsRequest {
+  1: required string dbName,
+  2: required string tblName
+}
+struct GetValidWriteIdsResult {
+  1: required i64 lowWatermarkId,
+  2: required i64 highWatermarkId,
+  3: optional bool areIdsValid,
+  4: optional list<i64> ids
+}
+
 
 struct GetAllFunctionsResponse {
   1: optional list<Function> functions
@@ -1470,6 +1481,7 @@ service ThriftHiveMetastore extends fb303.FacebookService
   GetNextWriteIdResult get_next_write_id(1:GetNextWriteIdRequest req)
   FinalizeWriteIdResult finalize_write_id(1:FinalizeWriteIdRequest req)
   HeartbeatWriteIdResult heartbeat_write_id(1:HeartbeatWriteIdRequest req)
+  GetValidWriteIdsResult get_valid_write_ids(1:GetValidWriteIdsRequest req)
 }
 
 // * Note about the DDL_TIME: When creating or altering a table or a partition,

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/scripts/upgrade/derby/037-HIVE-14637.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/037-HIVE-14637.derby.sql b/metastore/scripts/upgrade/derby/037-HIVE-14637.derby.sql
index 8cea9f1..88a48f0 100644
--- a/metastore/scripts/upgrade/derby/037-HIVE-14637.derby.sql
+++ b/metastore/scripts/upgrade/derby/037-HIVE-14637.derby.sql
@@ -1,5 +1,5 @@
-ALTER TABLE "TBLS" ADD "MM_WATERMARK_WRITE_ID" BIGINT;
-ALTER TABLE "TBLS" ADD "MM_NEXT_WRITE_ID" BIGINT;
+ALTER TABLE "TBLS" ADD "MM_WATERMARK_WRITE_ID" BIGINT DEFAULT -1;
+ALTER TABLE "TBLS" ADD "MM_NEXT_WRITE_ID" BIGINT DEFAULT 0;
 CREATE TABLE "APP"."TBL_WRITES" ("TW_ID" BIGINT NOT NULL, "TBL_ID" BIGINT NOT NULL, "WRITE_ID" BIGINT NOT NULL, "STATE" CHAR(1) NOT NULL, "LAST_HEARTBEAT" BIGINT);
 ALTER TABLE "APP"."TBL_WRITES" ADD CONSTRAINT "TBL_WRITES_PK" PRIMARY KEY ("TW_ID");
 ALTER TABLE "APP"."TBL_WRITES" ADD CONSTRAINT "TBL_WRITES_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql b/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
index 3c4ba4b..f86ee4a 100644
--- a/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
@@ -60,7 +60,7 @@ CREATE TABLE "APP"."COLUMNS" ("SD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "
 
 CREATE TABLE "APP"."ROLES" ("ROLE_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "ROLE_NAME" VARCHAR(128));
 
-CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(128), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR, "MM_WATERMARK_WRITE_ID" BIGINT, "MM_NEXT_WRITE_ID" BIGINT);
+CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(128), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR, "MM_WATERMARK_WRITE_ID" BIGINT DEFAULT -1, "MM_NEXT_WRITE_ID" BIGINT DEFAULT 0);
 
 CREATE TABLE "APP"."PARTITION_KEYS" ("TBL_ID" BIGINT NOT NULL, "PKEY_COMMENT" VARCHAR(4000), "PKEY_NAME" VARCHAR(128) NOT NULL, "PKEY_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/scripts/upgrade/mssql/022-HIVE-14637.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/022-HIVE-14637.mssql.sql b/metastore/scripts/upgrade/mssql/022-HIVE-14637.mssql.sql
index bb42935..5d6f99f 100644
--- a/metastore/scripts/upgrade/mssql/022-HIVE-14637.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/022-HIVE-14637.mssql.sql
@@ -1,5 +1,5 @@
-ALTER TABLE TBLS ADD MM_WATERMARK_WRITE_ID BIGINT NULL;
-ALTER TABLE TBLS ADD MM_NEXT_WRITE_ID BIGINT NULL;
+ALTER TABLE TBLS ADD MM_WATERMARK_WRITE_ID BIGINT DEFAULT -1;
+ALTER TABLE TBLS ADD MM_NEXT_WRITE_ID BIGINT DEFAULT 0;
 
 CREATE TABLE TBL_WRITES 
 (

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
index 6bd0d87..26b2ab3 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
@@ -359,8 +359,8 @@ CREATE TABLE TBLS
     TBL_TYPE nvarchar(128) NULL,
     VIEW_EXPANDED_TEXT text NULL,
     VIEW_ORIGINAL_TEXT text NULL,
-    MM_WATERMARK_WRITE_ID BIGINT NULL,
-    MM_NEXT_WRITE_ID BIGINT NULL
+    MM_WATERMARK_WRITE_ID BIGINT NULL DEFAULT -1,
+    MM_NEXT_WRITE_ID BIGINT NULL DEFAULT 0
 );
 
 ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/scripts/upgrade/mysql/037-HIVE-14637.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/037-HIVE-14637.mysql.sql b/metastore/scripts/upgrade/mysql/037-HIVE-14637.mysql.sql
index 1b740d5..c024584 100644
--- a/metastore/scripts/upgrade/mysql/037-HIVE-14637.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/037-HIVE-14637.mysql.sql
@@ -1,5 +1,5 @@
-alter table `TBLS` ADD COLUMN `MM_WATERMARK_WRITE_ID` bigint(20);
-alter table `TBLS` ADD COLUMN `MM_NEXT_WRITE_ID` bigint(20);
+alter table `TBLS` ADD COLUMN `MM_WATERMARK_WRITE_ID` bigint(20) DEFAULT -1;
+alter table `TBLS` ADD COLUMN `MM_NEXT_WRITE_ID` bigint(20) DEFAULT 0;
 
 CREATE TABLE IF NOT EXISTS `TBL_WRITES`
 (

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
index f7ef948..b295950 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
@@ -587,8 +587,8 @@ CREATE TABLE IF NOT EXISTS `TBLS` (
   `TBL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `VIEW_EXPANDED_TEXT` mediumtext,
   `VIEW_ORIGINAL_TEXT` mediumtext,
-  `MM_WATERMARK_WRITE_ID` bigint(20),
-  `MM_NEXT_WRITE_ID` bigint(20),
+  `MM_WATERMARK_WRITE_ID` bigint(20) DEFAULT -1,
+  `MM_NEXT_WRITE_ID` bigint(20) DEFAULT 0,
   PRIMARY KEY (`TBL_ID`),
   UNIQUE KEY `UNIQUETABLE` (`TBL_NAME`,`DB_ID`),
   KEY `TBLS_N50` (`SD_ID`),

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/scripts/upgrade/oracle/037-HIVE-14637.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/037-HIVE-14637.oracle.sql b/metastore/scripts/upgrade/oracle/037-HIVE-14637.oracle.sql
index bc5fb6b..9f6dbb2 100644
--- a/metastore/scripts/upgrade/oracle/037-HIVE-14637.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/037-HIVE-14637.oracle.sql
@@ -1,5 +1,5 @@
-ALTER TABLE TBLS ADD MM_WATERMARK_WRITE_ID NUMBER;
-ALTER TABLE TBLS ADD MM_NEXT_WRITE_ID NUMBER;
+ALTER TABLE TBLS ADD MM_WATERMARK_WRITE_ID NUMBER DEFAULT -1;
+ALTER TABLE TBLS ADD MM_NEXT_WRITE_ID NUMBER DEFAULT 0;
 
 CREATE TABLE TBL_WRITES
 (

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
index 503ce09..6972c20 100644
--- a/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
@@ -376,8 +376,8 @@ CREATE TABLE TBLS
     TBL_TYPE VARCHAR2(128) NULL,
     VIEW_EXPANDED_TEXT CLOB NULL,
     VIEW_ORIGINAL_TEXT CLOB NULL,
-    MM_WATERMARK_WRITE_ID NUMBER NULL,
-    MM_NEXT_WRITE_ID NUMBER NULL
+    MM_WATERMARK_WRITE_ID NUMBER DEFAULT -1,
+    MM_NEXT_WRITE_ID NUMBER DEFAULT 0
 );
 
 ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/scripts/upgrade/postgres/036-HIVE-14637.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/036-HIVE-14637.postgres.sql b/metastore/scripts/upgrade/postgres/036-HIVE-14637.postgres.sql
index d94c19d..f153837 100644
--- a/metastore/scripts/upgrade/postgres/036-HIVE-14637.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/036-HIVE-14637.postgres.sql
@@ -1,6 +1,6 @@
 
-ALTER TABLE "TBLS" ADD COLUMN "MM_WATERMARK_WRITE_ID" bigint;
-ALTER TABLE "TBLS" ADD COLUMN "MM_NEXT_WRITE_ID" bigint;
+ALTER TABLE "TBLS" ADD COLUMN "MM_WATERMARK_WRITE_ID" bigint DEFAULT -1;
+ALTER TABLE "TBLS" ADD COLUMN "MM_NEXT_WRITE_ID" bigint DEFAULT 0;
 
 CREATE TABLE "TBL_WRITES"
 (

http://git-wip-us.apache.org/repos/asf/hive/blob/3e481b47/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
index bf1d769..de997d3 100644
--- a/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
@@ -373,8 +373,8 @@ CREATE TABLE "TBLS" (
     "TBL_TYPE" character varying(128) DEFAULT NULL::character varying,
     "VIEW_EXPANDED_TEXT" text,
     "VIEW_ORIGINAL_TEXT" text,
-    "MM_WATERMARK_WRITE_ID" bigint,
-    "MM_NEXT_WRITE_ID" bigint
+    "MM_WATERMARK_WRITE_ID" bigint DEFAULT -1,
+    "MM_NEXT_WRITE_ID" bigint DEFAULT 0
 );
 
 


[04/31] hive git commit: HIVE-14217: Druid integration (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/explain_ddl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_ddl.q.out b/ql/src/test/results/clientpositive/explain_ddl.q.out
index fa73d99..e8438a1 100644
--- a/ql/src/test/results/clientpositive/explain_ddl.q.out
+++ b/ql/src/test/results/clientpositive/explain_ddl.q.out
@@ -251,6 +251,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: string), value (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/explain_logical.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_logical.q.out b/ql/src/test/results/clientpositive/explain_logical.q.out
index cb01724..4a25a38 100644
--- a/ql/src/test/results/clientpositive/explain_logical.q.out
+++ b/ql/src/test/results/clientpositive/explain_logical.q.out
@@ -263,6 +263,8 @@ LOGICAL PLAN:
 src 
   TableScan (TS_0)
     alias: src
+    properties:
+      insideView TRUE
     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
     Select Operator (SEL_1)
       expressions: key (type: string), value (type: string)
@@ -278,6 +280,8 @@ LOGICAL PLAN:
 srcpart 
   TableScan (TS_0)
     alias: srcpart
+    properties:
+      insideView TRUE
     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
     Select Operator (SEL_2)
       expressions: ds (type: string), key (type: string), value (type: string)
@@ -293,6 +297,8 @@ LOGICAL PLAN:
 $hdt$_0:srcpart 
   TableScan (TS_0)
     alias: srcpart
+    properties:
+      insideView TRUE
     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
     Filter Operator (FIL_11)
       predicate: key is not null (type: boolean)
@@ -328,6 +334,8 @@ $hdt$_0:srcpart
 $hdt$_1:src2 
   TableScan (TS_3)
     alias: src2
+    properties:
+      insideView TRUE
     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
     Filter Operator (FIL_12)
       predicate: key is not null (type: boolean)
@@ -359,6 +367,8 @@ LOGICAL PLAN:
 $hdt$_0:src 
   TableScan (TS_0)
     alias: src
+    properties:
+      insideView TRUE
     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
     Filter Operator (FIL_15)
       predicate: key is not null (type: boolean)
@@ -396,6 +406,8 @@ $hdt$_0:src
 $hdt$_1:srcpart 
   TableScan (TS_3)
     alias: srcpart
+    properties:
+      insideView TRUE
     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
     Filter Operator (FIL_16)
       predicate: key is not null (type: boolean)
@@ -423,6 +435,8 @@ $hdt$_1:srcpart
 $hdt$_2:src3 
   TableScan (TS_6)
     alias: src3
+    properties:
+      insideView TRUE
     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
     Filter Operator (FIL_17)
       predicate: key is not null (type: boolean)
@@ -470,6 +484,8 @@ LOGICAL PLAN:
 srcpart 
   TableScan (TS_0)
     alias: srcpart
+    properties:
+      insideView TRUE
     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
     Filter Operator (FIL_4)
       predicate: (ds = '10') (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/join_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join_view.q.out b/ql/src/test/results/clientpositive/join_view.q.out
index 84a5cb8..b1bfb00 100644
--- a/ql/src/test/results/clientpositive/join_view.q.out
+++ b/ql/src/test/results/clientpositive/join_view.q.out
@@ -53,6 +53,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: invites
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (ds = '2011-09-01') (type: boolean)
@@ -67,6 +69,8 @@ STAGE PLANS:
                   value expressions: _col0 (type: string)
           TableScan
             alias: invites2
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (ds = '2011-09-01') (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
index be09d28..ef01018 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -1951,7 +1951,7 @@ Stage-0
               Filter Operator [FIL_15] (rows=166 width=178)
                 predicate:((value > 'val_9') and key is not null)
                 TableScan [TS_0] (rows=500 width=178)
-                  default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                  default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"],properties:{"insideView":"TRUE"}
         <-Map 3 [SIMPLE_EDGE] llap
           SHUFFLE [RS_9]
             PartitionCols:_col0, _col1

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/masking_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/masking_2.q.out b/ql/src/test/results/clientpositive/masking_2.q.out
index ff045a9..4900a5a 100644
--- a/ql/src/test/results/clientpositive/masking_2.q.out
+++ b/ql/src/test/results/clientpositive/masking_2.q.out
@@ -22,6 +22,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -74,6 +76,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)
@@ -139,6 +143,8 @@ STAGE PLANS:
                   value expressions: _col1 (type: string)
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0) and reverse(value) is not null) (type: boolean)
@@ -190,6 +196,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -206,6 +214,8 @@ STAGE PLANS:
                   value expressions: _col0 (type: int), _col1 (type: string)
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0) and reverse(value) is not null) (type: boolean)
@@ -257,6 +267,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -279,6 +291,8 @@ STAGE PLANS:
                       Statistics: Num rows: 110 Data size: 1167 Basic stats: COMPLETE Column stats: NONE
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/masking_6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/masking_6.q.out b/ql/src/test/results/clientpositive/masking_6.q.out
index fb8c90f..56e4006 100644
--- a/ql/src/test/results/clientpositive/masking_6.q.out
+++ b/ql/src/test/results/clientpositive/masking_6.q.out
@@ -26,6 +26,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -78,6 +80,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)
@@ -149,6 +153,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -201,6 +207,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/masking_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/masking_7.q.out b/ql/src/test/results/clientpositive/masking_7.q.out
index 1fa9311..c4531f8 100644
--- a/ql/src/test/results/clientpositive/masking_7.q.out
+++ b/ql/src/test/results/clientpositive/masking_7.q.out
@@ -26,6 +26,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -78,6 +80,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)
@@ -149,6 +153,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -201,6 +207,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/serde_user_properties.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/serde_user_properties.q.out b/ql/src/test/results/clientpositive/serde_user_properties.q.out
index d5b81ed..c671d33 100644
--- a/ql/src/test/results/clientpositive/serde_user_properties.q.out
+++ b/ql/src/test/results/clientpositive/serde_user_properties.q.out
@@ -184,6 +184,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              user.defined.key some.value
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             GatherStats: false
             Select Operator
@@ -310,6 +312,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: a
+            properties:
+              user.defined.key some.value
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             GatherStats: false
             Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/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 953c98a..18c7b0e 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -87,6 +87,14 @@ field
 find_in_set
 first_value
 floor
+floor_day
+floor_hour
+floor_minute
+floor_month
+floor_quarter
+floor_second
+floor_week
+floor_year
 format_number
 from_unixtime
 from_utc_timestamp
@@ -286,6 +294,7 @@ e
 encode
 explode
 first_value
+floor_minute
 from_unixtime
 in_file
 inline

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/spark/join_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join_view.q.out b/ql/src/test/results/clientpositive/spark/join_view.q.out
index 8a8aa0a..fc78988 100644
--- a/ql/src/test/results/clientpositive/spark/join_view.q.out
+++ b/ql/src/test/results/clientpositive/spark/join_view.q.out
@@ -58,6 +58,8 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: invites
+                  properties:
+                    insideView TRUE
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Filter Operator
                     predicate: (ds = '2011-09-01') (type: boolean)
@@ -74,6 +76,8 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: invites2
+                  properties:
+                    insideView TRUE
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Filter Operator
                     predicate: (ds = '2011-09-01') (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/subquery_notin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_notin.q.out b/ql/src/test/results/clientpositive/subquery_notin.q.out
index 1c6b545..c184806 100644
--- a/ql/src/test/results/clientpositive/subquery_notin.q.out
+++ b/ql/src/test/results/clientpositive/subquery_notin.q.out
@@ -1473,6 +1473,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: ((key < '11') and CASE WHEN ((key > '104')) THEN (true) ELSE (key is null) END) (type: boolean)
@@ -1511,6 +1513,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key < '11') (type: boolean)
@@ -1554,6 +1558,8 @@ STAGE PLANS:
               Statistics: Num rows: 166 Data size: 3257 Basic stats: COMPLETE Column stats: NONE
           TableScan
             alias: src
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: ((key < '11') and CASE WHEN ((key > '104')) THEN (null) ELSE ((key < '11')) END) (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/subquery_views.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_views.q.out b/ql/src/test/results/clientpositive/subquery_views.q.out
index 046f0fe..610bf24 100644
--- a/ql/src/test/results/clientpositive/subquery_views.q.out
+++ b/ql/src/test/results/clientpositive/subquery_views.q.out
@@ -175,6 +175,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: b
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key < '11') (type: boolean)
@@ -330,6 +332,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: b
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key < '11') (type: boolean)


[03/31] hive git commit: HIVE-14710 : unify DB product type treatment in directsql and txnhandler (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
HIVE-14710 : unify DB product type treatment in directsql and txnhandler (Sergey Shelukhin, 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/8cec20d9
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8cec20d9
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8cec20d9

Branch: refs/heads/hive-14535
Commit: 8cec20d97c8c61e41d38b52f435c7ed55b1d1761
Parents: cd6c3cd
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Sep 7 19:03:29 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed Sep 7 19:03:29 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/DatabaseProduct.java  | 75 ++++++++++++++++++++
 .../hive/metastore/MetaStoreDirectSql.java      | 58 ++++++---------
 .../hadoop/hive/metastore/txn/TxnHandler.java   | 52 ++++----------
 .../hadoop/hive/metastore/txn/TestTxnUtils.java |  7 +-
 4 files changed, 112 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8cec20d9/metastore/src/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java b/metastore/src/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
new file mode 100644
index 0000000..33abbb2
--- /dev/null
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import java.sql.SQLException;
+import java.sql.SQLTransactionRollbackException;
+
+/** Database product infered via JDBC. */
+public enum DatabaseProduct {
+  DERBY, MYSQL, POSTGRES, ORACLE, SQLSERVER, OTHER;
+
+  /**
+   * Determine the database product type
+   * @param conn database connection
+   * @return database product type
+   */
+  public static DatabaseProduct determineDatabaseProduct(String productName) throws SQLException {
+    if (productName == null) {
+      return OTHER;
+    }
+    productName = productName.toLowerCase();
+    if (productName.contains("derby")) {
+      return DERBY;
+    } else if (productName.contains("microsoft sql server")) {
+      return SQLSERVER;
+    } else if (productName.contains("mysql")) {
+      return MYSQL;
+    } else if (productName.contains("oracle")) {
+      return ORACLE;
+    } else if (productName.contains("postgresql")) {
+      return POSTGRES;
+    } else {
+      return OTHER;
+    }
+  }
+
+  public static boolean isDeadlock(DatabaseProduct dbProduct, SQLException e) {
+    return e instanceof SQLTransactionRollbackException
+        || ((dbProduct == MYSQL || dbProduct == POSTGRES || dbProduct == SQLSERVER)
+            && e.getSQLState().equals("40001"))
+        || (dbProduct == POSTGRES && e.getSQLState().equals("40P01"))
+        || (dbProduct == ORACLE && (e.getMessage().contains("deadlock detected")
+            || e.getMessage().contains("can't serialize access for this transaction")));
+  }
+
+  /**
+   * Whether the RDBMS has restrictions on IN list size (explicit, or poor perf-based).
+   */
+  public static boolean needsInBatching(DatabaseProduct dbType) {
+    return dbType == ORACLE || dbType == SQLSERVER;
+  }
+
+  /**
+   * Whether the RDBMS has a bug in join and filter operation order described in DERBY-6358.
+   */
+  public static boolean hasJoinOperationOrderBug(DatabaseProduct dbType) {
+    return dbType == DERBY || dbType == ORACLE;
+  }
+};

http://git-wip-us.apache.org/repos/asf/hive/blob/8cec20d9/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index 8eeb1c4..561f3e3 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -88,14 +88,6 @@ import com.google.common.collect.Lists;
  * to SQL stores only. There's always a way to do without direct SQL.
  */
 class MetaStoreDirectSql {
-  private static enum DB {
-    MYSQL,
-    ORACLE,
-    MSSQL,
-    DERBY,
-    OTHER
-  }
-
   private static final int NO_BATCHING = -1, DETECT_BATCHING = 0;
 
   private static final Logger LOG = LoggerFactory.getLogger(MetaStoreDirectSql.class);
@@ -109,7 +101,7 @@ class MetaStoreDirectSql {
    *
    * Use sparingly, we don't want to devolve into another DataNucleus...
    */
-  private final DB dbType;
+  private final DatabaseProduct dbType;
   private final int batchSize;
   private final boolean convertMapNullsToEmptyStrings;
   private final String defaultPartName;
@@ -123,10 +115,17 @@ class MetaStoreDirectSql {
 
   public MetaStoreDirectSql(PersistenceManager pm, Configuration conf) {
     this.pm = pm;
-    this.dbType = determineDbType();
+    DatabaseProduct dbType = null;
+    try {
+      dbType = DatabaseProduct.determineDatabaseProduct(getProductName());
+    } catch (SQLException e) {
+      LOG.warn("Cannot determine database product; assuming OTHER", e);
+      dbType = DatabaseProduct.OTHER;
+    }
+    this.dbType = dbType;
     int batchSize = HiveConf.getIntVar(conf, ConfVars.METASTORE_DIRECT_SQL_PARTITION_BATCH_SIZE);
     if (batchSize == DETECT_BATCHING) {
-      batchSize = (dbType == DB.ORACLE || dbType == DB.MSSQL) ? 1000 : NO_BATCHING;
+      batchSize = DatabaseProduct.needsInBatching(dbType) ? 1000 : NO_BATCHING;
     }
     this.batchSize = batchSize;
 
@@ -136,7 +135,7 @@ class MetaStoreDirectSql {
 
     String jdoIdFactory = HiveConf.getVar(conf, ConfVars.METASTORE_IDENTIFIER_FACTORY);
     if (! ("datanucleus1".equalsIgnoreCase(jdoIdFactory))){
-      LOG.warn("Underlying metastore does not use 'datanuclues1' for its ORM naming scheme."
+      LOG.warn("Underlying metastore does not use 'datanucleus1' for its ORM naming scheme."
           + " Disabling directSQL as it uses hand-hardcoded SQL with that assumption.");
       isCompatibleDatastore = false;
     } else {
@@ -146,30 +145,13 @@ class MetaStoreDirectSql {
       }
     }
 
-    isAggregateStatsCacheEnabled = HiveConf.getBoolVar(conf, ConfVars.METASTORE_AGGREGATE_STATS_CACHE_ENABLED);
+    isAggregateStatsCacheEnabled = HiveConf.getBoolVar(
+        conf, ConfVars.METASTORE_AGGREGATE_STATS_CACHE_ENABLED);
     if (isAggregateStatsCacheEnabled) {
       aggrStatsCache = AggregateStatsCache.getInstance(conf);
     }
   }
 
-  private DB determineDbType() {
-    DB dbType = DB.OTHER;
-    String productName = getProductName();
-    if (productName != null) {
-      productName = productName.toLowerCase();
-      if (productName.contains("mysql")) {
-        dbType = DB.MYSQL;
-      } else if (productName.contains("oracle")) {
-        dbType = DB.ORACLE;
-      } else if (productName.contains("microsoft sql server")) {
-        dbType = DB.MSSQL;
-      } else if (productName.contains("derby")) {
-        dbType = DB.DERBY;
-      }
-    }
-    return dbType;
-  }
-
   private String getProductName() {
     JDOConnection jdoConn = pm.getDataStoreConnection();
     try {
@@ -391,7 +373,7 @@ class MetaStoreDirectSql {
   public boolean generateSqlFilterForPushdown(
       Table table, ExpressionTree tree, SqlFilterForPushdown result) throws MetaException {
     // Derby and Oracle do not interpret filters ANSI-properly in some cases and need a workaround.
-    boolean dbHasJoinCastBug = (dbType == DB.DERBY || dbType == DB.ORACLE);
+    boolean dbHasJoinCastBug = DatabaseProduct.hasJoinOperationOrderBug(dbType);
     result.table = table;
     result.filter = PartitionFilterGenerator.generateSqlFilter(
         table, tree, result.params, result.joins, dbHasJoinCastBug, defaultPartName, dbType);
@@ -960,10 +942,10 @@ class MetaStoreDirectSql {
     private final List<String> joins;
     private final boolean dbHasJoinCastBug;
     private final String defaultPartName;
-    private final DB dbType;
+    private final DatabaseProduct dbType;
 
     private PartitionFilterGenerator(Table table, List<Object> params, List<String> joins,
-        boolean dbHasJoinCastBug, String defaultPartName, DB dbType) {
+        boolean dbHasJoinCastBug, String defaultPartName, DatabaseProduct dbType) {
       this.table = table;
       this.params = params;
       this.joins = joins;
@@ -981,8 +963,8 @@ class MetaStoreDirectSql {
      * @return the string representation of the expression tree
      */
     private static String generateSqlFilter(Table table, ExpressionTree tree, List<Object> params,
-        List<String> joins, boolean dbHasJoinCastBug, String defaultPartName, DB dbType)
-            throws MetaException {
+        List<String> joins, boolean dbHasJoinCastBug, String defaultPartName,
+        DatabaseProduct dbType) throws MetaException {
       assert table != null;
       if (tree == null) {
         // consistent with other APIs like makeExpressionTree, null is returned to indicate that
@@ -1129,7 +1111,7 @@ class MetaStoreDirectSql {
         if (colType == FilterType.Integral) {
           tableValue = "cast(" + tableValue + " as decimal(21,0))";
         } else if (colType == FilterType.Date) {
-          if (dbType == DB.ORACLE) {
+          if (dbType == DatabaseProduct.ORACLE) {
             // Oracle requires special treatment... as usual.
             tableValue = "TO_DATE(" + tableValue + ", 'YYYY-MM-DD')";
           } else {
@@ -1756,7 +1738,7 @@ class MetaStoreDirectSql {
    * effect will apply to the connection that is executing the queries otherwise.
    */
   public void prepareTxn() throws MetaException {
-    if (dbType != DB.MYSQL) return;
+    if (dbType != DatabaseProduct.MYSQL) return;
     try {
       assert pm.currentTransaction().isActive(); // must be inside tx together with queries
       executeNoResult("SET @@session.sql_mode=ANSI_QUOTES");

http://git-wip-us.apache.org/repos/asf/hive/blob/8cec20d9/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 a7a1cf9..fe16ebd 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
@@ -27,6 +27,7 @@ import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.hive.common.ServerUtils;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.HouseKeeperService;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.slf4j.Logger;
@@ -1875,12 +1876,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       if(dbProduct == null) {
         throw new IllegalStateException("DB Type not determined yet.");
       }
-      if (e instanceof SQLTransactionRollbackException ||
-        ((dbProduct == DatabaseProduct.MYSQL || dbProduct == DatabaseProduct.POSTGRES ||
-          dbProduct == DatabaseProduct.SQLSERVER) && e.getSQLState().equals("40001")) ||
-        (dbProduct == DatabaseProduct.POSTGRES && e.getSQLState().equals("40P01")) ||
-        (dbProduct == DatabaseProduct.ORACLE && (e.getMessage().contains("deadlock detected")
-          || e.getMessage().contains("can't serialize access for this transaction")))) {
+      if (DatabaseProduct.isDeadlock(dbProduct, e)) {
         if (deadlockCnt++ < ALLOWED_REPEATED_DEADLOCKS) {
           long waitInterval = deadlockRetryInterval * deadlockCnt;
           LOG.warn("Deadlock detected in " + caller + ". Will wait " + waitInterval +
@@ -1985,44 +1981,22 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     return identifierQuoteString;
   }
 
-  protected enum DatabaseProduct { DERBY, MYSQL, POSTGRES, ORACLE, SQLSERVER}
 
-  /**
-   * Determine the database product type
-   * @param conn database connection
-   * @return database product type
-   */
-  private DatabaseProduct determineDatabaseProduct(Connection conn) {
-    if (dbProduct == null) {
-      try {
-        String s = conn.getMetaData().getDatabaseProductName();
-        if (s == null) {
-          String msg = "getDatabaseProductName returns null, can't determine database product";
-          LOG.error(msg);
-          throw new IllegalStateException(msg);
-        } else if (s.equals("Apache Derby")) {
-          dbProduct = DatabaseProduct.DERBY;
-        } else if (s.equals("Microsoft SQL Server")) {
-          dbProduct = DatabaseProduct.SQLSERVER;
-        } else if (s.equals("MySQL")) {
-          dbProduct = DatabaseProduct.MYSQL;
-        } else if (s.equals("Oracle")) {
-          dbProduct = DatabaseProduct.ORACLE;
-        } else if (s.equals("PostgreSQL")) {
-          dbProduct = DatabaseProduct.POSTGRES;
-        } else {
-          String msg = "Unrecognized database product name <" + s + ">";
-          LOG.error(msg);
-          throw new IllegalStateException(msg);
-        }
-
-      } catch (SQLException e) {
-        String msg = "Unable to get database product name: " + e.getMessage();
+  private void determineDatabaseProduct(Connection conn) {
+    if (dbProduct != null) return;
+    try {
+      String s = conn.getMetaData().getDatabaseProductName();
+      dbProduct = DatabaseProduct.determineDatabaseProduct(s);
+      if (dbProduct == DatabaseProduct.OTHER) {
+        String msg = "Unrecognized database product name <" + s + ">";
         LOG.error(msg);
         throw new IllegalStateException(msg);
       }
+    } catch (SQLException e) {
+      String msg = "Unable to get database product name";
+      LOG.error(msg, e);
+      throw new IllegalStateException(msg, e);
     }
-    return dbProduct;
   }
 
   private static class LockInfo {

http://git-wip-us.apache.org/repos/asf/hive/blob/8cec20d9/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnUtils.java b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnUtils.java
index ebcbaff..8fada2c 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnUtils.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnUtils.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.metastore.txn;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -118,7 +119,7 @@ public class TestTxnUtils {
   public void testSQLGenerator() throws Exception {
     //teseted on Oracle Database 11g Express Edition Release 11.2.0.2.0 - 64bit Production
     TxnHandler.SQLGenerator sqlGenerator =
-      new TxnHandler.SQLGenerator(TxnHandler.DatabaseProduct.ORACLE, conf);
+      new TxnHandler.SQLGenerator(DatabaseProduct.ORACLE, conf);
     List<String> rows = new ArrayList<>();
     rows.add("'yellow', 1");
     List<String> sql = sqlGenerator.createInsertValuesStmt("colors(name, category)", rows);
@@ -140,7 +141,7 @@ public class TestTxnUtils {
     Assert.assertEquals("Wrong stmt", "insert all into colors(name, category) values('G',997) into colors(name, category) values('G',998) into colors(name, category) values('G',999) select * from dual", sql.get(1));
     
     sqlGenerator =
-      new TxnHandler.SQLGenerator(TxnHandler.DatabaseProduct.MYSQL, conf);
+      new TxnHandler.SQLGenerator(DatabaseProduct.MYSQL, conf);
     rows.clear();
     rows.add("'yellow', 1");
     sql = sqlGenerator.createInsertValuesStmt("colors(name, category)", rows);
@@ -159,7 +160,7 @@ public class TestTxnUtils {
     Assert.assertEquals("Wrong stmt", "insert into colors(name, category) values('yellow', 1),('red', 2),('orange', 3),('G',0),('G',1),('G',2),('G',3),('G',4),('G',5),('G',6),('G',7),('G',8),('G',9),('G',10),('G',11),('G',12),('G',13),('G',14),('G',15),('G',16),('G',17),('G',18),('G',19),('G',20),('G',21),('G',22),('G',23),('G',24),('G',25),('G',26),('G',27),('G',28),('G',29),('G',30),('G',31),('G',32),('G',33),('G',34),('G',35),('G',36),('G',37),('G',38),('G',39),('G',40),('G',41),('G',42),('G',43),('G',44),('G',45),('G',46),('G',47),('G',48),('G',49),('G',50),('G',51),('G',52),('G',53),('G',54),('G',55),('G',56),('G',57),('G',58),('G',59),('G',60),('G',61),('G',62),('G',63),('G',64),('G',65),('G',66),('G',67),('G',68),('G',69),('G',70),('G',71),('G',72),('G',73),('G',74),('G',75),('G',76),('G',77),('G',78),('G',79),('G',80),('G',81),('G',82),('G',83),('G',84),('G',85),('G',86),('G',87),('G',88),('G',89),('G',90),('G',91),('G',92),('G',93),('G',94),('G',95),('G',96),('G',97),('G',9
 8),('G',99),('G',100),('G',101),('G',102),('G',103),('G',104),('G',105),('G',106),('G',107),('G',108),('G',109),('G',110),('G',111),('G',112),('G',113),('G',114),('G',115),('G',116),('G',117),('G',118),('G',119),('G',120),('G',121),('G',122),('G',123),('G',124),('G',125),('G',126),('G',127),('G',128),('G',129),('G',130),('G',131),('G',132),('G',133),('G',134),('G',135),('G',136),('G',137),('G',138),('G',139),('G',140),('G',141),('G',142),('G',143),('G',144),('G',145),('G',146),('G',147),('G',148),('G',149),('G',150),('G',151),('G',152),('G',153),('G',154),('G',155),('G',156),('G',157),('G',158),('G',159),('G',160),('G',161),('G',162),('G',163),('G',164),('G',165),('G',166),('G',167),('G',168),('G',169),('G',170),('G',171),('G',172),('G',173),('G',174),('G',175),('G',176),('G',177),('G',178),('G',179),('G',180),('G',181),('G',182),('G',183),('G',184),('G',185),('G',186),('G',187),('G',188),('G',189),('G',190),('G',191),('G',192),('G',193),('G',194),('G',195),('G',196),('G',197),('G',
 198),('G',199),('G',200),('G',201),('G',202),('G',203),('G',204),('G',205),('G',206),('G',207),('G',208),('G',209),('G',210),('G',211),('G',212),('G',213),('G',214),('G',215),('G',216),('G',217),('G',218),('G',219),('G',220),('G',221),('G',222),('G',223),('G',224),('G',225),('G',226),('G',227),('G',228),('G',229),('G',230),('G',231),('G',232),('G',233),('G',234),('G',235),('G',236),('G',237),('G',238),('G',239),('G',240),('G',241),('G',242),('G',243),('G',244),('G',245),('G',246),('G',247),('G',248),('G',249),('G',250),('G',251),('G',252),('G',253),('G',254),('G',255),('G',256),('G',257),('G',258),('G',259),('G',260),('G',261),('G',262),('G',263),('G',264),('G',265),('G',266),('G',267),('G',268),('G',269),('G',270),('G',271),('G',272),('G',273),('G',274),('G',275),('G',276),('G',277),('G',278),('G',279),('G',280),('G',281),('G',282),('G',283),('G',284),('G',285),('G',286),('G',287),('G',288),('G',289),('G',290),('G',291),('G',292),('G',293),('G',294),('G',295),('G',296),('G',297),('
 G',298),('G',299),('G',300),('G',301),('G',302),('G',303),('G',304),('G',305),('G',306),('G',307),('G',308),('G',309),('G',310),('G',311),('G',312),('G',313),('G',314),('G',315),('G',316),('G',317),('G',318),('G',319),('G',320),('G',321),('G',322),('G',323),('G',324),('G',325),('G',326),('G',327),('G',328),('G',329),('G',330),('G',331),('G',332),('G',333),('G',334),('G',335),('G',336),('G',337),('G',338),('G',339),('G',340),('G',341),('G',342),('G',343),('G',344),('G',345),('G',346),('G',347),('G',348),('G',349),('G',350),('G',351),('G',352),('G',353),('G',354),('G',355),('G',356),('G',357),('G',358),('G',359),('G',360),('G',361),('G',362),('G',363),('G',364),('G',365),('G',366),('G',367),('G',368),('G',369),('G',370),('G',371),('G',372),('G',373),('G',374),('G',375),('G',376),('G',377),('G',378),('G',379),('G',380),('G',381),('G',382),('G',383),('G',384),('G',385),('G',386),('G',387),('G',388),('G',389),('G',390),('G',391),('G',392),('G',393),('G',394),('G',395),('G',396),('G',397)
 ,('G',398),('G',399),('G',400),('G',401),('G',402),('G',403),('G',404),('G',405),('G',406),('G',407),('G',408),('G',409),('G',410),('G',411),('G',412),('G',413),('G',414),('G',415),('G',416),('G',417),('G',418),('G',419),('G',420),('G',421),('G',422),('G',423),('G',424),('G',425),('G',426),('G',427),('G',428),('G',429),('G',430),('G',431),('G',432),('G',433),('G',434),('G',435),('G',436),('G',437),('G',438),('G',439),('G',440),('G',441),('G',442),('G',443),('G',444),('G',445),('G',446),('G',447),('G',448),('G',449),('G',450),('G',451),('G',452),('G',453),('G',454),('G',455),('G',456),('G',457),('G',458),('G',459),('G',460),('G',461),('G',462),('G',463),('G',464),('G',465),('G',466),('G',467),('G',468),('G',469),('G',470),('G',471),('G',472),('G',473),('G',474),('G',475),('G',476),('G',477),('G',478),('G',479),('G',480),('G',481),('G',482),('G',483),('G',484),('G',485),('G',486),('G',487),('G',488),('G',489),('G',490),('G',491),('G',492),('G',493),('G',494),('G',495),('G',496),('G',4
 97),('G',498),('G',499),('G',500),('G',501),('G',502),('G',503),('G',504),('G',505),('G',506),('G',507),('G',508),('G',509),('G',510),('G',511),('G',512),('G',513),('G',514),('G',515),('G',516),('G',517),('G',518),('G',519),('G',520),('G',521),('G',522),('G',523),('G',524),('G',525),('G',526),('G',527),('G',528),('G',529),('G',530),('G',531),('G',532),('G',533),('G',534),('G',535),('G',536),('G',537),('G',538),('G',539),('G',540),('G',541),('G',542),('G',543),('G',544),('G',545),('G',546),('G',547),('G',548),('G',549),('G',550),('G',551),('G',552),('G',553),('G',554),('G',555),('G',556),('G',557),('G',558),('G',559),('G',560),('G',561),('G',562),('G',563),('G',564),('G',565),('G',566),('G',567),('G',568),('G',569),('G',570),('G',571),('G',572),('G',573),('G',574),('G',575),('G',576),('G',577),('G',578),('G',579),('G',580),('G',581),('G',582),('G',583),('G',584),('G',585),('G',586),('G',587),('G',588),('G',589),('G',590),('G',591),('G',592),('G',593),('G',594),('G',595),('G',596),('G
 ',597),('G',598),('G',599),('G',600),('G',601),('G',602),('G',603),('G',604),('G',605),('G',606),('G',607),('G',608),('G',609),('G',610),('G',611),('G',612),('G',613),('G',614),('G',615),('G',616),('G',617),('G',618),('G',619),('G',620),('G',621),('G',622),('G',623),('G',624),('G',625),('G',626),('G',627),('G',628),('G',629),('G',630),('G',631),('G',632),('G',633),('G',634),('G',635),('G',636),('G',637),('G',638),('G',639),('G',640),('G',641),('G',642),('G',643),('G',644),('G',645),('G',646),('G',647),('G',648),('G',649),('G',650),('G',651),('G',652),('G',653),('G',654),('G',655),('G',656),('G',657),('G',658),('G',659),('G',660),('G',661),('G',662),('G',663),('G',664),('G',665),('G',666),('G',667),('G',668),('G',669),('G',670),('G',671),('G',672),('G',673),('G',674),('G',675),('G',676),('G',677),('G',678),('G',679),('G',680),('G',681),('G',682),('G',683),('G',684),('G',685),('G',686),('G',687),('G',688),('G',689),('G',690),('G',691),('G',692),('G',693),('G',694),('G',695),('G',696),
 ('G',697),('G',698),('G',699),('G',700),('G',701),('G',702),('G',703),('G',704),('G',705),('G',706),('G',707),('G',708),('G',709),('G',710),('G',711),('G',712),('G',713),('G',714),('G',715),('G',716),('G',717),('G',718),('G',719),('G',720),('G',721),('G',722),('G',723),('G',724),('G',725),('G',726),('G',727),('G',728),('G',729),('G',730),('G',731),('G',732),('G',733),('G',734),('G',735),('G',736),('G',737),('G',738),('G',739),('G',740),('G',741),('G',742),('G',743),('G',744),('G',745),('G',746),('G',747),('G',748),('G',749),('G',750),('G',751),('G',752),('G',753),('G',754),('G',755),('G',756),('G',757),('G',758),('G',759),('G',760),('G',761),('G',762),('G',763),('G',764),('G',765),('G',766),('G',767),('G',768),('G',769),('G',770),('G',771),('G',772),('G',773),('G',774),('G',775),('G',776),('G',777),('G',778),('G',779),('G',780),('G',781),('G',782),('G',783),('G',784),('G',785),('G',786),('G',787),('G',788),('G',789),('G',790),('G',791),('G',792),('G',793),('G',794),('G',795),('G',79
 6),('G',797),('G',798),('G',799),('G',800),('G',801),('G',802),('G',803),('G',804),('G',805),('G',806),('G',807),('G',808),('G',809),('G',810),('G',811),('G',812),('G',813),('G',814),('G',815),('G',816),('G',817),('G',818),('G',819),('G',820),('G',821),('G',822),('G',823),('G',824),('G',825),('G',826),('G',827),('G',828),('G',829),('G',830),('G',831),('G',832),('G',833),('G',834),('G',835),('G',836),('G',837),('G',838),('G',839),('G',840),('G',841),('G',842),('G',843),('G',844),('G',845),('G',846),('G',847),('G',848),('G',849),('G',850),('G',851),('G',852),('G',853),('G',854),('G',855),('G',856),('G',857),('G',858),('G',859),('G',860),('G',861),('G',862),('G',863),('G',864),('G',865),('G',866),('G',867),('G',868),('G',869),('G',870),('G',871),('G',872),('G',873),('G',874),('G',875),('G',876),('G',877),('G',878),('G',879),('G',880),('G',881),('G',882),('G',883),('G',884),('G',885),('G',886),('G',887),('G',888),('G',889),('G',890),('G',891),('G',892),('G',893),('G',894),('G',895),('G'
 ,896),('G',897),('G',898),('G',899),('G',900),('G',901),('G',902),('G',903),('G',904),('G',905),('G',906),('G',907),('G',908),('G',909),('G',910),('G',911),('G',912),('G',913),('G',914),('G',915),('G',916),('G',917),('G',918),('G',919),('G',920),('G',921),('G',922),('G',923),('G',924),('G',925),('G',926),('G',927),('G',928),('G',929),('G',930),('G',931),('G',932),('G',933),('G',934),('G',935),('G',936),('G',937),('G',938),('G',939),('G',940),('G',941),('G',942),('G',943),('G',944),('G',945),('G',946),('G',947),('G',948),('G',949),('G',950),('G',951),('G',952),('G',953),('G',954),('G',955),('G',956),('G',957),('G',958),('G',959),('G',960),('G',961),('G',962),('G',963),('G',964),('G',965),('G',966),('G',967),('G',968),('G',969),('G',970),('G',971),('G',972),('G',973),('G',974),('G',975),('G',976),('G',977),('G',978),('G',979),('G',980),('G',981),('G',982),('G',983),('G',984),('G',985),('G',986),('G',987),('G',988),('G',989),('G',990),('G',991),('G',992),('G',993),('G',994),('G',995),(
 'G',996)", sql.get(0));
     Assert.assertEquals("Wrong stmt", "insert into colors(name, category) values('G',997),('G',998),('G',999)", sql.get(1));
 
-    sqlGenerator = new TxnHandler.SQLGenerator(TxnHandler.DatabaseProduct.SQLSERVER, conf);
+    sqlGenerator = new TxnHandler.SQLGenerator(DatabaseProduct.SQLSERVER, conf);
     String modSql = sqlGenerator.addForUpdateClause("select nl_next from NEXT_LOCK_ID");
     Assert.assertEquals("select nl_next from NEXT_LOCK_ID with (updlock)", modSql);
     modSql = sqlGenerator.addForUpdateClause("select MT_COMMENT from AUX_TABLE where MT_KEY1='CheckLock' and MT_KEY2=0");


[19/31] hive git commit: HIVE-14011: MessageFactory is not pluggable (Sravya Tirukkovalur, reviewed by Mohit Sabharwal)

Posted by se...@apache.org.
HIVE-14011: MessageFactory is not pluggable (Sravya Tirukkovalur, reviewed by Mohit Sabharwal)


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

Branch: refs/heads/hive-14535
Commit: e09f3c7a0c641c580483eb52f1f91bce8a0828c7
Parents: d4ffa5d
Author: Sravya Tirukkovalur <sr...@cloudera.com>
Authored: Mon Sep 12 09:58:32 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Mon Sep 12 09:58:32 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/hive/hcatalog/messaging/MessageFactory.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e09f3c7a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
index 71dc048..0710dd0 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
@@ -36,7 +36,7 @@ import java.util.Map;
  */
 public abstract class MessageFactory {
 
-  private static MessageFactory instance = new JSONMessageFactory();
+  private static MessageFactory instance = null;
 
   protected static final HiveConf hiveConf = new HiveConf();
   static {