You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by om...@apache.org on 2015/11/17 21:19:12 UTC

[01/43] hive git commit: HIVE-12232 : BucketingSortingReduceSinkOptimizer throws IOB exception for duplicate columns

Repository: hive
Updated Branches:
  refs/heads/master-fixed 96c45a349 -> d78fea102


HIVE-12232 : BucketingSortingReduceSinkOptimizer throws IOB exception for duplicate columns

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: c8023e9690fefcc9acd5260bfab0f1f829ab0e0a
Parents: 1a297a1
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Thu Nov 5 18:25:24 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:29 2015 -0800

----------------------------------------------------------------------
 .../BucketingSortingReduceSinkOptimizer.java    |  6 ++
 .../clientpositive/insertoverwrite_bucket.q     |  9 +++
 .../clientpositive/insertoverwrite_bucket.q.out | 78 ++++++++++++++++++++
 3 files changed, 93 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c8023e96/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
index a090a5b..d5df34c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
@@ -588,6 +588,12 @@ public class BucketingSortingReduceSinkOptimizer implements Transform {
             }
             // Only columns can be selected for both sorted and bucketed positions
             for (int pos : bucketPositions) {
+              if (pos >= selectDesc.getColList().size()) {
+                // e.g., INSERT OVERWRITE TABLE temp1 SELECT  c0,  c0 FROM temp2;
+                // In such a case Select Op will only have one instance of c0 and RS would have two.
+                // So, locating bucketCol in such cases will generate error. So, bail out.
+                return null;
+              }
               ExprNodeDesc selectColList = selectDesc.getColList().get(pos);
               if (!(selectColList instanceof ExprNodeColumnDesc)) {
                 return null;

http://git-wip-us.apache.org/repos/asf/hive/blob/c8023e96/ql/src/test/queries/clientpositive/insertoverwrite_bucket.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/insertoverwrite_bucket.q b/ql/src/test/queries/clientpositive/insertoverwrite_bucket.q
index d939710..5a10f94 100644
--- a/ql/src/test/queries/clientpositive/insertoverwrite_bucket.q
+++ b/ql/src/test/queries/clientpositive/insertoverwrite_bucket.q
@@ -18,10 +18,19 @@ insert into table bucketinput values ("firstinsert3");
 set hive.enforce.bucketing = true; 
 set hive.enforce.sorting=true;
 insert overwrite table bucketoutput1 select * from bucketinput where data like 'first%'; 
+CREATE TABLE temp1
+(
+    change string,
+    num string
+)
+CLUSTERED BY (num) SORTED BY (num) INTO 4 BUCKETS;
+explain insert overwrite table temp1 select data, data from bucketinput;
+
 set hive.auto.convert.sortmerge.join=true; 
 set hive.optimize.bucketmapjoin = true; 
 set hive.optimize.bucketmapjoin.sortedmerge = true; 
 select * from bucketoutput1 a join bucketoutput2 b on (a.data=b.data);
+drop table temp1;
 drop table buckettestinput;
 drop table buckettestoutput1;
 drop table buckettestoutput2;

http://git-wip-us.apache.org/repos/asf/hive/blob/c8023e96/ql/src/test/results/clientpositive/insertoverwrite_bucket.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/insertoverwrite_bucket.q.out b/ql/src/test/results/clientpositive/insertoverwrite_bucket.q.out
index 9b7b85d..4add20c 100644
--- a/ql/src/test/results/clientpositive/insertoverwrite_bucket.q.out
+++ b/ql/src/test/results/clientpositive/insertoverwrite_bucket.q.out
@@ -80,6 +80,76 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@bucketinput
 POSTHOOK: Output: default@bucketoutput1
 POSTHOOK: Lineage: bucketoutput1.data SIMPLE [(bucketinput)bucketinput.FieldSchema(name:data, type:string, comment:null), ]
+PREHOOK: query: CREATE TABLE temp1
+(
+    change string,
+    num string
+)
+CLUSTERED BY (num) SORTED BY (num) INTO 4 BUCKETS
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@temp1
+POSTHOOK: query: CREATE TABLE temp1
+(
+    change string,
+    num string
+)
+CLUSTERED BY (num) SORTED BY (num) INTO 4 BUCKETS
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@temp1
+PREHOOK: query: explain insert overwrite table temp1 select data, data from bucketinput
+PREHOOK: type: QUERY
+POSTHOOK: query: explain insert overwrite table temp1 select data, data from bucketinput
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+  Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: bucketinput
+            Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: data (type: string)
+              outputColumnNames: _col1
+              Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col1 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col1 (type: string)
+                Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey0 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                name: default.temp1
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: true
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.temp1
+
+  Stage: Stage-2
+    Stats-Aggr Operator
+
 PREHOOK: query: select * from bucketoutput1 a join bucketoutput2 b on (a.data=b.data)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@bucketoutput1
@@ -90,6 +160,14 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@bucketoutput1
 POSTHOOK: Input: default@bucketoutput2
 #### A masked pattern was here ####
+PREHOOK: query: drop table temp1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@temp1
+PREHOOK: Output: default@temp1
+POSTHOOK: query: drop table temp1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@temp1
+POSTHOOK: Output: default@temp1
 PREHOOK: query: drop table buckettestinput
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: drop table buckettestinput


[15/43] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 3d7cb18..cea9000 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -43,6 +43,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) = 0;
   virtual void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) = 0;
   virtual void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern) = 0;
+  virtual void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types) = 0;
   virtual void get_all_tables(std::vector<std::string> & _return, const std::string& db_name) = 0;
   virtual void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) = 0;
   virtual void get_table_objects_by_name(std::vector<Table> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names) = 0;
@@ -246,6 +247,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void get_tables(std::vector<std::string> & /* _return */, const std::string& /* db_name */, const std::string& /* pattern */) {
     return;
   }
+  void get_table_meta(std::vector<TableMeta> & /* _return */, const std::string& /* db_patterns */, const std::string& /* tbl_patterns */, const std::vector<std::string> & /* tbl_types */) {
+    return;
+  }
   void get_all_tables(std::vector<std::string> & /* _return */, const std::string& /* db_name */) {
     return;
   }
@@ -3199,6 +3203,132 @@ class ThriftHiveMetastore_get_tables_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_get_table_meta_args__isset {
+  _ThriftHiveMetastore_get_table_meta_args__isset() : db_patterns(false), tbl_patterns(false), tbl_types(false) {}
+  bool db_patterns :1;
+  bool tbl_patterns :1;
+  bool tbl_types :1;
+} _ThriftHiveMetastore_get_table_meta_args__isset;
+
+class ThriftHiveMetastore_get_table_meta_args {
+ public:
+
+  ThriftHiveMetastore_get_table_meta_args(const ThriftHiveMetastore_get_table_meta_args&);
+  ThriftHiveMetastore_get_table_meta_args& operator=(const ThriftHiveMetastore_get_table_meta_args&);
+  ThriftHiveMetastore_get_table_meta_args() : db_patterns(), tbl_patterns() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_table_meta_args() throw();
+  std::string db_patterns;
+  std::string tbl_patterns;
+  std::vector<std::string>  tbl_types;
+
+  _ThriftHiveMetastore_get_table_meta_args__isset __isset;
+
+  void __set_db_patterns(const std::string& val);
+
+  void __set_tbl_patterns(const std::string& val);
+
+  void __set_tbl_types(const std::vector<std::string> & val);
+
+  bool operator == (const ThriftHiveMetastore_get_table_meta_args & rhs) const
+  {
+    if (!(db_patterns == rhs.db_patterns))
+      return false;
+    if (!(tbl_patterns == rhs.tbl_patterns))
+      return false;
+    if (!(tbl_types == rhs.tbl_types))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_table_meta_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_table_meta_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_table_meta_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_table_meta_pargs() throw();
+  const std::string* db_patterns;
+  const std::string* tbl_patterns;
+  const std::vector<std::string> * tbl_types;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_table_meta_result__isset {
+  _ThriftHiveMetastore_get_table_meta_result__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_table_meta_result__isset;
+
+class ThriftHiveMetastore_get_table_meta_result {
+ public:
+
+  ThriftHiveMetastore_get_table_meta_result(const ThriftHiveMetastore_get_table_meta_result&);
+  ThriftHiveMetastore_get_table_meta_result& operator=(const ThriftHiveMetastore_get_table_meta_result&);
+  ThriftHiveMetastore_get_table_meta_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_table_meta_result() throw();
+  std::vector<TableMeta>  success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_table_meta_result__isset __isset;
+
+  void __set_success(const std::vector<TableMeta> & val);
+
+  void __set_o1(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_table_meta_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_table_meta_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_table_meta_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_table_meta_presult__isset {
+  _ThriftHiveMetastore_get_table_meta_presult__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_table_meta_presult__isset;
+
+class ThriftHiveMetastore_get_table_meta_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_table_meta_presult() throw();
+  std::vector<TableMeta> * success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_table_meta_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_get_all_tables_args__isset {
   _ThriftHiveMetastore_get_all_tables_args__isset() : db_name(false) {}
   bool db_name :1;
@@ -16967,6 +17097,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern);
   void send_get_tables(const std::string& db_name, const std::string& pattern);
   void recv_get_tables(std::vector<std::string> & _return);
+  void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
+  void send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
+  void recv_get_table_meta(std::vector<TableMeta> & _return);
   void get_all_tables(std::vector<std::string> & _return, const std::string& db_name);
   void send_get_all_tables(const std::string& db_name);
   void recv_get_all_tables(std::vector<std::string> & _return);
@@ -17325,6 +17458,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_table_meta(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_all_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_table_objects_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -17459,6 +17593,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["drop_table"] = &ThriftHiveMetastoreProcessor::process_drop_table;
     processMap_["drop_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context;
     processMap_["get_tables"] = &ThriftHiveMetastoreProcessor::process_get_tables;
+    processMap_["get_table_meta"] = &ThriftHiveMetastoreProcessor::process_get_table_meta;
     processMap_["get_all_tables"] = &ThriftHiveMetastoreProcessor::process_get_all_tables;
     processMap_["get_table"] = &ThriftHiveMetastoreProcessor::process_get_table;
     processMap_["get_table_objects_by_name"] = &ThriftHiveMetastoreProcessor::process_get_table_objects_by_name;
@@ -17801,6 +17936,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_table_meta(_return, db_patterns, tbl_patterns, tbl_types);
+    }
+    ifaces_[i]->get_table_meta(_return, db_patterns, tbl_patterns, tbl_types);
+    return;
+  }
+
   void get_all_tables(std::vector<std::string> & _return, const std::string& db_name) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -18929,6 +19074,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern);
   int32_t send_get_tables(const std::string& db_name, const std::string& pattern);
   void recv_get_tables(std::vector<std::string> & _return, const int32_t seqid);
+  void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
+  int32_t send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
+  void recv_get_table_meta(std::vector<TableMeta> & _return, const int32_t seqid);
   void get_all_tables(std::vector<std::string> & _return, const std::string& db_name);
   int32_t send_get_all_tables(const std::string& db_name);
   void recv_get_all_tables(std::vector<std::string> & _return, const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 a395729..c0d9401 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -127,6 +127,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("get_tables\n");
   }
 
+  void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types) {
+    // Your implementation goes here
+    printf("get_table_meta\n");
+  }
+
   void get_all_tables(std::vector<std::string> & _return, const std::string& db_name) {
     // Your implementation goes here
     printf("get_all_tables\n");

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 5fd4a90..ee28d0d 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -15157,6 +15157,163 @@ void GetAllFunctionsResponse::printTo(std::ostream& out) const {
 }
 
 
+TableMeta::~TableMeta() throw() {
+}
+
+
+void TableMeta::__set_dbName(const std::string& val) {
+  this->dbName = val;
+}
+
+void TableMeta::__set_tableName(const std::string& val) {
+  this->tableName = val;
+}
+
+void TableMeta::__set_tableType(const std::string& val) {
+  this->tableType = val;
+}
+
+void TableMeta::__set_comments(const std::string& val) {
+  this->comments = val;
+__isset.comments = true;
+}
+
+uint32_t TableMeta::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_dbName = false;
+  bool isset_tableName = false;
+  bool isset_tableType = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->dbName);
+          isset_dbName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tableName);
+          isset_tableName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tableType);
+          isset_tableType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->comments);
+          this->__isset.comments = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_dbName)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_tableName)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_tableType)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t TableMeta::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("TableMeta");
+
+  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->dbName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->tableName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->tableType);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.comments) {
+    xfer += oprot->writeFieldBegin("comments", ::apache::thrift::protocol::T_STRING, 4);
+    xfer += oprot->writeString(this->comments);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TableMeta &a, TableMeta &b) {
+  using ::std::swap;
+  swap(a.dbName, b.dbName);
+  swap(a.tableName, b.tableName);
+  swap(a.tableType, b.tableType);
+  swap(a.comments, b.comments);
+  swap(a.__isset, b.__isset);
+}
+
+TableMeta::TableMeta(const TableMeta& other682) {
+  dbName = other682.dbName;
+  tableName = other682.tableName;
+  tableType = other682.tableType;
+  comments = other682.comments;
+  __isset = other682.__isset;
+}
+TableMeta& TableMeta::operator=(const TableMeta& other683) {
+  dbName = other683.dbName;
+  tableName = other683.tableName;
+  tableType = other683.tableType;
+  comments = other683.comments;
+  __isset = other683.__isset;
+  return *this;
+}
+void TableMeta::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "TableMeta(";
+  out << "dbName=" << to_string(dbName);
+  out << ", " << "tableName=" << to_string(tableName);
+  out << ", " << "tableType=" << to_string(tableType);
+  out << ", " << "comments="; (__isset.comments ? (out << to_string(comments)) : (out << "<null>"));
+  out << ")";
+}
+
+
 MetaException::~MetaException() throw() {
 }
 
@@ -15226,13 +15383,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other682) : TException() {
-  message = other682.message;
-  __isset = other682.__isset;
+MetaException::MetaException(const MetaException& other684) : TException() {
+  message = other684.message;
+  __isset = other684.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other683) {
-  message = other683.message;
-  __isset = other683.__isset;
+MetaException& MetaException::operator=(const MetaException& other685) {
+  message = other685.message;
+  __isset = other685.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -15323,13 +15480,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other684) : TException() {
-  message = other684.message;
-  __isset = other684.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other686) : TException() {
+  message = other686.message;
+  __isset = other686.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other685) {
-  message = other685.message;
-  __isset = other685.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other687) {
+  message = other687.message;
+  __isset = other687.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -15420,13 +15577,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other686) : TException() {
-  message = other686.message;
-  __isset = other686.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other688) : TException() {
+  message = other688.message;
+  __isset = other688.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other687) {
-  message = other687.message;
-  __isset = other687.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other689) {
+  message = other689.message;
+  __isset = other689.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -15517,13 +15674,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other688) : TException() {
-  message = other688.message;
-  __isset = other688.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other690) : TException() {
+  message = other690.message;
+  __isset = other690.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other689) {
-  message = other689.message;
-  __isset = other689.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other691) {
+  message = other691.message;
+  __isset = other691.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -15614,13 +15771,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other690) : TException() {
-  message = other690.message;
-  __isset = other690.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other692) : TException() {
+  message = other692.message;
+  __isset = other692.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other691) {
-  message = other691.message;
-  __isset = other691.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other693) {
+  message = other693.message;
+  __isset = other693.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -15711,13 +15868,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other692) : TException() {
-  message = other692.message;
-  __isset = other692.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other694) : TException() {
+  message = other694.message;
+  __isset = other694.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other693) {
-  message = other693.message;
-  __isset = other693.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other695) {
+  message = other695.message;
+  __isset = other695.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -15808,13 +15965,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other694) : TException() {
-  message = other694.message;
-  __isset = other694.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other696) : TException() {
+  message = other696.message;
+  __isset = other696.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other695) {
-  message = other695.message;
-  __isset = other695.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other697) {
+  message = other697.message;
+  __isset = other697.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -15905,13 +16062,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other696) : TException() {
-  message = other696.message;
-  __isset = other696.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other698) : TException() {
+  message = other698.message;
+  __isset = other698.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other697) {
-  message = other697.message;
-  __isset = other697.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other699) {
+  message = other699.message;
+  __isset = other699.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -16002,13 +16159,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other698) : TException() {
-  message = other698.message;
-  __isset = other698.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other700) : TException() {
+  message = other700.message;
+  __isset = other700.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other699) {
-  message = other699.message;
-  __isset = other699.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other701) {
+  message = other701.message;
+  __isset = other701.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -16099,13 +16256,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other700) : TException() {
-  message = other700.message;
-  __isset = other700.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other702) : TException() {
+  message = other702.message;
+  __isset = other702.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other701) {
-  message = other701.message;
-  __isset = other701.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other703) {
+  message = other703.message;
+  __isset = other703.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -16196,13 +16353,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other702) : TException() {
-  message = other702.message;
-  __isset = other702.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other704) : TException() {
+  message = other704.message;
+  __isset = other704.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other703) {
-  message = other703.message;
-  __isset = other703.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other705) {
+  message = other705.message;
+  __isset = other705.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -16293,13 +16450,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other704) : TException() {
-  message = other704.message;
-  __isset = other704.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other706) : TException() {
+  message = other706.message;
+  __isset = other706.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other705) {
-  message = other705.message;
-  __isset = other705.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other707) {
+  message = other707.message;
+  __isset = other707.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -16390,13 +16547,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other706) : TException() {
-  message = other706.message;
-  __isset = other706.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other708) : TException() {
+  message = other708.message;
+  __isset = other708.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other707) {
-  message = other707.message;
-  __isset = other707.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other709) {
+  message = other709.message;
+  __isset = other709.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -16487,13 +16644,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other708) : TException() {
-  message = other708.message;
-  __isset = other708.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other710) : TException() {
+  message = other710.message;
+  __isset = other710.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other709) {
-  message = other709.message;
-  __isset = other709.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other711) {
+  message = other711.message;
+  __isset = other711.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -16584,13 +16741,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other710) : TException() {
-  message = other710.message;
-  __isset = other710.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other712) : TException() {
+  message = other712.message;
+  __isset = other712.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other711) {
-  message = other711.message;
-  __isset = other711.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other713) {
+  message = other713.message;
+  __isset = other713.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -16681,13 +16838,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other712) : TException() {
-  message = other712.message;
-  __isset = other712.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other714) : TException() {
+  message = other714.message;
+  __isset = other714.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other713) {
-  message = other713.message;
-  __isset = other713.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other715) {
+  message = other715.message;
+  __isset = other715.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 53ab272..05c288c 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -349,6 +349,8 @@ class ClearFileMetadataRequest;
 
 class GetAllFunctionsResponse;
 
+class TableMeta;
+
 class MetaException;
 
 class UnknownTableException;
@@ -6158,6 +6160,69 @@ inline std::ostream& operator<<(std::ostream& out, const GetAllFunctionsResponse
   return out;
 }
 
+typedef struct _TableMeta__isset {
+  _TableMeta__isset() : comments(false) {}
+  bool comments :1;
+} _TableMeta__isset;
+
+class TableMeta {
+ public:
+
+  TableMeta(const TableMeta&);
+  TableMeta& operator=(const TableMeta&);
+  TableMeta() : dbName(), tableName(), tableType(), comments() {
+  }
+
+  virtual ~TableMeta() throw();
+  std::string dbName;
+  std::string tableName;
+  std::string tableType;
+  std::string comments;
+
+  _TableMeta__isset __isset;
+
+  void __set_dbName(const std::string& val);
+
+  void __set_tableName(const std::string& val);
+
+  void __set_tableType(const std::string& val);
+
+  void __set_comments(const std::string& val);
+
+  bool operator == (const TableMeta & rhs) const
+  {
+    if (!(dbName == rhs.dbName))
+      return false;
+    if (!(tableName == rhs.tableName))
+      return false;
+    if (!(tableType == rhs.tableType))
+      return false;
+    if (__isset.comments != rhs.__isset.comments)
+      return false;
+    else if (__isset.comments && !(comments == rhs.comments))
+      return false;
+    return true;
+  }
+  bool operator != (const TableMeta &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const TableMeta & ) 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(TableMeta &a, TableMeta &b);
+
+inline std::ostream& operator<<(std::ostream& out, const TableMeta& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _MetaException__isset {
   _MetaException__isset() : message(false) {}
   bool message :1;

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java
new file mode 100644
index 0000000..08a8e36
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java
@@ -0,0 +1,701 @@
+/**
+ * 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 TableMeta implements org.apache.thrift.TBase<TableMeta, TableMeta._Fields>, java.io.Serializable, Cloneable, Comparable<TableMeta> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TableMeta");
+
+  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 TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TABLE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableType", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField COMMENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("comments", org.apache.thrift.protocol.TType.STRING, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TableMetaStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TableMetaTupleSchemeFactory());
+  }
+
+  private String dbName; // required
+  private String tableName; // required
+  private String tableType; // required
+  private String comments; // 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 {
+    DB_NAME((short)1, "dbName"),
+    TABLE_NAME((short)2, "tableName"),
+    TABLE_TYPE((short)3, "tableType"),
+    COMMENTS((short)4, "comments");
+
+    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: // TABLE_NAME
+          return TABLE_NAME;
+        case 3: // TABLE_TYPE
+          return TABLE_TYPE;
+        case 4: // COMMENTS
+          return COMMENTS;
+        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 _Fields optionals[] = {_Fields.COMMENTS};
+  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.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE_TYPE, new org.apache.thrift.meta_data.FieldMetaData("tableType", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.COMMENTS, new org.apache.thrift.meta_data.FieldMetaData("comments", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TableMeta.class, metaDataMap);
+  }
+
+  public TableMeta() {
+  }
+
+  public TableMeta(
+    String dbName,
+    String tableName,
+    String tableType)
+  {
+    this();
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.tableType = tableType;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TableMeta(TableMeta other) {
+    if (other.isSetDbName()) {
+      this.dbName = other.dbName;
+    }
+    if (other.isSetTableName()) {
+      this.tableName = other.tableName;
+    }
+    if (other.isSetTableType()) {
+      this.tableType = other.tableType;
+    }
+    if (other.isSetComments()) {
+      this.comments = other.comments;
+    }
+  }
+
+  public TableMeta deepCopy() {
+    return new TableMeta(this);
+  }
+
+  @Override
+  public void clear() {
+    this.dbName = null;
+    this.tableName = null;
+    this.tableType = null;
+    this.comments = null;
+  }
+
+  public String getDbName() {
+    return this.dbName;
+  }
+
+  public void setDbName(String dbName) {
+    this.dbName = dbName;
+  }
+
+  public void unsetDbName() {
+    this.dbName = null;
+  }
+
+  /** Returns true if field dbName is set (has been assigned a value) and false otherwise */
+  public boolean isSetDbName() {
+    return this.dbName != null;
+  }
+
+  public void setDbNameIsSet(boolean value) {
+    if (!value) {
+      this.dbName = null;
+    }
+  }
+
+  public String getTableName() {
+    return this.tableName;
+  }
+
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  public void unsetTableName() {
+    this.tableName = null;
+  }
+
+  /** Returns true if field tableName is set (has been assigned a value) and false otherwise */
+  public boolean isSetTableName() {
+    return this.tableName != null;
+  }
+
+  public void setTableNameIsSet(boolean value) {
+    if (!value) {
+      this.tableName = null;
+    }
+  }
+
+  public String getTableType() {
+    return this.tableType;
+  }
+
+  public void setTableType(String tableType) {
+    this.tableType = tableType;
+  }
+
+  public void unsetTableType() {
+    this.tableType = null;
+  }
+
+  /** Returns true if field tableType is set (has been assigned a value) and false otherwise */
+  public boolean isSetTableType() {
+    return this.tableType != null;
+  }
+
+  public void setTableTypeIsSet(boolean value) {
+    if (!value) {
+      this.tableType = null;
+    }
+  }
+
+  public String getComments() {
+    return this.comments;
+  }
+
+  public void setComments(String comments) {
+    this.comments = comments;
+  }
+
+  public void unsetComments() {
+    this.comments = null;
+  }
+
+  /** Returns true if field comments is set (has been assigned a value) and false otherwise */
+  public boolean isSetComments() {
+    return this.comments != null;
+  }
+
+  public void setCommentsIsSet(boolean value) {
+    if (!value) {
+      this.comments = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case DB_NAME:
+      if (value == null) {
+        unsetDbName();
+      } else {
+        setDbName((String)value);
+      }
+      break;
+
+    case TABLE_NAME:
+      if (value == null) {
+        unsetTableName();
+      } else {
+        setTableName((String)value);
+      }
+      break;
+
+    case TABLE_TYPE:
+      if (value == null) {
+        unsetTableType();
+      } else {
+        setTableType((String)value);
+      }
+      break;
+
+    case COMMENTS:
+      if (value == null) {
+        unsetComments();
+      } else {
+        setComments((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDbName();
+
+    case TABLE_NAME:
+      return getTableName();
+
+    case TABLE_TYPE:
+      return getTableType();
+
+    case COMMENTS:
+      return getComments();
+
+    }
+    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 TABLE_NAME:
+      return isSetTableName();
+    case TABLE_TYPE:
+      return isSetTableType();
+    case COMMENTS:
+      return isSetComments();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TableMeta)
+      return this.equals((TableMeta)that);
+    return false;
+  }
+
+  public boolean equals(TableMeta that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_dbName = true && this.isSetDbName();
+    boolean that_present_dbName = true && that.isSetDbName();
+    if (this_present_dbName || that_present_dbName) {
+      if (!(this_present_dbName && that_present_dbName))
+        return false;
+      if (!this.dbName.equals(that.dbName))
+        return false;
+    }
+
+    boolean this_present_tableName = true && this.isSetTableName();
+    boolean that_present_tableName = true && that.isSetTableName();
+    if (this_present_tableName || that_present_tableName) {
+      if (!(this_present_tableName && that_present_tableName))
+        return false;
+      if (!this.tableName.equals(that.tableName))
+        return false;
+    }
+
+    boolean this_present_tableType = true && this.isSetTableType();
+    boolean that_present_tableType = true && that.isSetTableType();
+    if (this_present_tableType || that_present_tableType) {
+      if (!(this_present_tableType && that_present_tableType))
+        return false;
+      if (!this.tableType.equals(that.tableType))
+        return false;
+    }
+
+    boolean this_present_comments = true && this.isSetComments();
+    boolean that_present_comments = true && that.isSetComments();
+    if (this_present_comments || that_present_comments) {
+      if (!(this_present_comments && that_present_comments))
+        return false;
+      if (!this.comments.equals(that.comments))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_dbName = true && (isSetDbName());
+    list.add(present_dbName);
+    if (present_dbName)
+      list.add(dbName);
+
+    boolean present_tableName = true && (isSetTableName());
+    list.add(present_tableName);
+    if (present_tableName)
+      list.add(tableName);
+
+    boolean present_tableType = true && (isSetTableType());
+    list.add(present_tableType);
+    if (present_tableType)
+      list.add(tableType);
+
+    boolean present_comments = true && (isSetComments());
+    list.add(present_comments);
+    if (present_comments)
+      list.add(comments);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TableMeta other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDbName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, other.dbName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTableName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTableType()).compareTo(other.isSetTableType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTableType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableType, other.tableType);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetComments()).compareTo(other.isSetComments());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetComments()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.comments, other.comments);
+      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("TableMeta(");
+    boolean first = true;
+
+    sb.append("dbName:");
+    if (this.dbName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.dbName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tableName:");
+    if (this.tableName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tableName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tableType:");
+    if (this.tableType == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tableType);
+    }
+    first = false;
+    if (isSetComments()) {
+      if (!first) sb.append(", ");
+      sb.append("comments:");
+      if (this.comments == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.comments);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetDbName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTableName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTableType()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableType' 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 TableMetaStandardSchemeFactory implements SchemeFactory {
+    public TableMetaStandardScheme getScheme() {
+      return new TableMetaStandardScheme();
+    }
+  }
+
+  private static class TableMetaStandardScheme extends StandardScheme<TableMeta> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TableMeta 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: // TABLE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tableName = iprot.readString();
+              struct.setTableNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TABLE_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tableType = iprot.readString();
+              struct.setTableTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // COMMENTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.comments = iprot.readString();
+              struct.setCommentsIsSet(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, TableMeta 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.tableName != null) {
+        oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
+        oprot.writeString(struct.tableName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tableType != null) {
+        oprot.writeFieldBegin(TABLE_TYPE_FIELD_DESC);
+        oprot.writeString(struct.tableType);
+        oprot.writeFieldEnd();
+      }
+      if (struct.comments != null) {
+        if (struct.isSetComments()) {
+          oprot.writeFieldBegin(COMMENTS_FIELD_DESC);
+          oprot.writeString(struct.comments);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TableMetaTupleSchemeFactory implements SchemeFactory {
+    public TableMetaTupleScheme getScheme() {
+      return new TableMetaTupleScheme();
+    }
+  }
+
+  private static class TableMetaTupleScheme extends TupleScheme<TableMeta> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TableMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.dbName);
+      oprot.writeString(struct.tableName);
+      oprot.writeString(struct.tableType);
+      BitSet optionals = new BitSet();
+      if (struct.isSetComments()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetComments()) {
+        oprot.writeString(struct.comments);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TableMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.dbName = iprot.readString();
+      struct.setDbNameIsSet(true);
+      struct.tableName = iprot.readString();
+      struct.setTableNameIsSet(true);
+      struct.tableType = iprot.readString();
+      struct.setTableTypeIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.comments = iprot.readString();
+        struct.setCommentsIsSet(true);
+      }
+    }
+  }
+
+}
+


[26/43] hive git commit: HIVE-12385 : schematool failed on MySQL (Sergey Shelukhin, reviewed by Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-12385 : schematool failed on MySQL (Sergey Shelukhin, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master-fixed
Commit: b029d039da1066d239027e3a119a024f9d53b065
Parents: f480940
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Nov 12 11:46:03 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:33 2015 -0800

----------------------------------------------------------------------
 .../scripts/upgrade/mysql/022-HIVE-11970.mysql.sql      | 12 ++++++------
 .../scripts/upgrade/mysql/hive-schema-1.3.0.mysql.sql   |  2 +-
 .../scripts/upgrade/mysql/hive-schema-2.0.0.mysql.sql   |  2 +-
 3 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b029d039/metastore/scripts/upgrade/mysql/022-HIVE-11970.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/022-HIVE-11970.mysql.sql b/metastore/scripts/upgrade/mysql/022-HIVE-11970.mysql.sql
index 4517e00..583b5f4 100644
--- a/metastore/scripts/upgrade/mysql/022-HIVE-11970.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/022-HIVE-11970.mysql.sql
@@ -1,6 +1,6 @@
-ALTER TABLE `COLUMNS_V2` MODIFY `COLUMN_NAME` varchar(1000) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
-ALTER TABLE `PART_COL_PRIVS` MODIFY `COLUMN_NAME` varchar(1000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
-ALTER TABLE `TBL_COL_PRIVS` MODIFY `COLUMN_NAME` varchar(1000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
-ALTER TABLE `SORT_COLS` MODIFY `COLUMN_NAME` varchar(1000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
-ALTER TABLE `TAB_COL_STATS` MODIFY `COLUMN_NAME` varchar(1000) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
-ALTER TABLE `PART_COL_STATS` MODIFY `COLUMN_NAME` varchar(1000) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
+ALTER TABLE `COLUMNS_V2` MODIFY `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
+ALTER TABLE `PART_COL_PRIVS` MODIFY `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
+ALTER TABLE `TBL_COL_PRIVS` MODIFY `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
+ALTER TABLE `SORT_COLS` MODIFY `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
+ALTER TABLE `TAB_COL_STATS` MODIFY `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;
+ALTER TABLE `PART_COL_STATS` MODIFY `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL;

http://git-wip-us.apache.org/repos/asf/hive/blob/b029d039/metastore/scripts/upgrade/mysql/hive-schema-1.3.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-1.3.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-1.3.0.mysql.sql
index ce0ac54..6c211a6 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-1.3.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-1.3.0.mysql.sql
@@ -52,7 +52,7 @@ CREATE TABLE IF NOT EXISTS `CDS` (
 CREATE TABLE IF NOT EXISTS `COLUMNS_V2` (
   `CD_ID` bigint(20) NOT NULL,
   `COMMENT` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
-  `COLUMN_NAME` varchar(1000) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TYPE_NAME` varchar(4000) DEFAULT NULL,
   `INTEGER_IDX` int(11) NOT NULL,
   PRIMARY KEY (`CD_ID`,`COLUMN_NAME`),

http://git-wip-us.apache.org/repos/asf/hive/blob/b029d039/metastore/scripts/upgrade/mysql/hive-schema-2.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-2.0.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-2.0.0.mysql.sql
index 3a2c0e2..ff0b643 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-2.0.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-2.0.0.mysql.sql
@@ -52,7 +52,7 @@ CREATE TABLE IF NOT EXISTS `CDS` (
 CREATE TABLE IF NOT EXISTS `COLUMNS_V2` (
   `CD_ID` bigint(20) NOT NULL,
   `COMMENT` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
-  `COLUMN_NAME` varchar(1000) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TYPE_NAME` varchar(4000) DEFAULT NULL,
   `INTEGER_IDX` int(11) NOT NULL,
   PRIMARY KEY (`CD_ID`,`COLUMN_NAME`),


[07/43] hive git commit: HIVE-12358: Categorize vectorization benchmarks into arithmetic, comparison, logic(Teddy Choi, reviewed by Ashutosh Chauhan, Ferdinand Xu)

Posted by om...@apache.org.
HIVE-12358: Categorize vectorization benchmarks into arithmetic, comparison, logic(Teddy Choi, reviewed by Ashutosh Chauhan, Ferdinand Xu)


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

Branch: refs/heads/master-fixed
Commit: 20e980552986657706a38d6378bc7983e37a8334
Parents: c8023e9
Author: Teddy Choi <tc...@hortonworks.com>
Authored: Sun Nov 8 20:08:07 2015 -0500
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:30 2015 -0800

----------------------------------------------------------------------
 .../vectorization/AbstractExpression.java       | 150 ++++++
 .../vectorization/VectorizationBench.java       | 506 -------------------
 .../VectorizedArithmeticBench.java              | 112 ++++
 .../VectorizedComparisonBench.java              | 215 ++++++++
 .../vectorization/VectorizedLogicBench.java     | 147 ++++++
 5 files changed, 624 insertions(+), 506 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/20e98055/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java
new file mode 100644
index 0000000..94af3e0
--- /dev/null
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java
@@ -0,0 +1,150 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.benchmark.vectorization;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+@BenchmarkMode(Mode.AverageTime)
+@Fork(1)
+@State(Scope.Thread)
+@OutputTimeUnit(TimeUnit.NANOSECONDS)
+public abstract class AbstractExpression {
+  private static final int DEFAULT_ITER_TIME = 1000000;
+  protected VectorExpression expression;
+  protected VectorizedRowBatch rowBatch;
+
+  protected VectorizedRowBatch buildRowBatch(ColumnVector output, int colNum, ColumnVector...
+    cols) {
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(colNum + 1);
+    for (int i = 0; i < cols.length; i++) {
+      rowBatch.cols[i] = cols[i];
+    }
+    rowBatch.cols[colNum] = output;
+    return rowBatch;
+  }
+
+  @Setup
+  public abstract void setup();
+
+  @Benchmark
+  @Warmup(iterations = 2, time = 2, timeUnit = TimeUnit.MILLISECONDS)
+  @Measurement(iterations = 2, time = 2, timeUnit = TimeUnit.MILLISECONDS)
+  public void bench() {
+    for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
+      expression.evaluate(rowBatch);
+    }
+  }
+
+  protected LongColumnVector getLongColumnVector() {
+    LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+    Random random = new Random();
+    for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      columnVector.vector[i] = random.nextLong();
+    }
+    return columnVector;
+  }
+
+  protected LongColumnVector getRepeatingLongColumnVector() {
+    LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+    columnVector.fill(2);
+    return columnVector;
+  }
+
+  protected LongColumnVector getLongColumnVectorWithNull() {
+    LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+    columnVector.noNulls = false;
+    Random random = new Random();
+    for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      if (i % 100 == 0) {
+        columnVector.isNull[i] = true;
+      }
+      columnVector.vector[i] = random.nextLong();
+    }
+    return columnVector;
+  }
+
+  protected LongColumnVector getBooleanLongColumnVector() {
+    LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+    Random random = new Random();
+    for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      columnVector.vector[i] = random.nextInt(2);
+    }
+    return columnVector;
+  }
+
+  protected LongColumnVector getBooleanRepeatingLongColumnVector() {
+    LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+    columnVector.fill(1);
+    return columnVector;
+  }
+
+  protected LongColumnVector getBooleanLongColumnVectorWithNull() {
+    LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+    columnVector.noNulls = false;
+    Random random = new Random();
+    for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      if (i % 100 == 0) {
+        columnVector.isNull[i] = true;
+      }
+      columnVector.vector[i] = random.nextInt(2);
+    }
+    return columnVector;
+  }
+
+  protected DoubleColumnVector getDoubleColumnVector() {
+    DoubleColumnVector columnVector = new DoubleColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+    Random random = new Random();
+    for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      columnVector.vector[i] = random.nextDouble();
+    }
+    return columnVector;
+  }
+
+  protected DoubleColumnVector getRepeatingDoubleColumnVector() {
+    DoubleColumnVector columnVector = new DoubleColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+    columnVector.fill(2.0d);
+    return columnVector;
+  }
+
+  protected DoubleColumnVector getDoubleColumnVectorWithNull() {
+    DoubleColumnVector columnVector = new DoubleColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+    columnVector.noNulls = false;
+    Random random = new Random();
+    for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      if (i % 100 == 0) {
+        columnVector.isNull[i] = true;
+      }
+      columnVector.vector[i] = random.nextDouble();
+    }
+    return columnVector;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/20e98055/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizationBench.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizationBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizationBench.java
deleted file mode 100644
index 642c5e1..0000000
--- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizationBench.java
+++ /dev/null
@@ -1,506 +0,0 @@
-/**
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.benchmark.vectorization;
-
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprLongColumnLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprDoubleColumnDoubleColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.ColAndCol;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.ColOrCol;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColDivideLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.NotCol;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarGreaterEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarGreaterLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarLessEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarLessLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarNotEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColDivideLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DoubleColAddDoubleColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DoubleColDivideDoubleColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColAddLongColumn;
-import org.openjdk.jmh.annotations.Benchmark;
-import org.openjdk.jmh.annotations.BenchmarkMode;
-import org.openjdk.jmh.annotations.Fork;
-import org.openjdk.jmh.annotations.Measurement;
-import org.openjdk.jmh.annotations.Mode;
-import org.openjdk.jmh.annotations.OutputTimeUnit;
-import org.openjdk.jmh.annotations.Scope;
-import org.openjdk.jmh.annotations.Setup;
-import org.openjdk.jmh.annotations.State;
-import org.openjdk.jmh.annotations.Warmup;
-import org.openjdk.jmh.runner.Runner;
-import org.openjdk.jmh.runner.RunnerException;
-import org.openjdk.jmh.runner.options.Options;
-import org.openjdk.jmh.runner.options.OptionsBuilder;
-
-import java.util.Arrays;
-import java.lang.Override;
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-
-@State(Scope.Benchmark)
-public class VectorizationBench {
-  /**
-   * This test measures the performance for vectorization.
-   * <p/>
-   * This test uses JMH framework for benchmarking.
-   * You may execute this benchmark tool using JMH command line in different ways:
-   * <p/>
-   * To use the settings shown in the main() function, use:
-   * $ java -cp target/benchmarks.jar org.apache.hive.benchmark.vectorization.VectorizationBench
-   * <p/>
-   * To use the default settings used by JMH, use:
-   * $ java -jar target/benchmarks.jar org.apache.hive.benchmark.vectorization VectorizationBench
-   * <p/>
-   * To specify different parameters, use:
-   * - This command will use 10 warm-up iterations, 5 test iterations, and 2 forks. And it will
-   * display the Average Time (avgt) in Microseconds (us)
-   * - Benchmark mode. Available modes are:
-   * [Throughput/thrpt, AverageTime/avgt, SampleTime/sample, SingleShotTime/ss, All/all]
-   * - Output time unit. Available time units are: [m, s, ms, us, ns].
-   * <p/>
-   * $ java -jar target/benchmarks.jar org.apache.hive.benchmark.vectorization VectorizationBench
-   * -wi 10 -i 5 -f 2 -bm avgt -tu us
-   */
-
-  @BenchmarkMode(Mode.AverageTime)
-  @Fork(1)
-  @State(Scope.Thread)
-  @OutputTimeUnit(TimeUnit.NANOSECONDS)
-  public static abstract class AbstractExpression {
-    private static final int DEFAULT_ITER_TIME = 1000000;
-    protected VectorExpression expression;
-    protected VectorizedRowBatch rowBatch;
-
-    protected VectorizedRowBatch buildRowBatch(ColumnVector output, int colNum, ColumnVector...
-      cols) {
-      VectorizedRowBatch rowBatch = new VectorizedRowBatch(colNum + 1);
-      for (int i = 0; i < cols.length; i++) {
-        rowBatch.cols[i] = cols[i];
-      }
-      rowBatch.cols[colNum] = output;
-//      rowBatch.selectedInUse = true;
-      return rowBatch;
-    }
-
-    @Setup
-    public abstract void setup();
-
-    @Benchmark
-    @Warmup(iterations = 2, time = 2, timeUnit = TimeUnit.MILLISECONDS)
-    @Measurement(iterations = 2, time = 2, timeUnit = TimeUnit.MILLISECONDS)
-    public void bench() {
-      for (int i = 0; i < DEFAULT_ITER_TIME; i++) {
-        expression.evaluate(rowBatch);
-      }
-    }
-
-    protected LongColumnVector getLongColumnVector() {
-      LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
-      Random random = new Random();
-      for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
-        columnVector.vector[i] = random.nextLong();
-      }
-      return columnVector;
-    }
-
-    protected LongColumnVector getRepeatingLongColumnVector() {
-      LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
-      columnVector.fill(2);
-      return columnVector;
-    }
-
-    protected LongColumnVector getLongColumnVectorWithNull() {
-      LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
-      columnVector.noNulls = false;
-      Random random = new Random();
-      for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
-        if (i % 100 == 0) {
-          columnVector.isNull[i] = true;
-        }
-        columnVector.vector[i] = random.nextLong();
-      }
-      return columnVector;
-    }
-
-    protected LongColumnVector getBooleanLongColumnVector() {
-      LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
-      Random random = new Random();
-      for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
-        columnVector.vector[i] = random.nextInt(2);
-      }
-      return columnVector;
-    }
-
-    protected LongColumnVector getBooleanRepeatingLongColumnVector() {
-      LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
-      columnVector.fill(1);
-      return columnVector;
-    }
-
-    protected LongColumnVector getBooleanLongColumnVectorWithNull() {
-      LongColumnVector columnVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
-      columnVector.noNulls = false;
-      Random random = new Random();
-      for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
-        if (i % 100 == 0) {
-          columnVector.isNull[i] = true;
-        }
-        columnVector.vector[i] = random.nextInt(2);
-      }
-      return columnVector;
-    }
-
-    protected DoubleColumnVector getDoubleColumnVector() {
-      DoubleColumnVector columnVector = new DoubleColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
-      Random random = new Random();
-      for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
-        columnVector.vector[i] = random.nextDouble();
-      }
-      return columnVector;
-    }
-
-    protected DoubleColumnVector getRepeatingDoubleColumnVector() {
-      DoubleColumnVector columnVector = new DoubleColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
-      columnVector.fill(2.0d);
-      return columnVector;
-    }
-
-    protected DoubleColumnVector getDoubleColumnVectorWithNull() {
-      DoubleColumnVector columnVector = new DoubleColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
-      columnVector.noNulls = false;
-      Random random = new Random();
-      for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) {
-        if (i % 100 == 0) {
-          columnVector.isNull[i] = true;
-        }
-        columnVector.vector[i] = random.nextDouble();
-      }
-      return columnVector;
-    }
-
-  }
-
-  public static class DoubleColAddRepeatingDoubleColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new DoubleColumnVector(), 2, getDoubleColumnVector(),
-        getRepeatingDoubleColumnVector());
-      expression = new DoubleColAddDoubleColumn(0, 1, 2);
-    }
-  }
-
-  public static class LongColAddRepeatingLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(),
-        getRepeatingLongColumnVector());
-      expression = new LongColAddLongColumn(0, 1, 2);
-    }
-  }
-
-
-  public static class DoubleColDivideDoubleColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new DoubleColumnVector(), 2, getDoubleColumnVector(),
-        getDoubleColumnVector());
-      expression = new DoubleColDivideDoubleColumn(0, 1, 2);
-    }
-  }
-
-  public static class DoubleColDivideRepeatingDoubleColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new DoubleColumnVector(), 2, getDoubleColumnVector(),
-        getRepeatingDoubleColumnVector());
-      expression = new DoubleColDivideDoubleColumn(0, 1, 2);
-    }
-  }
-
-  public static class LongColDivideLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new DoubleColumnVector(), 2, getLongColumnVector(),
-        getLongColumnVector());
-      expression = new LongColDivideLongColumn(0, 1, 2);
-    }
-  }
-
-  public static class LongColDivideRepeatingLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new DoubleColumnVector(), 2, getLongColumnVector(),
-        getRepeatingLongColumnVector());
-      expression = new LongColDivideLongColumn(0, 1, 2);
-    }
-  }
-
-  public static class ColAndColBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
-        getBooleanLongColumnVector());
-      expression = new ColAndCol(0, 1, 2);
-    }
-  }
-
-  public static class ColAndRepeatingColBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
-          getBooleanRepeatingLongColumnVector());
-      expression = new ColAndCol(0, 1, 2);
-    }
-  }
-
-  public static class RepeatingColAndColBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanRepeatingLongColumnVector(),
-          getBooleanLongColumnVector());
-      expression = new ColAndCol(0, 1, 2);
-    }
-  }
-
-  public static class ColOrColBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
-          getBooleanLongColumnVector());
-      expression = new ColOrCol(0, 1, 2);
-    }
-  }
-
-  public static class ColOrRepeatingColBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
-          getBooleanRepeatingLongColumnVector());
-      expression = new ColOrCol(0, 1, 2);
-    }
-  }
-
-  public static class RepeatingColOrColBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanRepeatingLongColumnVector(),
-          getBooleanLongColumnVector());
-      expression = new ColOrCol(0, 1, 2);
-    }
-  }
-
-  public static class NotColBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getBooleanLongColumnVector());
-      expression = new NotCol(0, 1);
-    }
-  }
-
-  public static class IfExprLongColumnLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 3, getBooleanLongColumnVector(),
-        getLongColumnVector(), getLongColumnVector());
-      expression = new IfExprLongColumnLongColumn(0, 1, 2, 3);
-    }
-  }
-
-  public static class IfExprRepeatingLongColumnLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 3, getBooleanLongColumnVector(),
-          getRepeatingLongColumnVector(), getLongColumnVector());
-      expression = new IfExprLongColumnLongColumn(0, 1, 2, 3);
-    }
-  }
-
-  public static class IfExprLongColumnRepeatingLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 3, getBooleanLongColumnVector(),
-          getLongColumnVector(), getRepeatingLongColumnVector());
-      expression = new IfExprLongColumnLongColumn(0, 1, 2, 3);
-    }
-  }
-
-  public static class LongColEqualLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
-      expression = new LongColEqualLongColumn(0, 1, 2);
-    }
-  }
-
-  public static class LongColGreaterEqualLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
-      expression = new LongColGreaterEqualLongColumn(0, 1, 2);
-    }
-  }
-
-  public static class LongColGreaterLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
-      expression = new LongColGreaterLongColumn(0, 1, 2);
-    }
-  }
-
-  public static class LongColLessEqualLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
-      expression = new LongColLessEqualLongColumn(0, 1, 2);
-    }
-  }
-
-  public static class LongColLessLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
-      expression = new LongColLessLongColumn(0, 1, 2);
-    }
-  }
-
-  public static class LongColNotEqualLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
-      expression = new LongColNotEqualLongColumn(0, 1, 2);
-    }
-  }
-
-  public static class LongColEqualLongScalarBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongColEqualLongScalar(0, 0, 1);
-    }
-  }
-
-  public static class LongColGreaterEqualLongScalarBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongColGreaterEqualLongScalar(0, 0, 1);
-    }
-  }
-
-  public static class LongColGreaterLongScalarBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongColGreaterLongScalar(0, 0, 1);
-    }
-  }
-
-  public static class LongColLessEqualLongScalarBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongColLessEqualLongScalar(0, 0, 1);
-    }
-  }
-
-  public static class LongColLessLongScalarBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongColLessLongScalar(0, 0, 1);
-    }
-  }
-
-  public static class LongColNotEqualLongScalarBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongColNotEqualLongScalar(0, 0, 1);
-    }
-  }
-
-
-  public static class LongScalarEqualLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongScalarEqualLongColumn(0, 0, 1);
-    }
-  }
-
-  public static class LongScalarGreaterEqualLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongScalarGreaterEqualLongColumn(0, 0, 1);
-    }
-  }
-
-  public static class LongScalarGreaterLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongScalarGreaterLongColumn(0, 0, 1);
-    }
-  }
-
-  public static class LongScalarLessEqualLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongScalarLessEqualLongColumn(0, 0, 1);
-    }
-  }
-
-  public static class LongScalarLessLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongScalarLessLongColumn(0, 0, 1);
-    }
-  }
-
-  public static class LongScalarNotEqualLongColumnBench extends AbstractExpression {
-    @Override
-    public void setup() {
-      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
-      expression = new LongScalarNotEqualLongColumn(0, 0, 1);
-    }
-  }
-
-  public static void main(String[] args) throws RunnerException {
-    Options opt = new OptionsBuilder().include(".*" + VectorizationBench.class.getSimpleName() +
-      ".*").build();
-    new Runner(opt).run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/20e98055/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedArithmeticBench.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedArithmeticBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedArithmeticBench.java
new file mode 100644
index 0000000..b6e2fec
--- /dev/null
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedArithmeticBench.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.benchmark.vectorization;
+
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColDivideLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DoubleColAddDoubleColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DoubleColDivideDoubleColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColAddLongColumn;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+/**
+ * This test measures the performance for vectorization.
+ * <p/>
+ * This test uses JMH framework for benchmarking.
+ * You may execute this benchmark tool using JMH command line in different ways:
+ * <p/>
+ * To use the settings shown in the main() function, use:
+ * $ java -cp target/benchmarks.jar org.apache.hive.benchmark.vectorization.VectorizedArithmeticBench
+ * <p/>
+ * To use the default settings used by JMH, use:
+ * $ java -jar target/benchmarks.jar org.apache.hive.benchmark.vectorization.VectorizedArithmeticBench
+ * <p/>
+ * To specify different parameters, use:
+ * - This command will use 10 warm-up iterations, 5 test iterations, and 2 forks. And it will
+ * display the Average Time (avgt) in Microseconds (us)
+ * - Benchmark mode. Available modes are:
+ * [Throughput/thrpt, AverageTime/avgt, SampleTime/sample, SingleShotTime/ss, All/all]
+ * - Output time unit. Available time units are: [m, s, ms, us, ns].
+ * <p/>
+ * $ java -jar target/benchmarks.jar org.apache.hive.benchmark.vectorization.VectorizedArithmeticBench
+ * -wi 10 -i 5 -f 2 -bm avgt -tu us
+ */
+@State(Scope.Benchmark)
+public class VectorizedArithmeticBench {
+  public static class DoubleColAddRepeatingDoubleColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new DoubleColumnVector(), 2, getDoubleColumnVector(),
+          getRepeatingDoubleColumnVector());
+      expression = new DoubleColAddDoubleColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColAddRepeatingLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(),
+          getRepeatingLongColumnVector());
+      expression = new LongColAddLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class DoubleColDivideDoubleColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new DoubleColumnVector(), 2, getDoubleColumnVector(),
+          getDoubleColumnVector());
+      expression = new DoubleColDivideDoubleColumn(0, 1, 2);
+    }
+  }
+
+  public static class DoubleColDivideRepeatingDoubleColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new DoubleColumnVector(), 2, getDoubleColumnVector(),
+          getRepeatingDoubleColumnVector());
+      expression = new DoubleColDivideDoubleColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColDivideLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new DoubleColumnVector(), 2, getLongColumnVector(),
+          getLongColumnVector());
+      expression = new LongColDivideLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColDivideRepeatingLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new DoubleColumnVector(), 2, getLongColumnVector(),
+          getRepeatingLongColumnVector());
+      expression = new LongColDivideLongColumn(0, 1, 2);
+    }
+  }
+
+  public static void main(String[] args) throws RunnerException {
+    Options opt = new OptionsBuilder().include(".*" + VectorizedArithmeticBench.class.getSimpleName() +
+        ".*").build();
+    new Runner(opt).run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/20e98055/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedComparisonBench.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedComparisonBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedComparisonBench.java
new file mode 100644
index 0000000..536ef76
--- /dev/null
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedComparisonBench.java
@@ -0,0 +1,215 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.benchmark.vectorization;
+
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterEqualLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessEqualLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarGreaterEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarGreaterLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarLessEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarLessLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarNotEqualLongColumn;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+/**
+ * This test measures the performance for vectorization.
+ * <p/>
+ * This test uses JMH framework for benchmarking.
+ * You may execute this benchmark tool using JMH command line in different ways:
+ * <p/>
+ * To use the settings shown in the main() function, use:
+ * $ java -cp target/benchmarks.jar org.apache.hive.benchmark.vectorization.VectorizedComparisonBench
+ * <p/>
+ * To use the default settings used by JMH, use:
+ * $ java -jar target/benchmarks.jar org.apache.hive.benchmark.vectorization.VectorizedComparisonBench
+ * <p/>
+ * To specify different parameters, use:
+ * - This command will use 10 warm-up iterations, 5 test iterations, and 2 forks. And it will
+ * display the Average Time (avgt) in Microseconds (us)
+ * - Benchmark mode. Available modes are:
+ * [Throughput/thrpt, AverageTime/avgt, SampleTime/sample, SingleShotTime/ss, All/all]
+ * - Output time unit. Available time units are: [m, s, ms, us, ns].
+ * <p/>
+ * $ java -jar target/benchmarks.jar org.apache.hive.benchmark.vectorization.VectorizedComparisonBench
+ * -wi 10 -i 5 -f 2 -bm avgt -tu us
+ */
+@State(Scope.Benchmark)
+public class VectorizedComparisonBench {
+  public static class LongColEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColEqualLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColGreaterEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColGreaterEqualLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColGreaterLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColGreaterLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColLessEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColLessEqualLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColLessLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColLessLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColNotEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColNotEqualLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColEqualLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColEqualLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongColGreaterEqualLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColGreaterEqualLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongColGreaterLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColGreaterLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongColLessEqualLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColLessEqualLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongColLessLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColLessLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongColNotEqualLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColNotEqualLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarEqualLongColumn(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarGreaterEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarGreaterEqualLongColumn(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarGreaterLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarGreaterLongColumn(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarLessEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarLessEqualLongColumn(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarLessLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarLessLongColumn(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarNotEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarNotEqualLongColumn(0, 0, 1);
+    }
+  }
+
+  public static void main(String[] args) throws RunnerException {
+    Options opt = new OptionsBuilder().include(".*" + VectorizedComparisonBench.class.getSimpleName() +
+        ".*").build();
+    new Runner(opt).run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/20e98055/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
new file mode 100644
index 0000000..50dadb2
--- /dev/null
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.benchmark.vectorization;
+
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.ColAndCol;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.ColOrCol;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprLongColumnLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.NotCol;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+/**
+ * This test measures the performance for vectorization.
+ * <p/>
+ * This test uses JMH framework for benchmarking.
+ * You may execute this benchmark tool using JMH command line in different ways:
+ * <p/>
+ * To use the settings shown in the main() function, use:
+ * $ java -cp target/benchmarks.jar org.apache.hive.benchmark.vectorization.VectorizedLogicBench
+ * <p/>
+ * To use the default settings used by JMH, use:
+ * $ java -jar target/benchmarks.jar org.apache.hive.benchmark.vectorization.VectorizedLogicBench
+ * <p/>
+ * To specify different parameters, use:
+ * - This command will use 10 warm-up iterations, 5 test iterations, and 2 forks. And it will
+ * display the Average Time (avgt) in Microseconds (us)
+ * - Benchmark mode. Available modes are:
+ * [Throughput/thrpt, AverageTime/avgt, SampleTime/sample, SingleShotTime/ss, All/all]
+ * - Output time unit. Available time units are: [m, s, ms, us, ns].
+ * <p/>
+ * $ java -jar target/benchmarks.jar org.apache.hive.benchmark.vectorization.VectorizedLogicBench
+ * -wi 10 -i 5 -f 2 -bm avgt -tu us
+ */
+@State(Scope.Benchmark)
+public class VectorizedLogicBench {
+
+  public static class ColAndColBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
+          getBooleanLongColumnVector());
+      expression = new ColAndCol(0, 1, 2);
+    }
+  }
+
+  public static class ColAndRepeatingColBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
+          getBooleanRepeatingLongColumnVector());
+      expression = new ColAndCol(0, 1, 2);
+    }
+  }
+
+  public static class RepeatingColAndColBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanRepeatingLongColumnVector(),
+          getBooleanLongColumnVector());
+      expression = new ColAndCol(0, 1, 2);
+    }
+  }
+
+  public static class ColOrColBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
+          getBooleanLongColumnVector());
+      expression = new ColOrCol(0, 1, 2);
+    }
+  }
+
+  public static class ColOrRepeatingColBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
+          getBooleanRepeatingLongColumnVector());
+      expression = new ColOrCol(0, 1, 2);
+    }
+  }
+
+  public static class RepeatingColOrColBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanRepeatingLongColumnVector(),
+          getBooleanLongColumnVector());
+      expression = new ColOrCol(0, 1, 2);
+    }
+  }
+
+  public static class NotColBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getBooleanLongColumnVector());
+      expression = new NotCol(0, 1);
+    }
+  }
+
+  public static class IfExprLongColumnLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 3, getBooleanLongColumnVector(),
+          getLongColumnVector(), getLongColumnVector());
+      expression = new IfExprLongColumnLongColumn(0, 1, 2, 3);
+    }
+  }
+
+  public static class IfExprRepeatingLongColumnLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 3, getBooleanLongColumnVector(),
+          getRepeatingLongColumnVector(), getLongColumnVector());
+      expression = new IfExprLongColumnLongColumn(0, 1, 2, 3);
+    }
+  }
+
+  public static class IfExprLongColumnRepeatingLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 3, getBooleanLongColumnVector(),
+          getLongColumnVector(), getRepeatingLongColumnVector());
+      expression = new IfExprLongColumnLongColumn(0, 1, 2, 3);
+    }
+  }
+
+  public static void main(String[] args) throws RunnerException {
+    Options opt = new OptionsBuilder().include(".*" + VectorizedLogicBench.class.getSimpleName() +
+        ".*").build();
+    new Runner(opt).run();
+  }
+}
\ No newline at end of file


[24/43] hive git commit: HIVE-12309 : TableScan should colStats when available for better data size estimate (Ashutosh Chauhan via Prasanth J)

Posted by om...@apache.org.
HIVE-12309 : TableScan should colStats when available for better data size estimate (Ashutosh Chauhan via Prasanth J)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: dcaf0c6300daccf66d6d863a8562989ee26ef4d9
Parents: 558b35c
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Fri Oct 30 15:03:42 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:32 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |  71 ++--
 .../annotate_stats_deep_filters.q.out           |   2 +-
 .../clientpositive/annotate_stats_filter.q.out  |  48 +--
 .../clientpositive/annotate_stats_groupby.q.out |  56 +--
 .../annotate_stats_groupby2.q.out               |  32 +-
 .../clientpositive/annotate_stats_join.q.out    |  34 +-
 .../annotate_stats_join_pkfk.q.out              |  50 +--
 .../clientpositive/annotate_stats_limit.q.out   |   8 +-
 .../clientpositive/annotate_stats_part.q.out    |  14 +-
 .../clientpositive/annotate_stats_select.q.out  |  24 +-
 .../clientpositive/annotate_stats_table.q.out   |  12 +-
 .../clientpositive/annotate_stats_union.q.out   |  20 +-
 .../clientpositive/cbo_rp_auto_join0.q.out      |   8 +-
 .../clientpositive/cbo_rp_auto_join1.q.out      |  30 +-
 .../results/clientpositive/cbo_rp_join0.q.out   |  14 +-
 .../extrapolate_part_stats_full.q.out           |   8 +-
 .../extrapolate_part_stats_partial.q.out        |  12 +-
 .../extrapolate_part_stats_partial_ndv.q.out    |   6 +-
 .../clientpositive/llap/llapdecider.q.out       |  46 +--
 .../spark/annotate_stats_join.q.out             |  34 +-
 .../results/clientpositive/stats_ppr_all.q.out  |   8 +-
 .../clientpositive/tez/explainuser_1.q.out      | 352 +++++++++----------
 .../clientpositive/tez/llapdecider.q.out        |  46 +--
 23 files changed, 473 insertions(+), 462 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index e1f8ebc..71ed31c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -138,6 +138,39 @@ public class StatsUtils {
         fetchColStats, fetchPartStats);
   }
 
+  private static long getDataSize(HiveConf conf, Table table) {
+    long ds = getRawDataSize(table);
+    if (ds <= 0) {
+      ds = getTotalSize(table);
+
+      // if data size is still 0 then get file size
+      if (ds <= 0) {
+        ds = getFileSizeForTable(conf, table);
+      }
+      float deserFactor =
+          HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR);
+      ds = (long) (ds * deserFactor);
+    }
+
+    return ds;
+  }
+
+  private static long getNumRows(HiveConf conf, List<ColumnInfo> schema, List<String> neededColumns, Table table, long ds) {
+    long nr = getNumRows(table);
+ // number of rows -1 means that statistics from metastore is not reliable
+    // and 0 means statistics gathering is disabled
+    if (nr <= 0) {
+      int avgRowSize = estimateRowSizeFromSchema(conf, schema, neededColumns);
+      if (avgRowSize > 0) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Estimated average row size: " + avgRowSize);
+        }
+        nr = ds / avgRowSize;
+      }
+    }
+    return nr == 0 ? 1 : nr;
+  }
+
   public static Statistics collectStatistics(HiveConf conf, PrunedPartitionList partList,
       Table table, List<ColumnInfo> schema, List<String> neededColumns,
       List<String> referencedColumns, boolean fetchColStats, boolean fetchPartStats)
@@ -149,41 +182,17 @@ public class StatsUtils {
         HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR);
 
     if (!table.isPartitioned()) {
-      long nr = getNumRows(table);
-      long ds = getRawDataSize(table);
-      if (ds <= 0) {
-        ds = getTotalSize(table);
 
-        // if data size is still 0 then get file size
-        if (ds <= 0) {
-          ds = getFileSizeForTable(conf, table);
-        }
-
-        ds = (long) (ds * deserFactor);
-      }
-
-      // number of rows -1 means that statistics from metastore is not reliable
-      // and 0 means statistics gathering is disabled
-      if (nr <= 0) {
-        int avgRowSize = estimateRowSizeFromSchema(conf, schema, neededColumns);
-        if (avgRowSize > 0) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Estimated average row size: " + avgRowSize);
-          }
-          nr = ds / avgRowSize;
-        }
-      }
-      if (nr == 0) {
-        nr = 1;
-      }
+      long ds = getDataSize(conf, table);
+      long nr = getNumRows(conf, schema, neededColumns, table, ds);
       stats.setNumRows(nr);
-      stats.setDataSize(ds);
-
       List<ColStatistics> colStats = Lists.newArrayList();
       if (fetchColStats) {
         colStats = getTableColumnStats(table, schema, neededColumns);
+        long betterDS = getDataSizeFromColumnStats(nr, colStats);
+        ds = betterDS < 1 ? ds : betterDS;
       }
-
+       stats.setDataSize(ds);
       // infer if any column can be primary key based on column statistics
       inferAndSetPrimaryKey(stats.getNumRows(), colStats);
 
@@ -276,11 +285,13 @@ public class StatsUtils {
             LOG.debug("Column stats requested for : " + neededColumns.size() + " columns. Able to" +
                 " retrieve for " + colStats.size() + " columns");
           }
+
           List<ColStatistics> columnStats = convertColStats(colStats, table.getTableName());
 
           addParitionColumnStats(conf, neededColumns, referencedColumns, schema, table, partList,
               columnStats);
-
+          long betterDS = getDataSizeFromColumnStats(nr, columnStats);
+          stats.setDataSize(betterDS < 1 ? ds : betterDS);
           // infer if any column can be primary key based on column statistics
           inferAndSetPrimaryKey(stats.getNumRows(), columnStats);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_deep_filters.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_deep_filters.q.out b/ql/src/test/results/clientpositive/annotate_stats_deep_filters.q.out
index fc4f294..20ccda5 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_deep_filters.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_deep_filters.q.out
@@ -118,7 +118,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: over1k
-            Statistics: Num rows: 2098 Data size: 211174 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2098 Data size: 16736 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((t = 1) and (si = 2)) or ((t = 2) and (si = 3)) or ((t = 3) and (si = 4)) or ((t = 4) and (si = 5)) or ((t = 5) and (si = 6)) or ((t = 6) and (si = 7)) or ((t = 7) and (si = 8)) or ((t = 9) and (si = 10)) or ((t = 10) and (si = 11)) or ((t = 11) and (si = 12)) or ((t = 12) and (si = 13)) or ((t = 13) and (si = 14)) or ((t = 14) and (si = 15)) or ((t = 15) and (si = 16)) or ((t = 16) and (si = 17)) or ((t = 17) and (si = 18)) or ((t = 27) and (si = 28)) or ((t = 37) and (si = 38)) or ((t = 47) and (si = 48)) or ((t = 52) and (si = 53))) (type: boolean)
               Statistics: Num rows: 280 Data size: 2232 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_filter.q.out b/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
index 054b573..f13fdb7 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
@@ -141,7 +141,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (state = 'OH') (type: boolean)
               Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE
@@ -181,7 +181,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (state <> 'OH') (type: boolean)
               Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
@@ -217,7 +217,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (state <> 'OH') (type: boolean)
               Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
@@ -257,7 +257,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: zip is null (type: boolean)
               Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE
@@ -295,7 +295,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: zip is null (type: boolean)
               Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE
@@ -335,7 +335,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: zip is not null (type: boolean)
               Statistics: Num rows: 7 Data size: 702 Basic stats: COMPLETE Column stats: COMPLETE
@@ -373,7 +373,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: zip is not null (type: boolean)
               Statistics: Num rows: 7 Data size: 702 Basic stats: COMPLETE Column stats: COMPLETE
@@ -413,7 +413,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3
@@ -436,7 +436,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: false (type: boolean)
               Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE
@@ -476,7 +476,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3
@@ -499,7 +499,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: 'foo' (type: string)
               Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
@@ -537,7 +537,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3
@@ -560,7 +560,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: false (type: boolean)
               Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE
@@ -598,7 +598,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: false (type: boolean)
               Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE
@@ -636,7 +636,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: false (type: boolean)
               Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE
@@ -676,7 +676,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((state = 'OH') or (state = 'CA')) (type: boolean)
               Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
@@ -716,7 +716,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((year = 2001) and year is null) (type: boolean)
               Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE
@@ -754,7 +754,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((year = 2001) and (state = 'OH')) and (state = 'FL')) (type: boolean)
               Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE
@@ -794,7 +794,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((year = 2001) and year is null) or (state = 'CA')) (type: boolean)
               Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
@@ -834,7 +834,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((year = 2001) or year is null) and (state = 'CA')) (type: boolean)
               Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE
@@ -874,7 +874,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (locid < 30) (type: boolean)
               Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
@@ -910,7 +910,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (locid > 30) (type: boolean)
               Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
@@ -946,7 +946,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (locid <= 30) (type: boolean)
               Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
@@ -982,7 +982,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (locid >= 30) (type: boolean)
               Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out b/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
index 1b9ec68..68acacf 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
@@ -154,11 +154,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: PARTIAL
+            Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: PARTIAL
             Select Operator
               expressions: state (type: string), locid (type: int)
               outputColumnNames: state, locid
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: PARTIAL
+              Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: PARTIAL
               Group By Operator
                 aggregations: count()
                 keys: state (type: string), locid (type: int)
@@ -248,11 +248,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: year (type: int)
               outputColumnNames: year
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: year (type: int)
                 mode: hash
@@ -301,11 +301,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
               outputColumnNames: state, locid
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), locid (type: int)
                 mode: hash
@@ -354,11 +354,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
               outputColumnNames: state, locid
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
@@ -408,11 +408,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
               outputColumnNames: state, locid
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
@@ -462,11 +462,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
               outputColumnNames: state, locid
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
@@ -516,11 +516,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
               outputColumnNames: state, locid
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
@@ -570,11 +570,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
               outputColumnNames: state, locid
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
@@ -624,11 +624,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
               outputColumnNames: state, locid
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
@@ -682,30 +682,30 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: year (type: int)
               outputColumnNames: year
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: year (type: int)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 4 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 4 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -735,11 +735,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int)
               outputColumnNames: state, locid
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), locid (type: int), '0' (type: string)
                 mode: hash
@@ -791,11 +791,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: PARTIAL
+            Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: PARTIAL
             Select Operator
               expressions: state (type: string), zip (type: bigint)
               outputColumnNames: state, zip
-              Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: PARTIAL
+              Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: PARTIAL
               Group By Operator
                 keys: state (type: string), zip (type: bigint)
                 mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out b/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out
index be3fa1d..0d53b70 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out
@@ -199,21 +199,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: location
-            Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3460 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), country (type: string)
               outputColumnNames: state, country
-              Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 3460 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), country (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 8 Data size: 1384 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 10 Data size: 1730 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                  Statistics: Num rows: 8 Data size: 1384 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 10 Data size: 1730 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
@@ -254,21 +254,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: location
-            Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: PARTIAL
+            Statistics: Num rows: 20 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
             Select Operator
               expressions: state (type: string), votes (type: bigint)
               outputColumnNames: state, votes
-              Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: PARTIAL
+              Statistics: Num rows: 20 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
               Group By Operator
                 keys: state (type: string), votes (type: bigint)
                 mode: hash
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 10 Data size: 860 Basic stats: COMPLETE Column stats: PARTIAL
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: bigint)
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
-                  Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 10 Data size: 860 Basic stats: COMPLETE Column stats: PARTIAL
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: bigint)
@@ -307,21 +307,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: location
-            Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3460 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), country (type: string)
               outputColumnNames: state, country
-              Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 3460 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), country (type: string), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 32 Data size: 8256 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 40 Data size: 10320 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                   sort order: +++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string)
-                  Statistics: Num rows: 32 Data size: 8256 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 40 Data size: 10320 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
@@ -361,11 +361,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: location
-            Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3460 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), country (type: string)
               outputColumnNames: state, country
-              Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 3460 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), country (type: string)
                 mode: hash
@@ -414,11 +414,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: location
-            Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3460 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), country (type: string)
               outputColumnNames: state, country
-              Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 20 Data size: 3460 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 keys: state (type: string), country (type: string), '0' (type: string)
                 mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_join.q.out b/ql/src/test/results/clientpositive/annotate_stats_join.q.out
index bc44cc3..ee05e6e 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_join.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_join.q.out
@@ -164,7 +164,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: e
-            Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: deptid is not null (type: boolean)
               Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -180,7 +180,7 @@ STAGE PLANS:
                   value expressions: _col0 (type: string), _col2 (type: int)
           TableScan
             alias: d
-            Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: deptid is not null (type: boolean)
               Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -235,7 +235,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: emp
-            Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (deptid is not null and lastname is not null) (type: boolean)
               Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -251,7 +251,7 @@ STAGE PLANS:
                   value expressions: _col2 (type: int)
           TableScan
             alias: dept
-            Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (deptid is not null and deptname is not null) (type: boolean)
               Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -301,7 +301,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: e
-            Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (deptid is not null and lastname is not null) (type: boolean)
               Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -317,7 +317,7 @@ STAGE PLANS:
                   value expressions: _col2 (type: int)
           TableScan
             alias: d
-            Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (deptid is not null and deptname is not null) (type: boolean)
               Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -371,7 +371,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: emp
-            Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (deptid is not null and lastname is not null) (type: boolean)
               Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -387,7 +387,7 @@ STAGE PLANS:
                   value expressions: _col2 (type: int)
           TableScan
             alias: dept
-            Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (deptid is not null and deptname is not null) (type: boolean)
               Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -441,7 +441,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: e
-            Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: deptid is not null (type: boolean)
               Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -457,7 +457,7 @@ STAGE PLANS:
                   value expressions: _col0 (type: string), _col2 (type: int)
           TableScan
             alias: d
-            Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: deptid is not null (type: boolean)
               Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -473,7 +473,7 @@ STAGE PLANS:
                   value expressions: _col1 (type: string)
           TableScan
             alias: e
-            Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: deptid is not null (type: boolean)
               Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -528,7 +528,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: e
-            Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: deptid is not null (type: boolean)
               Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -544,7 +544,7 @@ STAGE PLANS:
                   value expressions: _col0 (type: string), _col2 (type: int)
           TableScan
             alias: d
-            Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: deptid is not null (type: boolean)
               Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -560,7 +560,7 @@ STAGE PLANS:
                   value expressions: _col1 (type: string)
           TableScan
             alias: l
-            Statistics: Num rows: 8 Data size: 109 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: locid is not null (type: boolean)
               Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
@@ -617,7 +617,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: e
-            Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (deptid is not null and lastname is not null) (type: boolean)
               Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -633,7 +633,7 @@ STAGE PLANS:
                   value expressions: _col2 (type: int)
           TableScan
             alias: d
-            Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (deptid is not null and deptname is not null) (type: boolean)
               Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -648,7 +648,7 @@ STAGE PLANS:
                   Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: l
-            Statistics: Num rows: 8 Data size: 109 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (locid is not null and state is not null) (type: boolean)
               Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out b/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out
index c864c04..aa380b2 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out
@@ -274,7 +274,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1000 Data size: 130523 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 3856 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ss_store_sk is not null (type: boolean)
               Statistics: Num rows: 964 Data size: 3716 Basic stats: COMPLETE Column stats: COMPLETE
@@ -289,7 +289,7 @@ STAGE PLANS:
                   Statistics: Num rows: 964 Data size: 3716 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: s_store_sk is not null (type: boolean)
               Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -343,7 +343,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1000 Data size: 130523 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 3856 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (ss_store_sk > 0) (type: boolean)
               Statistics: Num rows: 333 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE
@@ -358,7 +358,7 @@ STAGE PLANS:
                   Statistics: Num rows: 333 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (s_store_sk > 0) (type: boolean)
               Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
@@ -412,7 +412,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1000 Data size: 130523 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 7668 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((ss_quantity > 10) and ss_store_sk is not null) (type: boolean)
               Statistics: Num rows: 321 Data size: 2460 Basic stats: COMPLETE Column stats: COMPLETE
@@ -427,7 +427,7 @@ STAGE PLANS:
                   Statistics: Num rows: 321 Data size: 2460 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: PARTIAL
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: PARTIAL
             Filter Operator
               predicate: ((s_company_id > 0) and s_store_sk is not null) (type: boolean)
               Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL
@@ -481,7 +481,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1000 Data size: 130523 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 3856 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ss_store_sk is not null (type: boolean)
               Statistics: Num rows: 964 Data size: 3716 Basic stats: COMPLETE Column stats: COMPLETE
@@ -496,7 +496,7 @@ STAGE PLANS:
                   Statistics: Num rows: 964 Data size: 3716 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((s_floor_space > 0) and s_store_sk is not null) (type: boolean)
               Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
@@ -550,7 +550,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1000 Data size: 130523 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 7668 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((ss_quantity > 10) and ss_store_sk is not null) (type: boolean)
               Statistics: Num rows: 321 Data size: 2460 Basic stats: COMPLETE Column stats: COMPLETE
@@ -565,7 +565,7 @@ STAGE PLANS:
                   Statistics: Num rows: 321 Data size: 2460 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: s_store_sk is not null (type: boolean)
               Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -619,7 +619,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1000 Data size: 130523 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 3856 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ss_store_sk is not null (type: boolean)
               Statistics: Num rows: 964 Data size: 3716 Basic stats: COMPLETE Column stats: COMPLETE
@@ -634,7 +634,7 @@ STAGE PLANS:
                   Statistics: Num rows: 964 Data size: 3716 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: s_store_sk is not null (type: boolean)
               Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -649,7 +649,7 @@ STAGE PLANS:
                   Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: s_store_sk is not null (type: boolean)
               Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -705,7 +705,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1000 Data size: 130523 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 3856 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (ss_store_sk > 1000) (type: boolean)
               Statistics: Num rows: 333 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE
@@ -720,7 +720,7 @@ STAGE PLANS:
                   Statistics: Num rows: 333 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (s_store_sk > 1000) (type: boolean)
               Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
@@ -735,7 +735,7 @@ STAGE PLANS:
                   Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (s_store_sk > 1000) (type: boolean)
               Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
@@ -791,7 +791,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1000 Data size: 130523 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 3856 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ss_store_sk is not null (type: boolean)
               Statistics: Num rows: 964 Data size: 3716 Basic stats: COMPLETE Column stats: COMPLETE
@@ -806,7 +806,7 @@ STAGE PLANS:
                   Statistics: Num rows: 964 Data size: 3716 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((s_floor_space > 1000) and s_store_sk is not null) (type: boolean)
               Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
@@ -821,7 +821,7 @@ STAGE PLANS:
                   Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: s_store_sk is not null (type: boolean)
               Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -877,7 +877,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1000 Data size: 130523 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 7668 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ((ss_quantity > 10) and ss_store_sk is not null) (type: boolean)
               Statistics: Num rows: 321 Data size: 2460 Basic stats: COMPLETE Column stats: COMPLETE
@@ -892,7 +892,7 @@ STAGE PLANS:
                   Statistics: Num rows: 321 Data size: 2460 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: s_store_sk is not null (type: boolean)
               Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -907,7 +907,7 @@ STAGE PLANS:
                   Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: s_store_sk is not null (type: boolean)
               Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -964,7 +964,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1000 Data size: 130523 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1000 Data size: 7656 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (ss_addr_sk is not null and ss_store_sk is not null) (type: boolean)
               Statistics: Num rows: 916 Data size: 7012 Basic stats: COMPLETE Column stats: COMPLETE
@@ -980,7 +980,7 @@ STAGE PLANS:
                   value expressions: _col1 (type: int)
           TableScan
             alias: ca
-            Statistics: Num rows: 20 Data size: 2114 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: ca_address_sk is not null (type: boolean)
               Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1020,7 +1020,7 @@ STAGE PLANS:
               Statistics: Num rows: 210 Data size: 840 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: s
-            Statistics: Num rows: 12 Data size: 3143 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: s_store_sk is not null (type: boolean)
               Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_limit.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_limit.q.out b/ql/src/test/results/clientpositive/annotate_stats_limit.q.out
index 7300ea0..3c4109b 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_limit.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_limit.q.out
@@ -76,7 +76,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3
@@ -99,7 +99,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3
@@ -127,7 +127,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3
@@ -153,7 +153,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_part.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_part.q.out b/ql/src/test/results/clientpositive/annotate_stats_part.q.out
index cf523cb..186f7af 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_part.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_part.q.out
@@ -348,7 +348,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 774 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: PARTIAL
           Select Operator
             expressions: state (type: string)
             outputColumnNames: _col0
@@ -396,7 +396,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 774 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 8 Data size: 720 Basic stats: COMPLETE Column stats: PARTIAL
           Select Operator
             expressions: state (type: string), locid (type: int)
             outputColumnNames: _col0, _col1
@@ -419,7 +419,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 7 Data size: 678 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 7 Data size: 630 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: state (type: string), locid (type: int)
             outputColumnNames: _col0, _col1
@@ -465,7 +465,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 774 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 8 Data size: 2192 Basic stats: COMPLETE Column stats: PARTIAL
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3
@@ -490,7 +490,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 7 Data size: 678 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
           Filter Operator
             predicate: (locid > 0) (type: boolean)
             Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -514,7 +514,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 7 Data size: 678 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
           Filter Operator
             predicate: (locid > 0) (type: boolean)
             Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -538,7 +538,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 7 Data size: 678 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
           Filter Operator
             predicate: (locid > 0) (type: boolean)
             Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_select.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_select.q.out b/ql/src/test/results/clientpositive/annotate_stats_select.q.out
index 877037d..bd645c8 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_select.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_select.q.out
@@ -138,7 +138,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: alltypes_orc
-          Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 2 Data size: 420 Basic stats: COMPLETE Column stats: PARTIAL
           Select Operator
             expressions: bo1 (type: boolean), ti1 (type: tinyint), si1 (type: smallint), i1 (type: int), bi1 (type: bigint), f1 (type: float), d1 (type: double), de1 (type: decimal(10,0)), ts1 (type: timestamp), da1 (type: timestamp), s1 (type: string), vc1 (type: varchar(5)), m1 (type: map<string,string>), l1 (type: array<int>), st1 (type: struct<c1:int,c2:string>)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -161,7 +161,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: alltypes_orc
-          Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: bo1 (type: boolean)
             outputColumnNames: _col0
@@ -186,7 +186,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: alltypes_orc
-          Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: i1 (type: int)
             outputColumnNames: _col0
@@ -209,7 +209,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: alltypes_orc
-          Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: s1 (type: string)
             outputColumnNames: _col0
@@ -257,7 +257,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: alltypes_orc
-          Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 246 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: bo1 (type: boolean), ti1 (type: tinyint), si1 (type: smallint), i1 (type: int), bi1 (type: bigint), f1 (type: float), d1 (type: double), s1 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
@@ -794,7 +794,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: alltypes_orc
-          Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 2 Data size: 420 Basic stats: COMPLETE Column stats: PARTIAL
           Select Operator
             expressions: bo1 (type: boolean), ti1 (type: tinyint), si1 (type: smallint), i1 (type: int), bi1 (type: bigint), f1 (type: float), d1 (type: double), de1 (type: decimal(10,0)), ts1 (type: timestamp), da1 (type: timestamp), s1 (type: string), vc1 (type: varchar(5)), m1 (type: map<string,string>), l1 (type: array<int>), st1 (type: struct<c1:int,c2:string>), 11 (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15
@@ -821,7 +821,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: alltypes_orc
-          Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: i1 (type: int)
             outputColumnNames: _col0
@@ -849,7 +849,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: alltypes_orc
-          Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: i1 (type: int)
             outputColumnNames: _col0
@@ -877,7 +877,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypes_orc
-            Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: i1 (type: int)
               outputColumnNames: _col0
@@ -961,7 +961,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypes_orc
-            Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: i1 (type: int)
               outputColumnNames: _col0
@@ -1090,7 +1090,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypes_orc
-            Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: bo1 (type: boolean)
               Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1128,7 +1128,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypes_orc
-            Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (not bo1) (type: boolean)
               Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_table.q.out b/ql/src/test/results/clientpositive/annotate_stats_table.q.out
index ebc6c5b..0f80755 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_table.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_table.q.out
@@ -155,7 +155,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: emp_orc
-          Statistics: Num rows: 48 Data size: 4512 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 48 Data size: 192 Basic stats: COMPLETE Column stats: PARTIAL
           Select Operator
             expressions: lastname (type: string), deptid (type: int)
             outputColumnNames: _col0, _col1
@@ -180,7 +180,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: emp_orc
-          Statistics: Num rows: 48 Data size: 4512 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 48 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: deptid (type: int)
             outputColumnNames: _col0
@@ -213,7 +213,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: emp_orc
-          Statistics: Num rows: 48 Data size: 4512 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 48 Data size: 4560 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: lastname (type: string), deptid (type: int)
             outputColumnNames: _col0, _col1
@@ -236,7 +236,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: emp_orc
-          Statistics: Num rows: 48 Data size: 4512 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 48 Data size: 4368 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: lastname (type: string)
             outputColumnNames: _col0
@@ -259,7 +259,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: emp_orc
-          Statistics: Num rows: 48 Data size: 4512 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 48 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: deptid (type: int)
             outputColumnNames: _col0
@@ -282,7 +282,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: emp_orc
-          Statistics: Num rows: 48 Data size: 4512 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 48 Data size: 4560 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: lastname (type: string), deptid (type: int)
             outputColumnNames: _col0, _col1

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/annotate_stats_union.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_union.q.out b/ql/src/test/results/clientpositive/annotate_stats_union.q.out
index e09dde3..af7e90a 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_union.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_union.q.out
@@ -76,7 +76,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: state (type: string)
             outputColumnNames: _col0
@@ -99,7 +99,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string)
               outputColumnNames: _col0
@@ -115,7 +115,7 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string)
               outputColumnNames: _col0
@@ -152,7 +152,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
             outputColumnNames: _col0, _col1, _col2, _col3
@@ -175,7 +175,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
               outputColumnNames: _col0, _col1, _col2, _col3
@@ -191,7 +191,7 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
               outputColumnNames: _col0, _col1, _col2, _col3
@@ -318,7 +318,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string)
               outputColumnNames: _col0
@@ -334,7 +334,7 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string)
               outputColumnNames: _col0
@@ -371,7 +371,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: loc_staging
-            Statistics: Num rows: 8 Data size: 109 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string)
               outputColumnNames: _col0
@@ -387,7 +387,7 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             alias: loc_orc
-            Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 688 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: state (type: string)
               outputColumnNames: _col0

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
index d1bc6d4..7822ad9 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
@@ -38,7 +38,7 @@ STAGE PLANS:
         a:cbo_t1:cbo_t3 
           TableScan
             alias: cbo_t3
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3060 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 10) (type: boolean)
               Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE
@@ -56,7 +56,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: cbo_t3
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3060 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 10) (type: boolean)
               Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE
@@ -171,7 +171,7 @@ STAGE PLANS:
         a:cbo_t1:cbo_t3 
           TableScan
             alias: cbo_t3
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3060 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 10) (type: boolean)
               Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE
@@ -189,7 +189,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: cbo_t3
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 3060 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 10) (type: boolean)
               Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE


[06/43] hive git commit: HIVE-11201 : HCatalog is ignoring user specified avro schema in the table definition (Bing Li via Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-11201 : HCatalog  is ignoring user specified avro schema in the table definition (Bing Li via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: 9b263ae78b0cb01c03284cd554aaaed86d681cba
Parents: 526c507
Author: Bing Li <sa...@gmail.com>
Authored: Tue Jul 7 23:56:00 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:30 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hive/hcatalog/mapreduce/SpecialCases.java  | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9b263ae7/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
index 1bf6f07..756abf8 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
@@ -112,14 +112,14 @@ public class SpecialCases {
         colTypes.add(TypeInfoUtils.getTypeInfoFromTypeString(field.getTypeString()));
       }
 
-      jobProperties.put(AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName(),
+      if (jobProperties.get(AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName())==null
+          || jobProperties.get(AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName()).isEmpty()) {
+     
+        jobProperties.put(AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName(),
           AvroSerDe.getSchemaFromCols(properties, colNames, colTypes, null).toString());
-
-
-      for (String propName : jobProperties.keySet()){
-        String propVal = jobProperties.get(propName);
       }
 
+
     }
   }
 


[41/43] hive git commit: HIVE-1841 : datanucleus.fixedDatastore should be true and autoCreate should be false by default (Ashutosh Chauhan via Sushanth Sowmyan)

Posted by om...@apache.org.
HIVE-1841 : datanucleus.fixedDatastore should be true and autoCreate should be false by default (Ashutosh Chauhan via Sushanth Sowmyan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: f28ed81b1ce6bd851ee9a4074d3f9b20924cb8f7
Parents: 87ceba6
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Thu Nov 12 16:27:02 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:35 2015 -0800

----------------------------------------------------------------------
 .../test/org/apache/hive/beeline/cli/TestHiveCli.java    |  7 ++++---
 beeline/src/test/resources/hive-site.xml                 | 11 +++++++++++
 .../src/java/org/apache/hadoop/hive/conf/HiveConf.java   |  4 ++--
 data/conf/hive-site.xml                                  | 10 ++++++++++
 data/conf/llap/hive-site.xml                             | 10 ++++++++++
 data/conf/spark/standalone/hive-site.xml                 | 10 ++++++++++
 data/conf/spark/yarn-client/hive-site.xml                | 11 +++++++++++
 data/conf/tez/hive-site.xml                              | 10 ++++++++++
 8 files changed, 68 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f28ed81b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
----------------------------------------------------------------------
diff --git a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
index e1a565b..21ba690 100644
--- a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
+++ b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java
@@ -17,11 +17,11 @@
  */
 package org.apache.hive.beeline.cli;
 
-import junit.framework.Assert;
 import org.apache.commons.io.IOUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -38,7 +38,6 @@ public class TestHiveCli {
   private static final Logger LOG = LoggerFactory.getLogger(TestHiveCli.class.getName());
   private static final int ERRNO_OK = 0;
   private static final int ERRNO_ARGS = 1;
-  private static final int ERRNO_OTHER = 2;
 
   private final static String SOURCE_CONTEXT =
       "create table if not exists test.testSrcTbl(sc1 string);";
@@ -277,9 +276,11 @@ public class TestHiveCli {
 
   @Before
   public void setup() {
+    System.setProperty("datanucleus.fixedDatastore", "false");
+    System.setProperty("datanucleus.autoCreateSchema", "true");
     cli = new HiveCli();
-    initFromFile();
     redirectOutputStream();
+    initFromFile();
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hive/blob/f28ed81b/beeline/src/test/resources/hive-site.xml
----------------------------------------------------------------------
diff --git a/beeline/src/test/resources/hive-site.xml b/beeline/src/test/resources/hive-site.xml
index d2df03c..b2347c7 100644
--- a/beeline/src/test/resources/hive-site.xml
+++ b/beeline/src/test/resources/hive-site.xml
@@ -23,6 +23,17 @@
     <value>true</value>
     <description>Internal marker for test. Used for masking env-dependent values</description>
   </property>
+
+<property>
+  <name>datanucleus.autoCreateSchema</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>datanucleus.fixedDatastore</name>
+  <value>false</value>
+</property>
+
   <property>
     <name>javax.jdo.option.ConnectionURL</name>
     <value>jdbc:derby:;databaseName=${test.tmp.dir}/metastore_db;create=true</value>

http://git-wip-us.apache.org/repos/asf/hive/blob/f28ed81b/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 01cd731..425c7d9 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -544,9 +544,9 @@ public class HiveConf extends Configuration {
     METASTORE_VALIDATE_CONSTRAINTS("datanucleus.validateConstraints", false,
         "validates existing schema against code. turn this on if you want to verify existing schema"),
     METASTORE_STORE_MANAGER_TYPE("datanucleus.storeManagerType", "rdbms", "metadata store type"),
-    METASTORE_AUTO_CREATE_SCHEMA("datanucleus.autoCreateSchema", true,
+    METASTORE_AUTO_CREATE_SCHEMA("datanucleus.autoCreateSchema", false,
         "creates necessary schema on a startup if one doesn't exist. set this to false, after creating it once"),
-    METASTORE_FIXED_DATASTORE("datanucleus.fixedDatastore", false, ""),
+    METASTORE_FIXED_DATASTORE("datanucleus.fixedDatastore", true, "Dictates whether to allow updates to schema or not."),
     METASTORE_SCHEMA_VERIFICATION("hive.metastore.schema.verification", false,
         "Enforce metastore schema version consistency.\n" +
         "True: Verify that version information stored in metastore matches with one from Hive jars.  Also disable automatic\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/f28ed81b/data/conf/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/hive-site.xml b/data/conf/hive-site.xml
index 1b2359b..d15cc17 100644
--- a/data/conf/hive-site.xml
+++ b/data/conf/hive-site.xml
@@ -59,6 +59,16 @@
 </property>
 
 <property>
+  <name>datanucleus.autoCreateSchema</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>datanucleus.fixedDatastore</name>
+  <value>false</value>
+</property>
+
+<property>
   <name>javax.jdo.option.ConnectionURL</name>
   <value>jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true</value>
 </property>

http://git-wip-us.apache.org/repos/asf/hive/blob/f28ed81b/data/conf/llap/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/llap/hive-site.xml b/data/conf/llap/hive-site.xml
index 77f6945..becb5b2 100644
--- a/data/conf/llap/hive-site.xml
+++ b/data/conf/llap/hive-site.xml
@@ -69,6 +69,16 @@
 </property>
 
 <property>
+  <name>datanucleus.autoCreateSchema</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>datanucleus.fixedDatastore</name>
+  <value>false</value>
+</property>
+
+<property>
   <name>javax.jdo.option.ConnectionURL</name>
   <value>jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true</value>
 </property>

http://git-wip-us.apache.org/repos/asf/hive/blob/f28ed81b/data/conf/spark/standalone/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/spark/standalone/hive-site.xml b/data/conf/spark/standalone/hive-site.xml
index 759f3f7..38d0832 100644
--- a/data/conf/spark/standalone/hive-site.xml
+++ b/data/conf/spark/standalone/hive-site.xml
@@ -45,6 +45,16 @@
 </property>
 
 <property>
+  <name>datanucleus.autoCreateSchema</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>datanucleus.fixedDatastore</name>
+  <value>false</value>
+</property>
+
+<property>
   <name>javax.jdo.option.ConnectionURL</name>
   <value>jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true</value>
 </property>

http://git-wip-us.apache.org/repos/asf/hive/blob/f28ed81b/data/conf/spark/yarn-client/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/spark/yarn-client/hive-site.xml b/data/conf/spark/yarn-client/hive-site.xml
index 40ff688..ada3f3b 100644
--- a/data/conf/spark/yarn-client/hive-site.xml
+++ b/data/conf/spark/yarn-client/hive-site.xml
@@ -45,6 +45,17 @@
 </property>
 
 <property>
+  <name>datanucleus.autoCreateSchema</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>datanucleus.fixedDatastore</name>
+  <value>false</value>
+</property>
+
+
+<property>
   <name>javax.jdo.option.ConnectionURL</name>
   <value>jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true</value>
 </property>

http://git-wip-us.apache.org/repos/asf/hive/blob/f28ed81b/data/conf/tez/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/tez/hive-site.xml b/data/conf/tez/hive-site.xml
index 5074427..d008ad1 100644
--- a/data/conf/tez/hive-site.xml
+++ b/data/conf/tez/hive-site.xml
@@ -63,6 +63,16 @@
 </property>
 
 <property>
+  <name>datanucleus.autoCreateSchema</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>datanucleus.fixedDatastore</name>
+  <value>false</value>
+</property>
+
+<property>
   <name>hive.exec.local.scratchdir</name>
   <value>${test.tmp.dir}/localscratchdir/</value>
   <description>Local scratch space for Hive jobs</description>


[16/43] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 6a80db7..0443f80 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 _size714;
-            ::apache::thrift::protocol::TType _etype717;
-            xfer += iprot->readListBegin(_etype717, _size714);
-            this->success.resize(_size714);
-            uint32_t _i718;
-            for (_i718 = 0; _i718 < _size714; ++_i718)
+            uint32_t _size716;
+            ::apache::thrift::protocol::TType _etype719;
+            xfer += iprot->readListBegin(_etype719, _size716);
+            this->success.resize(_size716);
+            uint32_t _i720;
+            for (_i720 = 0; _i720 < _size716; ++_i720)
             {
-              xfer += iprot->readString(this->success[_i718]);
+              xfer += iprot->readString(this->success[_i720]);
             }
             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 _iter719;
-      for (_iter719 = this->success.begin(); _iter719 != this->success.end(); ++_iter719)
+      std::vector<std::string> ::const_iterator _iter721;
+      for (_iter721 = this->success.begin(); _iter721 != this->success.end(); ++_iter721)
       {
-        xfer += oprot->writeString((*_iter719));
+        xfer += oprot->writeString((*_iter721));
       }
       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 _size720;
-            ::apache::thrift::protocol::TType _etype723;
-            xfer += iprot->readListBegin(_etype723, _size720);
-            (*(this->success)).resize(_size720);
-            uint32_t _i724;
-            for (_i724 = 0; _i724 < _size720; ++_i724)
+            uint32_t _size722;
+            ::apache::thrift::protocol::TType _etype725;
+            xfer += iprot->readListBegin(_etype725, _size722);
+            (*(this->success)).resize(_size722);
+            uint32_t _i726;
+            for (_i726 = 0; _i726 < _size722; ++_i726)
             {
-              xfer += iprot->readString((*(this->success))[_i724]);
+              xfer += iprot->readString((*(this->success))[_i726]);
             }
             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 _size725;
-            ::apache::thrift::protocol::TType _etype728;
-            xfer += iprot->readListBegin(_etype728, _size725);
-            this->success.resize(_size725);
-            uint32_t _i729;
-            for (_i729 = 0; _i729 < _size725; ++_i729)
+            uint32_t _size727;
+            ::apache::thrift::protocol::TType _etype730;
+            xfer += iprot->readListBegin(_etype730, _size727);
+            this->success.resize(_size727);
+            uint32_t _i731;
+            for (_i731 = 0; _i731 < _size727; ++_i731)
             {
-              xfer += iprot->readString(this->success[_i729]);
+              xfer += iprot->readString(this->success[_i731]);
             }
             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 _iter730;
-      for (_iter730 = this->success.begin(); _iter730 != this->success.end(); ++_iter730)
+      std::vector<std::string> ::const_iterator _iter732;
+      for (_iter732 = this->success.begin(); _iter732 != this->success.end(); ++_iter732)
       {
-        xfer += oprot->writeString((*_iter730));
+        xfer += oprot->writeString((*_iter732));
       }
       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 _size731;
-            ::apache::thrift::protocol::TType _etype734;
-            xfer += iprot->readListBegin(_etype734, _size731);
-            (*(this->success)).resize(_size731);
-            uint32_t _i735;
-            for (_i735 = 0; _i735 < _size731; ++_i735)
+            uint32_t _size733;
+            ::apache::thrift::protocol::TType _etype736;
+            xfer += iprot->readListBegin(_etype736, _size733);
+            (*(this->success)).resize(_size733);
+            uint32_t _i737;
+            for (_i737 = 0; _i737 < _size733; ++_i737)
             {
-              xfer += iprot->readString((*(this->success))[_i735]);
+              xfer += iprot->readString((*(this->success))[_i737]);
             }
             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 _size736;
-            ::apache::thrift::protocol::TType _ktype737;
-            ::apache::thrift::protocol::TType _vtype738;
-            xfer += iprot->readMapBegin(_ktype737, _vtype738, _size736);
-            uint32_t _i740;
-            for (_i740 = 0; _i740 < _size736; ++_i740)
+            uint32_t _size738;
+            ::apache::thrift::protocol::TType _ktype739;
+            ::apache::thrift::protocol::TType _vtype740;
+            xfer += iprot->readMapBegin(_ktype739, _vtype740, _size738);
+            uint32_t _i742;
+            for (_i742 = 0; _i742 < _size738; ++_i742)
             {
-              std::string _key741;
-              xfer += iprot->readString(_key741);
-              Type& _val742 = this->success[_key741];
-              xfer += _val742.read(iprot);
+              std::string _key743;
+              xfer += iprot->readString(_key743);
+              Type& _val744 = this->success[_key743];
+              xfer += _val744.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 _iter743;
-      for (_iter743 = this->success.begin(); _iter743 != this->success.end(); ++_iter743)
+      std::map<std::string, Type> ::const_iterator _iter745;
+      for (_iter745 = this->success.begin(); _iter745 != this->success.end(); ++_iter745)
       {
-        xfer += oprot->writeString(_iter743->first);
-        xfer += _iter743->second.write(oprot);
+        xfer += oprot->writeString(_iter745->first);
+        xfer += _iter745->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 _size744;
-            ::apache::thrift::protocol::TType _ktype745;
-            ::apache::thrift::protocol::TType _vtype746;
-            xfer += iprot->readMapBegin(_ktype745, _vtype746, _size744);
-            uint32_t _i748;
-            for (_i748 = 0; _i748 < _size744; ++_i748)
+            uint32_t _size746;
+            ::apache::thrift::protocol::TType _ktype747;
+            ::apache::thrift::protocol::TType _vtype748;
+            xfer += iprot->readMapBegin(_ktype747, _vtype748, _size746);
+            uint32_t _i750;
+            for (_i750 = 0; _i750 < _size746; ++_i750)
             {
-              std::string _key749;
-              xfer += iprot->readString(_key749);
-              Type& _val750 = (*(this->success))[_key749];
-              xfer += _val750.read(iprot);
+              std::string _key751;
+              xfer += iprot->readString(_key751);
+              Type& _val752 = (*(this->success))[_key751];
+              xfer += _val752.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 _size751;
-            ::apache::thrift::protocol::TType _etype754;
-            xfer += iprot->readListBegin(_etype754, _size751);
-            this->success.resize(_size751);
-            uint32_t _i755;
-            for (_i755 = 0; _i755 < _size751; ++_i755)
+            uint32_t _size753;
+            ::apache::thrift::protocol::TType _etype756;
+            xfer += iprot->readListBegin(_etype756, _size753);
+            this->success.resize(_size753);
+            uint32_t _i757;
+            for (_i757 = 0; _i757 < _size753; ++_i757)
             {
-              xfer += this->success[_i755].read(iprot);
+              xfer += this->success[_i757].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 _iter756;
-      for (_iter756 = this->success.begin(); _iter756 != this->success.end(); ++_iter756)
+      std::vector<FieldSchema> ::const_iterator _iter758;
+      for (_iter758 = this->success.begin(); _iter758 != this->success.end(); ++_iter758)
       {
-        xfer += (*_iter756).write(oprot);
+        xfer += (*_iter758).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 _size757;
-            ::apache::thrift::protocol::TType _etype760;
-            xfer += iprot->readListBegin(_etype760, _size757);
-            (*(this->success)).resize(_size757);
-            uint32_t _i761;
-            for (_i761 = 0; _i761 < _size757; ++_i761)
+            uint32_t _size759;
+            ::apache::thrift::protocol::TType _etype762;
+            xfer += iprot->readListBegin(_etype762, _size759);
+            (*(this->success)).resize(_size759);
+            uint32_t _i763;
+            for (_i763 = 0; _i763 < _size759; ++_i763)
             {
-              xfer += (*(this->success))[_i761].read(iprot);
+              xfer += (*(this->success))[_i763].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 _size762;
-            ::apache::thrift::protocol::TType _etype765;
-            xfer += iprot->readListBegin(_etype765, _size762);
-            this->success.resize(_size762);
-            uint32_t _i766;
-            for (_i766 = 0; _i766 < _size762; ++_i766)
+            uint32_t _size764;
+            ::apache::thrift::protocol::TType _etype767;
+            xfer += iprot->readListBegin(_etype767, _size764);
+            this->success.resize(_size764);
+            uint32_t _i768;
+            for (_i768 = 0; _i768 < _size764; ++_i768)
             {
-              xfer += this->success[_i766].read(iprot);
+              xfer += this->success[_i768].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 _iter767;
-      for (_iter767 = this->success.begin(); _iter767 != this->success.end(); ++_iter767)
+      std::vector<FieldSchema> ::const_iterator _iter769;
+      for (_iter769 = this->success.begin(); _iter769 != this->success.end(); ++_iter769)
       {
-        xfer += (*_iter767).write(oprot);
+        xfer += (*_iter769).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 _size768;
-            ::apache::thrift::protocol::TType _etype771;
-            xfer += iprot->readListBegin(_etype771, _size768);
-            (*(this->success)).resize(_size768);
-            uint32_t _i772;
-            for (_i772 = 0; _i772 < _size768; ++_i772)
+            uint32_t _size770;
+            ::apache::thrift::protocol::TType _etype773;
+            xfer += iprot->readListBegin(_etype773, _size770);
+            (*(this->success)).resize(_size770);
+            uint32_t _i774;
+            for (_i774 = 0; _i774 < _size770; ++_i774)
             {
-              xfer += (*(this->success))[_i772].read(iprot);
+              xfer += (*(this->success))[_i774].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 _size773;
-            ::apache::thrift::protocol::TType _etype776;
-            xfer += iprot->readListBegin(_etype776, _size773);
-            this->success.resize(_size773);
-            uint32_t _i777;
-            for (_i777 = 0; _i777 < _size773; ++_i777)
+            uint32_t _size775;
+            ::apache::thrift::protocol::TType _etype778;
+            xfer += iprot->readListBegin(_etype778, _size775);
+            this->success.resize(_size775);
+            uint32_t _i779;
+            for (_i779 = 0; _i779 < _size775; ++_i779)
             {
-              xfer += this->success[_i777].read(iprot);
+              xfer += this->success[_i779].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 _iter778;
-      for (_iter778 = this->success.begin(); _iter778 != this->success.end(); ++_iter778)
+      std::vector<FieldSchema> ::const_iterator _iter780;
+      for (_iter780 = this->success.begin(); _iter780 != this->success.end(); ++_iter780)
       {
-        xfer += (*_iter778).write(oprot);
+        xfer += (*_iter780).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 _size779;
-            ::apache::thrift::protocol::TType _etype782;
-            xfer += iprot->readListBegin(_etype782, _size779);
-            (*(this->success)).resize(_size779);
-            uint32_t _i783;
-            for (_i783 = 0; _i783 < _size779; ++_i783)
+            uint32_t _size781;
+            ::apache::thrift::protocol::TType _etype784;
+            xfer += iprot->readListBegin(_etype784, _size781);
+            (*(this->success)).resize(_size781);
+            uint32_t _i785;
+            for (_i785 = 0; _i785 < _size781; ++_i785)
             {
-              xfer += (*(this->success))[_i783].read(iprot);
+              xfer += (*(this->success))[_i785].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 _size784;
-            ::apache::thrift::protocol::TType _etype787;
-            xfer += iprot->readListBegin(_etype787, _size784);
-            this->success.resize(_size784);
-            uint32_t _i788;
-            for (_i788 = 0; _i788 < _size784; ++_i788)
+            uint32_t _size786;
+            ::apache::thrift::protocol::TType _etype789;
+            xfer += iprot->readListBegin(_etype789, _size786);
+            this->success.resize(_size786);
+            uint32_t _i790;
+            for (_i790 = 0; _i790 < _size786; ++_i790)
             {
-              xfer += this->success[_i788].read(iprot);
+              xfer += this->success[_i790].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 _iter789;
-      for (_iter789 = this->success.begin(); _iter789 != this->success.end(); ++_iter789)
+      std::vector<FieldSchema> ::const_iterator _iter791;
+      for (_iter791 = this->success.begin(); _iter791 != this->success.end(); ++_iter791)
       {
-        xfer += (*_iter789).write(oprot);
+        xfer += (*_iter791).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 _size790;
-            ::apache::thrift::protocol::TType _etype793;
-            xfer += iprot->readListBegin(_etype793, _size790);
-            (*(this->success)).resize(_size790);
-            uint32_t _i794;
-            for (_i794 = 0; _i794 < _size790; ++_i794)
+            uint32_t _size792;
+            ::apache::thrift::protocol::TType _etype795;
+            xfer += iprot->readListBegin(_etype795, _size792);
+            (*(this->success)).resize(_size792);
+            uint32_t _i796;
+            for (_i796 = 0; _i796 < _size792; ++_i796)
             {
-              xfer += (*(this->success))[_i794].read(iprot);
+              xfer += (*(this->success))[_i796].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5099,14 +5099,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size795;
-            ::apache::thrift::protocol::TType _etype798;
-            xfer += iprot->readListBegin(_etype798, _size795);
-            this->success.resize(_size795);
-            uint32_t _i799;
-            for (_i799 = 0; _i799 < _size795; ++_i799)
+            uint32_t _size797;
+            ::apache::thrift::protocol::TType _etype800;
+            xfer += iprot->readListBegin(_etype800, _size797);
+            this->success.resize(_size797);
+            uint32_t _i801;
+            for (_i801 = 0; _i801 < _size797; ++_i801)
             {
-              xfer += iprot->readString(this->success[_i799]);
+              xfer += iprot->readString(this->success[_i801]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5145,10 +5145,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 _iter800;
-      for (_iter800 = this->success.begin(); _iter800 != this->success.end(); ++_iter800)
+      std::vector<std::string> ::const_iterator _iter802;
+      for (_iter802 = this->success.begin(); _iter802 != this->success.end(); ++_iter802)
       {
-        xfer += oprot->writeString((*_iter800));
+        xfer += oprot->writeString((*_iter802));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5193,14 +5193,313 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size801;
-            ::apache::thrift::protocol::TType _etype804;
-            xfer += iprot->readListBegin(_etype804, _size801);
-            (*(this->success)).resize(_size801);
-            uint32_t _i805;
-            for (_i805 = 0; _i805 < _size801; ++_i805)
+            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))[_i805]);
+              xfer += iprot->readString((*(this->success))[_i807]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_table_meta_args::~ThriftHiveMetastore_get_table_meta_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->db_patterns);
+          this->__isset.db_patterns = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tbl_patterns);
+          this->__isset.tbl_patterns = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->tbl_types.clear();
+            uint32_t _size808;
+            ::apache::thrift::protocol::TType _etype811;
+            xfer += iprot->readListBegin(_etype811, _size808);
+            this->tbl_types.resize(_size808);
+            uint32_t _i812;
+            for (_i812 = 0; _i812 < _size808; ++_i812)
+            {
+              xfer += iprot->readString(this->tbl_types[_i812]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.tbl_types = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_args");
+
+  xfer += oprot->writeFieldBegin("db_patterns", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->db_patterns);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tbl_patterns", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->tbl_patterns);
+  xfer += oprot->writeFieldEnd();
+
+  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 _iter813;
+    for (_iter813 = this->tbl_types.begin(); _iter813 != this->tbl_types.end(); ++_iter813)
+    {
+      xfer += oprot->writeString((*_iter813));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_table_meta_pargs::~ThriftHiveMetastore_get_table_meta_pargs() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_pargs");
+
+  xfer += oprot->writeFieldBegin("db_patterns", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->db_patterns)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tbl_patterns", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString((*(this->tbl_patterns)));
+  xfer += oprot->writeFieldEnd();
+
+  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 _iter814;
+    for (_iter814 = (*(this->tbl_types)).begin(); _iter814 != (*(this->tbl_types)).end(); ++_iter814)
+    {
+      xfer += oprot->writeString((*_iter814));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_table_meta_result::~ThriftHiveMetastore_get_table_meta_result() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->success.clear();
+            uint32_t _size815;
+            ::apache::thrift::protocol::TType _etype818;
+            xfer += iprot->readListBegin(_etype818, _size815);
+            this->success.resize(_size815);
+            uint32_t _i819;
+            for (_i819 = 0; _i819 < _size815; ++_i819)
+            {
+              xfer += this->success[_i819].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_result");
+
+  if (this->__isset.success) {
+    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 _iter820;
+      for (_iter820 = this->success.begin(); _iter820 != this->success.end(); ++_iter820)
+      {
+        xfer += (*_iter820).write(oprot);
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_table_meta_presult::~ThriftHiveMetastore_get_table_meta_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            (*(this->success)).clear();
+            uint32_t _size821;
+            ::apache::thrift::protocol::TType _etype824;
+            xfer += iprot->readListBegin(_etype824, _size821);
+            (*(this->success)).resize(_size821);
+            uint32_t _i825;
+            for (_i825 = 0; _i825 < _size821; ++_i825)
+            {
+              xfer += (*(this->success))[_i825].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5338,14 +5637,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size806;
-            ::apache::thrift::protocol::TType _etype809;
-            xfer += iprot->readListBegin(_etype809, _size806);
-            this->success.resize(_size806);
-            uint32_t _i810;
-            for (_i810 = 0; _i810 < _size806; ++_i810)
+            uint32_t _size826;
+            ::apache::thrift::protocol::TType _etype829;
+            xfer += iprot->readListBegin(_etype829, _size826);
+            this->success.resize(_size826);
+            uint32_t _i830;
+            for (_i830 = 0; _i830 < _size826; ++_i830)
             {
-              xfer += iprot->readString(this->success[_i810]);
+              xfer += iprot->readString(this->success[_i830]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5384,10 +5683,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 _iter811;
-      for (_iter811 = this->success.begin(); _iter811 != this->success.end(); ++_iter811)
+      std::vector<std::string> ::const_iterator _iter831;
+      for (_iter831 = this->success.begin(); _iter831 != this->success.end(); ++_iter831)
       {
-        xfer += oprot->writeString((*_iter811));
+        xfer += oprot->writeString((*_iter831));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5432,14 +5731,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size812;
-            ::apache::thrift::protocol::TType _etype815;
-            xfer += iprot->readListBegin(_etype815, _size812);
-            (*(this->success)).resize(_size812);
-            uint32_t _i816;
-            for (_i816 = 0; _i816 < _size812; ++_i816)
+            uint32_t _size832;
+            ::apache::thrift::protocol::TType _etype835;
+            xfer += iprot->readListBegin(_etype835, _size832);
+            (*(this->success)).resize(_size832);
+            uint32_t _i836;
+            for (_i836 = 0; _i836 < _size832; ++_i836)
             {
-              xfer += iprot->readString((*(this->success))[_i816]);
+              xfer += iprot->readString((*(this->success))[_i836]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5749,14 +6048,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 _size817;
-            ::apache::thrift::protocol::TType _etype820;
-            xfer += iprot->readListBegin(_etype820, _size817);
-            this->tbl_names.resize(_size817);
-            uint32_t _i821;
-            for (_i821 = 0; _i821 < _size817; ++_i821)
+            uint32_t _size837;
+            ::apache::thrift::protocol::TType _etype840;
+            xfer += iprot->readListBegin(_etype840, _size837);
+            this->tbl_names.resize(_size837);
+            uint32_t _i841;
+            for (_i841 = 0; _i841 < _size837; ++_i841)
             {
-              xfer += iprot->readString(this->tbl_names[_i821]);
+              xfer += iprot->readString(this->tbl_names[_i841]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5789,10 +6088,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 _iter822;
-    for (_iter822 = this->tbl_names.begin(); _iter822 != this->tbl_names.end(); ++_iter822)
+    std::vector<std::string> ::const_iterator _iter842;
+    for (_iter842 = this->tbl_names.begin(); _iter842 != this->tbl_names.end(); ++_iter842)
     {
-      xfer += oprot->writeString((*_iter822));
+      xfer += oprot->writeString((*_iter842));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5820,10 +6119,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 _iter823;
-    for (_iter823 = (*(this->tbl_names)).begin(); _iter823 != (*(this->tbl_names)).end(); ++_iter823)
+    std::vector<std::string> ::const_iterator _iter843;
+    for (_iter843 = (*(this->tbl_names)).begin(); _iter843 != (*(this->tbl_names)).end(); ++_iter843)
     {
-      xfer += oprot->writeString((*_iter823));
+      xfer += oprot->writeString((*_iter843));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5864,14 +6163,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 _size824;
-            ::apache::thrift::protocol::TType _etype827;
-            xfer += iprot->readListBegin(_etype827, _size824);
-            this->success.resize(_size824);
-            uint32_t _i828;
-            for (_i828 = 0; _i828 < _size824; ++_i828)
+            uint32_t _size844;
+            ::apache::thrift::protocol::TType _etype847;
+            xfer += iprot->readListBegin(_etype847, _size844);
+            this->success.resize(_size844);
+            uint32_t _i848;
+            for (_i848 = 0; _i848 < _size844; ++_i848)
             {
-              xfer += this->success[_i828].read(iprot);
+              xfer += this->success[_i848].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5926,10 +6225,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 _iter829;
-      for (_iter829 = this->success.begin(); _iter829 != this->success.end(); ++_iter829)
+      std::vector<Table> ::const_iterator _iter849;
+      for (_iter849 = this->success.begin(); _iter849 != this->success.end(); ++_iter849)
       {
-        xfer += (*_iter829).write(oprot);
+        xfer += (*_iter849).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5982,14 +6281,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 _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 _size850;
+            ::apache::thrift::protocol::TType _etype853;
+            xfer += iprot->readListBegin(_etype853, _size850);
+            (*(this->success)).resize(_size850);
+            uint32_t _i854;
+            for (_i854 = 0; _i854 < _size850; ++_i854)
             {
-              xfer += (*(this->success))[_i834].read(iprot);
+              xfer += (*(this->success))[_i854].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6175,14 +6474,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 _size835;
-            ::apache::thrift::protocol::TType _etype838;
-            xfer += iprot->readListBegin(_etype838, _size835);
-            this->success.resize(_size835);
-            uint32_t _i839;
-            for (_i839 = 0; _i839 < _size835; ++_i839)
+            uint32_t _size855;
+            ::apache::thrift::protocol::TType _etype858;
+            xfer += iprot->readListBegin(_etype858, _size855);
+            this->success.resize(_size855);
+            uint32_t _i859;
+            for (_i859 = 0; _i859 < _size855; ++_i859)
             {
-              xfer += iprot->readString(this->success[_i839]);
+              xfer += iprot->readString(this->success[_i859]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6237,10 +6536,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 _iter840;
-      for (_iter840 = this->success.begin(); _iter840 != this->success.end(); ++_iter840)
+      std::vector<std::string> ::const_iterator _iter860;
+      for (_iter860 = this->success.begin(); _iter860 != this->success.end(); ++_iter860)
       {
-        xfer += oprot->writeString((*_iter840));
+        xfer += oprot->writeString((*_iter860));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6293,14 +6592,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 _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 _size861;
+            ::apache::thrift::protocol::TType _etype864;
+            xfer += iprot->readListBegin(_etype864, _size861);
+            (*(this->success)).resize(_size861);
+            uint32_t _i865;
+            for (_i865 = 0; _i865 < _size861; ++_i865)
             {
-              xfer += iprot->readString((*(this->success))[_i845]);
+              xfer += iprot->readString((*(this->success))[_i865]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7634,14 +7933,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size846;
-            ::apache::thrift::protocol::TType _etype849;
-            xfer += iprot->readListBegin(_etype849, _size846);
-            this->new_parts.resize(_size846);
-            uint32_t _i850;
-            for (_i850 = 0; _i850 < _size846; ++_i850)
+            uint32_t _size866;
+            ::apache::thrift::protocol::TType _etype869;
+            xfer += iprot->readListBegin(_etype869, _size866);
+            this->new_parts.resize(_size866);
+            uint32_t _i870;
+            for (_i870 = 0; _i870 < _size866; ++_i870)
             {
-              xfer += this->new_parts[_i850].read(iprot);
+              xfer += this->new_parts[_i870].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7670,10 +7969,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 _iter851;
-    for (_iter851 = this->new_parts.begin(); _iter851 != this->new_parts.end(); ++_iter851)
+    std::vector<Partition> ::const_iterator _iter871;
+    for (_iter871 = this->new_parts.begin(); _iter871 != this->new_parts.end(); ++_iter871)
     {
-      xfer += (*_iter851).write(oprot);
+      xfer += (*_iter871).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7697,10 +7996,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 _iter852;
-    for (_iter852 = (*(this->new_parts)).begin(); _iter852 != (*(this->new_parts)).end(); ++_iter852)
+    std::vector<Partition> ::const_iterator _iter872;
+    for (_iter872 = (*(this->new_parts)).begin(); _iter872 != (*(this->new_parts)).end(); ++_iter872)
     {
-      xfer += (*_iter852).write(oprot);
+      xfer += (*_iter872).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7909,14 +8208,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 _size853;
-            ::apache::thrift::protocol::TType _etype856;
-            xfer += iprot->readListBegin(_etype856, _size853);
-            this->new_parts.resize(_size853);
-            uint32_t _i857;
-            for (_i857 = 0; _i857 < _size853; ++_i857)
+            uint32_t _size873;
+            ::apache::thrift::protocol::TType _etype876;
+            xfer += iprot->readListBegin(_etype876, _size873);
+            this->new_parts.resize(_size873);
+            uint32_t _i877;
+            for (_i877 = 0; _i877 < _size873; ++_i877)
             {
-              xfer += this->new_parts[_i857].read(iprot);
+              xfer += this->new_parts[_i877].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7945,10 +8244,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 _iter858;
-    for (_iter858 = this->new_parts.begin(); _iter858 != this->new_parts.end(); ++_iter858)
+    std::vector<PartitionSpec> ::const_iterator _iter878;
+    for (_iter878 = this->new_parts.begin(); _iter878 != this->new_parts.end(); ++_iter878)
     {
-      xfer += (*_iter858).write(oprot);
+      xfer += (*_iter878).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7972,10 +8271,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 _iter859;
-    for (_iter859 = (*(this->new_parts)).begin(); _iter859 != (*(this->new_parts)).end(); ++_iter859)
+    std::vector<PartitionSpec> ::const_iterator _iter879;
+    for (_iter879 = (*(this->new_parts)).begin(); _iter879 != (*(this->new_parts)).end(); ++_iter879)
     {
-      xfer += (*_iter859).write(oprot);
+      xfer += (*_iter879).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8200,14 +8499,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size860;
-            ::apache::thrift::protocol::TType _etype863;
-            xfer += iprot->readListBegin(_etype863, _size860);
-            this->part_vals.resize(_size860);
-            uint32_t _i864;
-            for (_i864 = 0; _i864 < _size860; ++_i864)
+            uint32_t _size880;
+            ::apache::thrift::protocol::TType _etype883;
+            xfer += iprot->readListBegin(_etype883, _size880);
+            this->part_vals.resize(_size880);
+            uint32_t _i884;
+            for (_i884 = 0; _i884 < _size880; ++_i884)
             {
-              xfer += iprot->readString(this->part_vals[_i864]);
+              xfer += iprot->readString(this->part_vals[_i884]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8244,10 +8543,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 _iter865;
-    for (_iter865 = this->part_vals.begin(); _iter865 != this->part_vals.end(); ++_iter865)
+    std::vector<std::string> ::const_iterator _iter885;
+    for (_iter885 = this->part_vals.begin(); _iter885 != this->part_vals.end(); ++_iter885)
     {
-      xfer += oprot->writeString((*_iter865));
+      xfer += oprot->writeString((*_iter885));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8279,10 +8578,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 _iter866;
-    for (_iter866 = (*(this->part_vals)).begin(); _iter866 != (*(this->part_vals)).end(); ++_iter866)
+    std::vector<std::string> ::const_iterator _iter886;
+    for (_iter886 = (*(this->part_vals)).begin(); _iter886 != (*(this->part_vals)).end(); ++_iter886)
     {
-      xfer += oprot->writeString((*_iter866));
+      xfer += oprot->writeString((*_iter886));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8754,14 +9053,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size867;
-            ::apache::thrift::protocol::TType _etype870;
-            xfer += iprot->readListBegin(_etype870, _size867);
-            this->part_vals.resize(_size867);
-            uint32_t _i871;
-            for (_i871 = 0; _i871 < _size867; ++_i871)
+            uint32_t _size887;
+            ::apache::thrift::protocol::TType _etype890;
+            xfer += iprot->readListBegin(_etype890, _size887);
+            this->part_vals.resize(_size887);
+            uint32_t _i891;
+            for (_i891 = 0; _i891 < _size887; ++_i891)
             {
-              xfer += iprot->readString(this->part_vals[_i871]);
+              xfer += iprot->readString(this->part_vals[_i891]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8806,10 +9105,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 _iter872;
-    for (_iter872 = this->part_vals.begin(); _iter872 != this->part_vals.end(); ++_iter872)
+    std::vector<std::string> ::const_iterator _iter892;
+    for (_iter892 = this->part_vals.begin(); _iter892 != this->part_vals.end(); ++_iter892)
     {
-      xfer += oprot->writeString((*_iter872));
+      xfer += oprot->writeString((*_iter892));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8845,10 +9144,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 _iter873;
-    for (_iter873 = (*(this->part_vals)).begin(); _iter873 != (*(this->part_vals)).end(); ++_iter873)
+    std::vector<std::string> ::const_iterator _iter893;
+    for (_iter893 = (*(this->part_vals)).begin(); _iter893 != (*(this->part_vals)).end(); ++_iter893)
     {
-      xfer += oprot->writeString((*_iter873));
+      xfer += oprot->writeString((*_iter893));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9651,14 +9950,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size874;
-            ::apache::thrift::protocol::TType _etype877;
-            xfer += iprot->readListBegin(_etype877, _size874);
-            this->part_vals.resize(_size874);
-            uint32_t _i878;
-            for (_i878 = 0; _i878 < _size874; ++_i878)
+            uint32_t _size894;
+            ::apache::thrift::protocol::TType _etype897;
+            xfer += iprot->readListBegin(_etype897, _size894);
+            this->part_vals.resize(_size894);
+            uint32_t _i898;
+            for (_i898 = 0; _i898 < _size894; ++_i898)
             {
-              xfer += iprot->readString(this->part_vals[_i878]);
+              xfer += iprot->readString(this->part_vals[_i898]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9703,10 +10002,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 _iter879;
-    for (_iter879 = this->part_vals.begin(); _iter879 != this->part_vals.end(); ++_iter879)
+    std::vector<std::string> ::const_iterator _iter899;
+    for (_iter899 = this->part_vals.begin(); _iter899 != this->part_vals.end(); ++_iter899)
     {
-      xfer += oprot->writeString((*_iter879));
+      xfer += oprot->writeString((*_iter899));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9742,10 +10041,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 _iter880;
-    for (_iter880 = (*(this->part_vals)).begin(); _iter880 != (*(this->part_vals)).end(); ++_iter880)
+    std::vector<std::string> ::const_iterator _iter900;
+    for (_iter900 = (*(this->part_vals)).begin(); _iter900 != (*(this->part_vals)).end(); ++_iter900)
     {
-      xfer += oprot->writeString((*_iter880));
+      xfer += oprot->writeString((*_iter900));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9954,14 +10253,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size881;
-            ::apache::thrift::protocol::TType _etype884;
-            xfer += iprot->readListBegin(_etype884, _size881);
-            this->part_vals.resize(_size881);
-            uint32_t _i885;
-            for (_i885 = 0; _i885 < _size881; ++_i885)
+            uint32_t _size901;
+            ::apache::thrift::protocol::TType _etype904;
+            xfer += iprot->readListBegin(_etype904, _size901);
+            this->part_vals.resize(_size901);
+            uint32_t _i905;
+            for (_i905 = 0; _i905 < _size901; ++_i905)
             {
-              xfer += iprot->readString(this->part_vals[_i885]);
+              xfer += iprot->readString(this->part_vals[_i905]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10014,10 +10313,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 _iter886;
-    for (_iter886 = this->part_vals.begin(); _iter886 != this->part_vals.end(); ++_iter886)
+    std::vector<std::string> ::const_iterator _iter906;
+    for (_iter906 = this->part_vals.begin(); _iter906 != this->part_vals.end(); ++_iter906)
     {
-      xfer += oprot->writeString((*_iter886));
+      xfer += oprot->writeString((*_iter906));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10057,10 +10356,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 _iter887;
-    for (_iter887 = (*(this->part_vals)).begin(); _iter887 != (*(this->part_vals)).end(); ++_iter887)
+    std::vector<std::string> ::const_iterator _iter907;
+    for (_iter907 = (*(this->part_vals)).begin(); _iter907 != (*(this->part_vals)).end(); ++_iter907)
     {
-      xfer += oprot->writeString((*_iter887));
+      xfer += oprot->writeString((*_iter907));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11066,14 +11365,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size888;
-            ::apache::thrift::protocol::TType _etype891;
-            xfer += iprot->readListBegin(_etype891, _size888);
-            this->part_vals.resize(_size888);
-            uint32_t _i892;
-            for (_i892 = 0; _i892 < _size888; ++_i892)
+            uint32_t _size908;
+            ::apache::thrift::protocol::TType _etype911;
+            xfer += iprot->readListBegin(_etype911, _size908);
+            this->part_vals.resize(_size908);
+            uint32_t _i912;
+            for (_i912 = 0; _i912 < _size908; ++_i912)
             {
-              xfer += iprot->readString(this->part_vals[_i892]);
+              xfer += iprot->readString(this->part_vals[_i912]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11110,10 +11409,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 _iter893;
-    for (_iter893 = this->part_vals.begin(); _iter893 != this->part_vals.end(); ++_iter893)
+    std::vector<std::string> ::const_iterator _iter913;
+    for (_iter913 = this->part_vals.begin(); _iter913 != this->part_vals.end(); ++_iter913)
     {
-      xfer += oprot->writeString((*_iter893));
+      xfer += oprot->writeString((*_iter913));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11145,10 +11444,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 _iter894;
-    for (_iter894 = (*(this->part_vals)).begin(); _iter894 != (*(this->part_vals)).end(); ++_iter894)
+    std::vector<std::string> ::const_iterator _iter914;
+    for (_iter914 = (*(this->part_vals)).begin(); _iter914 != (*(this->part_vals)).end(); ++_iter914)
     {
-      xfer += oprot->writeString((*_iter894));
+      xfer += oprot->writeString((*_iter914));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11337,17 +11636,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size895;
-            ::apache::thrift::protocol::TType _ktype896;
-            ::apache::thrift::protocol::TType _vtype897;
-            xfer += iprot->readMapBegin(_ktype896, _vtype897, _size895);
-            uint32_t _i899;
-            for (_i899 = 0; _i899 < _size895; ++_i899)
+            uint32_t _size915;
+            ::apache::thrift::protocol::TType _ktype916;
+            ::apache::thrift::protocol::TType _vtype917;
+            xfer += iprot->readMapBegin(_ktype916, _vtype917, _size915);
+            uint32_t _i919;
+            for (_i919 = 0; _i919 < _size915; ++_i919)
             {
-              std::string _key900;
-              xfer += iprot->readString(_key900);
-              std::string& _val901 = this->partitionSpecs[_key900];
-              xfer += iprot->readString(_val901);
+              std::string _key920;
+              xfer += iprot->readString(_key920);
+              std::string& _val921 = this->partitionSpecs[_key920];
+              xfer += iprot->readString(_val921);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11408,11 +11707,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 _iter902;
-    for (_iter902 = this->partitionSpecs.begin(); _iter902 != this->partitionSpecs.end(); ++_iter902)
+    std::map<std::string, std::string> ::const_iterator _iter922;
+    for (_iter922 = this->partitionSpecs.begin(); _iter922 != this->partitionSpecs.end(); ++_iter922)
     {
-      xfer += oprot->writeString(_iter902->first);
-      xfer += oprot->writeString(_iter902->second);
+      xfer += oprot->writeString(_iter922->first);
+      xfer += oprot->writeString(_iter922->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11452,11 +11751,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 _iter903;
-    for (_iter903 = (*(this->partitionSpecs)).begin(); _iter903 != (*(this->partitionSpecs)).end(); ++_iter903)
+    std::map<std::string, std::string> ::const_iterator _iter923;
+    for (_iter923 = (*(this->partitionSpecs)).begin(); _iter923 != (*(this->partitionSpecs)).end(); ++_iter923)
     {
-      xfer += oprot->writeString(_iter903->first);
-      xfer += oprot->writeString(_iter903->second);
+      xfer += oprot->writeString(_iter923->first);
+      xfer += oprot->writeString(_iter923->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11701,17 +12000,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size904;
-            ::apache::thrift::protocol::TType _ktype905;
-            ::apache::thrift::protocol::TType _vtype906;
-            xfer += iprot->readMapBegin(_ktype905, _vtype906, _size904);
-            uint32_t _i908;
-            for (_i908 = 0; _i908 < _size904; ++_i908)
+            uint32_t _size924;
+            ::apache::thrift::protocol::TType _ktype925;
+            ::apache::thrift::protocol::TType _vtype926;
+            xfer += iprot->readMapBegin(_ktype925, _vtype926, _size924);
+            uint32_t _i928;
+            for (_i928 = 0; _i928 < _size924; ++_i928)
             {
-              std::string _key909;
-              xfer += iprot->readString(_key909);
-              std::string& _val910 = this->partitionSpecs[_key909];
-              xfer += iprot->readString(_val910);
+              std::string _key929;
+              xfer += iprot->readString(_key929);
+              std::string& _val930 = this->partitionSpecs[_key929];
+              xfer += iprot->readString(_val930);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11772,11 +12071,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 _iter911;
-    for (_iter911 = this->partitionSpecs.begin(); _iter911 != this->partitionSpecs.end(); ++_iter911)
+    std::map<std::string, std::string> ::const_iterator _iter931;
+    for (_iter931 = this->partitionSpecs.begin(); _iter931 != this->partitionSpecs.end(); ++_iter931)
     {
-      xfer += oprot->writeString(_iter911->first);
-      xfer += oprot->writeString(_iter911->second);
+      xfer += oprot->writeString(_iter931->first);
+      xfer += oprot->writeString(_iter931->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11816,11 +12115,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 _iter912;
-    for (_iter912 = (*(this->partitionSpecs)).begin(); _iter912 != (*(this->partitionSpecs)).end(); ++_iter912)
+    std::map<std::string, std::string> ::const_iterator _iter932;
+    for (_iter932 = (*(this->partitionSpecs)).begin(); _iter932 != (*(this->partitionSpecs)).end(); ++_iter932)
     {
-      xfer += oprot->writeString(_iter912->first);
-      xfer += oprot->writeString(_iter912->second);
+      xfer += oprot->writeString(_iter932->first);
+      xfer += oprot->writeString(_iter932->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11877,14 +12176,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size913;
-            ::apache::thrift::protocol::TType _etype916;
-            xfer += iprot->readListBegin(_etype916, _size913);
-            this->success.resize(_size913);
-            uint32_t _i917;
-            for (_i917 = 0; _i917 < _size913; ++_i917)
+            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 += this->success[_i917].read(iprot);
+              xfer += this->success[_i937].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11947,10 +12246,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 _iter918;
-      for (_iter918 = this->success.begin(); _iter918 != this->success.end(); ++_iter918)
+      std::vector<Partition> ::const_iterator _iter938;
+      for (_iter938 = this->success.begin(); _iter938 != this->success.end(); ++_iter938)
       {
-        xfer += (*_iter918).write(oprot);
+        xfer += (*_iter938).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12007,14 +12306,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size919;
-            ::apache::thrift::protocol::TType _etype922;
-            xfer += iprot->readListBegin(_etype922, _size919);
-            (*(this->success)).resize(_size919);
-            uint32_t _i923;
-            for (_i923 = 0; _i923 < _size919; ++_i923)
+            uint32_t _size939;
+            ::apache::thrift::protocol::TType _etype942;
+            xfer += iprot->readListBegin(_etype942, _size939);
+            (*(this->success)).resize(_size939);
+            uint32_t _i943;
+            for (_i943 = 0; _i943 < _size939; ++_i943)
             {
-              xfer += (*(this->success))[_i923].read(iprot);
+              xfer += (*(this->success))[_i943].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12113,14 +12412,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 _size924;
-            ::apache::thrift::protocol::TType _etype927;
-            xfer += iprot->readListBegin(_etype927, _size924);
-            this->part_vals.resize(_size924);
-            uint32_t _i928;
-            for (_i928 = 0; _i928 < _size924; ++_i928)
+            uint32_t _size944;
+            ::apache::thrift::protocol::TType _etype947;
+            xfer += iprot->readListBegin(_etype947, _size944);
+            this->part_vals.resize(_size944);
+            uint32_t _i948;
+            for (_i948 = 0; _i948 < _size944; ++_i948)
             {
-              xfer += iprot->readString(this->part_vals[_i928]);
+              xfer += iprot->readString(this->part_vals[_i948]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12141,14 +12440,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 _size929;
-            ::apache::thrift::protocol::TType _etype932;
-            xfer += iprot->readListBegin(_etype932, _size929);
-            this->group_names.resize(_size929);
-            uint32_t _i933;
-            for (_i933 = 0; _i933 < _size929; ++_i933)
+            uint32_t _size949;
+            ::apache::thrift::protocol::TType _etype952;
+            xfer += iprot->readListBegin(_etype952, _size949);
+            this->group_names.resize(_size949);
+            uint32_t _i953;
+            for (_i953 = 0; _i953 < _size949; ++_i953)
             {
-              xfer += iprot->readString(this->group_names[_i933]);
+              xfer += iprot->readString(this->group_names[_i953]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12185,10 +12484,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 _iter934;
-    for (_iter934 = this->part_vals.begin(); _iter934 != this->part_vals.end(); ++_iter934)
+    std::vector<std::string> ::const_iterator _iter954;
+    for (_iter954 = this->part_vals.begin(); _iter954 != this->part_vals.end(); ++_iter954)
     {
-      xfer += oprot->writeString((*_iter934));
+      xfer += oprot->writeString((*_iter954));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12201,10 +12500,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 _iter935;
-    for (_iter935 = this->group_names.begin(); _iter935 != this->group_names.end(); ++_iter935)
+    std::vector<std::string> ::const_iterator _iter955;
+    for (_iter955 = this->group_names.begin(); _iter955 != this->group_names.end(); ++_iter955)
     {
-      xfer += oprot->writeString((*_iter935));
+      xfer += oprot->writeString((*_iter955));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12236,10 +12535,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 _iter936;
-    for (_iter936 = (*(this->part_vals)).begin(); _iter936 != (*(this->part_vals)).end(); ++_iter936)
+    std::vector<std::string> ::const_iterator _iter956;
+    for (_iter956 = (*(this->part_vals)).begin(); _iter956 != (*(this->part_vals)).end(); ++_iter956)
     {
-      xfer += oprot->writeString((*_iter936));
+      xfer += oprot->writeString((*_iter956));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12252,10 +12551,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 _iter937;
-    for (_iter937 = (*(this->group_names)).begin(); _iter937 != (*(this->group_names)).end(); ++_iter937)
+    std::vector<std::string> ::const_iterator _iter957;
+    for (_iter957 = (*(this->group_names)).begin(); _iter957 != (*(this->group_names)).end(); ++_iter957)
     {
-      xfer += oprot->writeString((*_iter937));
+      xfer += oprot->writeString((*_iter957));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12814,14 +13113,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size938;
-            ::apache::thrift::protocol::TType _etype941;
-            xfer += iprot->readListBegin(_etype941, _size938);
-            this->success.resize(_size938);
-            uint32_t _i942;
-            for (_i942 = 0; _i942 < _size938; ++_i942)
+            uint32_t _size958;
+            ::apache::thrift::protocol::TType _etype961;
+            xfer += iprot->readListBegin(_etype961, _size958);
+            this->success.resize(_size958);
+            uint32_t _i962;
+            for (_i962 = 0; _i962 < _size958; ++_i962)
             {
-              xfer += this->success[_i942].read(iprot);
+              xfer += this->success[_i962].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12868,10 +13167,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 _iter943;
-      for (_iter943 = this->success.begin(); _iter943 != this->success.end(); ++_iter943)
+      std::vector<Partition> ::const_iterator _iter963;
+      for (_iter963 = this->success.begin(); _iter963 != this->success.end(); ++_iter963)
       {
-        xfer += (*_iter943).write(oprot);
+        xfer += (*_iter963).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12920,14 +13219,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size944;
-            ::apache::thrift::protocol::TType _etype947;
-            xfer += iprot->readListBegin(_etype947, _size944);
-            (*(this->success)).resize(_size944);
-            uint32_t _i948;
-            for (_i948 = 0; _i948 < _size944; ++_i948)
+            uint32_t _size964;
+            ::apache::thrift::protocol::TType _etype967;
+            xfer += iprot->readListBegin(_etype967, _size964);
+            (*(this->success)).resize(_size964);
+            uint32_t _i968;
+            for (_i968 = 0; _i968 < _size964; ++_i968)
             {
-              xfer += (*(this->success))[_i948].read(iprot);
+              xfer += (*(this->success))[_i968].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13026,14 +13325,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 _size949;
-            ::apache::thrift::protocol::TType _etype952;
-            xfer += iprot->readListBegin(_etype952, _size949);
-            this->group_names.resize(_size949);
-            uint32_t _i953;
-            for (_i953 = 0; _i953 < _size949; ++_i953)
+            uint32_t _size969;
+            ::apache::thrift::protocol::TType _etype972;
+            xfer += iprot->readListBegin(_etype972, _size969);
+            this->group_names.resize(_size969);
+            uint32_t _i973;
+            for (_i973 = 0; _i973 < _size969; ++_i973)
             {
-              xfer += iprot->readString(this->group_names[_i953]);
+              xfer += iprot->readString(this->group_names[_i973]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13078,10 +13377,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 _iter954;
-    for (_iter954 = this->group_names.begin(); _iter954 != this->group_names.end(); ++_iter954)
+    std::vector<std::string> ::const_iterator _iter974;
+    for (_iter974 = this->group_names.begin(); _iter974 != this->group_names.end(); ++_iter974)
     {
-      xfer += oprot->writeString((*_iter954));
+      xfer += oprot->writeString((*_iter974));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13121,10 +13420,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 _iter955;
-    for (_iter955 = (*(this->group_names)).begin(); _iter955 != (*(this->group_names)).end(); ++_iter955)
+    std::vector<std::string> ::const_iterator _iter975;
+    for (_iter975 = (*(this->group_names)).begin(); _iter975 != (*(this->group_names)).end(); ++_iter975)
     {
-      xfer += oprot->writeString((*_iter955));
+      xfer += oprot->writeString((*_iter975));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13165,14 +13464,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            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)
+            uint32_t _size976;
+            ::apache::thrift::protocol::TType _etype979;
+            xfer += iprot->readListBegin(_etype979, _size976);
+            this->success.resize(_size976);
+            uint32_t _i980;
+            for (_i980 = 0; _i980 < _size976; ++_i980)
             {
-              xfer += this->success[_i960].read(iprot);
+              xfer += this->success[_i980].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13219,10 +13518,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 _iter961;
-      for (_iter961 = this->success.begin(); _iter961 != this->success.end(); ++_iter961)
+      std::vector<Partition> ::const_iterator _iter981;
+      for (_iter981 = this->success.begin(); _iter981 != this->success.end(); ++_iter981)
       {
-        xfer += (*_iter961).write(oprot);
+        xfer += (*_iter981).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13271,14 +13570,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            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)
+            uint32_t _size982;
+            ::apache::thrift::protocol::TType _etype985;
+            xfer += iprot->readListBegin(_etype985, _size982);
+            (*(this->success)).resize(_size982);
+            uint32_t _i986;
+            for (_i986 = 0; _i986 < _size982; ++_i986)
             {
-              xfer += (*(this->success))[_i966].read(iprot);
+              xfer += (*(this->success))[_i986].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13456,14 +13755,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size967;
-            ::apache::thrift::protocol::TType _etype970;
-            xfer += iprot->readListBegin(_etype970, _size967);
-            this->success.resize(_size967);
-            uint32_t _i971;
-            for (_i971 = 0; _i971 < _size967; ++_i971)
+            uint32_t _size987;
+            ::apache::thrift::protocol::TType _etype990;
+            xfer += iprot->readListBegin(_etype990, _size987);
+            this->success.resize(_size987);
+            uint32_t _i991;
+            for (_i991 = 0; _i991 < _size987; ++_i991)
             {
-              xfer += this->success[_i971].read(iprot);
+              xfer += this->success[_i991].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13510,10 +13809,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 _iter972;
-      for (_iter972 = this->success.begin(); _iter972 != this->success.end(); ++_iter972)
+      std::vector<PartitionSpec> ::const_iterator _iter992;
+      for (_iter992 = this->success.begin(); _iter992 != this->success.end(); ++_iter992)
       {
-        xfer += (*_iter972).write(oprot);
+        xfer += (*_iter992).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13562,14 +13861,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size973;
-            ::apache::thrift::protocol::TType _etype976;
-            xfer += iprot->readListBegin(_etype976, _size973);
-            (*(this->success)).resize(_size973);
-            uint32_t _i977;
-            for (_i977 = 0; _i977 < _size973; ++_i977)
+            uint32_t _size993;
+            ::apache::thrift::protocol::TType _etype996;
+            xfer += iprot->readListBegin(_etype996, _size993);
+            (*(this->success)).resize(_size993);
+            uint32_t _i997;
+            for (_i997 = 0; _i997 < _size993; ++_i997)
             {
-              xfer += (*(this->success))[_i977].read(iprot);
+              xfer += (*(this->success))[_i997].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13747,14 +14046,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size978;
-            ::apache::thrift::protocol::TType _etype981;
-            xfer += iprot->readListBegin(_etype981, _size978);
-            this->success.resize(_size978);
-            uint32_t _i982;
-            for (_i982 = 0; _i982 < _size978; ++_i982)
+            uint32_t _size998;
+            ::apache::thrift::protocol::TType _etype1001;
+            xfer += iprot->readListBegin(_etype1001, _size998);
+            this->success.resize(_size998);
+            uint32_t _i1002;
+            for (_i1002 = 0; _i1002 < _size998; ++_i1002)
             {
-              xfer += iprot->readString(this->success[_i982]);
+              xfer += iprot->readString(this->success[_i1002]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13793,10 +14092,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift:
     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 _iter983;
-      for (_iter983 = this->success.begin(); _iter983 != this->success.end(); ++_iter983)
+      std::vector<std::string> ::const_iterator _iter1003;
+      for (_iter1003 = this->success.begin(); _iter1003 != this->success.end(); ++_iter1003)
       {
-        xfer += oprot->writeString((*_iter983));
+        xfer += oprot->writeString((*_iter1003));
       }
       xfer += oprot->writeListEnd();
     }
@@ -13841,14 +14140,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size984;
-            ::apache::thrift::protocol::TType _etype987;
-            xfer += iprot->readListBegin(_etype987, _size984);
-            (*(this->success)).resize(_size984);
-            uint32_t _i988;
-            for (_i988 = 0; _i988 < _size984; ++_i988)
+            uint32_t _size1004;
+            ::apache::thrift::protocol::TType _etype1007;
+            xfer += iprot->readListBegin(_etype1007, _size1004);
+            (*(this->success)).resize(_size1004);
+            uint32_t

<TRUNCATED>

[18/43] hive git commit: HIVE-12365: Added resource path is sent to cluster as an empty string when externally removed (Chaoyu Tang, reviewed by Xuefu Zhang)

Posted by om...@apache.org.
HIVE-12365: Added resource path is sent to cluster as an empty string when externally removed (Chaoyu Tang, reviewed by Xuefu Zhang)


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

Branch: refs/heads/master-fixed
Commit: 6949855194289487d57d8ae03afe63a271ddc8a3
Parents: cf0729c
Author: ctang <ct...@gmail.com>
Authored: Wed Nov 11 18:37:19 2015 -0500
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:32 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/Utilities.java   |  7 ++++-
 .../clientpositive/add_jar_with_file_removed.q  | 15 +++++++++++
 .../add_jar_with_file_removed.q.out             | 27 ++++++++++++++++++++
 3 files changed, 48 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/69498551/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 02adf0c..fc04f18 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
@@ -2150,7 +2150,12 @@ public final class Utilities {
       List<String> realFiles = new ArrayList<String>(files.size());
       for (String one : files) {
         try {
-          realFiles.add(realFile(one, conf));
+          String onefile = realFile(one, conf);
+          if (onefile != null) {
+            realFiles.add(realFile(one, conf));
+          } else {
+            LOG.warn("The file " + one + " does not exist.");
+          }
         } catch (IOException e) {
           throw new RuntimeException("Cannot validate file " + one + "due to exception: "
               + e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/hive/blob/69498551/ql/src/test/queries/clientpositive/add_jar_with_file_removed.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/add_jar_with_file_removed.q b/ql/src/test/queries/clientpositive/add_jar_with_file_removed.q
new file mode 100644
index 0000000..95d7001
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/add_jar_with_file_removed.q
@@ -0,0 +1,15 @@
+!mkdir ${system:test.tmp.dir}/tmpjars;
+!touch ${system:test.tmp.dir}/tmpjars/added1.jar;
+!touch ${system:test.tmp.dir}/tmpjars/added2.jar;
+
+select count(key) from src;
+
+add jar ${system:test.tmp.dir}/tmpjars/added1.jar;
+add jar ${system:test.tmp.dir}/tmpjars/added2.jar;
+
+select count(key) from src;
+
+!rm ${system:test.tmp.dir}/tmpjars/added1.jar;
+
+select count(key) from src;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/69498551/ql/src/test/results/clientpositive/add_jar_with_file_removed.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/add_jar_with_file_removed.q.out b/ql/src/test/results/clientpositive/add_jar_with_file_removed.q.out
new file mode 100644
index 0000000..cb495f4
--- /dev/null
+++ b/ql/src/test/results/clientpositive/add_jar_with_file_removed.q.out
@@ -0,0 +1,27 @@
+PREHOOK: query: select count(key) from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select count(key) from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+500
+PREHOOK: query: select count(key) from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select count(key) from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+500
+PREHOOK: query: select count(key) from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select count(key) from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+500


[04/43] hive git commit: HIVE-12325 : Turn hive.map.groupby.sorted on by default (Chetna Chaudhari via Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-12325 : Turn hive.map.groupby.sorted on by default (Chetna Chaudhari via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: 8a5040c2a57242bc1926b3c2dabe7b30e59003a5
Parents: 96c45a3
Author: Chetna Chaudhari <ch...@gmail.com>
Authored: Thu Nov 5 20:44:00 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:29 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   6 +-
 .../hive/ql/optimizer/GroupByOptimizer.java     |   8 --
 .../queries/clientpositive/groupby_sort_8.q     |   6 --
 .../clientpositive/groupby_sort_test_1.q        |   1 -
 .../clientpositive/auto_sortmerge_join_10.q.out | 100 +++++++------------
 .../results/clientpositive/bucket_groupby.q.out |  46 +++------
 .../results/clientpositive/groupby_sort_8.q.out |  64 ------------
 .../clientpositive/groupby_sort_test_1.q.out    |  87 ++++++++++------
 .../spark/auto_sortmerge_join_10.q.out          |  45 +++------
 .../tez/auto_sortmerge_join_10.q.out            |  71 ++++++-------
 10 files changed, 155 insertions(+), 279 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8a5040c2/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 7272ea4..7a8517b 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -862,14 +862,10 @@ public class HiveConf extends Configuration {
     HIVEMULTIGROUPBYSINGLEREDUCER("hive.multigroupby.singlereducer", true,
         "Whether to optimize multi group by query to generate single M/R  job plan. If the multi group by query has \n" +
         "common group by keys, it will be optimized to generate single M/R job."),
-    HIVE_MAP_GROUPBY_SORT("hive.map.groupby.sorted", false,
+    HIVE_MAP_GROUPBY_SORT("hive.map.groupby.sorted", true,
         "If the bucketing/sorting properties of the table exactly match the grouping key, whether to perform \n" +
         "the group by in the mapper by using BucketizedHiveInputFormat. The only downside to this\n" +
         "is that it limits the number of mappers to the number of files."),
-    HIVE_MAP_GROUPBY_SORT_TESTMODE("hive.map.groupby.sorted.testmode", false,
-        "If the bucketing/sorting properties of the table exactly match the grouping key, whether to perform \n" +
-        "the group by in the mapper by using BucketizedHiveInputFormat. If the test mode is set, the plan\n" +
-        "is not converted, but a query property is set to denote the same."),
     HIVE_GROUPBY_ORDERBY_POSITION_ALIAS("hive.groupby.orderby.position.alias", false,
         "Whether to enable using Column Position Alias in Group By or Order By"),
     HIVE_NEW_JOB_GROUPING_SET_CARDINALITY("hive.new.job.grouping.set.cardinality", 30,

http://git-wip-us.apache.org/repos/asf/hive/blob/8a5040c2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
index f758776..fe459f7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
@@ -212,11 +212,7 @@ public class GroupByOptimizer implements Transform {
         convertGroupByMapSideSortedGroupBy(hiveConf, groupByOp, depth);
       }
       else if (optimizeDistincts && !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED)) {
-        // In test mode, dont change the query plan. However, setup a query property
         pGraphContext.getQueryProperties().setHasMapGroupBy(true);
-        if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_MAP_GROUPBY_SORT_TESTMODE)) {
-          return;
-        }
         ReduceSinkOperator reduceSinkOp =
             (ReduceSinkOperator)groupByOp.getChildOperators().get(0);
         GroupByDesc childGroupByDesc =
@@ -518,11 +514,7 @@ public class GroupByOptimizer implements Transform {
     // The operators specified by depth and removed from the tree.
     protected void convertGroupByMapSideSortedGroupBy(
         HiveConf conf, GroupByOperator groupByOp, int depth) {
-      // In test mode, dont change the query plan. However, setup a query property
       pGraphContext.getQueryProperties().setHasMapGroupBy(true);
-      if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MAP_GROUPBY_SORT_TESTMODE)) {
-        return;
-      }
 
       if (removeChildren(groupByOp, depth)) {
         // Use bucketized hive input format - that makes sure that one mapper reads the entire file

http://git-wip-us.apache.org/repos/asf/hive/blob/8a5040c2/ql/src/test/queries/clientpositive/groupby_sort_8.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/groupby_sort_8.q b/ql/src/test/queries/clientpositive/groupby_sort_8.q
index f53295e..f0d3a59 100644
--- a/ql/src/test/queries/clientpositive/groupby_sort_8.q
+++ b/ql/src/test/queries/clientpositive/groupby_sort_8.q
@@ -18,10 +18,4 @@ EXPLAIN
 select count(distinct key) from T1;
 select count(distinct key) from T1;
 
-set hive.map.groupby.sorted.testmode=true;
--- In testmode, the plan is not changed
-EXPLAIN
-select count(distinct key) from T1;
-select count(distinct key) from T1;
-
 DROP TABLE T1;

http://git-wip-us.apache.org/repos/asf/hive/blob/8a5040c2/ql/src/test/queries/clientpositive/groupby_sort_test_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/groupby_sort_test_1.q b/ql/src/test/queries/clientpositive/groupby_sort_test_1.q
index 4ec138e..70eef33 100644
--- a/ql/src/test/queries/clientpositive/groupby_sort_test_1.q
+++ b/ql/src/test/queries/clientpositive/groupby_sort_test_1.q
@@ -2,7 +2,6 @@ set hive.enforce.bucketing = true;
 set hive.enforce.sorting = true;
 set hive.exec.reducers.max = 10;
 set hive.map.groupby.sorted=true;
-set hive.map.groupby.sorted.testmode=true;
 
 CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/8a5040c2/ql/src/test/results/clientpositive/auto_sortmerge_join_10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_10.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_10.q.out
index e7f6de3..fb1e656 100644
--- a/ql/src/test/results/clientpositive/auto_sortmerge_join_10.q.out
+++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_10.q.out
@@ -242,15 +242,19 @@ select count(*) from
   on subq1.key = subq2.key
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-6 depends on stages: Stage-1
-  Stage-3 depends on stages: Stage-6
-  Stage-0 depends on stages: Stage-3
+  Stage-5 is a root stage
+  Stage-2 depends on stages: Stage-5
+  Stage-0 depends on stages: Stage-2
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
+  Stage: Stage-5
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        subq1:a 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        subq1:a 
           TableScan
             alias: a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
@@ -259,43 +263,22 @@ STAGE PLANS:
               Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count()
-                bucketGroup: true
                 keys: key (type: int)
-                mode: hash
+                mode: final
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col0 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col1 (type: bigint)
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: count(VALUE._col0)
-          keys: KEY._col0 (type: int)
-          mode: mergepartial
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
-          Select Operator
-            expressions: _col0 (type: int)
-            outputColumnNames: _col0
-            Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+                Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: int)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+                  HashTable Sink Operator
+                    keys:
+                      0 _col0 (type: int)
+                      1 _col0 (type: int)
 
-  Stage: Stage-6
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        subq2:a 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
-        subq2:a 
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
           TableScan
             alias: a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
@@ -306,31 +289,22 @@ STAGE PLANS:
                 expressions: key (type: int)
                 outputColumnNames: _col0
                 Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
-                HashTable Sink Operator
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
                   keys:
                     0 _col0 (type: int)
                     1 _col0 (type: int)
-
-  Stage: Stage-3
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Map Join Operator
-              condition map:
-                   Inner Join 0 to 1
-              keys:
-                0 _col0 (type: int)
-                1 _col0 (type: int)
-              Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE
-              Group By Operator
-                aggregations: count()
-                mode: hash
-                outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  sort order: 
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: bigint)
+                  Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    aggregations: count()
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: bigint)
       Local Work:
         Map Reduce Local Work
       Reduce Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/8a5040c2/ql/src/test/results/clientpositive/bucket_groupby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket_groupby.q.out b/ql/src/test/results/clientpositive/bucket_groupby.q.out
index 1b48d3a..1ac5287 100644
--- a/ql/src/test/results/clientpositive/bucket_groupby.q.out
+++ b/ql/src/test/results/clientpositive/bucket_groupby.q.out
@@ -1191,38 +1191,24 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: count(1)
-                bucketGroup: true
                 keys: _col0 (type: string), _col1 (type: string)
-                mode: hash
+                mode: final
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col0 (type: string), _col1 (type: string)
-                  sort order: ++
-                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col2 (type: bigint)
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: count(VALUE._col0)
-          keys: KEY._col0 (type: string), KEY._col1 (type: string)
-          mode: mergepartial
-          outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-          Select Operator
-            expressions: _col0 (type: string), _col2 (type: bigint)
-            outputColumnNames: _col0, _col1
-            Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
-            Limit
-              Number of rows: 10
-              Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
-                Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
-                table:
-                    input format: org.apache.hadoop.mapred.TextInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: string), _col2 (type: bigint)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                  Limit
+                    Number of rows: 10
+                    Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.TextInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/8a5040c2/ql/src/test/results/clientpositive/groupby_sort_8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_sort_8.q.out b/ql/src/test/results/clientpositive/groupby_sort_8.q.out
index 5152385..5d8f513 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_8.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_8.q.out
@@ -101,70 +101,6 @@ POSTHOOK: Input: default@t1
 POSTHOOK: Input: default@t1@ds=1
 #### A masked pattern was here ####
 5
-PREHOOK: query: -- In testmode, the plan is not changed
-EXPLAIN
-select count(distinct key) from T1
-PREHOOK: type: QUERY
-POSTHOOK: query: -- In testmode, the plan is not changed
-EXPLAIN
-select count(distinct key) from T1
-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: t1
-            Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: key (type: string)
-              outputColumnNames: key
-              Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-              Group By Operator
-                aggregations: count(DISTINCT key)
-                keys: key (type: string)
-                mode: hash
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col0 (type: string)
-                  sort order: +
-                  Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: count(DISTINCT KEY._col0:0._col0)
-          mode: mergepartial
-          outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-            table:
-                input format: org.apache.hadoop.mapred.TextInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
-  Stage: Stage-0
-    Fetch Operator
-      limit: -1
-      Processor Tree:
-        ListSink
-
-PREHOOK: query: select count(distinct key) from T1
-PREHOOK: type: QUERY
-PREHOOK: Input: default@t1
-PREHOOK: Input: default@t1@ds=1
-#### A masked pattern was here ####
-POSTHOOK: query: select count(distinct key) from T1
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@t1
-POSTHOOK: Input: default@t1@ds=1
-#### A masked pattern was here ####
-5
 PREHOOK: query: DROP TABLE T1
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@t1

http://git-wip-us.apache.org/repos/asf/hive/blob/8a5040c2/ql/src/test/results/clientpositive/groupby_sort_test_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_sort_test_1.q.out b/ql/src/test/results/clientpositive/groupby_sort_test_1.q.out
index 8c1765d..dfe0ff1 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_test_1.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_test_1.q.out
@@ -50,8 +50,13 @@ SELECT key, count(1) FROM T1 GROUP BY key
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
+  Stage-4
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
+  Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
   Stage: Stage-1
@@ -67,34 +72,30 @@ STAGE PLANS:
               Group By Operator
                 aggregations: count(1)
                 keys: _col0 (type: string)
-                mode: hash
+                mode: final
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col0 (type: string)
-                  sort order: +
-                  Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col1 (type: bigint)
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: count(VALUE._col0)
-          keys: KEY._col0 (type: string)
-          mode: mergepartial
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
-          Select Operator
-            expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int)
-            outputColumnNames: _col0, _col1
-            Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  name: default.outputtbl1
+                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        name: default.outputtbl1
+
+  Stage: Stage-7
+    Conditional Operator
+
+  Stage: Stage-4
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
 
   Stage: Stage-0
     Move Operator
@@ -109,3 +110,33 @@ STAGE PLANS:
   Stage: Stage-2
     Stats-Aggr Operator
 
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.outputtbl1
+
+  Stage: Stage-5
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.outputtbl1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+

http://git-wip-us.apache.org/repos/asf/hive/blob/8a5040c2/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out
index ee9f448..17d20cb 100644
--- a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out
@@ -206,8 +206,6 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-2
     Spark
-      Edges:
-        Reducer 2 <- Map 1 (GROUP, 1)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -220,43 +218,28 @@ STAGE PLANS:
                     Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      bucketGroup: true
                       keys: key (type: int)
-                      mode: hash
+                      mode: final
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: bigint)
-        Reducer 2 
+                      Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+                      Select Operator
+                        expressions: _col0 (type: int)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+                        Spark HashTable Sink Operator
+                          keys:
+                            0 _col0 (type: int)
+                            1 _col0 (type: int)
             Local Work:
               Map Reduce Local Work
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: count(VALUE._col0)
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: int)
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
-                  Spark HashTable Sink Operator
-                    keys:
-                      0 _col0 (type: int)
-                      1 _col0 (type: int)
 
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 4 <- Map 3 (GROUP, 1)
+        Reducer 3 <- Map 2 (GROUP, 1)
 #### A masked pattern was here ####
       Vertices:
-        Map 3 
+        Map 2 
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -275,7 +258,7 @@ STAGE PLANS:
                           0 _col0 (type: int)
                           1 _col0 (type: int)
                         input vertices:
-                          0 Reducer 2
+                          0 Map 1
                         Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
                           aggregations: count()
@@ -288,7 +271,7 @@ STAGE PLANS:
                             value expressions: _col0 (type: bigint)
             Local Work:
               Map Reduce Local Work
-        Reducer 4 
+        Reducer 3 
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0)

http://git-wip-us.apache.org/repos/asf/hive/blob/8a5040c2/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_10.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_10.q.out
index 0d22ea7..98e099c 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_10.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_10.q.out
@@ -245,8 +245,8 @@ STAGE PLANS:
   Stage: Stage-1
     Tez
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (BROADCAST_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Map 1 <- Map 3 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -259,18 +259,34 @@ STAGE PLANS:
                     Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
-                      bucketGroup: true
                       keys: key (type: int)
-                      mode: hash
+                      mode: final
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: bigint)
-        Map 4 
+                      Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+                      Select Operator
+                        expressions: _col0 (type: int)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col0 (type: int)
+                            1 _col0 (type: int)
+                          input vertices:
+                            1 Map 3
+                          Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+                          HybridGraceHashJoin: true
+                          Group By Operator
+                            aggregations: count()
+                            mode: hash
+                            outputColumnNames: _col0
+                            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                            Reduce Output Operator
+                              sort order: 
+                              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                              value expressions: _col0 (type: bigint)
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -291,37 +307,6 @@ STAGE PLANS:
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0)
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: int)
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
-                  Map Join Operator
-                    condition map:
-                         Inner Join 0 to 1
-                    keys:
-                      0 _col0 (type: int)
-                      1 _col0 (type: int)
-                    input vertices:
-                      1 Map 4
-                    Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE
-                    HybridGraceHashJoin: true
-                    Group By Operator
-                      aggregations: count()
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        sort order: 
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: bigint)
-        Reducer 3 
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE


[22/43] hive git commit: HIVE-12309 : TableScan should colStats when available for better data size estimate (Ashutosh Chauhan via Prasanth J)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
index ee70033..fa29dfe 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
@@ -420,10 +420,10 @@ Stage-0
                         Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator [SEL_1]
                            outputColumnNames:["key","c_int","c_float"]
-                           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                            TableScan [TS_0]
                               alias:cbo_t1
-                              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from cbo_t1 group by c_float, cbo_t1.c_int, key) R group by y, x
 PREHOOK: type: QUERY
@@ -486,10 +486,10 @@ Stage-0
                                     Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
                                     Select Operator [SEL_1]
                                        outputColumnNames:["key","c_int","c_float"]
-                                       Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                       Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                                        TableScan [TS_0]
                                           alias:cbo_t1
-                                          Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                          Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c
 PREHOOK: type: QUERY
@@ -571,7 +571,7 @@ Stage-0
                                     |           Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
                                     |           TableScan [TS_29]
                                     |              alias:cbo_t3
-                                    |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |              Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                                     |<-Reducer 4 [SIMPLE_EDGE]
                                        Reduce Output Operator [RS_32]
                                           key expressions:_col0 (type: string)
@@ -631,7 +631,7 @@ Stage-0
                                                 |                          Statistics:Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
                                                 |                          TableScan [TS_11]
                                                 |                             alias:cbo_t2
-                                                |                             Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |                             Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                                                 |<-Reducer 3 [SIMPLE_EDGE]
                                                    Reduce Output Operator [RS_23]
                                                       key expressions:_col0 (type: string)
@@ -673,7 +673,7 @@ Stage-0
                                                                            Statistics:Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
                                                                            TableScan [TS_0]
                                                                               alias:cbo_t1
-                                                                              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                                              Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)  group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key  having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c  having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0  order by cbo_t3.c_int % c asc, cbo_t3.c_int desc
 PREHOOK: type: QUERY
@@ -751,7 +751,7 @@ Stage-0
                                     |        Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                                     |        TableScan [TS_28]
                                     |           alias:cbo_t3
-                                    |           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |           Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                                     |<-Reducer 4 [SIMPLE_EDGE]
                                        Reduce Output Operator [RS_30]
                                           key expressions:_col0 (type: string)
@@ -814,7 +814,7 @@ Stage-0
                                                 |                             Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                                                 |                             TableScan [TS_0]
                                                 |                                alias:cbo_t1
-                                                |                                Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                |                                Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                                                 |<-Reducer 9 [SIMPLE_EDGE]
                                                    Reduce Output Operator [RS_24]
                                                       key expressions:_col0 (type: string)
@@ -847,7 +847,7 @@ Stage-0
                                                                         Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                                                                         TableScan [TS_13]
                                                                            alias:cbo_t2
-                                                                           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                                           Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)  group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) cbo_t1 right outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 2) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c
 PREHOOK: type: QUERY
@@ -914,7 +914,7 @@ Stage-0
                               |        Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                               |        TableScan [TS_23]
                               |           alias:cbo_t3
-                              |           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                              |           Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                               |<-Reducer 3 [SIMPLE_EDGE]
                               |  Reduce Output Operator [RS_25]
                               |     key expressions:_col0 (type: string)
@@ -959,7 +959,7 @@ Stage-0
                               |                             Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                               |                             TableScan [TS_0]
                               |                                alias:cbo_t1
-                              |                                Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                              |                                Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                               |<-Reducer 7 [SIMPLE_EDGE]
                                  Reduce Output Operator [RS_26]
                                     key expressions:_col0 (type: string)
@@ -992,7 +992,7 @@ Stage-0
                                                       Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                                                       TableScan [TS_13]
                                                          alias:cbo_t2
-                                                         Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                         Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)  group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) cbo_t1 full outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by p+q desc, r asc) cbo_t2 on cbo_t1.a=p full outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int
 PREHOOK: type: QUERY
@@ -1070,7 +1070,7 @@ Stage-0
                                     |        Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                                     |        TableScan [TS_26]
                                     |           alias:cbo_t3
-                                    |           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |           Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                                     |<-Reducer 3 [SIMPLE_EDGE]
                                     |  Reduce Output Operator [RS_28]
                                     |     key expressions:_col0 (type: string)
@@ -1115,7 +1115,7 @@ Stage-0
                                     |                             Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                                     |                             TableScan [TS_0]
                                     |                                alias:cbo_t1
-                                    |                                Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |                                Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                                     |<-Reducer 9 [SIMPLE_EDGE]
                                        Reduce Output Operator [RS_29]
                                           key expressions:_col0 (type: string)
@@ -1160,7 +1160,7 @@ Stage-0
                                                                   Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                                                                   TableScan [TS_13]
                                                                      alias:cbo_t2
-                                                                     Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                                     Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)  group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c
 PREHOOK: type: QUERY
@@ -1230,7 +1230,7 @@ Stage-0
                               |           Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
                               |           TableScan [TS_27]
                               |              alias:cbo_t3
-                              |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                              |              Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                               |<-Reducer 3 [SIMPLE_EDGE]
                                  Reduce Output Operator [RS_30]
                                     key expressions:_col0 (type: string)
@@ -1284,7 +1284,7 @@ Stage-0
                                           |                       Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                                           |                       TableScan [TS_0]
                                           |                          alias:cbo_t1
-                                          |                          Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                          |                          Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                                           |<-Reducer 7 [SIMPLE_EDGE]
                                              Reduce Output Operator [RS_23]
                                                 key expressions:_col0 (type: string)
@@ -1317,7 +1317,7 @@ Stage-0
                                                                   Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                                                                   TableScan [TS_10]
                                                                      alias:cbo_t2
-                                                                     Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                                     Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select unionsrc.key FROM (select 'tst1' as key, count(1) as value from src) unionsrc
 PREHOOK: type: QUERY
@@ -1355,10 +1355,10 @@ Stage-0
                         Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator [SEL_1]
                            outputColumnNames:["key"]
-                           Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                            TableScan [TS_0]
                               alias:src
-                              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select unionsrc.key FROM (select 'max' as key, max(c_int) as value from cbo_t3 s1
 	UNION  ALL
@@ -1416,10 +1416,10 @@ Stage-0
                |                 Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                |                 Select Operator [SEL_1]
                |                    outputColumnNames:["key"]
-               |                    Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |                    Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                |                    TableScan [TS_0]
                |                       alias:s1
-               |                       Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |                       Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Reducer 6 [CONTAINS]
                |  Reduce Output Operator [RS_24]
                |     key expressions:_col0 (type: string)
@@ -1443,10 +1443,10 @@ Stage-0
                |                 Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                |                 Select Operator [SEL_8]
                |                    outputColumnNames:["key"]
-               |                    Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |                    Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                |                    TableScan [TS_7]
                |                       alias:s1
-               |                       Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |                       Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Reducer 8 [CONTAINS]
                   Reduce Output Operator [RS_24]
                      key expressions:_col0 (type: string)
@@ -1470,10 +1470,10 @@ Stage-0
                                  Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                                  Select Operator [SEL_17]
                                     outputColumnNames:["key"]
-                                    Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                    Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                                     TableScan [TS_16]
                                        alias:s1
-                                       Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                       Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select unionsrc.key, count(1) FROM (select 'max' as key, max(c_int) as value from cbo_t3 s1
     UNION  ALL
@@ -1550,10 +1550,10 @@ Stage-0
                      |                    Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                      |                    Select Operator [SEL_1]
                      |                       outputColumnNames:["key"]
-                     |                       Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                     |                       Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                      |                       TableScan [TS_0]
                      |                          alias:s1
-                     |                          Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                     |                          Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                      |<-Reducer 7 [CONTAINS]
                      |  Reduce Output Operator [RS_26]
                      |     key expressions:_col0 (type: string)
@@ -1584,10 +1584,10 @@ Stage-0
                      |                    Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                      |                    Select Operator [SEL_8]
                      |                       outputColumnNames:["key"]
-                     |                       Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                     |                       Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                      |                       TableScan [TS_7]
                      |                          alias:s1
-                     |                          Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                     |                          Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                      |<-Reducer 9 [CONTAINS]
                         Reduce Output Operator [RS_26]
                            key expressions:_col0 (type: string)
@@ -1618,10 +1618,10 @@ Stage-0
                                           Statistics:Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                                           Select Operator [SEL_17]
                                              outputColumnNames:["key"]
-                                             Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                             Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                                              TableScan [TS_16]
                                                 alias:s1
-                                                Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t1.key from cbo_t1 join cbo_t3 where cbo_t1.key=cbo_t3.key and cbo_t1.key >= 1
 PREHOOK: type: QUERY
@@ -1660,7 +1660,7 @@ Stage-0
             |           Statistics:Num rows: 6 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE
             |           TableScan [TS_0]
             |              alias:cbo_t1
-            |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            |              Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 3 [SIMPLE_EDGE]
                Reduce Output Operator [RS_9]
                   key expressions:_col0 (type: string)
@@ -1675,7 +1675,7 @@ Stage-0
                         Statistics:Num rows: 6 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE
                         TableScan [TS_3]
                            alias:cbo_t3
-                           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t1.c_int, cbo_t2.c_int from cbo_t1 left outer join  cbo_t2 on cbo_t1.key=cbo_t2.key
 PREHOOK: type: QUERY
@@ -1715,7 +1715,7 @@ Stage-0
                |        Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                |        TableScan [TS_0]
                |           alias:cbo_t1
-               |           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |           Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 3 [SIMPLE_EDGE]
                   Reduce Output Operator [RS_5]
                      key expressions:_col0 (type: string)
@@ -1728,7 +1728,7 @@ Stage-0
                         Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                         TableScan [TS_2]
                            alias:cbo_t2
-                           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t1.c_int, cbo_t2.c_int from cbo_t1 full outer join  cbo_t2 on cbo_t1.key=cbo_t2.key
 PREHOOK: type: QUERY
@@ -1768,7 +1768,7 @@ Stage-0
                |        Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                |        TableScan [TS_0]
                |           alias:cbo_t1
-               |           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |           Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 3 [SIMPLE_EDGE]
                   Reduce Output Operator [RS_5]
                      key expressions:_col0 (type: string)
@@ -1781,7 +1781,7 @@ Stage-0
                         Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                         TableScan [TS_2]
                            alias:cbo_t2
-                           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select b, cbo_t1.c, cbo_t2.p, q, cbo_t3.c_int from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1) cbo_t1 join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key
 PREHOOK: type: QUERY
@@ -1824,7 +1824,7 @@ Stage-0
                |           Statistics:Num rows: 18 Data size: 1488 Basic stats: COMPLETE Column stats: COMPLETE
                |           TableScan [TS_0]
                |              alias:cbo_t1
-               |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |              Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 3 [SIMPLE_EDGE]
                |  Reduce Output Operator [RS_9]
                |     key expressions:_col0 (type: string)
@@ -1840,7 +1840,7 @@ Stage-0
                |           Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
                |           TableScan [TS_2]
                |              alias:cbo_t3
-               |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |              Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 4 [SIMPLE_EDGE]
                   Reduce Output Operator [RS_11]
                      key expressions:_col0 (type: string)
@@ -1856,7 +1856,7 @@ Stage-0
                            Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
                            TableScan [TS_4]
                               alias:cbo_t2
-                              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select key, cbo_t1.c_int, cbo_t2.p, q from cbo_t1 join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2) cbo_t2 on cbo_t1.key=p join (select key as a, c_int as b, cbo_t3.c_float as c from cbo_t3)cbo_t3 on cbo_t1.key=a
 PREHOOK: type: QUERY
@@ -1899,7 +1899,7 @@ Stage-0
                |           Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
                |           TableScan [TS_0]
                |              alias:cbo_t1
-               |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |              Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 3 [SIMPLE_EDGE]
                |  Reduce Output Operator [RS_9]
                |     key expressions:_col0 (type: string)
@@ -1914,7 +1914,7 @@ Stage-0
                |           Statistics:Num rows: 18 Data size: 1360 Basic stats: COMPLETE Column stats: COMPLETE
                |           TableScan [TS_2]
                |              alias:cbo_t3
-               |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |              Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 4 [SIMPLE_EDGE]
                   Reduce Output Operator [RS_11]
                      key expressions:_col0 (type: string)
@@ -1930,7 +1930,7 @@ Stage-0
                            Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
                            TableScan [TS_4]
                               alias:cbo_t2
-                              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select * from (select q, b, cbo_t2.p, cbo_t1.c, cbo_t3.c_int from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1  where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 full outer join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2  where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0)
 PREHOOK: type: QUERY
@@ -1977,7 +1977,7 @@ Stage-0
                   |           Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
                   |           TableScan [TS_11]
                   |              alias:cbo_t3
-                  |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                  |              Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Reducer 2 [SIMPLE_EDGE]
                      Reduce Output Operator [RS_14]
                         key expressions:_col0 (type: string)
@@ -2008,7 +2008,7 @@ Stage-0
                            |           Statistics:Num rows: 6 Data size: 465 Basic stats: COMPLETE Column stats: COMPLETE
                            |           TableScan [TS_0]
                            |              alias:cbo_t1
-                           |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                           |              Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                            |<-Map 4 [SIMPLE_EDGE]
                               Reduce Output Operator [RS_7]
                                  key expressions:_col0 (type: string)
@@ -2024,7 +2024,7 @@ Stage-0
                                        Statistics:Num rows: 6 Data size: 465 Basic stats: COMPLETE Column stats: COMPLETE
                                        TableScan [TS_3]
                                           alias:cbo_t2
-                                          Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                          Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select * from (select q, b, cbo_t2.p, cbo_t1.c, cbo_t3.c_int from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1  where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 right outer join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2  where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0)
 PREHOOK: type: QUERY
@@ -2070,7 +2070,7 @@ Stage-0
                   |           Statistics:Num rows: 6 Data size: 465 Basic stats: COMPLETE Column stats: COMPLETE
                   |           TableScan [TS_0]
                   |              alias:cbo_t1
-                  |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                  |              Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Map 3 [SIMPLE_EDGE]
                   |  Reduce Output Operator [RS_9]
                   |     key expressions:_col0 (type: string)
@@ -2086,7 +2086,7 @@ Stage-0
                   |           Statistics:Num rows: 6 Data size: 465 Basic stats: COMPLETE Column stats: COMPLETE
                   |           TableScan [TS_3]
                   |              alias:cbo_t2
-                  |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                  |              Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Map 4 [SIMPLE_EDGE]
                      Reduce Output Operator [RS_10]
                         key expressions:_col0 (type: string)
@@ -2099,7 +2099,7 @@ Stage-0
                            Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                            TableScan [TS_6]
                               alias:cbo_t3
-                              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select key, (c_int+1)+2 as x, sum(c_int) from cbo_t1 group by c_float, cbo_t1.c_int, key order by x limit 1
 PREHOOK: type: QUERY
@@ -2154,10 +2154,10 @@ Stage-0
                                  Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
                                  Select Operator [SEL_1]
                                     outputColumnNames:["key","c_int","c_float"]
-                                    Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                    Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                                     TableScan [TS_0]
                                        alias:cbo_t1
-                                       Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                       Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from cbo_t1 group by c_float, cbo_t1.c_int, key) R group by y, x order by x,y limit 1
 PREHOOK: type: QUERY
@@ -2233,10 +2233,10 @@ Stage-0
                                              Statistics:Num rows: 10 Data size: 917 Basic stats: COMPLETE Column stats: COMPLETE
                                              Select Operator [SEL_1]
                                                 outputColumnNames:["key","c_int","c_float"]
-                                                Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                                                 TableScan [TS_0]
                                                    alias:cbo_t1
-                                                   Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                   Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select key from(select key from (select key from cbo_t1 limit 5)cbo_t2  limit 5)cbo_t3  limit 5
 PREHOOK: type: QUERY
@@ -2293,7 +2293,7 @@ Stage-0
                                           Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                                           TableScan [TS_0]
                                              alias:cbo_t1
-                                             Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                             Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select key, c_int from(select key, c_int from (select key, c_int from cbo_t1 order by c_int limit 5)cbo_t1  order by c_int limit 5)cbo_t2  order by c_int limit 5
 PREHOOK: type: QUERY
@@ -2356,7 +2356,7 @@ Stage-0
                                           Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                                           TableScan [TS_0]
                                              alias:cbo_t1
-                                             Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                             Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a limit 5) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc limit 5) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c limit 5
 PREHOOK: type: QUERY
@@ -2441,7 +2441,7 @@ Stage-0
                                        |           Statistics:Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
                                        |           TableScan [TS_31]
                                        |              alias:cbo_t3
-                                       |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                       |              Statistics:Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
                                        |<-Reducer 4 [SIMPLE_EDGE]
                                           Reduce Output Operator [RS_34]
                                              key expressions:_col0 (type: string)
@@ -2507,7 +2507,7 @@ Stage-0
                                                    |                                Statistics:Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
                                                    |                                TableScan [TS_12]
                                                    |                                   alias:cbo_t2
-                                                   |                                   Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                   |                                   Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                                                    |<-Reducer 3 [SIMPLE_EDGE]
                                                       Reduce Output Operator [RS_25]
                                                          key expressions:_col0 (type: string)
@@ -2555,7 +2555,7 @@ Stage-0
                                                                                     Statistics:Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
                                                                                     TableScan [TS_0]
                                                                                        alias:cbo_t1
-                                                                                       Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                                                       Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t1.c_int           from cbo_t1 left semi join   cbo_t2 on cbo_t1.key=cbo_t2.key where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)
 PREHOOK: type: QUERY
@@ -2598,7 +2598,7 @@ Stage-0
                |           Statistics:Num rows: 5 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
                |           TableScan [TS_0]
                |              alias:cbo_t1
-               |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |              Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 3 [SIMPLE_EDGE]
                   Reduce Output Operator [RS_10]
                      key expressions:_col0 (type: string)
@@ -2617,7 +2617,7 @@ Stage-0
                               Statistics:Num rows: 18 Data size: 1360 Basic stats: COMPLETE Column stats: COMPLETE
                               TableScan [TS_3]
                                  alias:cbo_t2
-                                 Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                 Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select * from (select c, b, a from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1  where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2  where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 == 2) and (b > 0 or c >= 0)) R where  (b + 1 = 2) and (R.b > 0 or c >= 0)
 PREHOOK: type: QUERY
@@ -2660,7 +2660,7 @@ Stage-0
                |           Statistics:Num rows: 5 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
                |           TableScan [TS_0]
                |              alias:cbo_t1
-               |              Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |              Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 3 [SIMPLE_EDGE]
                |  Reduce Output Operator [RS_19]
                |     key expressions:_col0 (type: string)
@@ -2679,7 +2679,7 @@ Stage-0
                |              Statistics:Num rows: 5 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
                |              TableScan [TS_7]
                |                 alias:cbo_t2
-               |                 Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+               |                 Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Map 4 [SIMPLE_EDGE]
                   Reduce Output Operator [RS_21]
                      key expressions:_col0 (type: string)
@@ -2698,7 +2698,7 @@ Stage-0
                               Statistics:Num rows: 18 Data size: 1360 Basic stats: COMPLETE Column stats: COMPLETE
                               TableScan [TS_10]
                                  alias:cbo_t3
-                                 Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                 Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select a, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)  group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc) cbo_t1 left semi join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by q+r/10 desc, p) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1  >= 0) and (b > 0 or a >= 0) group by a, c  having a > 0 and (a >=1 or c >= 1) and (a + c) >= 0 order by c, a
 PREHOOK: type: QUERY
@@ -2773,7 +2773,7 @@ Stage-0
                            |              Statistics:Num rows: 6 Data size: 425 Basic stats: COMPLETE Column stats: COMPLETE
                            |              TableScan [TS_32]
                            |                 alias:cbo_t3
-                           |                 Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                           |                 Statistics:Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
                            |<-Reducer 3 [SIMPLE_EDGE]
                            |  Reduce Output Operator [RS_40]
                            |     key expressions:_col0 (type: string)
@@ -2827,7 +2827,7 @@ Stage-0
                            |                                      Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                            |                                      TableScan [TS_0]
                            |                                         alias:cbo_t1
-                           |                                         Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                           |                                         Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
                            |<-Reducer 9 [SIMPLE_EDGE]
                               Reduce Output Operator [RS_42]
                                  key expressions:_col0 (type: string)
@@ -2874,7 +2874,7 @@ Stage-0
                                                             Statistics:Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
                                                             TableScan [TS_17]
                                                                alias:cbo_t2
-                                                               Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                                                               Statistics:Num rows: 20 Data size: 1674 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select cbo_t1.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1
 PREHOOK: type: QUERY
@@ -3032,7 +3032,7 @@ Stage-0
                   |        Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   |        TableScan [TS_0]
                   |           alias:b
-                  |           Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  |           Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Reducer 4 [SIMPLE_EDGE]
                      Reduce Output Operator [RS_11]
                         key expressions:_col1 (type: string)
@@ -3061,7 +3061,7 @@ Stage-0
                                        Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                                        TableScan [TS_3]
                                           alias:b
-                                          Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                          Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select * 
 from src_cbo b 
@@ -3121,7 +3121,7 @@ Stage-0
                   |           Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                   |           TableScan [TS_7]
                   |              alias:b
-                  |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  |              Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Reducer 2 [SIMPLE_EDGE]
                      Reduce Output Operator [RS_10]
                         key expressions:_col1 (type: string), _col0 (type: string)
@@ -3144,10 +3144,10 @@ Stage-0
                                  Statistics:Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
                                  Select Operator [SEL_2]
                                     outputColumnNames:["key","value"]
-                                    Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                    Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                                     TableScan [TS_0]
                                        alias:b
-                                       Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                       Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: create view cv1 as 
 select * 
@@ -3208,7 +3208,7 @@ Stage-0
             |           Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
             |           TableScan [TS_0]
             |              alias:b
-            |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            |              Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 3 [SIMPLE_EDGE]
                Reduce Output Operator [RS_13]
                   key expressions:_col0 (type: string), _col1 (type: string)
@@ -3227,7 +3227,7 @@ Stage-0
                            Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                            TableScan [TS_5]
                               alias:b
-                              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select * 
 from (select * 
@@ -3280,7 +3280,7 @@ Stage-0
             |           Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
             |           TableScan [TS_0]
             |              alias:b
-            |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            |              Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 3 [SIMPLE_EDGE]
                Reduce Output Operator [RS_13]
                   key expressions:_col0 (type: string), _col1 (type: string)
@@ -3299,7 +3299,7 @@ Stage-0
                            Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                            TableScan [TS_5]
                               alias:b
-                              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select * 
 from src_cbo 
@@ -3343,7 +3343,7 @@ Stage-0
             |           Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
             |           TableScan [TS_0]
             |              alias:src_cbo
-            |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            |              Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 3 [SIMPLE_EDGE]
                Reduce Output Operator [RS_13]
                   key expressions:_col0 (type: string)
@@ -3362,7 +3362,7 @@ Stage-0
                            Statistics:Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
                            TableScan [TS_5]
                               alias:src_cbo
-                              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select p.p_partkey, li.l_suppkey 
 from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey 
@@ -3425,7 +3425,7 @@ Stage-0
                |     |           Statistics:Num rows: 16 Data size: 256 Basic stats: COMPLETE Column stats: COMPLETE
                |     |           TableScan [TS_0]
                |     |              alias:lineitem
-               |     |              Statistics:Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: COMPLETE
+               |     |              Statistics:Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
                |     |<-Map 4 [SIMPLE_EDGE]
                |        Reduce Output Operator [RS_19]
                |           key expressions:_col0 (type: int)
@@ -3444,7 +3444,7 @@ Stage-0
                |                    Statistics:Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE
                |                    TableScan [TS_3]
                |                       alias:lineitem
-               |                       Statistics:Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: COMPLETE
+               |                       Statistics:Num rows: 100 Data size: 9600 Basic stats: COMPLETE Column stats: COMPLETE
                |<-Reducer 6 [SIMPLE_EDGE]
                   Reduce Output Operator [RS_24]
                      key expressions:_col0 (type: int)
@@ -3470,7 +3470,7 @@ Stage-0
                                  Statistics:Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                                  TableScan [TS_8]
                                     alias:lineitem
-                                    Statistics:Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: COMPLETE
+                                    Statistics:Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select key, value, count(*) 
 from src_cbo b
@@ -3553,7 +3553,7 @@ Stage-0
             |                 |           Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
             |                 |           TableScan [TS_0]
             |                 |              alias:b
-            |                 |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            |                 |              Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             |                 |<-Map 5 [SIMPLE_EDGE]
             |                    Reduce Output Operator [RS_15]
             |                       key expressions:_col0 (type: string)
@@ -3572,7 +3572,7 @@ Stage-0
             |                                Statistics:Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
             |                                TableScan [TS_7]
             |                                   alias:b
-            |                                   Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            |                                   Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Reducer 7 [SIMPLE_EDGE]
                Reduce Output Operator [RS_34]
                   key expressions:_col0 (type: bigint)
@@ -3614,7 +3614,7 @@ Stage-0
                                           Statistics:Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
                                           TableScan [TS_22]
                                              alias:b
-                                             Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                             Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select p_mfgr, p_name, avg(p_size) 
 from part 
@@ -3681,7 +3681,7 @@ Stage-0
             |                    Statistics:Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
             |                    TableScan [TS_0]
             |                       alias:part
-            |                       Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+            |                       Statistics:Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Reducer 5 [SIMPLE_EDGE]
                Reduce Output Operator [RS_18]
                   key expressions:_col0 (type: string)
@@ -3709,11 +3709,11 @@ Stage-0
                                     key expressions:p_mfgr (type: string), p_size (type: int)
                                     Map-reduce partition columns:p_mfgr (type: string)
                                     sort order:++
-                                    Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                                    Statistics:Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                                     value expressions:p_name (type: string)
                                     TableScan [TS_7]
                                        alias:part
-                                       Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                                       Statistics:Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select * 
 from src_cbo 
@@ -3780,7 +3780,7 @@ Stage-0
                         |           Statistics:Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
                         |           TableScan [TS_14]
                         |              alias:src_cbo
-                        |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                        |              Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                         |<-Reducer 2 [SIMPLE_EDGE]
                            Reduce Output Operator [RS_21]
                               key expressions:_col0 (type: string)
@@ -3803,7 +3803,7 @@ Stage-0
                               |        Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                               |        TableScan [TS_0]
                               |           alias:src_cbo
-                              |           Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                              |           Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                               |<-Reducer 6 [SIMPLE_EDGE]
                                  Reduce Output Operator [RS_19]
                                     sort order:
@@ -3833,7 +3833,7 @@ Stage-0
                                                          Statistics:Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE
                                                          TableScan [TS_3]
                                                             alias:src_cbo
-                                                            Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                                            Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select p_mfgr, b.p_name, p_size 
 from part b 
@@ -3892,7 +3892,7 @@ Stage-0
                   |           Statistics:Num rows: 8 Data size: 1784 Basic stats: COMPLETE Column stats: COMPLETE
                   |           TableScan [TS_14]
                   |              alias:b
-                  |              Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                  |              Statistics:Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Reducer 2 [SIMPLE_EDGE]
                      Reduce Output Operator [RS_21]
                         key expressions:_col0 (type: string), _col1 (type: string)
@@ -3915,7 +3915,7 @@ Stage-0
                         |        Statistics:Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                         |        TableScan [TS_0]
                         |           alias:b
-                        |           Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                        |           Statistics:Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                         |<-Reducer 5 [SIMPLE_EDGE]
                            Reduce Output Operator [RS_19]
                               sort order:
@@ -3945,7 +3945,7 @@ Stage-0
                                                    Statistics:Num rows: 1 Data size: 223 Basic stats: COMPLETE Column stats: COMPLETE
                                                    TableScan [TS_3]
                                                       alias:b
-                                                      Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                                                      Statistics:Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select p_name, p_size 
 from 
@@ -4020,7 +4020,7 @@ Stage-0
                   |        |        Statistics:Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                   |        |        TableScan [TS_0]
                   |        |           alias:part
-                  |        |           Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                  |        |           Statistics:Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                   |        |<-Reducer 6 [SIMPLE_EDGE]
                   |           Reduce Output Operator [RS_28]
                   |              key expressions:_col0 (type: double)
@@ -4045,7 +4045,7 @@ Stage-0
                   |                          Statistics:Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                   |                          TableScan [TS_3]
                   |                             alias:part
-                  |                             Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                  |                             Statistics:Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Reducer 8 [SIMPLE_EDGE]
                      Reduce Output Operator [RS_31]
                         sort order:
@@ -4082,7 +4082,7 @@ Stage-0
                                                    Statistics:Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                                                    TableScan [TS_10]
                                                       alias:part
-                                                      Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                                                      Statistics:Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select b.p_mfgr, min(p_retailprice) 
 from part b 
@@ -4175,7 +4175,7 @@ Stage-0
                         |                    Statistics:Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
                         |                    TableScan [TS_22]
                         |                       alias:b
-                        |                       Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                        |                       Statistics:Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE
                         |<-Reducer 3 [SIMPLE_EDGE]
                            Reduce Output Operator [RS_33]
                               key expressions:_col0 (type: string), _col1 (type: double)
@@ -4211,10 +4211,10 @@ Stage-0
                               |              Statistics:Num rows: 5 Data size: 530 Basic stats: COMPLETE Column stats: COMPLETE
                               |              Select Operator [SEL_2]
                               |                 outputColumnNames:["p_mfgr","p_retailprice"]
-                              |                 Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                              |                 Statistics:Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE
                               |                 TableScan [TS_0]
                               |                    alias:b
-                              |                    Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                              |                    Statistics:Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE
                               |<-Reducer 8 [SIMPLE_EDGE]
                                  Reduce Output Operator [RS_31]
                                     sort order:
@@ -4261,10 +4261,10 @@ Stage-0
                                                                   Statistics:Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
                                                                   Select Operator [SEL_8]
                                                                      outputColumnNames:["p_mfgr","p_retailprice"]
-                                                                     Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                                                                     Statistics:Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE
                                                                      TableScan [TS_7]
                                                                         alias:b
-                                                                        Statistics:Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
+                                                                        Statistics:Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select count(c_int) over(), sum(c_float) over(), max(c_int) over(), min(c_int) over(), row_number() over(), rank() over(), dense_rank() over(), percent_rank() over(), lead(c_int, 2, c_int) over(), lag(c_float, 2, c_float) over() from cbo_t1
 PREHOOK: type: QUERY
@@ -4298,11 +4298,11 @@ Stage-0
                         key expressions:0 (type: int)
                         Map-reduce partition columns:0 (type: int)
                         sort order:+
-                        Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics:Num rows: 20 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions:c_int (type: int), c_float (type: float)
                         TableScan [TS_0]
                            alias:cbo_t1
-                           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 20 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select * from (select count(c_int) over(), sum(c_float) over(), max(c_int) over(), min(c_int) over(), row_number() over(), rank() over(), dense_rank() over(), percent_rank() over(), lead(c_int, 2, c_int) over(), lag(c_float, 2, c_float) over() from cbo_t1) cbo_t1
 PREHOOK: type: QUERY
@@ -4336,11 +4336,11 @@ Stage-0
                         key expressions:0 (type: int)
                         Map-reduce partition columns:0 (type: int)
                         sort order:+
-                        Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics:Num rows: 20 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions:c_int (type: int), c_float (type: float)
                         TableScan [TS_0]
                            alias:cbo_t1
-                           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 20 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select i, a, h, b, c, d, e, f, g, a as x, a +1 as y from (select max(c_int) over (partition by key order by value range UNBOUNDED PRECEDING) a, min(c_int) over (partition by key order by value range current row) b, count(c_int) over(partition by key order by value range 1 PRECEDING) c, avg(value) over (partition by key order by value range between unbounded preceding and unbounded following) d, sum(value) over (partition by key order by value range between unbounded preceding and current row) e, avg(c_float) over (partition by key order by value range between 1 preceding and unbounded following) f, sum(c_float) over (partition by key order by value range between 1 preceding and current row) g, max(c_float) over (partition by key order by value range between 1 preceding and unbounded following) h, min(c_float) over (partition by key order by value range between 1 preceding and 1 following) i from cbo_t1) cbo_t1
 PREHOOK: type: QUERY
@@ -4374,11 +4374,11 @@ Stage-0
                         key expressions:key (type: string), value (type: string)
                         Map-reduce partition columns:key (type: string)
                         sort order:++
-                        Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics:Num rows: 20 Data size: 3204 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions:c_int (type: int), c_float (type: float)
                         TableScan [TS_0]
                            alias:cbo_t1
-                           Statistics:Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 20 Data size: 3204 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select *, rank() over(partition by key order by value) as rr from src1
 PREHOOK: type: QUERY
@@ -4412,10 +4412,10 @@ Stage-0
                         key expressions:key (type: string), value (type: string)
                         Map-reduce partition columns:key (type: string)
                         sort order:++
-                        Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics:Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                         TableScan [TS_0]
                            alias:src1
-                           Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain
 select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt))
@@ -4500,7 +4500,7 @@ Stage-0
                                     |           Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                                     |           TableScan [TS_0]
                                     |              alias:y
-                                    |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |              Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                                     |<-Map 5 [SIMPLE_EDGE]
                                        Reduce Output Operator [RS_7]
                                           key expressions:_col0 (type: string)
@@ -4515,7 +4515,7 @@ Stage-0
                                                 Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                                                 TableScan [TS_2]
                                                    alias:x
-                                                   Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+                                                   Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain
 select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt))
@@ -4600,7 +4600,7 @@ Stage-0
                                     |           Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                                     |           TableScan [TS_0]
                                     |              alias:y
-                                    |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |              Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                                     |<-Map 5 [SIMPLE_EDGE]
                                        Reduce Output Operator [RS_7]
                                           key expressions:_col0 (type: string)
@@ -4615,7 +4615,7 @@ Stage-0
                                                 Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                                                 TableScan [TS_2]
                                                    alias:x
-                                                   Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+                                                   Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain
 select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt))
@@ -4701,7 +4701,7 @@ Stage-0
                                     |           Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                                     |           TableScan [TS_2]
                                     |              alias:x
-                                    |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+                                    |              Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                                     |<-Select Operator [SEL_1]
                                           outputColumnNames:["_col0"]
                                           Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4710,7 +4710,7 @@ Stage-0
                                              Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                                              TableScan [TS_0]
                                                 alias:y
-                                                Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                                Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain
 select SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt))
@@ -4792,7 +4792,7 @@ Stage-0
                                  |           Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                                  |           TableScan [TS_0]
                                  |              alias:x
-                                 |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+                                 |              Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                                  |<-Map 5 [SIMPLE_EDGE]
                                     Reduce Output Operator [RS_9]
                                        key expressions:_col0 (type: string)
@@ -4811,7 +4811,7 @@ Stage-0
                                                 Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                                                 TableScan [TS_2]
                                                    alias:y
-                                                   Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                                   Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain create table abcd (a int, b int, c int, d int)
 PREHOOK: type: CREATETABLE
@@ -5198,7 +5198,7 @@ Stage-0
             |        Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
             |        TableScan [TS_0]
             |           alias:src
-            |           Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            |           Statistics:Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 3 [SIMPLE_EDGE]
                Reduce Output Operator [RS_6]
                   sort order:
@@ -5281,7 +5281,7 @@ Stage-3
                                              Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                                              TableScan [TS_0]
                                                 alias:src
-                                                Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                                Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Stage-0
                Move Operator
                    Please refer to the previous Stage-1
@@ -5349,7 +5349,7 @@ Stage-3
                                              Statistics:Num rows: 500 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
                                              TableScan [TS_0]
                                                 alias:src
-                                                Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                                Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Stage-0
                Move Operator
                    Please refer to the previous Stage-1
@@ -5451,7 +5451,7 @@ Stage-0
                   |           Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                   |           TableScan [TS_0]
                   |              alias:src
-                  |              Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  |              Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   |<-Map 4 [SIMPLE_EDGE]
                      Reduce Output Operator [RS_7]
                         sort order:
@@ -5465,7 +5465,7 @@ Stage-0
                               Statistics:Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                               TableScan [TS_3]
                                  alias:src
-                                 Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                                 Statistics:Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: CREATE TABLE myinput1(key int, value int)
 PREHOOK: type: CREATETABLE
@@ -6437,7 +6437,7 @@ Stage-0
             |     |           Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             |     |           TableScan [TS_3]
             |     |              alias:src1
-            |     |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+            |     |              Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             |     |<-Select Operator [SEL_2]
             |           outputColumnNames:["_col0","_col1"]
             |           Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
@@ -6446,7 +6446,7 @@ Stage-0
             |              Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
             |              TableScan [TS_0]
             |                 alias:srcpart
-            |                 Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE
+            |                 Statistics:Num rows: 2000 Data size: 356000 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 4 [SIMPLE_EDGE]
                Reduce Output Operator [RS_16]
                   key expressions:_col0 (type: string)
@@ -6461,7 +6461,7 @@ Stage-0
                         Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
                         TableScan [TS_5]
                            alias:src
-                           Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
 
 PREHOOK: query: explain select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450'
 PREHOOK: type: QUERY
@@ -6514,7 +6514,7 @@ Stage-0
             |     |           Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             |     |           TableScan [TS_3]
             |     |              alias:src1
-            |     |              Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+            |     |              Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             |     |<-Select Operator [SEL_2]
             |           outputColumnNames:["_col0","_col1"]
             |           Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
@@ -6523,7 +6523,7 @@ Stage-0
             |              Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
             |              TableScan [TS_0]
             |                 alias:srcpart
-            |                 Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE
+            |                 Statistics:Num rows: 2000 Data size: 356000 Basic stats: COMPLETE Column stats: COMPLETE
             |<-Map 4 [SIMPLE_EDGE]
                Reduce Output Operator [RS_16]
                   key expressions:_col0 (type: string)
@@ -6538,7 +6538,7 @@ Stage-0
                         Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
                         TableScan [TS_5]
                            alias:src
-                           Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                           Statistics:Num rows: 500 Data size: 45500 Basic stats: COMPL

<TRUNCATED>

[09/43] hive git commit: HIVE-12312 : Excessive logging in PPD code (Carter Shanklin via Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-12312 : Excessive logging in PPD code (Carter Shanklin via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: 954baa89bb4ee3e6ba7ed45157553dae633b3c9f
Parents: 8e3d9e9
Author: Carter Shanklin <ca...@hortonworks.com>
Authored: Sun Nov 8 08:11:00 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:30 2015 -0800

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/954baa89/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
index 3605484..1702628 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
@@ -709,7 +709,7 @@ public final class OpProcFactory {
      * @param ewi
      */
     protected void logExpr(Node nd, ExprWalkerInfo ewi) {
-      if (!LOG.isInfoEnabled()) return;
+      if (!LOG.isDebugEnabled()) return;
       for (Entry<String, List<ExprNodeDesc>> e : ewi.getFinalCandidates().entrySet()) {
         StringBuilder sb = new StringBuilder("Pushdown predicates of ").append(nd.getName())
             .append(" for alias ").append(e.getKey()).append(": ");
@@ -721,7 +721,7 @@ public final class OpProcFactory {
           isFirst = false;
           sb.append(n.getExprString());
         }
-        LOG.info(sb.toString());
+        LOG.debug(sb.toString());
       }
     }
 


[30/43] hive git commit: HIVE-11525: Tez Bucket pruning (Gopal V, reviewed by Sergey Shelukhin)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/ql/src/test/results/clientpositive/bucketpruning1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketpruning1.q.out b/ql/src/test/results/clientpositive/bucketpruning1.q.out
new file mode 100644
index 0000000..d11239f
--- /dev/null
+++ b/ql/src/test/results/clientpositive/bucketpruning1.q.out
@@ -0,0 +1,2282 @@
+PREHOOK: query: CREATE TABLE srcbucket_pruned(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 16 BUCKETS STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srcbucket_pruned
+POSTHOOK: query: CREATE TABLE srcbucket_pruned(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 16 BUCKETS STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srcbucket_pruned
+PREHOOK: query: -- cannot prune 2-key scenarios without a smarter optimizer
+CREATE TABLE srcbucket_unpruned(key int, value string) partitioned by (ds string) CLUSTERED BY (key,value) INTO 16 BUCKETS STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srcbucket_unpruned
+POSTHOOK: query: -- cannot prune 2-key scenarios without a smarter optimizer
+CREATE TABLE srcbucket_unpruned(key int, value string) partitioned by (ds string) CLUSTERED BY (key,value) INTO 16 BUCKETS STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srcbucket_unpruned
+PREHOOK: query: -- good cases
+
+explain extended
+select * from srcbucket_pruned where key = 1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- good cases
+
+explain extended
+select * from srcbucket_pruned where key = 1
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            1
+
+
+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: srcbucket_pruned
+            filterExpr: (key = 1) (type: boolean)
+            buckets included: [1,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (key = 1) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 1 (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 16
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 16
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            16
+
+
+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: srcbucket_pruned
+            filterExpr: (key = 16) (type: boolean)
+            buckets included: [0,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (key = 16) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 16 (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 17
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 17
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            17
+
+
+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: srcbucket_pruned
+            filterExpr: (key = 17) (type: boolean)
+            buckets included: [1,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (key = 17) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 17 (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 16+1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 16+1
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            +
+               16
+               1
+
+
+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: srcbucket_pruned
+            filterExpr: (key = 17) (type: boolean)
+            buckets included: [1,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (key = 17) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 17 (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = '11'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = '11'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            '11'
+
+
+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: srcbucket_pruned
+            filterExpr: (key = 11) (type: boolean)
+            buckets included: [11,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (key = 11) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 11 (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            =
+               TOK_TABLE_OR_COL
+                  key
+               1
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+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: srcbucket_pruned
+            filterExpr: ((key = 1) and (ds = '2008-04-08')) (type: boolean)
+            buckets included: [1,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: ((key = 1) and (ds = '2008-04-08')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 1 (type: int), value (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' and value='One'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' and value='One'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     ds
+                  '2008-04-08'
+            =
+               TOK_TABLE_OR_COL
+                  value
+               'One'
+
+
+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: srcbucket_pruned
+            filterExpr: (((key = 1) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+            buckets included: [1,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (((key = 1) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 1 (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key = 1 and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key = 1 and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'One'
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+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: srcbucket_pruned
+            filterExpr: (((value = 'One') and (key = 1)) and (ds = '2008-04-08')) (type: boolean)
+            buckets included: [1,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (((value = 'One') and (key = 1)) and (ds = '2008-04-08')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 1 (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3)
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         TOK_FUNCTION
+            in
+            TOK_TABLE_OR_COL
+               key
+            2
+            3
+
+
+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: srcbucket_pruned
+            filterExpr: (key) IN (2, 3) (type: boolean)
+            buckets included: [2,3,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (key) IN (2, 3) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3) and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3) and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            TOK_FUNCTION
+               in
+               TOK_TABLE_OR_COL
+                  key
+               2
+               3
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+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: srcbucket_pruned
+            filterExpr: ((key) IN (2, 3) and (ds = '2008-04-08')) (type: boolean)
+            buckets included: [2,3,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: ((key) IN (2, 3) and (ds = '2008-04-08')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3) and ds='2008-04-08' and value='One'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3) and ds='2008-04-08' and value='One'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               TOK_FUNCTION
+                  in
+                  TOK_TABLE_OR_COL
+                     key
+                  2
+                  3
+               =
+                  TOK_TABLE_OR_COL
+                     ds
+                  '2008-04-08'
+            =
+               TOK_TABLE_OR_COL
+                  value
+               'One'
+
+
+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: srcbucket_pruned
+            filterExpr: (((key) IN (2, 3) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+            buckets included: [2,3,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (((key) IN (2, 3) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key in (2,3) and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key in (2,3) and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'One'
+               TOK_FUNCTION
+                  in
+                  TOK_TABLE_OR_COL
+                     key
+                  2
+                  3
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+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: srcbucket_pruned
+            filterExpr: (((value = 'One') and (key) IN (2, 3)) and (ds = '2008-04-08')) (type: boolean)
+            buckets included: [2,3,] of 16
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (((value = 'One') and (key) IN (2, 3)) and (ds = '2008-04-08')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where (key=1 or key=2) and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where (key=1 or key=2) and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            or
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  2
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+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: srcbucket_pruned
+            filterExpr: (((key = 1) or (key = 2)) and (ds = '2008-04-08')) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (((key = 1) or (key = 2)) and (ds = '2008-04-08')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where (key=1 or key=2) and value = 'One' and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where (key=1 or key=2) and value = 'One' and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               or
+                  =
+                     TOK_TABLE_OR_COL
+                        key
+                     1
+                  =
+                     TOK_TABLE_OR_COL
+                        key
+                     2
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'One'
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+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: srcbucket_pruned
+            filterExpr: ((((key = 1) or (key = 2)) and (value = 'One')) and (ds = '2008-04-08')) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: ((((key = 1) or (key = 2)) and (value = 'One')) and (ds = '2008-04-08')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: -- valid but irrelevant case (all buckets selected)
+
+explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- valid but irrelevant case (all buckets selected)
+
+explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17)
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         TOK_FUNCTION
+            in
+            TOK_TABLE_OR_COL
+               key
+            1
+            2
+            3
+            4
+            5
+            6
+            7
+            8
+            9
+            10
+            11
+            12
+            13
+            14
+            15
+            16
+            17
+
+
+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: srcbucket_pruned
+            filterExpr: (key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            TOK_FUNCTION
+               in
+               TOK_TABLE_OR_COL
+                  key
+               1
+               2
+               3
+               4
+               5
+               6
+               7
+               8
+               9
+               10
+               11
+               12
+               13
+               14
+               15
+               16
+               17
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+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: srcbucket_pruned
+            filterExpr: ((key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) and (ds = '2008-04-08')) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: ((key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) and (ds = '2008-04-08')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08' and value='One'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08' and value='One'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               TOK_FUNCTION
+                  in
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+                  2
+                  3
+                  4
+                  5
+                  6
+                  7
+                  8
+                  9
+                  10
+                  11
+                  12
+                  13
+                  14
+                  15
+                  16
+                  17
+               =
+                  TOK_TABLE_OR_COL
+                     ds
+                  '2008-04-08'
+            =
+               TOK_TABLE_OR_COL
+                  value
+               'One'
+
+
+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: srcbucket_pruned
+            filterExpr: (((key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (((key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'One'
+               TOK_FUNCTION
+                  in
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+                  2
+                  3
+                  4
+                  5
+                  6
+                  7
+                  8
+                  9
+                  10
+                  11
+                  12
+                  13
+                  14
+                  15
+                  16
+                  17
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+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: srcbucket_pruned
+            filterExpr: (((value = 'One') and (key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) and (ds = '2008-04-08')) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (((value = 'One') and (key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) and (ds = '2008-04-08')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: -- valid, but unimplemented cases
+
+explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' or key = 2
+PREHOOK: type: QUERY
+POSTHOOK: query: -- valid, but unimplemented cases
+
+explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' or key = 2
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         or
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     ds
+                  '2008-04-08'
+            =
+               TOK_TABLE_OR_COL
+                  key
+               2
+
+
+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: srcbucket_pruned
+            filterExpr: (((key = 1) and (ds = '2008-04-08')) or (key = 2)) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (((key = 1) and (ds = '2008-04-08')) or (key = 2)) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' and (value='One' or value = 'Two')
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' and (value='One' or value = 'Two')
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     ds
+                  '2008-04-08'
+            or
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'One'
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'Two'
+
+
+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: srcbucket_pruned
+            filterExpr: (((key = 1) and (ds = '2008-04-08')) and ((value = 'One') or (value = 'Two'))) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (((key = 1) and (ds = '2008-04-08')) and ((value = 'One') or (value = 'Two'))) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 1 (type: int), value (type: string), '2008-04-08' (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One" or key = 2 and value = "Two"
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One" or key = 2 and value = "Two"
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         or
+            or
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  "One"
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  2
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  "Two"
+
+
+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: srcbucket_pruned
+            filterExpr: ((key = 1) or (value = 'One') or ((key = 2) and (value = 'Two'))) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: ((key = 1) or (value = 'One') or ((key = 2) and (value = 'Two'))) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: -- Invalid cases
+
+explain extended
+select * from srcbucket_pruned where key = 'x11'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Invalid cases
+
+explain extended
+select * from srcbucket_pruned where key = 'x11'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            'x11'
+
+
+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: srcbucket_pruned
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: false (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One"
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One"
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         or
+            =
+               TOK_TABLE_OR_COL
+                  key
+               1
+            =
+               TOK_TABLE_OR_COL
+                  value
+               "One"
+
+
+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: srcbucket_pruned
+            filterExpr: ((key = 1) or (value = 'One')) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: ((key = 1) or (value = 'One')) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One" or key = 2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One" or key = 2
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         or
+            or
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  "One"
+            =
+               TOK_TABLE_OR_COL
+                  key
+               2
+
+
+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: srcbucket_pruned
+            filterExpr: ((key = 1) or (value = 'One') or (key = 2)) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: ((key = 1) or (value = 'One') or (key = 2)) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_unpruned where key in (3, 5)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_unpruned where key in (3, 5)
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_unpruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         TOK_FUNCTION
+            in
+            TOK_TABLE_OR_COL
+               key
+            3
+            5
+
+
+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: srcbucket_unpruned
+            filterExpr: (key) IN (3, 5) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (key) IN (3, 5) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: key (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_unpruned where key = 1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_unpruned where key = 1
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_unpruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            1
+
+
+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: srcbucket_unpruned
+            filterExpr: (key = 1) (type: boolean)
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            GatherStats: false
+            Filter Operator
+              isSamplingPred: false
+              predicate: (key = 1) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: 1 (type: int), value (type: string), ds (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                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.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      properties:
+                        columns _col0,_col1,_col2
+                        columns.types int: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
+


[27/43] hive git commit: HIVE-12395: Turn off CBO for hive.support.special.characters.tablename tests until feature is complete (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-12395: Turn off CBO for hive.support.special.characters.tablename tests until feature is complete (Pengcheng Xiong, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master-fixed
Commit: be870a4a1677d3ad602baba79d5cc087016deeb5
Parents: b029d03
Author: Pengcheng Xiong <px...@apache.org>
Authored: Thu Nov 12 14:06:30 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:33 2015 -0800

----------------------------------------------------------------------
 .../clientpositive/special_character_in_tabnames_1.q      |  2 +-
 .../clientpositive/special_character_in_tabnames_2.q      |  2 ++
 .../clientpositive/special_character_in_tabnames_2.q.out  | 10 +++++-----
 3 files changed, 8 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/be870a4a/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q b/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
index 7540d27..7867ae1 100644
--- a/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
+++ b/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
@@ -1,4 +1,4 @@
-set hive.cbo.enable=true;
+set hive.cbo.enable=false;
 set hive.exec.check.crossproducts=false;
 set hive.stats.fetch.column.stats=true;
 set hive.auto.convert.join=false;

http://git-wip-us.apache.org/repos/asf/hive/blob/be870a4a/ql/src/test/queries/clientpositive/special_character_in_tabnames_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/special_character_in_tabnames_2.q b/ql/src/test/queries/clientpositive/special_character_in_tabnames_2.q
index 34dcc90..6110279 100644
--- a/ql/src/test/queries/clientpositive/special_character_in_tabnames_2.q
+++ b/ql/src/test/queries/clientpositive/special_character_in_tabnames_2.q
@@ -1,3 +1,5 @@
+set hive.cbo.enable=false;
+
 -- try the query without indexing, with manual indexing, and with automatic indexing
 -- SORT_QUERY_RESULTS
 

http://git-wip-us.apache.org/repos/asf/hive/blob/be870a4a/ql/src/test/results/clientpositive/special_character_in_tabnames_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/special_character_in_tabnames_2.q.out b/ql/src/test/results/clientpositive/special_character_in_tabnames_2.q.out
index 51d31e0..1cc672e 100644
--- a/ql/src/test/results/clientpositive/special_character_in_tabnames_2.q.out
+++ b/ql/src/test/results/clientpositive/special_character_in_tabnames_2.q.out
@@ -113,7 +113,7 @@ STAGE PLANS:
             alias: s/c
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: ((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) (type: boolean)
+              predicate: ((key > 80) and (key < 100)) (type: boolean)
               Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
@@ -183,9 +183,9 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: default__s/c_src_index__
-            filterExpr: ((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) (type: boolean)
+            filterExpr: ((key > 80) and (key < 100)) (type: boolean)
             Filter Operator
-              predicate: ((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) (type: boolean)
+              predicate: ((key > 80) and (key < 100)) (type: boolean)
               Select Operator
                 expressions: _bucketname (type: string), _offsets (type: array<bigint>)
                 outputColumnNames: _col0, _col1
@@ -216,10 +216,10 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s/c
-            filterExpr: ((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) (type: boolean)
+            filterExpr: ((key > 80) and (key < 100)) (type: boolean)
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: ((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) (type: boolean)
+              predicate: ((key > 80) and (key < 100)) (type: boolean)
               Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)


[19/43] hive git commit: HIVE-12208: Vectorized JOIN NPE on dynamically partitioned hash-join + map-join (Gunther Hagleitner, reviewed by Matt McCline)

Posted by om...@apache.org.
HIVE-12208: Vectorized JOIN NPE on dynamically partitioned hash-join + map-join (Gunther Hagleitner, reviewed by Matt McCline)


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

Branch: refs/heads/master-fixed
Commit: cf0729c13699546f38e9392c2039fa9214347690
Parents: 231247f
Author: Gunther Hagleitner <gu...@apache.org>
Authored: Wed Nov 11 13:41:26 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:32 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |  2 +-
 .../mapjoin/VectorMapJoinCommonOperator.java    | 22 +++++---------------
 2 files changed, 6 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cf0729c1/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
index 4af98e5..cab0fc8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
@@ -183,7 +183,7 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
 
   @SuppressWarnings("unchecked")
   @Override
-  protected final void completeInitializationOp(Object[] os) throws HiveException {
+  protected void completeInitializationOp(Object[] os) throws HiveException {
     if (os.length != 0) {
       Pair<MapJoinTableContainer[], MapJoinTableContainerSerDe[]> pair =
           (Pair<MapJoinTableContainer[], MapJoinTableContainerSerDe[]>) os[0];

http://git-wip-us.apache.org/repos/asf/hive/blob/cf0729c1/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
index 1667bf7..1d5a9de 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
@@ -614,46 +614,34 @@ public abstract class VectorMapJoinCommonOperator extends MapJoinOperator implem
   }
 
   @Override
-  protected Pair<MapJoinTableContainer[], MapJoinTableContainerSerDe[]> loadHashTable(
-      ExecMapperContext mapContext, MapredContext mrContext) throws HiveException {
-
-    Pair<MapJoinTableContainer[], MapJoinTableContainerSerDe[]> pair;
+  protected void completeInitializationOp(Object[] os) throws HiveException {
+    // setup mapJoinTables and serdes
+    super.completeInitializationOp(os);
 
     VectorMapJoinDesc vectorDesc = conf.getVectorDesc();
     HashTableImplementationType hashTableImplementationType = vectorDesc.hashTableImplementationType();
     switch (vectorDesc.hashTableImplementationType()) {
     case OPTIMIZED:
       {
-        // Using Tez's HashTableLoader, create either a MapJoinBytesTableContainer or
-        // HybridHashTableContainer.
-        pair = super.loadHashTable(mapContext, mrContext);
-
         // Create our vector map join optimized hash table variation *above* the
         // map join table container.
-        MapJoinTableContainer[] mapJoinTableContainers = pair.getLeft();
         vectorMapJoinHashTable = VectorMapJoinOptimizedCreateHashTable.createHashTable(conf,
-                mapJoinTableContainers[posSingleVectorMapJoinSmallTable]);
+                mapJoinTables[posSingleVectorMapJoinSmallTable]);
       }
       break;
 
     case FAST:
       {
-        // Use our VectorMapJoinFastHashTableLoader to create a VectorMapJoinTableContainer.
-        pair = super.loadHashTable(mapContext, mrContext);
-
         // Get our vector map join fast hash table variation from the
         // vector map join table container.
-        MapJoinTableContainer[] mapJoinTableContainers = pair.getLeft();
         VectorMapJoinTableContainer vectorMapJoinTableContainer =
-                (VectorMapJoinTableContainer) mapJoinTableContainers[posSingleVectorMapJoinSmallTable];
+                (VectorMapJoinTableContainer) mapJoinTables[posSingleVectorMapJoinSmallTable];
         vectorMapJoinHashTable = vectorMapJoinTableContainer.vectorMapJoinHashTable();
       }
       break;
     default:
       throw new RuntimeException("Unknown vector map join hash table implementation type " + hashTableImplementationType.name());
     }
-
-    return pair;
   }
 
   /*


[39/43] hive git commit: HIVE-12402: Split hive.root.logger separately to make it compatible with log4j1.x (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Posted by om...@apache.org.
HIVE-12402: Split hive.root.logger separately to make it compatible with log4j1.x (Prasanth Jayachandran reviewed by Sergey Shelukhin)


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

Branch: refs/heads/master-fixed
Commit: a3c02d0d6b155a3cc308ed7e60aa3cfdc22b4ea3
Parents: f28ed81
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Fri Nov 13 18:41:33 2015 -0600
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:35 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hive/cli/OptionsProcessor.java       | 22 +++++++++++++++++++-
 1 file changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a3c02d0d/cli/src/java/org/apache/hadoop/hive/cli/OptionsProcessor.java
----------------------------------------------------------------------
diff --git a/cli/src/java/org/apache/hadoop/hive/cli/OptionsProcessor.java b/cli/src/java/org/apache/hadoop/hive/cli/OptionsProcessor.java
index 3dee11a..b6a76d2 100644
--- a/cli/src/java/org/apache/hadoop/hive/cli/OptionsProcessor.java
+++ b/cli/src/java/org/apache/hadoop/hive/cli/OptionsProcessor.java
@@ -29,6 +29,7 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.logging.log4j.Level;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -117,7 +118,26 @@ public class OptionsProcessor {
       commandLine = new GnuParser().parse(options, argv);
       Properties confProps = commandLine.getOptionProperties("hiveconf");
       for (String propKey : confProps.stringPropertyNames()) {
-        System.setProperty(propKey, confProps.getProperty(propKey));
+        // with HIVE-11304, hive.root.logger cannot have both logger name and log level.
+        // if we still see it, split logger and level separately for hive.root.logger
+        // and hive.log.level respectively
+        if (propKey.equalsIgnoreCase("hive.root.logger")) {
+          String propVal = confProps.getProperty(propKey);
+          if (propVal.contains(",")) {
+            String[] tokens = propVal.split(",");
+            for (String token : tokens) {
+              if (Level.getLevel(token) == null) {
+                System.setProperty("hive.root.logger", token);
+              } else {
+                System.setProperty("hive.log.level", token);
+              }
+            }
+          } else {
+            System.setProperty(propKey, confProps.getProperty(propKey));
+          }
+        } else {
+          System.setProperty(propKey, confProps.getProperty(propKey));
+        }
       }
 
       Properties hiveVars = commandLine.getOptionProperties("define");


[08/43] hive git commit: HIVE-12354 : MapJoin with double keys is slow on MR (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by om...@apache.org.
HIVE-12354 : MapJoin with double keys is slow on MR (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/master-fixed
Commit: 526c507b03da70c83691f8c3729b080c5c0e1380
Parents: 954baa8
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Nov 9 16:32:31 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:30 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/KeyWrapperFactory.java  | 20 +--------------
 .../ql/exec/persistence/MapJoinKeyObject.java   |  6 ++---
 .../objectinspector/ObjectInspectorUtils.java   | 26 ++++++++++++++++++++
 3 files changed, 29 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/526c507b/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
index 1c409a2..5154a5f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
@@ -105,25 +105,7 @@ public class KeyWrapperFactory {
 
     @Override
     public void setHashKey() {
-      if (keys == null) {
-        hashcode = 0;
-      } else {
-        hashcode = 1;
-        for (Object element : keys) {
-          hashcode = 31 * hashcode;
-          if(element != null) {
-            if(element instanceof LazyDouble) {
-              long v = Double.doubleToLongBits(((LazyDouble)element).getWritableObject().get());
-              hashcode = hashcode + (int) (v ^ (v >>> 32));
-            } else if (element instanceof DoubleWritable){
-              long v = Double.doubleToLongBits(((DoubleWritable)element).get());
-              hashcode = hashcode + (int) (v ^ (v >>> 32));
-            } else {
-              hashcode = hashcode + element.hashCode();
-            }
-          }
-        }
-      }
+      hashcode = ObjectInspectorUtils.writableArrayHashCode(keys);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/526c507b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java
index e1fd6d3..7592f9e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java
@@ -78,11 +78,9 @@ public class MapJoinKeyObject extends MapJoinKey {
 
   @Override
   public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + Arrays.hashCode(key);
-    return result;
+    return ObjectInspectorUtils.writableArrayHashCode(key);
   }
+
   @Override
   public boolean equals(Object obj) {
     if (this == obj)

http://git-wip-us.apache.org/repos/asf/hive/blob/526c507b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
index 56597a2..7a13eb0 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.lazy.LazyDouble;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.AbstractPrimitiveWritableObjectInspector;
@@ -77,6 +78,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectIn
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableStringObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.StringUtils;
 
@@ -104,6 +106,30 @@ public final class ObjectInspectorUtils {
   }
 
   /**
+   * Calculates the hash code for array of Objects that contains writables. This is used
+   * to work around the buggy Hadoop DoubleWritable hashCode implementation. This should
+   * only be used for process-local hash codes; don't replace stored hash codes like bucketing.
+   */
+  public static int writableArrayHashCode(Object[] keys) {
+    if (keys == null) return 0;
+    int hashcode = 1;
+    for (Object element : keys) {
+      hashcode = 31 * hashcode;
+      if (element == null) continue;
+      if (element instanceof LazyDouble) {
+        long v = Double.doubleToLongBits(((LazyDouble)element).getWritableObject().get());
+        hashcode = hashcode + (int) (v ^ (v >>> 32));
+      } else if (element instanceof DoubleWritable){
+        long v = Double.doubleToLongBits(((DoubleWritable)element).get());
+        hashcode = hashcode + (int) (v ^ (v >>> 32));
+      } else {
+        hashcode = hashcode + element.hashCode();
+      }
+    }
+    return hashcode;
+  }
+
+  /**
    * Ensures that an ObjectInspector is Writable.
    */
   public static ObjectInspector getWritableObjectInspector(ObjectInspector oi) {


[10/43] hive git commit: HIVE-12311 : explain CTAS fails if the table already exists (Gunther Hagleitner via Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-12311 : explain CTAS fails if the table already exists (Gunther Hagleitner via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: 8e3d9e999973f053f6a08f5def6a5e750b105913
Parents: 20e9805
Author: Gunther Hagleitner <gu...@apache.org>
Authored: Mon Nov 2 14:29:00 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:30 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   4 +-
 .../test/queries/clientpositive/explain_ddl.q   |  28 +
 .../results/clientpositive/explain_ddl.q.out    | 604 +++++++++++++++++++
 3 files changed, 634 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8e3d9e99/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 f7e2039..1ca113c 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
@@ -10961,7 +10961,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
               + dbName + "." + tblName);
         }
         Map<String, Table> tables = SessionHiveMetaStoreClient.getTempTablesForDatabase(dbName);
-        if (tables != null && tables.containsKey(tblName)) {
+        if (tables != null && tables.containsKey(tblName) && !ctx.getExplain())  {
           throw new SemanticException("Temporary table " + dbName + "." + tblName
               + " already exists");
         }
@@ -10970,7 +10970,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         // dumpTable is only used to check the conflict for non-temporary tables
         try {
           Table dumpTable = db.newTable(dbDotTab);
-          if (null != db.getTable(dumpTable.getDbName(), dumpTable.getTableName(), false)) {
+          if (null != db.getTable(dumpTable.getDbName(), dumpTable.getTableName(), false) && !ctx.getExplain()) {
             throw new SemanticException(ErrorMsg.TABLE_ALREADY_EXISTS.getMsg(dbDotTab));
           }
         } catch (HiveException e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/8e3d9e99/ql/src/test/queries/clientpositive/explain_ddl.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/explain_ddl.q b/ql/src/test/queries/clientpositive/explain_ddl.q
new file mode 100644
index 0000000..e255179
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/explain_ddl.q
@@ -0,0 +1,28 @@
+-- This test is used for testing explain for DDL/DML statements
+
+-- Create some views and tabels
+CREATE VIEW V1 AS SELECT key, value from src;
+select count(*) from V1 where key > 0;
+
+CREATE TABLE M1 AS SELECT key, value from src;
+select count(*) from M1 where key > 0;
+
+EXPLAIN CREATE TABLE M1 AS select * from src;
+EXPLAIN CREATE TABLE M1 AS select * from M1;
+EXPLAIN CREATE TABLE M1 AS select * from V1;
+
+EXPLAIN CREATE TABLE V1 AS select * from M1;
+EXPLAIN CREATE VIEW V1 AS select * from M1;
+
+EXPLAIN CREATE TABLE M1 LIKE src;
+EXPLAIN CREATE TABLE M1 LIKE M1;
+
+EXPLAIN DROP TABLE M1;
+select count(*) from M1 where key > 0;
+
+EXPLAIN INSERT INTO M1 SELECT * FROM M1;
+select count(*) from M1 where key > 0;
+
+EXPLAIN TRUNCATE TABLE M1;
+select count(*) from M1 where key > 0;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/8e3d9e99/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
new file mode 100644
index 0000000..3371e7e
--- /dev/null
+++ b/ql/src/test/results/clientpositive/explain_ddl.q.out
@@ -0,0 +1,604 @@
+PREHOOK: query: -- This test is used for testing explain for DDL/DML statements
+
+-- Create some views and tabels
+CREATE VIEW V1 AS SELECT key, value from src
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@V1
+POSTHOOK: query: -- This test is used for testing explain for DDL/DML statements
+
+-- Create some views and tabels
+CREATE VIEW V1 AS SELECT key, value from src
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@V1
+PREHOOK: query: select count(*) from V1 where key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@v1
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from V1 where key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@v1
+#### A masked pattern was here ####
+497
+PREHOOK: query: CREATE TABLE M1 AS SELECT key, value from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@M1
+POSTHOOK: query: CREATE TABLE M1 AS SELECT key, value from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@M1
+PREHOOK: query: select count(*) from M1 where key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@m1
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from M1 where key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@m1
+#### A masked pattern was here ####
+497
+PREHOOK: query: EXPLAIN CREATE TABLE M1 AS select * from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: query: EXPLAIN CREATE TABLE M1 AS select * from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
+  Stage-4
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+  Stage-8 depends on stages: Stage-0
+  Stage-2 depends on stages: Stage-8
+  Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    name: default.M1
+
+  Stage: Stage-7
+    Conditional Operator
+
+  Stage: Stage-4
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-0
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-8
+      Create Table Operator:
+        Create Table
+          columns: key string, value string
+          input format: org.apache.hadoop.mapred.TextInputFormat
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          name: default.M1
+
+  Stage: Stage-2
+    Stats-Aggr Operator
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.M1
+
+  Stage: Stage-5
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.M1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+PREHOOK: query: EXPLAIN CREATE TABLE M1 AS select * from M1
+PREHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: query: EXPLAIN CREATE TABLE M1 AS select * from M1
+POSTHOOK: type: CREATETABLE_AS_SELECT
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
+  Stage-4
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+  Stage-8 depends on stages: Stage-0
+  Stage-2 depends on stages: Stage-8
+  Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: m1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    name: default.M1
+
+  Stage: Stage-7
+    Conditional Operator
+
+  Stage: Stage-4
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-0
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-8
+      Create Table Operator:
+        Create Table
+          columns: key string, value string
+          input format: org.apache.hadoop.mapred.TextInputFormat
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          name: default.M1
+
+  Stage: Stage-2
+    Stats-Aggr Operator
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.M1
+
+  Stage: Stage-5
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.M1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+PREHOOK: query: EXPLAIN CREATE TABLE M1 AS select * from V1
+PREHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: query: EXPLAIN CREATE TABLE M1 AS select * from V1
+POSTHOOK: type: CREATETABLE_AS_SELECT
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
+  Stage-4
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+  Stage-8 depends on stages: Stage-0
+  Stage-2 depends on stages: Stage-8
+  Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    name: default.M1
+
+  Stage: Stage-7
+    Conditional Operator
+
+  Stage: Stage-4
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-0
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-8
+      Create Table Operator:
+        Create Table
+          columns: key string, value string
+          input format: org.apache.hadoop.mapred.TextInputFormat
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          name: default.M1
+
+  Stage: Stage-2
+    Stats-Aggr Operator
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.M1
+
+  Stage: Stage-5
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.M1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+PREHOOK: query: EXPLAIN CREATE TABLE V1 AS select * from M1
+PREHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: query: EXPLAIN CREATE TABLE V1 AS select * from M1
+POSTHOOK: type: CREATETABLE_AS_SELECT
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
+  Stage-4
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+  Stage-8 depends on stages: Stage-0
+  Stage-2 depends on stages: Stage-8
+  Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: m1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    name: default.V1
+
+  Stage: Stage-7
+    Conditional Operator
+
+  Stage: Stage-4
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-0
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-8
+      Create Table Operator:
+        Create Table
+          columns: key string, value string
+          input format: org.apache.hadoop.mapred.TextInputFormat
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          name: default.V1
+
+  Stage: Stage-2
+    Stats-Aggr Operator
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.V1
+
+  Stage: Stage-5
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.V1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+PREHOOK: query: EXPLAIN CREATE VIEW V1 AS select * from M1
+PREHOOK: type: CREATEVIEW
+POSTHOOK: query: EXPLAIN CREATE VIEW V1 AS select * from M1
+POSTHOOK: type: CREATEVIEW
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Create View Operator:
+        Create View
+          or replace: false
+          columns: key string, value string
+          expanded text: select `m1`.`key`, `m1`.`value` from `default`.`M1`
+          name: default.V1
+          original text: select * from M1
+
+PREHOOK: query: EXPLAIN CREATE TABLE M1 LIKE src
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: EXPLAIN CREATE TABLE M1 LIKE src
+POSTHOOK: type: CREATETABLE
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Create Table Operator:
+        Create Table
+          default input format: org.apache.hadoop.mapred.TextInputFormat
+          default output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          default serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          like: src
+          name: default.M1
+
+PREHOOK: query: EXPLAIN CREATE TABLE M1 LIKE M1
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: EXPLAIN CREATE TABLE M1 LIKE M1
+POSTHOOK: type: CREATETABLE
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Create Table Operator:
+        Create Table
+          default input format: org.apache.hadoop.mapred.TextInputFormat
+          default output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          default serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          like: M1
+          name: default.M1
+
+PREHOOK: query: EXPLAIN DROP TABLE M1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: EXPLAIN DROP TABLE M1
+POSTHOOK: type: DROPTABLE
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Drop Table Operator:
+        Drop Table
+          table: M1
+
+PREHOOK: query: select count(*) from M1 where key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@m1
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from M1 where key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@m1
+#### A masked pattern was here ####
+497
+PREHOOK: query: EXPLAIN INSERT INTO M1 SELECT * FROM M1
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO M1 SELECT * FROM M1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
+  Stage-4
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+  Stage-2 depends on stages: Stage-0
+  Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: m1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    name: default.m1
+
+  Stage: Stage-7
+    Conditional Operator
+
+  Stage: Stage-4
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.m1
+
+  Stage: Stage-2
+    Stats-Aggr Operator
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.m1
+
+  Stage: Stage-5
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.m1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+PREHOOK: query: select count(*) from M1 where key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@m1
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from M1 where key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@m1
+#### A masked pattern was here ####
+497
+PREHOOK: query: EXPLAIN TRUNCATE TABLE M1
+PREHOOK: type: TRUNCATETABLE
+POSTHOOK: query: EXPLAIN TRUNCATE TABLE M1
+POSTHOOK: type: TRUNCATETABLE
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Truncate Table Operator:
+        Truncate Table or Partition
+          TableName: M1
+
+PREHOOK: query: select count(*) from M1 where key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@m1
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from M1 where key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@m1
+#### A masked pattern was here ####
+497


[20/43] hive git commit: HIVE-12363: Incorrect results with orc ppd across ORC versions (Gopal V, reviewed by Prasanth Jayachandran)

Posted by om...@apache.org.
HIVE-12363: Incorrect results with orc ppd across ORC versions (Gopal V, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/master-fixed
Commit: 31ba8c3beef1c722baee5057aeebc6eec3bd0bd8
Parents: dcaf0c6
Author: Gopal V <go...@apache.org>
Authored: Wed Nov 11 06:37:04 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:32 2015 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/31ba8c3b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
index 0696277..04b9eaf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.io.Text;
 public class RecordReaderImpl implements RecordReader {
   static final Logger LOG = LoggerFactory.getLogger(RecordReaderImpl.class);
   private static final boolean isLogDebugEnabled = LOG.isDebugEnabled();
+  private static final Object UNKNOWN_VALUE = new Object();
   private final Path path;
   private final long firstRow;
   private final List<StripeInformation> stripes =
@@ -310,7 +311,7 @@ public class RecordReaderImpl implements RecordReader {
         return Boolean.TRUE;
       }
     } else {
-      return null;
+      return UNKNOWN_VALUE; // null is not safe here
     }
   }
 
@@ -359,6 +360,8 @@ public class RecordReaderImpl implements RecordReader {
       } else {
         return TruthValue.NULL;
       }
+    } else if (min == UNKNOWN_VALUE) {
+      return TruthValue.YES_NO_NULL;
     }
 
     TruthValue result;


[17/43] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by om...@apache.org.
 HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)


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

Branch: refs/heads/master-fixed
Commit: 558b35cf22dc1145dde1dfed565b1dfe54c5a5a0
Parents: 3915a33
Author: aihuaxu <ai...@apache.org>
Authored: Tue Nov 10 15:37:08 2015 -0500
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:31 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |  208 +-
 .../apache/hive/jdbc/HiveDatabaseMetaData.java  |    4 +-
 metastore/if/hive_metastore.thrift              |    9 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2323 +++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  148 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  349 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   65 +
 .../hadoop/hive/metastore/api/TableMeta.java    |  701 ++++
 .../hive/metastore/api/ThriftHiveMetastore.java | 3406 ++++++++++++------
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1275 ++++---
 .../src/gen/thrift/gen-php/metastore/Types.php  |  144 +
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  883 +++--
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  110 +
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   25 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   65 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   21 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   32 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    7 +
 .../hadoop/hive/metastore/ObjectStore.java      |  112 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |    4 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   44 +-
 .../DummyRawStoreControlledCommit.java          |    7 +
 .../DummyRawStoreForJdoConnection.java          |    7 +
 .../ql/metadata/SessionHiveMetaStoreClient.java |   64 +-
 .../cli/operation/GetTablesOperation.java       |   47 +-
 .../cli/operation/MetadataOperation.java        |   23 +-
 28 files changed, 7128 insertions(+), 2967 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
index 2b3fdf1..5450eaa 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
@@ -42,6 +42,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.InputStream;
+import java.lang.Exception;
+import java.lang.Object;
+import java.lang.String;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
@@ -56,8 +59,11 @@ import java.sql.Timestamp;
 import java.sql.Types;
 import java.text.ParseException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Date;
+import java.util.HashSet;
 import java.util.HashMap;
+import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -134,6 +140,7 @@ public class TestJdbcDriver2 {
         stmt1.execute("DROP DATABASE " + db + " CASCADE");
       }
     }
+    stmt1.execute("create database testdb");
     stmt1.close();
     con1.close();
   }
@@ -147,12 +154,20 @@ public class TestJdbcDriver2 {
 
     stmt.execute("set hive.support.concurrency = false");
 
-    // drop table. ignore error.
-    try {
-      stmt.execute("drop table " + tableName);
-    } catch (Exception ex) {
-      fail(ex.toString());
-    }
+    createTestTables(stmt, "", true);
+    createTestTables(stmt, "testdb.", false);
+  }
+
+  private void createTestTables(Statement stmt, String prefix, boolean loadData)
+      throws SQLException {
+
+    // drop test tables/views
+    dropTestTables(stmt, prefix);
+
+    String tableName = prefix + this.tableName;
+    String partitionedTableName = prefix + this.partitionedTableName;
+    String dataTypeTableName = prefix + this.dataTypeTableName;
+    String viewName = prefix + this.viewName;
 
     // create table
     stmt.execute("create table " + tableName
@@ -160,35 +175,25 @@ public class TestJdbcDriver2 {
         + tableComment + "'");
 
     // load data
-    stmt.execute("load data local inpath '"
-        + dataFilePath.toString() + "' into table " + tableName);
-
-    // also initialize a paritioned table to test against.
-
-    // drop table. ignore error.
-    try {
-      stmt.execute("drop table " + partitionedTableName);
-    } catch (Exception ex) {
-      fail(ex.toString());
+    if (loadData) {
+      stmt.execute("load data local inpath '"
+          + dataFilePath.toString() + "' into table " + tableName);
     }
 
+    // also initialize a paritioned table to test against.
     stmt.execute("create table " + partitionedTableName
         + " (under_col int, value string) comment '"+partitionedTableComment
         +"' partitioned by (" + partitionedColumnName + " STRING)");
 
     // load data
-    stmt.execute("load data local inpath '"
-        + dataFilePath.toString() + "' into table " + partitionedTableName
-        + " PARTITION (" + partitionedColumnName + "="
-        + partitionedColumnValue + ")");
-
-    // drop table. ignore error.
-    try {
-      stmt.execute("drop table " + dataTypeTableName);
-    } catch (Exception ex) {
-      fail(ex.toString());
+    if (loadData) {
+      stmt.execute("load data local inpath '"
+          + dataFilePath.toString() + "' into table " + partitionedTableName
+          + " PARTITION (" + partitionedColumnName + "="
+          + partitionedColumnValue + ")");
     }
 
+    // tables with various types
     stmt.execute("create table " + dataTypeTableName
         + " (c1 int, c2 boolean, c3 double, c4 string,"
         + " c5 array<int>, c6 map<int,string>, c7 map<string,string>,"
@@ -208,15 +213,10 @@ public class TestJdbcDriver2 {
         + ") comment'" + dataTypeTableComment
         +"' partitioned by (dt STRING)");
 
-    stmt.execute("load data local inpath '"
-        + dataTypeDataFilePath.toString() + "' into table " + dataTypeTableName
-        + " PARTITION (dt='20090619')");
-
-    // drop view. ignore error.
-    try {
-      stmt.execute("drop view " + viewName);
-    } catch (Exception ex) {
-      fail(ex.toString());
+    if (loadData) {
+      stmt.execute("load data local inpath '"
+          + dataTypeDataFilePath.toString() + "' into table " + dataTypeTableName
+          + " PARTITION (dt='20090619')");
     }
 
     // create view
@@ -224,6 +224,28 @@ public class TestJdbcDriver2 {
         +"' as select * from "+ tableName);
   }
 
+  // drop test tables/views. ignore error.
+  private void dropTestTables(Statement stmt, String prefix) throws SQLException {
+    String tableName = prefix + this.tableName;
+    String partitionedTableName = prefix + this.partitionedTableName;
+    String dataTypeTableName = prefix + this.dataTypeTableName;
+    String viewName = prefix + this.viewName;
+
+    executeWithIgnore(stmt, "drop table " + tableName);
+    executeWithIgnore(stmt, "drop table " + partitionedTableName);
+    executeWithIgnore(stmt, "drop table " + dataTypeTableName);
+    executeWithIgnore(stmt, "drop view " + viewName);
+  }
+
+  private void executeWithIgnore(Statement stmt, String sql) throws SQLException {
+  // drop table. ignore error.
+    try {
+      stmt.execute(sql);
+    } catch (Exception ex) {
+      fail(ex.toString());
+    }
+  }
+
   private static Connection getConnection(String postfix) throws SQLException {
     Connection con1;
     if (standAloneServer) {
@@ -244,9 +266,8 @@ public class TestJdbcDriver2 {
     // drop table
     Statement stmt = con.createStatement();
     assertNotNull("Statement is null", stmt);
-    stmt.execute("drop table " + tableName);
-    stmt.execute("drop table " + partitionedTableName);
-    stmt.execute("drop table " + dataTypeTableName);
+    dropTestTables(stmt, "");
+    dropTestTables(stmt, "testdb.");
 
     con.close();
     assertTrue("Connection should be closed", con.isClosed());
@@ -1123,25 +1144,77 @@ public class TestJdbcDriver2 {
    * @throws SQLException
    */
   private void getTablesTest(String tableTypeName, String viewTypeName) throws SQLException {
-    Map<String, Object[]> tests = new HashMap<String, Object[]>();
-    tests.put("test%jdbc%", new Object[]{"testhivejdbcdriver_table"
-        , "testhivejdbcdriverpartitionedtable"
-        , "testhivejdbcdriverview"});
-    tests.put("%jdbcdriver\\_table", new Object[]{"testhivejdbcdriver_table"});
-    tests.put("testhivejdbcdriver\\_table", new Object[]{"testhivejdbcdriver_table"});
-    tests.put("test_ivejdbcdri_er\\_table", new Object[]{"testhivejdbcdriver_table"});
-    tests.put("test_ivejdbcdri_er_table", new Object[]{"testhivejdbcdriver_table"});
-    tests.put("test_ivejdbcdri_er%table", new Object[]{
-        "testhivejdbcdriver_table", "testhivejdbcdriverpartitionedtable" });
-    tests.put("%jdbc%", new Object[]{ "testhivejdbcdriver_table"
-        , "testhivejdbcdriverpartitionedtable"
-        , "testhivejdbcdriverview"});
-    tests.put("", new Object[]{});
-
-    for (String checkPattern: tests.keySet()) {
-      ResultSet rs = con.getMetaData().getTables("default", null, checkPattern, null);
+    String[] ALL = null;
+    String[] VIEW_ONLY = {viewTypeName};
+    String[] TABLE_ONLY = {tableTypeName};
+    String[] VIEWORTABLE = {tableTypeName, viewTypeName};
+
+    Map<Object[], String[]> tests = new IdentityHashMap<Object[], String[]>();
+    tests.put(new Object[] { null, "test%jdbc%", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "default.testhivejdbcdriverpartitionedtable",
+        "default.testhivejdbcdriverview",
+        "testdb.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriverpartitionedtable",
+        "testdb.testhivejdbcdriverview"});
+    tests.put(new Object[] { "test%", "test%jdbc%", ALL}, new String[]{
+        "testdb.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriverpartitionedtable",
+        "testdb.testhivejdbcdriverview"});
+    tests.put(new Object[] { "test%", "test%jdbc%", VIEW_ONLY}, new String[]{
+        "testdb.testhivejdbcdriverview"});
+
+    tests.put(new Object[] { null, "%jdbcdriver\\_table", VIEWORTABLE}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriver_table"});
+    tests.put(new Object[] { "def%", "%jdbcdriver\\_table", VIEWORTABLE}, new String[]{
+        "default.testhivejdbcdriver_table"});
+    tests.put(new Object[] { "def%", "%jdbcdriver\\_table", VIEW_ONLY}, new String[0]);
+
+    tests.put(new Object[] { null, "testhivejdbcdriver\\_table", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriver_table"});
+    tests.put(new Object[] { "%faul%", "testhivejdbcdriver\\_table", ALL}, new String[]{
+        "default.testhivejdbcdriver_table"});
+    tests.put(new Object[] { "%faul%", "testhivejdbcdriver\\_table", TABLE_ONLY}, new String[]{
+        "default.testhivejdbcdriver_table"});
+
+    tests.put(new Object[] { null, "test_ivejdbcdri_er\\_table", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriver_table"});
+    tests.put(new Object[] { "test__", "test_ivejdbcdri_er\\_table", ALL}, new String[]{
+        "testdb.testhivejdbcdriver_table"});
+
+    tests.put(new Object[] { null, "test_ivejdbcdri_er_table", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriver_table"});
+    tests.put(new Object[] { null, "test_ivejdbcdri_er%table", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "default.testhivejdbcdriverpartitionedtable",
+        "testdb.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriverpartitionedtable"});
+    tests.put(new Object[] { null, "%jdbc%", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "default.testhivejdbcdriverpartitionedtable",
+        "default.testhivejdbcdriverview",
+        "testdb.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriverpartitionedtable",
+        "testdb.testhivejdbcdriverview"});
+    tests.put(new Object[] { "%", "%jdbc%", VIEW_ONLY}, new String[]{
+        "default.testhivejdbcdriverview",
+        "testdb.testhivejdbcdriverview"});
+    tests.put(new Object[] { null, "", ALL}, new String[]{});
+
+    for (Map.Entry<Object[], String[]> entry : tests.entrySet()) {
+      Object[] checkPattern = entry.getKey();
+      String debugString = checkPattern[0] + ", " + checkPattern[1] + ", " +
+          Arrays.toString((String[]) checkPattern[2]);
+
+      Set<String> expectedTables = new HashSet<String>(Arrays.asList(entry.getValue()));
+      ResultSet rs = con.getMetaData().getTables(null,
+          (String)checkPattern[0], (String)checkPattern[1], (String[])checkPattern[2]);
       ResultSetMetaData resMeta = rs.getMetaData();
-      assertEquals(5, resMeta.getColumnCount());
+      assertEquals(10, resMeta.getColumnCount());
       assertEquals("TABLE_CAT", resMeta.getColumnName(1));
       assertEquals("TABLE_SCHEM", resMeta.getColumnName(2));
       assertEquals("TABLE_NAME", resMeta.getColumnName(3));
@@ -1150,9 +1223,11 @@ public class TestJdbcDriver2 {
 
       int cnt = 0;
       while (rs.next()) {
+        String resultDbName = rs.getString("TABLE_SCHEM");
         String resultTableName = rs.getString("TABLE_NAME");
-        assertEquals("Get by index different from get by name.", rs.getString(3), resultTableName);
-        assertEquals("Excpected a different table.", tests.get(checkPattern)[cnt], resultTableName);
+        assertTrue("Invalid table " + resultDbName + "." + resultTableName + " for test " + debugString,
+            expectedTables.contains(resultDbName + "." + resultTableName));
+
         String resultTableComment = rs.getString("REMARKS");
         assertTrue("Missing comment on the table.", resultTableComment.length()>0);
         String tableType = rs.getString("TABLE_TYPE");
@@ -1164,18 +1239,9 @@ public class TestJdbcDriver2 {
         cnt++;
       }
       rs.close();
-      assertEquals("Received an incorrect number of tables.", tests.get(checkPattern).length, cnt);
+      assertEquals("Received an incorrect number of tables for test " + debugString,
+          expectedTables.size(), cnt);
     }
-
-    // only ask for the views.
-    ResultSet rs = con.getMetaData().getTables("default", null, null
-        , new String[]{viewTypeName});
-    int cnt=0;
-    while (rs.next()) {
-      cnt++;
-    }
-    rs.close();
-    assertEquals("Incorrect number of views found.", 1, cnt);
   }
 
   @Test
@@ -1198,6 +1264,8 @@ public class TestJdbcDriver2 {
 
     assertTrue(rs.next());
     assertEquals("default", rs.getString(1));
+    assertTrue(rs.next());
+    assertEquals("testdb", rs.getString(1));
 
     assertFalse(rs.next());
     rs.close();
@@ -1270,7 +1338,7 @@ public class TestJdbcDriver2 {
     tests.put(new String[]{"%jdbcdriver\\_table%", "_%"}, 2);
 
     for (String[] checkPattern: tests.keySet()) {
-      ResultSet rs = con.getMetaData().getColumns(null, null, checkPattern[0],
+      ResultSet rs = con.getMetaData().getColumns(null, "default", checkPattern[0],
           checkPattern[1]);
 
       // validate the metadata for the getColumns result set

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java b/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
index 13e42b5..a73f443 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
@@ -646,9 +646,7 @@ public class HiveDatabaseMetaData implements DatabaseMetaData {
     if (types != null) {
       getTableReq.setTableTypes(Arrays.asList(types));
     }
-    if (schemaPattern != null) {
-      getTableReq.setSchemaName(schemaPattern);
-    }
+    getTableReq.setSchemaName(schemaPattern);
 
     try {
       getTableResp = client.GetTables(getTableReq);

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 98fd42b..bb754f1 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -772,6 +772,13 @@ struct GetAllFunctionsResponse {
   1: optional list<Function> functions
 }
 
+struct TableMeta {
+  1: required string dbName;
+  2: required string tableName;
+  3: required string tableType;
+  4: optional string comments;
+}
+
 exception MetaException {
   1: string message
 }
@@ -890,6 +897,8 @@ service ThriftHiveMetastore extends fb303.FacebookService
       4:EnvironmentContext environment_context)
                        throws(1:NoSuchObjectException o1, 2:MetaException o3)
   list<string> get_tables(1: string db_name, 2: string pattern) throws (1: MetaException o1)
+  list<TableMeta> get_table_meta(1: string db_patterns, 2: string tbl_patterns, 3: list<string> tbl_types)
+                       throws (1: MetaException o1)
   list<string> get_all_tables(1: string db_name) throws (1: MetaException o1)
 
   Table get_table(1:string dbname, 2:string tbl_name)


[25/43] hive git commit: HIVE-12186 : Upgrade Hive to Calcite 1.5 (Jesus Camacho Rodriguez via Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-12186 : Upgrade Hive to Calcite 1.5 (Jesus Camacho Rodriguez via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: 231247f35804bc918747a12ea630b30d33478ce2
Parents: 31ba8c3
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Nov 11 11:31:22 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:32 2015 -0800

----------------------------------------------------------------------
 pom.xml                                         |   2 +-
 .../calcite/reloperators/HiveTableScan.java     |   6 +-
 .../calcite/rules/HiveRelFieldTrimmer.java      | 143 ++-----------------
 .../calcite/translator/ASTConverter.java        |  34 ++---
 .../calcite/translator/HiveOpConverter.java     |   7 +-
 .../translator/PlanModifierForASTConv.java      |   3 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   8 +-
 .../bucketizedhiveinputformat.q.out             |   2 +
 .../spark/bucketizedhiveinputformat.q.out       |   2 +
 9 files changed, 38 insertions(+), 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/231247f3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 282d077..99aeff7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -105,7 +105,7 @@
     <antlr.version>3.4</antlr.version>
     <avro.version>1.7.7</avro.version>
     <bonecp.version>0.8.0.RELEASE</bonecp.version>
-    <calcite.version>1.4.0-incubating</calcite.version>
+    <calcite.version>1.5.0</calcite.version>
     <datanucleus-api-jdo.version>3.2.6</datanucleus-api-jdo.version>
     <datanucleus-core.version>3.2.10</datanucleus-core.version>
     <datanucleus-rdbms.version>3.2.9</datanucleus-rdbms.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/231247f3/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
index 1831d69..446dc73 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
@@ -29,13 +29,13 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.rel.core.RelFactories;
 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.RexNode;
+import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
@@ -152,7 +152,7 @@ public class HiveTableScan extends TableScan implements HiveRelNode {
 
   @Override
   public RelNode project(ImmutableBitSet fieldsUsed, Set<RelDataTypeField> extraFields,
-      RelFactories.ProjectFactory projectFactory) {
+      RelBuilder relBuilder) {
 
     // 1. If the schema is the same then bail out
     final int fieldCount = getRowType().getFieldCount();
@@ -183,7 +183,7 @@ public class HiveTableScan extends TableScan implements HiveRelNode {
         fieldNames));
 
     // 5. Add Proj on top of TS
-    return projectFactory.createProject(newHT, exprList, new ArrayList<String>(fieldNames));
+    return relBuilder.push(newHT).project(exprList, new ArrayList<String>(fieldNames)).build();
   }
 
   public List<Integer> getNeededColIndxsFrmReloptHT() {

http://git-wip-us.apache.org/repos/asf/hive/blob/231247f3/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
index a12fa2a..b543fbb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
@@ -23,10 +23,9 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptUtil;
 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.RelFactories;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -35,22 +34,23 @@ import org.apache.calcite.rex.RexPermuteInputsShuttle;
 import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.RelFieldTrimmer;
+import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.mapping.IntPair;
 import org.apache.calcite.util.mapping.Mapping;
 import org.apache.calcite.util.mapping.MappingType;
 import org.apache.calcite.util.mapping.Mappings;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
 
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-
 public class HiveRelFieldTrimmer extends RelFieldTrimmer {
 
-  private final RelFactories.AggregateFactory aggregateFactory;
+  protected static final Log LOG = LogFactory.getLog(HiveRelFieldTrimmer.class);
+
 
   public HiveRelFieldTrimmer(SqlValidator validator,
+      RelOptCluster cluster,
       RelFactories.ProjectFactory projectFactory,
       RelFactories.FilterFactory filterFactory,
       RelFactories.JoinFactory joinFactory,
@@ -58,9 +58,10 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
       RelFactories.SortFactory sortFactory,
       RelFactories.AggregateFactory aggregateFactory,
       RelFactories.SetOpFactory setOpFactory) {
-    super(validator, projectFactory, filterFactory, joinFactory,
-            semiJoinFactory, sortFactory, aggregateFactory, setOpFactory);
-    this.aggregateFactory = aggregateFactory;
+    super(validator,
+            RelBuilder.proto(projectFactory, filterFactory, joinFactory,
+                semiJoinFactory, sortFactory, aggregateFactory, setOpFactory)
+            .create(cluster, null));
   }
 
   /**
@@ -155,127 +156,5 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
 
     return new TrimResult(newJoin, mapping);
   }
-  /**
-   * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
-   * {@link org.apache.calcite.rel.logical.LogicalAggregate}.
-   */
-  @Override
-  public TrimResult trimFields(
-      Aggregate aggregate,
-      ImmutableBitSet fieldsUsed,
-      Set<RelDataTypeField> extraFields) {
-    // Fields:
-    //
-    // | sys fields | group fields | indicator fields | agg functions |
-    //
-    // Two kinds of trimming:
-    //
-    // 1. If agg rel has system fields but none of these are used, create an
-    // agg rel with no system fields.
-    //
-    // 2. If aggregate functions are not used, remove them.
-    //
-    // But group and indicator fields stay, even if they are not used.
-
-    final RelDataType rowType = aggregate.getRowType();
-
-    // Compute which input fields are used.
-    // 1. group fields are always used
-    final ImmutableBitSet.Builder inputFieldsUsed =
-        ImmutableBitSet.builder(aggregate.getGroupSet());
-    // 2. agg functions
-    for (AggregateCall aggCall : aggregate.getAggCallList()) {
-      for (int i : aggCall.getArgList()) {
-        inputFieldsUsed.set(i);
-      }
-      if (aggCall.filterArg >= 0) {
-        inputFieldsUsed.set(aggCall.filterArg);
-      }
-    }
-
-    // Create input with trimmed columns.
-    final RelNode input = aggregate.getInput();
-    final Set<RelDataTypeField> inputExtraFields = Collections.emptySet();
-    final TrimResult trimResult =
-        trimChild(aggregate, input, inputFieldsUsed.build(), inputExtraFields);
-    final RelNode newInput = trimResult.left;
-    final Mapping inputMapping = trimResult.right;
-
-    // We have to return group keys and (if present) indicators.
-    // So, pretend that the consumer asked for them.
-    final int groupCount = aggregate.getGroupSet().cardinality();
-    final int indicatorCount = aggregate.getIndicatorCount();
-    fieldsUsed =
-        fieldsUsed.union(ImmutableBitSet.range(groupCount + indicatorCount));
-
-    // If the input is unchanged, and we need to project all columns,
-    // there's nothing to do.
-    if (input == newInput
-        && fieldsUsed.equals(ImmutableBitSet.range(rowType.getFieldCount()))) {
-      return new TrimResult(
-          aggregate,
-          Mappings.createIdentity(rowType.getFieldCount()));
-    }
-
-    // Which agg calls are used by our consumer?
-    int j = groupCount + indicatorCount;
-    int usedAggCallCount = 0;
-    for (int i = 0; i < aggregate.getAggCallList().size(); i++) {
-      if (fieldsUsed.get(j++)) {
-        ++usedAggCallCount;
-      }
-    }
-
-    // Offset due to the number of system fields having changed.
-    Mapping mapping =
-        Mappings.create(
-            MappingType.INVERSE_SURJECTION,
-            rowType.getFieldCount(),
-            groupCount + indicatorCount + usedAggCallCount);
-
-    final ImmutableBitSet newGroupSet =
-        Mappings.apply(inputMapping, aggregate.getGroupSet());
-
-    final ImmutableList<ImmutableBitSet> newGroupSets =
-        ImmutableList.copyOf(
-            Iterables.transform(aggregate.getGroupSets(),
-                new Function<ImmutableBitSet, ImmutableBitSet>() {
-                  @Override
-                  public ImmutableBitSet apply(ImmutableBitSet input) {
-                    return Mappings.apply(inputMapping, input);
-                  }
-                }));
-
-    // Populate mapping of where to find the fields. System, group key and
-    // indicator fields first.
-    for (j = 0; j < groupCount + indicatorCount; j++) {
-      mapping.set(j, j);
-    }
-
-    // Now create new agg calls, and populate mapping for them.
-    final List<AggregateCall> newAggCallList = new ArrayList<>();
-    j = groupCount + indicatorCount;
-    for (AggregateCall aggCall : aggregate.getAggCallList()) {
-      if (fieldsUsed.get(j)) {
-        AggregateCall newAggCall =
-            aggCall.copy(Mappings.apply2(inputMapping, aggCall.getArgList()),
-                Mappings.apply(inputMapping, aggCall.filterArg));
-        if (newAggCall.equals(aggCall)) {
-          newAggCall = aggCall; // immutable -> canonize to save space
-        }
-        mapping.set(j, groupCount + indicatorCount + newAggCallList.size());
-        newAggCallList.add(newAggCall);
-      }
-      ++j;
-    }
-
-    RelNode newAggregate = aggregateFactory.createAggregate(newInput,
-        aggregate.indicator, newGroupSet, newGroupSets, newAggCallList);
-
-    assert newAggregate.getClass() == aggregate.getClass();
-
-    return new TrimResult(newAggregate, mapping);
-  }
-
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/231247f3/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 e4ac154..d026e58 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
@@ -77,8 +77,7 @@ public class ASTConverter {
   private Aggregate        groupBy;
   private Filter           having;
   private Project          select;
-  private Sort             order;
-  private Sort             limit;
+  private Sort             orderLimit;
 
   private Schema           schema;
 
@@ -203,27 +202,14 @@ public class ASTConverter {
      * parent hence we need to go top down; but OB at each block really belong
      * to its src/from. Hence the need to pass in sort for each block from
      * its parent.
+     * 8. Limit
      */
-    convertOBToASTNode((HiveSortLimit) order);
-
-    // 8. Limit
-    convertLimitToASTNode((HiveSortLimit) limit);
+    convertOrderLimitToASTNode((HiveSortLimit) orderLimit);
 
     return hiveAST.getAST();
   }
 
-  private void convertLimitToASTNode(HiveSortLimit limit) {
-    if (limit != null) {
-      HiveSortLimit hiveLimit = limit;
-      RexNode limitExpr = hiveLimit.getFetchExpr();
-      if (limitExpr != null) {
-        Object val = ((RexLiteral) limitExpr).getValue2();
-        hiveAST.limit = ASTBuilder.limit(val);
-      }
-    }
-  }
-
-  private void convertOBToASTNode(HiveSortLimit order) {
+  private void convertOrderLimitToASTNode(HiveSortLimit order) {
     if (order != null) {
       HiveSortLimit hiveSortLimit = order;
       if (!hiveSortLimit.getCollation().getFieldCollations().isEmpty()) {
@@ -264,6 +250,12 @@ public class ASTConverter {
         }
         hiveAST.order = orderAst;
       }
+
+      RexNode limitExpr = hiveSortLimit.getFetchExpr();
+      if (limitExpr != null) {
+        Object val = ((RexLiteral) limitExpr).getValue2();
+        hiveAST.limit = ASTBuilder.limit(val);
+      }
     }
   }
 
@@ -366,11 +358,7 @@ public class ASTConverter {
         if (ASTConverter.this.select != null) {
           ASTConverter.this.from = node;
         } else {
-          Sort hiveSortRel = (Sort) node;
-          if (hiveSortRel.getCollation().getFieldCollations().isEmpty())
-            ASTConverter.this.limit = hiveSortRel;
-          else
-            ASTConverter.this.order = hiveSortRel;
+          ASTConverter.this.orderLimit = (Sort) node;
         }
       }
       /*

http://git-wip-us.apache.org/repos/asf/hive/blob/231247f3/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
index f0f8aa8..130ee89 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
@@ -487,7 +487,7 @@ public class HiveOpConverter {
       if (this.semanticAnalyzer != null && semanticAnalyzer.getQB() != null
           && semanticAnalyzer.getQB().getParseInfo() != null)
         this.semanticAnalyzer.getQB().getParseInfo().setOuterQueryLimit(limit);
-      ArrayList<ColumnInfo> cinfoLst = createColInfos(inputOp);
+      ArrayList<ColumnInfo> cinfoLst = createColInfos(resultOp);
       resultOp = OperatorFactory.getAndMakeChild(limitDesc,
           new RowSchema(cinfoLst), resultOp);
 
@@ -1059,10 +1059,6 @@ public class HiveOpConverter {
   }
 
   private static JoinType extractJoinType(HiveJoin join) {
-    // UNIQUE
-    if (join.isDistinct()) {
-      return JoinType.UNIQUE;
-    }
     // SEMIJOIN
     if (join.isLeftSemiJoin()) {
       return JoinType.LEFTSEMI;
@@ -1080,6 +1076,7 @@ public class HiveOpConverter {
       resultJoinType = JoinType.RIGHTOUTER;
       break;
     default:
+      // TODO: UNIQUE JOIN
       resultJoinType = JoinType.INNER;
       break;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/231247f3/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
index e820496..29e08f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
@@ -312,7 +312,8 @@ public class PlanModifierForASTConv {
     boolean validChild = true;
     RelNode child = sortNode.getInput();
 
-    if (!(HiveCalciteUtil.limitRelNode(sortNode) && HiveCalciteUtil.orderRelNode(child))
+    if (!(HiveCalciteUtil.limitRelNode(sortNode) && HiveCalciteUtil.orderRelNode(child)
+            && HiveCalciteUtil.limitRelNode(child))
         && !(child instanceof Project)) {
       validChild = false;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/231247f3/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 e13356c..de67b54 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
@@ -58,8 +58,6 @@ import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -902,7 +900,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
                 HiveJoinToMultiJoinRule.INSTANCE, HiveProjectMergeRule.INSTANCE);
         // The previous rules can pull up projections through join operators,
         // thus we run the field trimmer again to push them back down
-        HiveRelFieldTrimmer fieldTrimmer = new HiveRelFieldTrimmer(null, HiveProject.DEFAULT_PROJECT_FACTORY,
+        HiveRelFieldTrimmer fieldTrimmer = new HiveRelFieldTrimmer(null,
+            cluster, HiveProject.DEFAULT_PROJECT_FACTORY,
             HiveFilter.DEFAULT_FILTER_FACTORY, HiveJoin.HIVE_JOIN_FACTORY,
             HiveSemiJoin.HIVE_SEMIJOIN_FACTORY, HiveSortLimit.HIVE_SORT_REL_FACTORY,
             HiveAggregate.HIVE_AGGR_REL_FACTORY, HiveUnion.UNION_REL_FACTORY);
@@ -983,7 +982,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
           new HivePartitionPruneRule(conf));
 
       // 5. Projection Pruning
-      HiveRelFieldTrimmer fieldTrimmer = new HiveRelFieldTrimmer(null, HiveProject.DEFAULT_PROJECT_FACTORY,
+      HiveRelFieldTrimmer fieldTrimmer = new HiveRelFieldTrimmer(null,
+          cluster, HiveProject.DEFAULT_PROJECT_FACTORY,
           HiveFilter.DEFAULT_FILTER_FACTORY, HiveJoin.HIVE_JOIN_FACTORY,
           HiveSemiJoin.HIVE_SEMIJOIN_FACTORY, HiveSortLimit.HIVE_SORT_REL_FACTORY,
           HiveAggregate.HIVE_AGGR_REL_FACTORY, HiveUnion.UNION_REL_FACTORY);

http://git-wip-us.apache.org/repos/asf/hive/blob/231247f3/ql/src/test/results/clientpositive/bucketizedhiveinputformat.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketizedhiveinputformat.q.out b/ql/src/test/results/clientpositive/bucketizedhiveinputformat.q.out
index 277b0f7..cfb95be 100644
--- a/ql/src/test/results/clientpositive/bucketizedhiveinputformat.q.out
+++ b/ql/src/test/results/clientpositive/bucketizedhiveinputformat.q.out
@@ -22,6 +22,8 @@ POSTHOOK: query: CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@T2
+Warning: Shuffle Join JOIN[13][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: INSERT OVERWRITE TABLE T2 SELECT * FROM (
 SELECT tmp1.name as name FROM (
   SELECT name, 'MMM' AS n FROM T1) tmp1 

http://git-wip-us.apache.org/repos/asf/hive/blob/231247f3/ql/src/test/results/clientpositive/spark/bucketizedhiveinputformat.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucketizedhiveinputformat.q.out b/ql/src/test/results/clientpositive/spark/bucketizedhiveinputformat.q.out
index 98c5802..c8fc4d3 100644
--- a/ql/src/test/results/clientpositive/spark/bucketizedhiveinputformat.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucketizedhiveinputformat.q.out
@@ -22,6 +22,8 @@ POSTHOOK: query: CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@T2
+Warning: Shuffle Join JOIN[10][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product
+Warning: Shuffle Join JOIN[13][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 3' is a cross product
 PREHOOK: query: INSERT OVERWRITE TABLE T2 SELECT * FROM (
 SELECT tmp1.name as name FROM (
   SELECT name, 'MMM' AS n FROM T1) tmp1 


[23/43] hive git commit: HIVE-12309 : TableScan should colStats when available for better data size estimate (Ashutosh Chauhan via Prasanth J)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
index 3b053fe..6537a8a 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
@@ -96,7 +96,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
@@ -184,7 +184,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: subq2:subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
@@ -327,7 +327,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src2:subq2:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
@@ -406,7 +406,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src1:subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
@@ -525,7 +525,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
@@ -622,7 +622,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: subq2:subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
@@ -743,7 +743,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: subq2:subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
@@ -854,7 +854,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 8) (type: boolean)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
@@ -942,7 +942,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key + 1) is not null (type: boolean)
               Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
@@ -957,7 +957,7 @@ STAGE PLANS:
                   Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: subq2:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key + 1) is not null (type: boolean)
               Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1063,7 +1063,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1152,7 +1152,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1259,7 +1259,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: a:subq2:subq1:a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1348,7 +1348,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1429,7 +1429,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: a
-            Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_join0.q.out b/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
index a8bcc90..3c6bb73 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
@@ -20,7 +20,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: cbo_t1
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
@@ -36,7 +36,7 @@ STAGE PLANS:
                   value expressions: c_int (type: int)
           TableScan
             alias: cbo_t2:cbo_t2
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
@@ -52,7 +52,7 @@ STAGE PLANS:
                   value expressions: c_int (type: int)
           TableScan
             alias: cbo_t3:cbo_t3
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: key (type: string)
               outputColumnNames: key
@@ -669,7 +669,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: cbo_t1
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
@@ -685,7 +685,7 @@ STAGE PLANS:
                   value expressions: c_int (type: int)
           TableScan
             alias: cbo_t2:cbo_t2
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
@@ -701,7 +701,7 @@ STAGE PLANS:
                   value expressions: c_int (type: int)
           TableScan
             alias: cbo_t3:cbo_t3
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: key (type: string)
               outputColumnNames: key
@@ -713,7 +713,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: cbo_t4:cbo_t1
-            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/extrapolate_part_stats_full.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/extrapolate_part_stats_full.q.out b/ql/src/test/results/clientpositive/extrapolate_part_stats_full.q.out
index f87a539..473ee0e 100644
--- a/ql/src/test/results/clientpositive/extrapolate_part_stats_full.q.out
+++ b/ql/src/test/results/clientpositive/extrapolate_part_stats_full.q.out
@@ -204,7 +204,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_1d
-          Statistics: Num rows: 6 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE
           GatherStats: false
           Select Operator
             expressions: state (type: string)
@@ -337,7 +337,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_1d
-          Statistics: Num rows: 6 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
           GatherStats: false
           Select Operator
             expressions: state (type: string), locid (type: int)
@@ -626,7 +626,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_2d
-          Statistics: Num rows: 6 Data size: 532 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE
           GatherStats: false
           Select Operator
             expressions: state (type: string)
@@ -845,7 +845,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_2d
-          Statistics: Num rows: 6 Data size: 532 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 6 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
           GatherStats: false
           Select Operator
             expressions: state (type: string), locid (type: int)

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/extrapolate_part_stats_partial.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/extrapolate_part_stats_partial.q.out b/ql/src/test/results/clientpositive/extrapolate_part_stats_partial.q.out
index 5903cd1..6c1fc13 100644
--- a/ql/src/test/results/clientpositive/extrapolate_part_stats_partial.q.out
+++ b/ql/src/test/results/clientpositive/extrapolate_part_stats_partial.q.out
@@ -307,7 +307,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_1d
-          Statistics: Num rows: 20 Data size: 1866 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 20 Data size: 1780 Basic stats: COMPLETE Column stats: PARTIAL
           GatherStats: false
           Select Operator
             expressions: state (type: string)
@@ -526,7 +526,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_1d
-          Statistics: Num rows: 20 Data size: 1866 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 20 Data size: 1860 Basic stats: COMPLETE Column stats: PARTIAL
           GatherStats: false
           Select Operator
             expressions: state (type: string), locid (type: int)
@@ -758,7 +758,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_1d
-          Statistics: Num rows: 20 Data size: 1866 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 20 Data size: 1740 Basic stats: COMPLETE Column stats: COMPLETE
           GatherStats: false
           Select Operator
             expressions: state (type: string)
@@ -973,7 +973,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_1d
-          Statistics: Num rows: 20 Data size: 1866 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 20 Data size: 1820 Basic stats: COMPLETE Column stats: PARTIAL
           GatherStats: false
           Select Operator
             expressions: state (type: string), locid (type: int)
@@ -1571,7 +1571,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_2d
-          Statistics: Num rows: 20 Data size: 1788 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 20 Data size: 1760 Basic stats: COMPLETE Column stats: PARTIAL
           GatherStats: false
           Select Operator
             expressions: state (type: string)
@@ -2098,7 +2098,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_2d
-          Statistics: Num rows: 20 Data size: 1788 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 20 Data size: 1840 Basic stats: COMPLETE Column stats: PARTIAL
           GatherStats: false
           Select Operator
             expressions: state (type: string), locid (type: int)

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/extrapolate_part_stats_partial_ndv.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/extrapolate_part_stats_partial_ndv.q.out b/ql/src/test/results/clientpositive/extrapolate_part_stats_partial_ndv.q.out
index 2ea1e6e..975dd50 100644
--- a/ql/src/test/results/clientpositive/extrapolate_part_stats_partial_ndv.q.out
+++ b/ql/src/test/results/clientpositive/extrapolate_part_stats_partial_ndv.q.out
@@ -384,7 +384,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_1d
-          Statistics: Num rows: 20 Data size: 4186 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 20 Data size: 4260 Basic stats: COMPLETE Column stats: PARTIAL
           GatherStats: false
           Select Operator
             expressions: state (type: string), locid (type: double), cnt (type: decimal(10,0)), zip (type: int)
@@ -697,7 +697,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_1d
-          Statistics: Num rows: 20 Data size: 4186 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 20 Data size: 4220 Basic stats: COMPLETE Column stats: COMPLETE
           GatherStats: false
           Select Operator
             expressions: state (type: string), locid (type: double), cnt (type: decimal(10,0)), zip (type: int)
@@ -1375,7 +1375,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc_2d
-          Statistics: Num rows: 20 Data size: 4028 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 20 Data size: 4160 Basic stats: COMPLETE Column stats: PARTIAL
           GatherStats: false
           Select Operator
             expressions: state (type: string), locid (type: int), cnt (type: decimal(10,0)), zip (type: int)

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/llap/llapdecider.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/llapdecider.q.out b/ql/src/test/results/clientpositive/llap/llapdecider.q.out
index 676a0e4..fd33181 100644
--- a/ql/src/test/results/clientpositive/llap/llapdecider.q.out
+++ b/ql/src/test/results/clientpositive/llap/llapdecider.q.out
@@ -20,11 +20,11 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: key, value
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(value)
                       keys: key (type: string)
@@ -251,11 +251,11 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: key, value
-                    Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(value)
                       keys: key (type: string)
@@ -324,7 +324,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -343,7 +343,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -412,7 +412,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -431,7 +431,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -503,7 +503,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -522,7 +522,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -593,7 +593,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -611,7 +611,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -680,7 +680,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -698,7 +698,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -767,7 +767,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -786,7 +786,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -856,7 +856,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -874,7 +874,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -943,7 +943,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -962,7 +962,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1039,7 +1039,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (UDFToInteger(key) > 1) (type: boolean)
                     Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1098,7 +1098,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (UDFToInteger(key) > 1) (type: boolean)
                     Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1155,7 +1155,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (UDFToInteger(GenericUDFTestGetJavaString(key)) > 1) (type: boolean)
                     Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out b/ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out
index 8955a61..9cbc411 100644
--- a/ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out
+++ b/ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out
@@ -169,7 +169,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: e
-                  Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: deptid is not null (type: boolean)
                     Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -187,7 +187,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: d
-                  Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: deptid is not null (type: boolean)
                     Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -248,7 +248,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: emp
-                  Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (deptid is not null and lastname is not null) (type: boolean)
                     Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -266,7 +266,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: dept
-                  Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (deptid is not null and deptname is not null) (type: boolean)
                     Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -322,7 +322,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: e
-                  Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (deptid is not null and lastname is not null) (type: boolean)
                     Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -340,7 +340,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: d
-                  Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (deptid is not null and deptname is not null) (type: boolean)
                     Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -400,7 +400,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: emp
-                  Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (deptid is not null and lastname is not null) (type: boolean)
                     Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -418,7 +418,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: dept
-                  Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (deptid is not null and deptname is not null) (type: boolean)
                     Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -478,7 +478,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: e
-                  Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: deptid is not null (type: boolean)
                     Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -496,7 +496,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: d
-                  Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: deptid is not null (type: boolean)
                     Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -514,7 +514,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: e
-                  Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: deptid is not null (type: boolean)
                     Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -575,7 +575,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: e
-                  Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: deptid is not null (type: boolean)
                     Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -593,7 +593,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: d
-                  Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: deptid is not null (type: boolean)
                     Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -611,7 +611,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: l
-                  Statistics: Num rows: 8 Data size: 109 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: locid is not null (type: boolean)
                     Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
@@ -674,7 +674,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: e
-                  Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (deptid is not null and lastname is not null) (type: boolean)
                     Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
@@ -692,7 +692,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: d
-                  Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (deptid is not null and deptname is not null) (type: boolean)
                     Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
@@ -709,7 +709,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: l
-                  Statistics: Num rows: 8 Data size: 109 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (locid is not null and state is not null) (type: boolean)
                     Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/stats_ppr_all.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_ppr_all.q.out b/ql/src/test/results/clientpositive/stats_ppr_all.q.out
index 7627f7a..c63c5b7 100644
--- a/ql/src/test/results/clientpositive/stats_ppr_all.q.out
+++ b/ql/src/test/results/clientpositive/stats_ppr_all.q.out
@@ -74,11 +74,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: order_amount (type: float)
               outputColumnNames: order_amount
-              Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: sum(order_amount)
                 mode: hash
@@ -173,11 +173,11 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ss
-            Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: order_amount (type: float)
               outputColumnNames: order_amount
-              Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
                 aggregations: sum(order_amount)
                 mode: hash


[29/43] hive git commit: HIVE-11525: Tez Bucket pruning (Gopal V, reviewed by Sergey Shelukhin)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/ql/src/test/results/clientpositive/tez/bucketpruning1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/bucketpruning1.q.out b/ql/src/test/results/clientpositive/tez/bucketpruning1.q.out
new file mode 100644
index 0000000..3b90687
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/bucketpruning1.q.out
@@ -0,0 +1,2360 @@
+PREHOOK: query: CREATE TABLE srcbucket_pruned(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 16 BUCKETS STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srcbucket_pruned
+POSTHOOK: query: CREATE TABLE srcbucket_pruned(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 16 BUCKETS STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srcbucket_pruned
+PREHOOK: query: -- cannot prune 2-key scenarios without a smarter optimizer
+CREATE TABLE srcbucket_unpruned(key int, value string) partitioned by (ds string) CLUSTERED BY (key,value) INTO 16 BUCKETS STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srcbucket_unpruned
+POSTHOOK: query: -- cannot prune 2-key scenarios without a smarter optimizer
+CREATE TABLE srcbucket_unpruned(key int, value string) partitioned by (ds string) CLUSTERED BY (key,value) INTO 16 BUCKETS STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srcbucket_unpruned
+PREHOOK: query: -- good cases
+
+explain extended
+select * from srcbucket_pruned where key = 1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- good cases
+
+explain extended
+select * from srcbucket_pruned where key = 1
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            1
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (key = 1) (type: boolean)
+                  buckets included: [1,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (key = 1) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: 1 (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 16
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 16
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            16
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (key = 16) (type: boolean)
+                  buckets included: [0,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (key = 16) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: 16 (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 17
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 17
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            17
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (key = 17) (type: boolean)
+                  buckets included: [1,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (key = 17) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: 17 (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 16+1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 16+1
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            +
+               16
+               1
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (key = 17) (type: boolean)
+                  buckets included: [1,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (key = 17) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: 17 (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = '11'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = '11'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            '11'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (key = 11) (type: boolean)
+                  buckets included: [11,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (key = 11) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: 11 (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            =
+               TOK_TABLE_OR_COL
+                  key
+               1
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: ((key = 1) and (ds = '2008-04-08')) (type: boolean)
+                  buckets included: [1,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: ((key = 1) and (ds = '2008-04-08')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: 1 (type: int), value (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' and value='One'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' and value='One'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     ds
+                  '2008-04-08'
+            =
+               TOK_TABLE_OR_COL
+                  value
+               'One'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (((key = 1) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+                  buckets included: [1,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (((key = 1) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: 1 (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key = 1 and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key = 1 and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'One'
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (((value = 'One') and (key = 1)) and (ds = '2008-04-08')) (type: boolean)
+                  buckets included: [1,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (((value = 'One') and (key = 1)) and (ds = '2008-04-08')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: 1 (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3)
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         TOK_FUNCTION
+            in
+            TOK_TABLE_OR_COL
+               key
+            2
+            3
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (key) IN (2, 3) (type: boolean)
+                  buckets included: [2,3,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (key) IN (2, 3) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3) and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3) and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            TOK_FUNCTION
+               in
+               TOK_TABLE_OR_COL
+                  key
+               2
+               3
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: ((key) IN (2, 3) and (ds = '2008-04-08')) (type: boolean)
+                  buckets included: [2,3,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: ((key) IN (2, 3) and (ds = '2008-04-08')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3) and ds='2008-04-08' and value='One'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key in (2,3) and ds='2008-04-08' and value='One'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               TOK_FUNCTION
+                  in
+                  TOK_TABLE_OR_COL
+                     key
+                  2
+                  3
+               =
+                  TOK_TABLE_OR_COL
+                     ds
+                  '2008-04-08'
+            =
+               TOK_TABLE_OR_COL
+                  value
+               'One'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (((key) IN (2, 3) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+                  buckets included: [2,3,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (((key) IN (2, 3) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key in (2,3) and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key in (2,3) and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'One'
+               TOK_FUNCTION
+                  in
+                  TOK_TABLE_OR_COL
+                     key
+                  2
+                  3
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (((value = 'One') and (key) IN (2, 3)) and (ds = '2008-04-08')) (type: boolean)
+                  buckets included: [2,3,] of 16
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (((value = 'One') and (key) IN (2, 3)) and (ds = '2008-04-08')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where (key=1 or key=2) and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where (key=1 or key=2) and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            or
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  2
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (((key = 1) or (key = 2)) and (ds = '2008-04-08')) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (((key = 1) or (key = 2)) and (ds = '2008-04-08')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where (key=1 or key=2) and value = 'One' and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where (key=1 or key=2) and value = 'One' and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               or
+                  =
+                     TOK_TABLE_OR_COL
+                        key
+                     1
+                  =
+                     TOK_TABLE_OR_COL
+                        key
+                     2
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'One'
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: ((((key = 1) or (key = 2)) and (value = 'One')) and (ds = '2008-04-08')) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: ((((key = 1) or (key = 2)) and (value = 'One')) and (ds = '2008-04-08')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: -- valid but irrelevant case (all buckets selected)
+
+explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- valid but irrelevant case (all buckets selected)
+
+explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17)
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         TOK_FUNCTION
+            in
+            TOK_TABLE_OR_COL
+               key
+            1
+            2
+            3
+            4
+            5
+            6
+            7
+            8
+            9
+            10
+            11
+            12
+            13
+            14
+            15
+            16
+            17
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            TOK_FUNCTION
+               in
+               TOK_TABLE_OR_COL
+                  key
+               1
+               2
+               3
+               4
+               5
+               6
+               7
+               8
+               9
+               10
+               11
+               12
+               13
+               14
+               15
+               16
+               17
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: ((key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) and (ds = '2008-04-08')) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: ((key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) and (ds = '2008-04-08')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08' and value='One'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08' and value='One'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               TOK_FUNCTION
+                  in
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+                  2
+                  3
+                  4
+                  5
+                  6
+                  7
+                  8
+                  9
+                  10
+                  11
+                  12
+                  13
+                  14
+                  15
+                  16
+                  17
+               =
+                  TOK_TABLE_OR_COL
+                     ds
+                  '2008-04-08'
+            =
+               TOK_TABLE_OR_COL
+                  value
+               'One'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (((key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (((key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) and (ds = '2008-04-08')) and (value = 'One')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where value='One' and key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'One'
+               TOK_FUNCTION
+                  in
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+                  2
+                  3
+                  4
+                  5
+                  6
+                  7
+                  8
+                  9
+                  10
+                  11
+                  12
+                  13
+                  14
+                  15
+                  16
+                  17
+            =
+               TOK_TABLE_OR_COL
+                  ds
+               '2008-04-08'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (((value = 'One') and (key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) and (ds = '2008-04-08')) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (((value = 'One') and (key) IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) and (ds = '2008-04-08')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), 'One' (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: -- valid, but unimplemented cases
+
+explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' or key = 2
+PREHOOK: type: QUERY
+POSTHOOK: query: -- valid, but unimplemented cases
+
+explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' or key = 2
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         or
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     ds
+                  '2008-04-08'
+            =
+               TOK_TABLE_OR_COL
+                  key
+               2
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (((key = 1) and (ds = '2008-04-08')) or (key = 2)) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (((key = 1) and (ds = '2008-04-08')) or (key = 2)) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' and (value='One' or value = 'Two')
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' and (value='One' or value = 'Two')
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         and
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     ds
+                  '2008-04-08'
+            or
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'One'
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  'Two'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: (((key = 1) and (ds = '2008-04-08')) and ((value = 'One') or (value = 'Two'))) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (((key = 1) and (ds = '2008-04-08')) and ((value = 'One') or (value = 'Two'))) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: 1 (type: int), value (type: string), '2008-04-08' (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One" or key = 2 and value = "Two"
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One" or key = 2 and value = "Two"
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         or
+            or
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  "One"
+            and
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  2
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  "Two"
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: ((key = 1) or (value = 'One') or ((key = 2) and (value = 'Two'))) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: ((key = 1) or (value = 'One') or ((key = 2) and (value = 'Two'))) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: -- Invalid cases
+
+explain extended
+select * from srcbucket_pruned where key = 'x11'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Invalid cases
+
+explain extended
+select * from srcbucket_pruned where key = 'x11'
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            'x11'
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: false (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One"
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One"
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         or
+            =
+               TOK_TABLE_OR_COL
+                  key
+               1
+            =
+               TOK_TABLE_OR_COL
+                  value
+               "One"
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: ((key = 1) or (value = 'One')) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: ((key = 1) or (value = 'One')) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One" or key = 2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_pruned where key = 1 or value = "One" or key = 2
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_pruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         or
+            or
+               =
+                  TOK_TABLE_OR_COL
+                     key
+                  1
+               =
+                  TOK_TABLE_OR_COL
+                     value
+                  "One"
+            =
+               TOK_TABLE_OR_COL
+                  key
+               2
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_pruned
+                  filterExpr: ((key = 1) or (value = 'One') or (key = 2)) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: ((key = 1) or (value = 'One') or (key = 2)) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_unpruned where key in (3, 5)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_unpruned where key in (3, 5)
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_unpruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         TOK_FUNCTION
+            in
+            TOK_TABLE_OR_COL
+               key
+            3
+            5
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_unpruned
+                  filterExpr: (key) IN (3, 5) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (key) IN (3, 5) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+
+PREHOOK: query: explain extended
+select * from srcbucket_unpruned where key = 1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select * from srcbucket_unpruned where key = 1
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  
+TOK_QUERY
+   TOK_FROM
+      TOK_TABREF
+         TOK_TABNAME
+            srcbucket_unpruned
+   TOK_INSERT
+      TOK_DESTINATION
+         TOK_DIR
+            TOK_TMP_FILE
+      TOK_SELECT
+         TOK_SELEXPR
+            TOK_ALLCOLREF
+      TOK_WHERE
+         =
+            TOK_TABLE_OR_COL
+               key
+            1
+
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: srcbucket_unpruned
+                  filterExpr: (key = 1) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (key = 1) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: 1 (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      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.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1,_col2
+                              columns.types int: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
+


[03/43] hive git commit: HIVE-12306: fix hbase_queries.q failure (Chaoyu Tang, reviewed by Jimmy Xiang)

Posted by om...@apache.org.
HIVE-12306: fix hbase_queries.q failure (Chaoyu Tang, reviewed by Jimmy Xiang)


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

Branch: refs/heads/master-fixed
Commit: c4abe2e414b4eb63d52d32fb72c50f390b0b4ed1
Parents: 8a5040c
Author: ctang <ct...@gmail.com>
Authored: Sat Nov 7 11:23:16 2015 -0500
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:29 2015 -0800

----------------------------------------------------------------------
 hbase-handler/src/test/results/positive/hbase_queries.q.out | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c4abe2e4/hbase-handler/src/test/results/positive/hbase_queries.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_queries.q.out b/hbase-handler/src/test/results/positive/hbase_queries.q.out
index d887566..d044c7e 100644
--- a/hbase-handler/src/test/results/positive/hbase_queries.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_queries.q.out
@@ -2,14 +2,14 @@ PREHOOK: query: DROP TABLE hbase_table_1
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: DROP TABLE hbase_table_1
 POSTHOOK: type: DROPTABLE
-PREHOOK: query: CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') 
+PREHOOK: query: CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value')
 STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
 WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string")
 TBLPROPERTIES ("hbase.table.name" = "hbase_table_0")
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@hbase_table_1
-POSTHOOK: query: CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') 
+POSTHOOK: query: CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value')
 STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
 WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string")
 TBLPROPERTIES ("hbase.table.name" = "hbase_table_0")


[36/43] hive git commit: HIVE-11955 : Add costing for join-groupby transpose rule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by om...@apache.org.
HIVE-11955 : Add costing for join-groupby transpose rule (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: fd54e595245b075e86c2191245f9938f9c64d1b4
Parents: 58e07b1
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Wed Nov 11 14:31:36 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:34 2015 -0800

----------------------------------------------------------------------
 .../calcite/rules/HiveAggregateJoinTransposeRule.java        | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/fd54e595/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
index 211b6fa..c59af39 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
@@ -296,6 +296,14 @@ public class HiveAggregateJoinTransposeRule extends AggregateJoinTransposeRule {
           Mappings.apply2(mapping, aggregate.getGroupSets()), newAggCalls);
     }
     call.transformTo(r);
+    // Add original tree as well for potential alternative transformation.
+    // This is modeled after LoptOptimizeJoinRule::findBestOrderings() in
+    // which rule adds multiple transformations and Planner picks the cheapest one.
+    // Hep planner will automatically pick the one with lower cost among two.
+    // For details, see: HepPlanner:applyTransformationResults()
+    // In this case, if ndv is close to # of rows, i.e., group by is not resulting
+    // in any deduction, doing this transformation is not useful.
+    call.transformTo(aggregate);
   }
 
   /** Computes the closure of a set of columns according to a given list of


[14/43] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 3bc7e10..4690093 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
@@ -84,6 +84,8 @@ public class ThriftHiveMetastore {
 
     public List<String> get_tables(String db_name, String pattern) throws MetaException, org.apache.thrift.TException;
 
+    public List<TableMeta> get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types) throws MetaException, org.apache.thrift.TException;
+
     public List<String> get_all_tables(String db_name) throws MetaException, org.apache.thrift.TException;
 
     public Table get_table(String dbname, String tbl_name) throws MetaException, NoSuchObjectException, org.apache.thrift.TException;
@@ -348,6 +350,8 @@ public class ThriftHiveMetastore {
 
     public void get_tables(String db_name, String pattern, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void get_all_tables(String db_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void get_table(String dbname, String tbl_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -1204,6 +1208,34 @@ public class ThriftHiveMetastore {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_tables failed: unknown result");
     }
 
+    public List<TableMeta> get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types) throws MetaException, org.apache.thrift.TException
+    {
+      send_get_table_meta(db_patterns, tbl_patterns, tbl_types);
+      return recv_get_table_meta();
+    }
+
+    public void send_get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types) throws org.apache.thrift.TException
+    {
+      get_table_meta_args args = new get_table_meta_args();
+      args.setDb_patterns(db_patterns);
+      args.setTbl_patterns(tbl_patterns);
+      args.setTbl_types(tbl_types);
+      sendBase("get_table_meta", args);
+    }
+
+    public List<TableMeta> recv_get_table_meta() throws MetaException, org.apache.thrift.TException
+    {
+      get_table_meta_result result = new get_table_meta_result();
+      receiveBase(result, "get_table_meta");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_table_meta failed: unknown result");
+    }
+
     public List<String> get_all_tables(String db_name) throws MetaException, org.apache.thrift.TException
     {
       send_get_all_tables(db_name);
@@ -5131,6 +5163,44 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_table_meta_call method_call = new get_table_meta_call(db_patterns, tbl_patterns, tbl_types, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_table_meta_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String db_patterns;
+      private String tbl_patterns;
+      private List<String> tbl_types;
+      public get_table_meta_call(String db_patterns, String tbl_patterns, List<String> tbl_types, 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.db_patterns = db_patterns;
+        this.tbl_patterns = tbl_patterns;
+        this.tbl_types = tbl_types;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_table_meta", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_table_meta_args args = new get_table_meta_args();
+        args.setDb_patterns(db_patterns);
+        args.setTbl_patterns(tbl_patterns);
+        args.setTbl_types(tbl_types);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<TableMeta> getResult() throws MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_table_meta();
+      }
+    }
+
     public void get_all_tables(String db_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       get_all_tables_call method_call = new get_all_tables_call(db_name, resultHandler, this, ___protocolFactory, ___transport);
@@ -9061,6 +9131,7 @@ public class ThriftHiveMetastore {
       processMap.put("drop_table", new drop_table());
       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
       processMap.put("get_tables", new get_tables());
+      processMap.put("get_table_meta", new get_table_meta());
       processMap.put("get_all_tables", new get_all_tables());
       processMap.put("get_table", new get_table());
       processMap.put("get_table_objects_by_name", new get_table_objects_by_name());
@@ -9731,6 +9802,30 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_table_meta<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_table_meta_args> {
+      public get_table_meta() {
+        super("get_table_meta");
+      }
+
+      public get_table_meta_args getEmptyArgsInstance() {
+        return new get_table_meta_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_table_meta_result getResult(I iface, get_table_meta_args args) throws org.apache.thrift.TException {
+        get_table_meta_result result = new get_table_meta_result();
+        try {
+          result.success = iface.get_table_meta(args.db_patterns, args.tbl_patterns, args.tbl_types);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
     public static class get_all_tables<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_all_tables_args> {
       public get_all_tables() {
         super("get_all_tables");
@@ -12558,6 +12653,7 @@ public class ThriftHiveMetastore {
       processMap.put("drop_table", new drop_table());
       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
       processMap.put("get_tables", new get_tables());
+      processMap.put("get_table_meta", new get_table_meta());
       processMap.put("get_all_tables", new get_all_tables());
       processMap.put("get_table", new get_table());
       processMap.put("get_table_objects_by_name", new get_table_objects_by_name());
@@ -13991,6 +14087,63 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_table_meta<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_meta_args, List<TableMeta>> {
+      public get_table_meta() {
+        super("get_table_meta");
+      }
+
+      public get_table_meta_args getEmptyArgsInstance() {
+        return new get_table_meta_args();
+      }
+
+      public AsyncMethodCallback<List<TableMeta>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<List<TableMeta>>() { 
+          public void onComplete(List<TableMeta> o) {
+            get_table_meta_result result = new get_table_meta_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_table_meta_result result = new get_table_meta_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_table_meta_args args, org.apache.thrift.async.AsyncMethodCallback<List<TableMeta>> resultHandler) throws TException {
+        iface.get_table_meta(args.db_patterns, args.tbl_patterns, args.tbl_types,resultHandler);
+      }
+    }
+
     public static class get_all_tables<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_all_tables_args, List<String>> {
       public get_all_tables() {
         super("get_all_tables");
@@ -40884,15 +41037,960 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class drop_table_with_environment_context_resultStandardSchemeFactory implements SchemeFactory {
-      public drop_table_with_environment_context_resultStandardScheme getScheme() {
-        return new drop_table_with_environment_context_resultStandardScheme();
+    private static class drop_table_with_environment_context_resultStandardSchemeFactory implements SchemeFactory {
+      public drop_table_with_environment_context_resultStandardScheme getScheme() {
+        return new drop_table_with_environment_context_resultStandardScheme();
+      }
+    }
+
+    private static class drop_table_with_environment_context_resultStandardScheme extends StandardScheme<drop_table_with_environment_context_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, drop_table_with_environment_context_result 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: // O1
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o1 = new NoSuchObjectException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // O3
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o3 = new MetaException();
+                struct.o3.read(iprot);
+                struct.setO3IsSet(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, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.o3 != null) {
+          oprot.writeFieldBegin(O3_FIELD_DESC);
+          struct.o3.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class drop_table_with_environment_context_resultTupleSchemeFactory implements SchemeFactory {
+      public drop_table_with_environment_context_resultTupleScheme getScheme() {
+        return new drop_table_with_environment_context_resultTupleScheme();
+      }
+    }
+
+    private static class drop_table_with_environment_context_resultTupleScheme extends TupleScheme<drop_table_with_environment_context_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetO1()) {
+          optionals.set(0);
+        }
+        if (struct.isSetO3()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
+        }
+        if (struct.isSetO3()) {
+          struct.o3.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.o1 = new NoSuchObjectException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.o3 = new MetaException();
+          struct.o3.read(iprot);
+          struct.setO3IsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class get_tables_args implements org.apache.thrift.TBase<get_tables_args, get_tables_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_args");
+
+    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("pattern", 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 get_tables_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_tables_argsTupleSchemeFactory());
+    }
+
+    private String db_name; // required
+    private String pattern; // 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, "db_name"),
+      PATTERN((short)2, "pattern");
+
+      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: // PATTERN
+            return PATTERN;
+          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("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.PATTERN, new org.apache.thrift.meta_data.FieldMetaData("pattern", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_args.class, metaDataMap);
+    }
+
+    public get_tables_args() {
+    }
+
+    public get_tables_args(
+      String db_name,
+      String pattern)
+    {
+      this();
+      this.db_name = db_name;
+      this.pattern = pattern;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_tables_args(get_tables_args other) {
+      if (other.isSetDb_name()) {
+        this.db_name = other.db_name;
+      }
+      if (other.isSetPattern()) {
+        this.pattern = other.pattern;
+      }
+    }
+
+    public get_tables_args deepCopy() {
+      return new get_tables_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.db_name = null;
+      this.pattern = null;
+    }
+
+    public String getDb_name() {
+      return this.db_name;
+    }
+
+    public void setDb_name(String db_name) {
+      this.db_name = db_name;
+    }
+
+    public void unsetDb_name() {
+      this.db_name = null;
+    }
+
+    /** Returns true if field db_name is set (has been assigned a value) and false otherwise */
+    public boolean isSetDb_name() {
+      return this.db_name != null;
+    }
+
+    public void setDb_nameIsSet(boolean value) {
+      if (!value) {
+        this.db_name = null;
+      }
+    }
+
+    public String getPattern() {
+      return this.pattern;
+    }
+
+    public void setPattern(String pattern) {
+      this.pattern = pattern;
+    }
+
+    public void unsetPattern() {
+      this.pattern = null;
+    }
+
+    /** Returns true if field pattern is set (has been assigned a value) and false otherwise */
+    public boolean isSetPattern() {
+      return this.pattern != null;
+    }
+
+    public void setPatternIsSet(boolean value) {
+      if (!value) {
+        this.pattern = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case DB_NAME:
+        if (value == null) {
+          unsetDb_name();
+        } else {
+          setDb_name((String)value);
+        }
+        break;
+
+      case PATTERN:
+        if (value == null) {
+          unsetPattern();
+        } else {
+          setPattern((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case DB_NAME:
+        return getDb_name();
+
+      case PATTERN:
+        return getPattern();
+
+      }
+      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 isSetDb_name();
+      case PATTERN:
+        return isSetPattern();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_tables_args)
+        return this.equals((get_tables_args)that);
+      return false;
+    }
+
+    public boolean equals(get_tables_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_db_name = true && this.isSetDb_name();
+      boolean that_present_db_name = true && that.isSetDb_name();
+      if (this_present_db_name || that_present_db_name) {
+        if (!(this_present_db_name && that_present_db_name))
+          return false;
+        if (!this.db_name.equals(that.db_name))
+          return false;
+      }
+
+      boolean this_present_pattern = true && this.isSetPattern();
+      boolean that_present_pattern = true && that.isSetPattern();
+      if (this_present_pattern || that_present_pattern) {
+        if (!(this_present_pattern && that_present_pattern))
+          return false;
+        if (!this.pattern.equals(that.pattern))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_db_name = true && (isSetDb_name());
+      list.add(present_db_name);
+      if (present_db_name)
+        list.add(db_name);
+
+      boolean present_pattern = true && (isSetPattern());
+      list.add(present_pattern);
+      if (present_pattern)
+        list.add(pattern);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_tables_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDb_name()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, other.db_name);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetPattern()).compareTo(other.isSetPattern());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetPattern()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pattern, other.pattern);
+        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("get_tables_args(");
+      boolean first = true;
+
+      sb.append("db_name:");
+      if (this.db_name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.db_name);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("pattern:");
+      if (this.pattern == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.pattern);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // 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 get_tables_argsStandardSchemeFactory implements SchemeFactory {
+      public get_tables_argsStandardScheme getScheme() {
+        return new get_tables_argsStandardScheme();
+      }
+    }
+
+    private static class get_tables_argsStandardScheme extends StandardScheme<get_tables_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_args 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.db_name = iprot.readString();
+                struct.setDb_nameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // PATTERN
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.pattern = iprot.readString();
+                struct.setPatternIsSet(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, get_tables_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.db_name != null) {
+          oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+          oprot.writeString(struct.db_name);
+          oprot.writeFieldEnd();
+        }
+        if (struct.pattern != null) {
+          oprot.writeFieldBegin(PATTERN_FIELD_DESC);
+          oprot.writeString(struct.pattern);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_tables_argsTupleSchemeFactory implements SchemeFactory {
+      public get_tables_argsTupleScheme getScheme() {
+        return new get_tables_argsTupleScheme();
+      }
+    }
+
+    private static class get_tables_argsTupleScheme extends TupleScheme<get_tables_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetDb_name()) {
+          optionals.set(0);
+        }
+        if (struct.isSetPattern()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetDb_name()) {
+          oprot.writeString(struct.db_name);
+        }
+        if (struct.isSetPattern()) {
+          oprot.writeString(struct.pattern);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.db_name = iprot.readString();
+          struct.setDb_nameIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.pattern = iprot.readString();
+          struct.setPatternIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class get_tables_result implements org.apache.thrift.TBase<get_tables_result, get_tables_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_tables_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_tables_resultTupleSchemeFactory());
+    }
+
+    private List<String> success; // required
+    private MetaException o1; // 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 {
+      SUCCESS((short)0, "success"),
+      O1((short)1, "o1");
+
+      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 0: // SUCCESS
+            return SUCCESS;
+          case 1: // O1
+            return O1;
+          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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          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.STRING))));
+      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_result.class, metaDataMap);
+    }
+
+    public get_tables_result() {
+    }
+
+    public get_tables_result(
+      List<String> success,
+      MetaException o1)
+    {
+      this();
+      this.success = success;
+      this.o1 = o1;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_tables_result(get_tables_result other) {
+      if (other.isSetSuccess()) {
+        List<String> __this__success = new ArrayList<String>(other.success);
+        this.success = __this__success;
+      }
+      if (other.isSetO1()) {
+        this.o1 = new MetaException(other.o1);
+      }
+    }
+
+    public get_tables_result deepCopy() {
+      return new get_tables_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.o1 = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public java.util.Iterator<String> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(String elem) {
+      if (this.success == null) {
+        this.success = new ArrayList<String>();
+      }
+      this.success.add(elem);
+    }
+
+    public List<String> getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(List<String> success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public MetaException getO1() {
+      return this.o1;
+    }
+
+    public void setO1(MetaException o1) {
+      this.o1 = o1;
+    }
+
+    public void unsetO1() {
+      this.o1 = null;
+    }
+
+    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO1() {
+      return this.o1 != null;
+    }
+
+    public void setO1IsSet(boolean value) {
+      if (!value) {
+        this.o1 = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((List<String>)value);
+        }
+        break;
+
+      case O1:
+        if (value == null) {
+          unsetO1();
+        } else {
+          setO1((MetaException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case O1:
+        return getO1();
+
+      }
+      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 SUCCESS:
+        return isSetSuccess();
+      case O1:
+        return isSetO1();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_tables_result)
+        return this.equals((get_tables_result)that);
+      return false;
+    }
+
+    public boolean equals(get_tables_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_o1 = true && this.isSetO1();
+      boolean that_present_o1 = true && that.isSetO1();
+      if (this_present_o1 || that_present_o1) {
+        if (!(this_present_o1 && that_present_o1))
+          return false;
+        if (!this.o1.equals(that.o1))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      boolean present_o1 = true && (isSetO1());
+      list.add(present_o1);
+      if (present_o1)
+        list.add(o1);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_tables_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetO1()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
+        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("get_tables_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("o1:");
+      if (this.o1 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.o1);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // 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 get_tables_resultStandardSchemeFactory implements SchemeFactory {
+      public get_tables_resultStandardScheme getScheme() {
+        return new get_tables_resultStandardScheme();
       }
     }
 
-    private static class drop_table_with_environment_context_resultStandardScheme extends StandardScheme<drop_table_with_environment_context_result> {
+    private static class get_tables_resultStandardScheme extends StandardScheme<get_tables_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -40902,20 +42000,29 @@ public class ThriftHiveMetastore {
             break;
           }
           switch (schemeField.id) {
-            case 1: // O1
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.o1 = new NoSuchObjectException();
-                struct.o1.read(iprot);
-                struct.setO1IsSet(true);
+            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)
+                  {
+                    _elem651 = iprot.readString();
+                    struct.success.add(_elem651);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // O3
+            case 1: // O1
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.o3 = new MetaException();
-                struct.o3.read(iprot);
-                struct.setO3IsSet(true);
+                struct.o1 = new MetaException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -40929,91 +42036,115 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          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)
+            {
+              oprot.writeString(_iter653);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
         if (struct.o1 != null) {
           oprot.writeFieldBegin(O1_FIELD_DESC);
           struct.o1.write(oprot);
           oprot.writeFieldEnd();
         }
-        if (struct.o3 != null) {
-          oprot.writeFieldBegin(O3_FIELD_DESC);
-          struct.o3.write(oprot);
-          oprot.writeFieldEnd();
-        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class drop_table_with_environment_context_resultTupleSchemeFactory implements SchemeFactory {
-      public drop_table_with_environment_context_resultTupleScheme getScheme() {
-        return new drop_table_with_environment_context_resultTupleScheme();
+    private static class get_tables_resultTupleSchemeFactory implements SchemeFactory {
+      public get_tables_resultTupleScheme getScheme() {
+        return new get_tables_resultTupleScheme();
       }
     }
 
-    private static class drop_table_with_environment_context_resultTupleScheme extends TupleScheme<drop_table_with_environment_context_result> {
+    private static class get_tables_resultTupleScheme extends TupleScheme<get_tables_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetO1()) {
+        if (struct.isSetSuccess()) {
           optionals.set(0);
         }
-        if (struct.isSetO3()) {
+        if (struct.isSetO1()) {
           optionals.set(1);
         }
         oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (String _iter654 : struct.success)
+            {
+              oprot.writeString(_iter654);
+            }
+          }
+        }
         if (struct.isSetO1()) {
           struct.o1.write(oprot);
         }
-        if (struct.isSetO3()) {
-          struct.o3.write(oprot);
-        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
-          struct.o1 = new NoSuchObjectException();
-          struct.o1.read(iprot);
-          struct.setO1IsSet(true);
+          {
+            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)
+            {
+              _elem656 = iprot.readString();
+              struct.success.add(_elem656);
+            }
+          }
+          struct.setSuccessIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.o3 = new MetaException();
-          struct.o3.read(iprot);
-          struct.setO3IsSet(true);
+          struct.o1 = new MetaException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
         }
       }
     }
 
   }
 
-  public static class get_tables_args implements org.apache.thrift.TBase<get_tables_args, get_tables_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_args");
+  public static class get_table_meta_args implements org.apache.thrift.TBase<get_table_meta_args, get_table_meta_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_table_meta_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_table_meta_args");
 
-    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
-    private static final org.apache.thrift.protocol.TField PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("pattern", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField DB_PATTERNS_FIELD_DESC = new org.apache.thrift.protocol.TField("db_patterns", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField TBL_PATTERNS_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_patterns", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField TBL_TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_types", org.apache.thrift.protocol.TType.LIST, (short)3);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_tables_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_tables_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_table_meta_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_table_meta_argsTupleSchemeFactory());
     }
 
-    private String db_name; // required
-    private String pattern; // required
+    private String db_patterns; // required
+    private String tbl_patterns; // required
+    private List<String> tbl_types; // 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, "db_name"),
-      PATTERN((short)2, "pattern");
+      DB_PATTERNS((short)1, "db_patterns"),
+      TBL_PATTERNS((short)2, "tbl_patterns"),
+      TBL_TYPES((short)3, "tbl_types");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -41028,10 +42159,12 @@ public class ThriftHiveMetastore {
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // DB_NAME
-            return DB_NAME;
-          case 2: // PATTERN
-            return PATTERN;
+          case 1: // DB_PATTERNS
+            return DB_PATTERNS;
+          case 2: // TBL_PATTERNS
+            return TBL_PATTERNS;
+          case 3: // TBL_TYPES
+            return TBL_TYPES;
           default:
             return null;
         }
@@ -41075,109 +42208,165 @@ public class ThriftHiveMetastore {
     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("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+      tmpMap.put(_Fields.DB_PATTERNS, new org.apache.thrift.meta_data.FieldMetaData("db_patterns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.PATTERN, new org.apache.thrift.meta_data.FieldMetaData("pattern", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+      tmpMap.put(_Fields.TBL_PATTERNS, new org.apache.thrift.meta_data.FieldMetaData("tbl_patterns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.TBL_TYPES, new org.apache.thrift.meta_data.FieldMetaData("tbl_types", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          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.STRING))));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_table_meta_args.class, metaDataMap);
     }
 
-    public get_tables_args() {
+    public get_table_meta_args() {
     }
 
-    public get_tables_args(
-      String db_name,
-      String pattern)
+    public get_table_meta_args(
+      String db_patterns,
+      String tbl_patterns,
+      List<String> tbl_types)
     {
       this();
-      this.db_name = db_name;
-      this.pattern = pattern;
+      this.db_patterns = db_patterns;
+      this.tbl_patterns = tbl_patterns;
+      this.tbl_types = tbl_types;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_tables_args(get_tables_args other) {
-      if (other.isSetDb_name()) {
-        this.db_name = other.db_name;
+    public get_table_meta_args(get_table_meta_args other) {
+      if (other.isSetDb_patterns()) {
+        this.db_patterns = other.db_patterns;
       }
-      if (other.isSetPattern()) {
-        this.pattern = other.pattern;
+      if (other.isSetTbl_patterns()) {
+        this.tbl_patterns = other.tbl_patterns;
+      }
+      if (other.isSetTbl_types()) {
+        List<String> __this__tbl_types = new ArrayList<String>(other.tbl_types);
+        this.tbl_types = __this__tbl_types;
       }
     }
 
-    public get_tables_args deepCopy() {
-      return new get_tables_args(this);
+    public get_table_meta_args deepCopy() {
+      return new get_table_meta_args(this);
     }
 
     @Override
     public void clear() {
-      this.db_name = null;
-      this.pattern = null;
+      this.db_patterns = null;
+      this.tbl_patterns = null;
+      this.tbl_types = null;
     }
 
-    public String getDb_name() {
-      return this.db_name;
+    public String getDb_patterns() {
+      return this.db_patterns;
     }
 
-    public void setDb_name(String db_name) {
-      this.db_name = db_name;
+    public void setDb_patterns(String db_patterns) {
+      this.db_patterns = db_patterns;
     }
 
-    public void unsetDb_name() {
-      this.db_name = null;
+    public void unsetDb_patterns() {
+      this.db_patterns = null;
     }
 
-    /** Returns true if field db_name is set (has been assigned a value) and false otherwise */
-    public boolean isSetDb_name() {
-      return this.db_name != null;
+    /** Returns true if field db_patterns is set (has been assigned a value) and false otherwise */
+    public boolean isSetDb_patterns() {
+      return this.db_patterns != null;
     }
 
-    public void setDb_nameIsSet(boolean value) {
+    public void setDb_patternsIsSet(boolean value) {
       if (!value) {
-        this.db_name = null;
+        this.db_patterns = null;
       }
     }
 
-    public String getPattern() {
-      return this.pattern;
+    public String getTbl_patterns() {
+      return this.tbl_patterns;
     }
 
-    public void setPattern(String pattern) {
-      this.pattern = pattern;
+    public void setTbl_patterns(String tbl_patterns) {
+      this.tbl_patterns = tbl_patterns;
     }
 
-    public void unsetPattern() {
-      this.pattern = null;
+    public void unsetTbl_patterns() {
+      this.tbl_patterns = null;
     }
 
-    /** Returns true if field pattern is set (has been assigned a value) and false otherwise */
-    public boolean isSetPattern() {
-      return this.pattern != null;
+    /** Returns true if field tbl_patterns is set (has been assigned a value) and false otherwise */
+    public boolean isSetTbl_patterns() {
+      return this.tbl_patterns != null;
     }
 
-    public void setPatternIsSet(boolean value) {
+    public void setTbl_patternsIsSet(boolean value) {
       if (!value) {
-        this.pattern = null;
+        this.tbl_patterns = null;
+      }
+    }
+
+    public int getTbl_typesSize() {
+      return (this.tbl_types == null) ? 0 : this.tbl_types.size();
+    }
+
+    public java.util.Iterator<String> getTbl_typesIterator() {
+      return (this.tbl_types == null) ? null : this.tbl_types.iterator();
+    }
+
+    public void addToTbl_types(String elem) {
+      if (this.tbl_types == null) {
+        this.tbl_types = new ArrayList<String>();
+      }
+      this.tbl_types.add(elem);
+    }
+
+    public List<String> getTbl_types() {
+      return this.tbl_types;
+    }
+
+    public void setTbl_types(List<String> tbl_types) {
+      this.tbl_types = tbl_types;
+    }
+
+    public void unsetTbl_types() {
+      this.tbl_types = null;
+    }
+
+    /** Returns true if field tbl_types is set (has been assigned a value) and false otherwise */
+    public boolean isSetTbl_types() {
+      return this.tbl_types != null;
+    }
+
+    public void setTbl_typesIsSet(boolean value) {
+      if (!value) {
+        this.tbl_types = null;
       }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case DB_NAME:
+      case DB_PATTERNS:
         if (value == null) {
-          unsetDb_name();
+          unsetDb_patterns();
         } else {
-          setDb_name((String)value);
+          setDb_patterns((String)value);
         }
         break;
 
-      case PATTERN:
+      case TBL_PATTERNS:
         if (value == null) {
-          unsetPattern();
+          unsetTbl_patterns();
         } else {
-          setPattern((String)value);
+          setTbl_patterns((String)value);
+        }
+        break;
+
+      case TBL_TYPES:
+        if (value == null) {
+          unsetTbl_types();
+        } else {
+          setTbl_types((List<String>)value);
         }
         break;
 
@@ -41186,11 +42375,14 @@ public class ThriftHiveMetastore {
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case DB_NAME:
-        return getDb_name();
+      case DB_PATTERNS:
+        return getDb_patterns();
 
-      case PATTERN:
-        return getPattern();
+      case TBL_PATTERNS:
+        return getTbl_patterns();
+
+      case TBL_TYPES:
+        return getTbl_types();
 
       }
       throw new IllegalStateException();
@@ -41203,10 +42395,12 @@ public class ThriftHiveMetastore {
       }
 
       switch (field) {
-      case DB_NAME:
-        return isSetDb_name();
-      case PATTERN:
-        return isSetPattern();
+      case DB_PATTERNS:
+        return isSetDb_patterns();
+      case TBL_PATTERNS:
+        return isSetTbl_patterns();
+      case TBL_TYPES:
+        return isSetTbl_types();
       }
       throw new IllegalStateException();
     }
@@ -41215,30 +42409,39 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_tables_args)
-        return this.equals((get_tables_args)that);
+      if (that instanceof get_table_meta_args)
+        return this.equals((get_table_meta_args)that);
       return false;
     }
 
-    public boolean equals(get_tables_args that) {
+    public boolean equals(get_table_meta_args that) {
       if (that == null)
         return false;
 
-      boolean this_present_db_name = true && this.isSetDb_name();
-      boolean that_present_db_name = true && that.isSetDb_name();
-      if (this_present_db_name || that_present_db_name) {
-        if (!(this_present_db_name && that_present_db_name))
+      boolean this_present_db_patterns = true && this.isSetDb_patterns();
+      boolean that_present_db_patterns = true && that.isSetDb_patterns();
+      if (this_present_db_patterns || that_present_db_patterns) {
+        if (!(this_present_db_patterns && that_present_db_patterns))
           return false;
-        if (!this.db_name.equals(that.db_name))
+        if (!this.db_patterns.equals(that.db_patterns))
           return false;
       }
 
-      boolean this_present_pattern = true && this.isSetPattern();
-      boolean that_present_pattern = true && that.isSetPattern();
-      if (this_present_pattern || that_present_pattern) {
-        if (!(this_present_pattern && that_present_pattern))
+      boolean this_present_tbl_patterns = true && this.isSetTbl_patterns();
+      boolean that_present_tbl_patterns = true && that.isSetTbl_patterns();
+      if (this_present_tbl_patterns || that_present_tbl_patterns) {
+        if (!(this_present_tbl_patterns && that_present_tbl_patterns))
           return false;
-        if (!this.pattern.equals(that.pattern))
+        if (!this.tbl_patterns.equals(that.tbl_patterns))
+          return false;
+      }
+
+      boolean this_present_tbl_types = true && this.isSetTbl_types();
+      boolean that_present_tbl_types = true && that.isSetTbl_types();
+      if (this_present_tbl_types || that_present_tbl_types) {
+        if (!(this_present_tbl_types && that_present_tbl_types))
+          return false;
+        if (!this.tbl_types.equals(that.tbl_types))
           return false;
       }
 
@@ -41249,43 +42452,58 @@ public class ThriftHiveMetastore {
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
-      boolean present_db_name = true && (isSetDb_name());
-      list.add(present_db_name);
-      if (present_db_name)
-        list.add(db_name);
+      boolean present_db_patterns = true && (isSetDb_patterns());
+      list.add(present_db_patterns);
+      if (present_db_patterns)
+        list.add(db_patterns);
 
-      boolean present_pattern = true && (isSetPattern());
-      list.add(present_pattern);
-      if (present_pattern)
-        list.add(pattern);
+      boolean present_tbl_patterns = true && (isSetTbl_patterns());
+      list.add(present_tbl_patterns);
+      if (present_tbl_patterns)
+        list.add(tbl_patterns);
+
+      boolean present_tbl_types = true && (isSetTbl_types());
+      list.add(present_tbl_types);
+      if (present_tbl_types)
+        list.add(tbl_types);
 
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_tables_args other) {
+    public int compareTo(get_table_meta_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
+      lastComparison = Boolean.valueOf(isSetDb_patterns()).compareTo(other.isSetDb_patterns());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetDb_name()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, other.db_name);
+      if (isSetDb_patterns()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_patterns, other.db_patterns);
         if (lastComparison != 0) {
           return lastComparison;
         }
       }
-      lastComparison = Boolean.valueOf(isSetPattern()).compareTo(other.isSetPattern());
+      lastComparison = Boolean.valueOf(isSetTbl_patterns()).compareTo(other.isSetTbl_patterns());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetPattern()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pattern, other.pattern);
+      if (isSetTbl_patterns()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_patterns, other.tbl_patterns);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetTbl_types()).compareTo(other.isSetTbl_types());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTbl_types()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_types, other.tbl_types);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -41307,22 +42525,30 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_tables_args(");
+      StringBuilder sb = new StringBuilder("get_table_meta_args(");
       boolean first = true;
 
-      sb.append("db_name:");
-      if (this.db_name == null) {
+      sb.append("db_patterns:");
+      if (this.db_patterns == null) {
         sb.append("null");
       } else {
-        sb.append(this.db_name);
+        sb.append(this.db_patterns);
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("pattern:");
-      if (this.pattern == null) {
+      sb.append("tbl_patterns:");
+      if (this.tbl_patterns == null) {
         sb.append("null");
       } else {
-        sb.append(this.pattern);
+        sb.append(this.tbl_patterns);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("tbl_types:");
+      if (this.tbl_types == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tbl_types);
       }
       first = false;
       sb.append(")");
@@ -41350,15 +42576,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_tables_argsStandardSchemeFactory implements SchemeFactory {
-      public get_tables_argsStandardScheme getScheme() {
-        return new get_tables_argsStandardScheme();
+    private static class get_table_meta_argsStandardSchemeFactory implements SchemeFactory {
+      public get_table_meta_argsStandardScheme getScheme() {
+        return new get_table_meta_argsStandardScheme();
       }
     }
 
-    private static class get_tables_argsStandardScheme extends StandardScheme<get_tables_args> {
+    private static class get_table_meta_argsStandardScheme extends StandardScheme<get_table_meta_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_meta_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -41368,18 +42594,36 @@ public class ThriftHiveMetastore {
             break;
           }
           switch (schemeField.id) {
-            case 1: // DB_NAME
+            case 1: // DB_PATTERNS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.db_name = iprot.readString();
-                struct.setDb_nameIsSet(true);
+                struct.db_patterns = iprot.readString();
+                struct.setDb_patternsIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // PATTERN
+            case 2: // TBL_PATTERNS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.pattern = iprot.readString();
-                struct.setPatternIsSet(true);
+                struct.tbl_patterns = iprot.readString();
+                struct.setTbl_patternsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // TBL_TYPES
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list658.size);
+                  String _elem659;
+                  for (int _i660 = 0; _i660 < _list658.size; ++_i660)
+                  {
+                    _elem659 = iprot.readString();
+                    struct.tbl_types.add(_elem659);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setTbl_typesIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -41393,18 +42637,30 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_meta_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.db_name != null) {
-          oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
-          oprot.writeString(struct.db_name);
+        if (struct.db_patterns != null) {
+          oprot.writeFieldBegin(DB_PATTERNS_FIELD_DESC);
+          oprot.writeString(struct.db_patterns);
           oprot.writeFieldEnd();
         }
-        if (struct.pattern != null) {
-          oprot.writeFieldBegin(PATTERN_FIELD_DESC);
-          oprot.writeString(struct.pattern);
+        if (struct.tbl_patterns != null) {
+          oprot.writeFieldBegin(TBL_PATTERNS_FIELD_DESC);
+          oprot.writeString(struct.tbl_patterns);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tbl_types != null) {
+          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 _iter661 : struct.tbl_types)
+            {
+              oprot.writeString(_iter661);
+            }
+            oprot.writeListEnd();
+          }
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -41413,63 +42669,88 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_tables_argsTupleSchemeFactory implements SchemeFactory {
-      public get_tables_argsTupleScheme getScheme() {
-        return new get_tables_argsTupleScheme();
+    private static class get_table_meta_argsTupleSchemeFactory implements SchemeFactory {
+      public get_table_meta_argsTupleScheme getScheme() {
+        return new get_table_meta_argsTupleScheme();
       }
     }
 
-    private static class get_tables_argsTupleScheme extends TupleScheme<get_tables_args> {
+    private static class get_table_meta_argsTupleScheme extends TupleScheme<get_table_meta_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_table_meta_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetDb_name()) {
+        if (struct.isSetDb_patterns()) {
           optionals.set(0);
         }
-        if (struct.isSetPattern()) {
+        if (struct.isSetTbl_patterns()) {
           optionals.set(1);
         }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetDb_name()) {
-          oprot.writeString(struct.db_name);
+        if (struct.isSetTbl_types()) {
+          optionals.set(2);
         }
-        if (struct.isSetPattern()) {
-          oprot.writeString(struct.pattern);
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetDb_patterns()) {
+          oprot.writeString(struct.db_patterns);
+        }
+        if (struct.isSetTbl_patterns()) {
+          oprot.writeString(struct.tbl_patterns);
+        }
+        if (struct.isSetTbl_types()) {
+          {
+            oprot.writeI32(struct.tbl_types.size());
+            for (String _iter662 : struct.tbl_types)
+            {
+              oprot.writeString(_iter662);
+            }
+          }
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_table_meta_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(2);
+        BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
-          struct.db_name = iprot.readString();
-          struct.setDb_nameIsSet(true);
+          struct.db_patterns = iprot.readString();
+          struct.setDb_patternsIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.pattern = iprot.readString();
-          struct.setPatternIsSet(true);
+          struct.tbl_patterns = iprot.readString();
+          struct.setTbl_patternsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          {
+            org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list663.size);
+            String _elem664;
+            for (int _i665 = 0; _i665 < _list663.size; ++_i665)
+            {
+              _elem664 = iprot.readString();
+              struct.tbl_types.add(_elem664);
+            }
+          }
+          struct.setTbl_typesIsSet(true);
         }
       }
     }
 
   }
 
-  public static class get_tables_result implements org.apache.thrift.TBase<get_tables_result, get_tables_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_result");
+  public static class get_table_meta_result implements org.apache.thrift.TBase<get_table_meta_result, get_table_meta_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_table_meta_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_table_meta_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
     private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_tables_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_tables_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_table_meta_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_table_meta_resultTupleSchemeFactory());
     }
 
-    private List<String> success; // required
+    private List<TableMeta> success; // required
     private MetaException o1; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -41539,18 +42820,18 @@ public class ThriftHiveMetastore {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           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.STRING))));
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TableMeta.class))));
       tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_table_meta_result.class, metaDataMap);
     }
 
-    public get_tables_result() {
+    public get_table_meta_result() {
     }
 
-    public get_tables_result(
-      List<String> success,
+    public get_table_meta_result(
+      List<TableMeta> success,
       MetaException o1)
     {
       this();
@@ -41561,9 +42842,12 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_tables_result(get_tables_result other) {
+    public get_table_meta_result(get_table_meta_result other) {
       if (other.isSetSuccess()) {
-        List<String> __this__success = new ArrayList<String>(other.success);
+        List<TableMeta> __this__success = new ArrayList<TableMeta>(other.success.size());
+        for (TableMeta other_element : other.success) {
+          __this__success.add(new TableMeta(other_element));
+        }
         this.success = __this__success;
       }
       if (other.isSetO1()) {
@@ -41571,8 +42855,8 @@ public class ThriftHiveMetastore {
       }
     }
 
-    public get_tables_result deepCopy() {
-      return new get_tables_result(this);
+    public get_table_meta_result deepCopy() {
+      return new get_table_meta_result(this);
     }
 
     @Override
@@ -41585,22 +42869,22 @@ public class ThriftHiveMetastore {
       return (this.success == null) ? 0 : this.success.size();
     }
 
-    public java.util.Iterator<String> getSuccessIterator() {
+    public java.util.Iterator<TableMeta> getSuccessIterator() {
       return (this.success == null) ? null : this.success.iterator();
     }
 
-    public void addToSuccess(String elem) {
+    public void addToSuccess(TableMeta elem) {
       if (this.success == null) {
-        this.success = new ArrayList<String>();
+        this.success = new ArrayList<TableMeta>();
       }
       this.success.add(elem);
     }
 
-    public List<String> getSuccess() {
+    public List<TableMeta> getSuccess() {
       return this.success;
     }
 
-    public void setSuccess(List<String> success) {
+    public void setSuccess(List<TableMeta> success) {
       this.success = success;
     }
 
@@ -41648,7 +42932,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((List<String>)value);
+          setSuccess((List<TableMeta>)value);
         }
         break;
 
@@ -41694,12 +42978,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_tables_result)
-        return this.equals((get_tables_result)that);
+      if (that instanceof get_table_meta_result)
+        return this.equals((get_table_meta_result)that);
       return false;
     }
 
-    public boolean equals(get_tables_result that) {
+    public boolean equals(get_table_meta_result that) {
       if (that == null)
         return false;
 
@@ -41742,7 +43026,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_tables_result other) {
+    public int compareTo(get_table_meta_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -41786,7 +43070,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_tables_result(");
+      StringBuilder sb = new StringBuilder("get_table_meta_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -41829,15 +43113,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_tables_resultStandardSchemeFactory implements SchemeFactory {
-      public get_tables_resultStandardScheme getScheme() {
-        return new get_tables_resultStandardScheme();
+    private static class get_table_meta_resultStandardSchemeFactory implements SchemeFactory {
+      public get_table_meta_resultStandardScheme getScheme() {
+        return new get_table_meta_resultStandardScheme();
       }
     }
 
-    private static class get_tables_resultStandardScheme extends StandardScheme<get_tables_result> {
+    private static class get_table_meta_resultStandardScheme extends StandardScheme<get_table_meta_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_meta_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -41850,13 +43134,14 @@ 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 _list666 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list666.size);
+                  TableMeta _elem667;
+                  for (int _i668 = 0; _i668 < _list666.size; ++_i668)
                   {
-                    _elem651 = iprot.readString();
-                    struct.success.add(_elem651);
+                    _elem667 = new TableMeta();
+                    _elem667.read(iprot);
+                    struct.success.add(_elem667);
                   }
                   iprot.readListEnd();
                 }
@@ -41883,17 +43168,17 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_meta_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
         if (struct.success != null) {
           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)
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
+            for (TableMeta _iter669 : struct.success)
             {
-              oprot.writeString(_iter653);
+              _iter669.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -41910,16 +43195,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_tables_resultTupleSchemeFactory implements SchemeFactory {
-      public get_tables_resultTupleScheme getScheme() {
-        return new get_tables_resultTupleScheme();
+    private static class get_table_meta_resultTupleSchemeFactory implements SchemeFactory {
+      public get_table_meta_resultTupleScheme getScheme() {
+        return new get_table_meta_resultTupleScheme();
       }
     }
 
-    private static class get_tables_resultTupleScheme extends TupleScheme<get_tables_result> {
+    private static class get_table_meta_resultTupleScheme extends TupleScheme<get_table_meta_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_table_meta_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -41932,9 +43217,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter654 : struct.success)
+            for (TableMeta _iter670 : struct.success)
             {
-              oprot.writeString(_iter654);
+              _iter670.write(oprot);
             }
           }
         }
@@ -41944,18 +43229,19 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_table_meta_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         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 _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list671.size);
+            TableMeta _elem672;
+            for (int _i673 = 0; _i673 < _list671.size; ++_i673)
             {
-              _elem656 = iprot.readString();
-              struct.success.add(_elem656);
+              _elem672 = new TableMeta();
+              _elem672.read(iprot);
+              struct.success.add(_elem672);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42721,13 +44007,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TList _list674 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list674.size);
+                  String _elem675;
+                  for (int _i676 = 0; _i676 < _list674.size; ++_i676)
                   {
-                    _elem659 = iprot.readString();
-                    struct.success.add(_elem659);
+                    _elem675 = iprot.readString();
+                    struct.success.add(_elem675);
                   }
                   i

<TRUNCATED>

[28/43] hive git commit: HIVE-12289 : Make sure slf4j-log4j12 jar is not in classpath (Ashutosh Chauhan via Prasanth J)

Posted by om...@apache.org.
HIVE-12289 : Make sure slf4j-log4j12 jar is not in classpath (Ashutosh Chauhan via Prasanth J)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: 04d92dd4f4baa83bd2a6dc7435af94374a5bc91f
Parents: be870a4
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Wed Oct 28 17:23:16 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:33 2015 -0800

----------------------------------------------------------------------
 accumulo-handler/pom.xml                        | 12 ++-
 common/pom.xml                                  | 24 ++++-
 .../apache/hadoop/hive/common/JavaUtils.java    | 11 +--
 hbase-handler/pom.xml                           | 44 ++++++++-
 hcatalog/core/pom.xml                           | 64 ++++++++++++-
 hcatalog/hcatalog-pig-adapter/pom.xml           | 24 ++++-
 hcatalog/streaming/pom.xml                      | 20 ++++
 hcatalog/webhcat/svr/pom.xml                    | 12 ++-
 hwi/pom.xml                                     | 12 ++-
 itests/hcatalog-unit/pom.xml                    | 48 +++++++++-
 itests/hive-minikdc/pom.xml                     | 48 +++++++++-
 itests/hive-unit-hadoop2/pom.xml                | 42 ++++++++-
 itests/hive-unit/pom.xml                        | 96 +++++++++++++++++++-
 .../metastore/hbase/TestHBaseSchemaTool.java    |  6 +-
 .../metastore/hbase/TestHBaseSchemaTool2.java   |  4 +-
 itests/qtest-accumulo/pom.xml                   | 84 +++++++++++++++--
 itests/qtest-spark/pom.xml                      | 50 ++++++++++
 itests/qtest/pom.xml                            | 72 +++++++++++++--
 itests/util/pom.xml                             | 24 ++++-
 llap-client/pom.xml                             | 24 ++++-
 llap-server/pom.xml                             | 58 +++++++++++-
 metastore/pom.xml                               | 36 +++++++-
 .../hive/metastore/hbase/HBaseSchemaTool.java   |  4 -
 pom.xml                                         | 46 +++++++++-
 ql/pom.xml                                      | 76 ++++++++++++++--
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    | 11 +--
 serde/pom.xml                                   | 24 ++++-
 service/pom.xml                                 | 12 ++-
 shims/0.23/pom.xml                              | 48 +++++++++-
 shims/common/pom.xml                            | 11 +--
 spark-client/pom.xml                            | 12 ++-
 storage-api/pom.xml                             | 12 ++-
 32 files changed, 975 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/accumulo-handler/pom.xml
----------------------------------------------------------------------
diff --git a/accumulo-handler/pom.xml b/accumulo-handler/pom.xml
index 329bf66..759c718 100644
--- a/accumulo-handler/pom.xml
+++ b/accumulo-handler/pom.xml
@@ -91,7 +91,17 @@
       <artifactId>hadoop-client</artifactId>
       <version>${hadoop.version}</version>
       <optional>true</optional>
-    </dependency>
+        <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-common</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index f9c5629..2292fdf 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -90,13 +90,33 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <optional>true</optional>
-    </dependency>
+          <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>
-    </dependency>
+           <exclusions>
+            <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+        </exclusions>
+  </dependency>
     <!-- test inter-project -->
     <dependency>
       <groupId>com.google.code.tempus-fugit</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java b/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
index dc3a4ae..87bc99e 100644
--- a/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
@@ -27,8 +27,8 @@ import java.lang.reflect.Method;
 import java.net.URLClassLoader;
 import java.util.Arrays;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Collection of Java class loading/reflection related utilities common across
@@ -36,7 +36,7 @@ import org.apache.commons.logging.LogFactory;
  */
 public final class JavaUtils {
 
-  private static final Log LOG = LogFactory.getLog(JavaUtils.class);
+  private static final Logger LOG = LoggerFactory.getLogger(JavaUtils.class);
   private static final Method SUN_MISC_UTIL_RELEASE;
 
   static {
@@ -56,10 +56,10 @@ public final class JavaUtils {
 
   /**
    * Standard way of getting classloader in Hive code (outside of Hadoop).
-   * 
+   *
    * Uses the context loader to get access to classpaths to auxiliary and jars
    * added with 'add jar' command. Falls back to current classloader.
-   * 
+   *
    * In Hadoop-related code, we use Configuration.getClassLoader().
    */
   public static ClassLoader getClassLoader() {
@@ -133,7 +133,6 @@ public final class JavaUtils {
         newOutputStream.close();
       }
     }
-    LogFactory.release(loader);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/hbase-handler/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-handler/pom.xml b/hbase-handler/pom.xml
index 0f10580..6ff353d 100644
--- a/hbase-handler/pom.xml
+++ b/hbase-handler/pom.xml
@@ -50,6 +50,16 @@
       <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>
@@ -71,6 +81,16 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-server</artifactId>
       <version>${hbase.version}</version>
+          <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.hbase</groupId>
@@ -110,7 +130,17 @@
       <version>${hbase.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
-    </dependency>
+           <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.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
@@ -130,7 +160,17 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+         <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>junit</groupId>
       <artifactId>junit</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/hcatalog/core/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/core/pom.xml b/hcatalog/core/pom.xml
index 70297bf..63e4f30 100644
--- a/hcatalog/core/pom.xml
+++ b/hcatalog/core/pom.xml
@@ -92,11 +92,31 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
+        <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>
+          <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>
@@ -116,6 +136,16 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
+        <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>
@@ -129,20 +159,50 @@
       <artifactId>hadoop-mapreduce-client-hs</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
+         <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-jobclient</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
-    </dependency>
+          <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-jobclient</artifactId>
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+          <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-yarn-server-tests</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/hcatalog/hcatalog-pig-adapter/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/hcatalog-pig-adapter/pom.xml b/hcatalog/hcatalog-pig-adapter/pom.xml
index fa02a36..87595e9 100644
--- a/hcatalog/hcatalog-pig-adapter/pom.xml
+++ b/hcatalog/hcatalog-pig-adapter/pom.xml
@@ -116,14 +116,34 @@
       <version>${hadoop.version}</version>
       <optional>true</optional>
       <scope>test</scope>
-    </dependency>
+               <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-common</artifactId>
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+              <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>com.sun.jersey</groupId>
       <artifactId>jersey-servlet</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/hcatalog/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/pom.xml b/hcatalog/streaming/pom.xml
index 39b3abd..e8dc4d8 100644
--- a/hcatalog/streaming/pom.xml
+++ b/hcatalog/streaming/pom.xml
@@ -78,6 +78,16 @@
       <artifactId>hadoop-mapreduce-client-core</artifactId>
       <optional>true</optional>
       <version>${hadoop.version}</version>
+             <exclusions>
+            <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+        </exclusions>
     </dependency>
 
     <!-- test -->
@@ -93,6 +103,16 @@
       <artifactId>hadoop-mapreduce-client-common</artifactId>
       <scope>test</scope>
       <version>${hadoop.version}</version>
+             <exclusions>
+            <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+        </exclusions>
     </dependency>
 
   </dependencies>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/hcatalog/webhcat/svr/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/svr/pom.xml b/hcatalog/webhcat/svr/pom.xml
index 67d73ae..d7bd923 100644
--- a/hcatalog/webhcat/svr/pom.xml
+++ b/hcatalog/webhcat/svr/pom.xml
@@ -106,7 +106,17 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-auth</artifactId>
       <version>${hadoop.version}</version>
-    </dependency>
+        <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-common</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/hwi/pom.xml
----------------------------------------------------------------------
diff --git a/hwi/pom.xml b/hwi/pom.xml
index 482ea2a..8507119 100644
--- a/hwi/pom.xml
+++ b/hwi/pom.xml
@@ -72,7 +72,17 @@
       <version>${hadoop.version}</version>
       <optional>true</optional>
       <scope>test</scope>
-    </dependency>
+             <exclusions>
+            <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+        </exclusions>
+   </dependency>
     <!-- test intra-project -->
     <dependency>
       <groupId>org.apache.hive</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/itests/hcatalog-unit/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/pom.xml b/itests/hcatalog-unit/pom.xml
index ec48a4d..6fa25d5 100644
--- a/itests/hcatalog-unit/pom.xml
+++ b/itests/hcatalog-unit/pom.xml
@@ -131,14 +131,34 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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-common</artifactId>
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+        <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-hdfs</artifactId>
@@ -164,7 +184,17 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+        <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-yarn-server-tests</artifactId>
@@ -182,7 +212,17 @@
       <artifactId>hadoop-mapreduce-client-hs</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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.hbase</groupId>
       <artifactId>hbase-client</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/itests/hive-minikdc/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-minikdc/pom.xml b/itests/hive-minikdc/pom.xml
index 6a20cad..906339d 100644
--- a/itests/hive-minikdc/pom.xml
+++ b/itests/hive-minikdc/pom.xml
@@ -135,7 +135,17 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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-hdfs</artifactId>
@@ -149,7 +159,17 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+        <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>
@@ -161,7 +181,17 @@
       <artifactId>hadoop-minikdc</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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.hbase</groupId>
       <artifactId>hbase-server</artifactId>
@@ -172,7 +202,17 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-minicluster</artifactId>
       <scope>test</scope>
-    </dependency>
+        <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>com.sun.jersey</groupId>
       <artifactId>jersey-servlet</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/itests/hive-unit-hadoop2/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-unit-hadoop2/pom.xml b/itests/hive-unit-hadoop2/pom.xml
index 006db5a..cd1cdb4 100644
--- a/itests/hive-unit-hadoop2/pom.xml
+++ b/itests/hive-unit-hadoop2/pom.xml
@@ -117,13 +117,33 @@
       <version>${project.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
+        <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.hive</groupId>
       <artifactId>hive-it-unit</artifactId>
       <version>${project.version}</version>
       <scope>test</scope>
-    </dependency>
+         <exclusions>
+             <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+      </exclusions>
+   </dependency>
     <!-- hadoop-2 dependencies -->
     <dependency>
       <groupId>org.apache.hadoop</groupId>
@@ -143,6 +163,16 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
+        <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>
@@ -160,6 +190,16 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-minicluster</artifactId>
       <scope>test</scope>
+        <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>com.sun.jersey</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/itests/hive-unit/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml
index 0c61164..c202001 100644
--- a/itests/hive-unit/pom.xml
+++ b/itests/hive-unit/pom.xml
@@ -69,12 +69,32 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
-    </dependency>
+        <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.tez</groupId>
       <artifactId>tez-tests</artifactId>
       <version>${tez.version}</version>
       <type>test-jar</type>
+        <exclusions>
+             <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+      </exclusions>
     </dependency>
     <!-- dependencies are always listed in sorted order by groupId, artifectId -->
     <!-- test intra-project -->
@@ -152,7 +172,17 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+        <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>
@@ -190,6 +220,16 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-minicluster</artifactId>
       <scope>test</scope>
+        <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>com.sun.jersey</groupId>
@@ -208,13 +248,33 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+        <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-hs</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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.tez</groupId>
       <artifactId>tez-api</artifactId>
@@ -226,6 +286,16 @@
       <artifactId>tez-runtime-library</artifactId>
       <version>${tez.version}</version>
       <scope>test</scope>
+        <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.tez</groupId>
@@ -238,6 +308,16 @@
       <artifactId>tez-dag</artifactId>
       <version>${tez.version}</version>
       <scope>test</scope>
+        <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>
@@ -275,6 +355,14 @@
           <version>${spark.version}</version>
           <scope>test</scope>
           <exclusions>
+              <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
             <exclusion>
               <groupId>com.esotericsoftware.kryo</groupId>
               <artifactId>kryo</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
index d3b8615..24bebb8 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.hive.metastore.hbase;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
@@ -56,8 +54,8 @@ import java.util.Collections;
 import java.util.List;
 
 public class TestHBaseSchemaTool extends HBaseIntegrationTests {
-  private static final Log LOG = LogFactory.getLog(TestHBaseSchemaTool.class.getName());
-  private String lsep = System.getProperty("line.separator");
+
+  private final String lsep = System.getProperty("line.separator");
 
   @BeforeClass
   public static void startup() throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool2.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool2.java
index e343035..0c95b2f 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool2.java
@@ -18,8 +18,6 @@
 package org.apache.hadoop.hive.metastore.hbase;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -32,7 +30,7 @@ import java.io.PrintStream;
  * This is in a separate class because install tests shouldn't set up the metastore first.
  */
 public class TestHBaseSchemaTool2 extends HBaseIntegrationTests {
-  private static final Log LOG = LogFactory.getLog(TestHBaseSchemaTool.class.getName());
+
   private String lsep = System.getProperty("line.separator");
 
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/itests/qtest-accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest-accumulo/pom.xml b/itests/qtest-accumulo/pom.xml
index 6014639..7403a15 100644
--- a/itests/qtest-accumulo/pom.xml
+++ b/itests/qtest-accumulo/pom.xml
@@ -157,14 +157,34 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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-common</artifactId>
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+         <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-hdfs</artifactId>
@@ -184,13 +204,33 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+        <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-hs</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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>
@@ -279,19 +319,49 @@
       <artifactId>tez-runtime-library</artifactId>
       <version>${tez.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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.tez</groupId>
       <artifactId>tez-mapreduce</artifactId>
       <version>${tez.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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.tez</groupId>
       <artifactId>tez-dag</artifactId>
       <version>${tez.version}</version>
       <scope>test</scope>
-    </dependency>
+        <exclusions>
+             <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+      </exclusions>
+   </dependency>
   </dependencies>
   <profiles>
     <profile>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/itests/qtest-spark/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest-spark/pom.xml b/itests/qtest-spark/pom.xml
index a0ccf66..fe16b45 100644
--- a/itests/qtest-spark/pom.xml
+++ b/itests/qtest-spark/pom.xml
@@ -51,6 +51,16 @@
       <artifactId>spark-core_${scala.binary.version}</artifactId>
       <version>${spark.version}</version>
       <scope>test</scope>
+        <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.eclipse.jetty</groupId>
@@ -164,6 +174,16 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
+        <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>
@@ -171,6 +191,16 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
+        <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>
@@ -191,12 +221,32 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
+        <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-hs</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
+        <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>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index cfa49ba..8db6fb0 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -135,14 +135,34 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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-common</artifactId>
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+        <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-distcp</artifactId>
@@ -168,13 +188,33 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+        <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-hs</artifactId>
       <version>${hadoop.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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>
@@ -276,7 +316,17 @@
       <artifactId>tez-runtime-library</artifactId>
       <version>${tez.version}</version>
       <scope>test</scope>
-    </dependency>
+        <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.tez</groupId>
       <artifactId>tez-mapreduce</artifactId>
@@ -288,7 +338,17 @@
       <artifactId>tez-dag</artifactId>
       <version>${tez.version}</version>
       <scope>test</scope>
-    </dependency>
+        <exclusions>
+             <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+      </exclusions>
+   </dependency>
   </dependencies>
   <profiles>
     <profile>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/itests/util/pom.xml
----------------------------------------------------------------------
diff --git a/itests/util/pom.xml b/itests/util/pom.xml
index 67e8e86..5bc5865 100644
--- a/itests/util/pom.xml
+++ b/itests/util/pom.xml
@@ -37,7 +37,17 @@
     <dependency>
       <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-minicluster</artifactId>
-    </dependency>
+        <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.hive</groupId>
       <artifactId>hive-accumulo-handler</artifactId>
@@ -128,7 +138,17 @@
       <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
-    </dependency>
+        <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.hbase</groupId>
       <artifactId>hbase-server</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/llap-client/pom.xml
----------------------------------------------------------------------
diff --git a/llap-client/pom.xml b/llap-client/pom.xml
index 0cd2ec9..3c85180 100644
--- a/llap-client/pom.xml
+++ b/llap-client/pom.xml
@@ -65,7 +65,17 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <optional>true</optional>
-    </dependency>
+          <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>
@@ -103,7 +113,17 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+          <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-hdfs</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/llap-server/pom.xml
----------------------------------------------------------------------
diff --git a/llap-server/pom.xml b/llap-server/pom.xml
index a733e2c..ab4f5a4 100644
--- a/llap-server/pom.xml
+++ b/llap-server/pom.xml
@@ -90,19 +90,49 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <optional>true</optional>
-    </dependency>
+            <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>
-    </dependency>
+             <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.tez</groupId>
       <artifactId>tez-runtime-internals</artifactId>
       <version>${tez.version}</version>
       <optional>true</optional>
-    </dependency>
+            <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.tez</groupId>
       <artifactId>tez-runtime-library</artifactId>
@@ -114,6 +144,16 @@
       <artifactId>tez-mapreduce</artifactId>
       <version>${tez.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.tez</groupId>
@@ -128,7 +168,17 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+            <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-hdfs</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/metastore/pom.xml
----------------------------------------------------------------------
diff --git a/metastore/pom.xml b/metastore/pom.xml
index e3f825b..9da3bfb 100644
--- a/metastore/pom.xml
+++ b/metastore/pom.xml
@@ -59,7 +59,17 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-client</artifactId>
       <version>${hbase.version}</version>
-    </dependency>
+       <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>com.jolbox</groupId>
       <artifactId>bonecp</artifactId>
@@ -121,13 +131,33 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <optional>true</optional>
-    </dependency>
+           <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>
-    </dependency>
+            <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.thrift</groupId>
       <artifactId>libfb303</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseSchemaTool.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseSchemaTool.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseSchemaTool.java
index 567970c..b4f8734 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseSchemaTool.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseSchemaTool.java
@@ -26,8 +26,6 @@ import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 
 import java.io.PrintStream;
@@ -40,8 +38,6 @@ import java.util.List;
  */
 public class HBaseSchemaTool {
 
-  static final private Log LOG = LogFactory.getLog(HBaseReadWrite.class.getName());
-
   public static void main(String[] args) {
     Options options = new Options();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 99aeff7..4a90cef 100644
--- a/pom.xml
+++ b/pom.xml
@@ -495,7 +495,23 @@
         <artifactId>zookeeper</artifactId>
         <version>${zookeeper.version}</version>
         <exclusions>
+            <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpcore</artifactId>
+          </exclusion>
           <exclusion>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+          </exclusion>
+         <exclusion>
             <groupId>org.jboss.netty</groupId>
             <artifactId>netty</artifactId>
           </exclusion>
@@ -602,7 +618,11 @@
         <artifactId>hadoop-common</artifactId>
         <version>${hadoop.version}</version>
         <exclusions>
-           <exclusion>
+            <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
             <groupId>commmons-logging</groupId>
             <artifactId>commons-logging</artifactId>
           </exclusion>
@@ -625,12 +645,32 @@
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
         <version>${hadoop.version}</version>
-      </dependency>
+          <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>
-      </dependency>
+         <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-minikdc</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index 005c232..9420a62 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -90,7 +90,17 @@
       <groupId>commons-httpclient</groupId>
       <artifactId>commons-httpclient</artifactId>
       <version>${commons-httpclient.version}</version>
-    </dependency>
+       <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>commons-io</groupId>
       <artifactId>commons-io</artifactId>
@@ -168,7 +178,15 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <exclusions>
-        <exclusion>
+             <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+       <exclusion>
           <groupId>javax.servlet</groupId>
           <artifactId>servlet-api</artifactId>
         </exclusion>
@@ -193,7 +211,17 @@
       <version>${hadoop.version}</version>
       <optional>true</optional>
       <scope>test</scope>
-    </dependency>
+        <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-hdfs</artifactId>
@@ -471,7 +499,15 @@
          <groupId>org.apache.hadoop</groupId>
          <artifactId>hadoop-yarn-client</artifactId>
        </exclusion>
-      </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.tez</groupId>
@@ -479,7 +515,15 @@
       <version>${tez.version}</version>
       <optional>true</optional>
       <exclusions>
-        <exclusion>
+              <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+       <exclusion>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
         </exclusion>
@@ -503,7 +547,15 @@
          <groupId>org.apache.hadoop</groupId>
          <artifactId>hadoop-yarn-client</artifactId>
        </exclusion>
-      </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.tez</groupId>
@@ -542,7 +594,17 @@
       <artifactId>spark-core_${scala.binary.version}</artifactId>
       <version>${spark.version}</version>
       <optional>true</optional>
-    </dependency>
+        <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>com.sun.jersey</groupId>
       <artifactId>jersey-servlet</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
index 44dfe3e..5f35630 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
@@ -30,8 +30,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.MapRedStats;
@@ -61,10 +59,11 @@ import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.Logger;
 import org.apache.logging.log4j.core.appender.FileAppender;
 import org.apache.logging.log4j.core.appender.RollingFileAppender;
+import org.slf4j.LoggerFactory;
 
 public class HadoopJobExecHelper {
 
-  static final private Log LOG = LogFactory.getLog(HadoopJobExecHelper.class.getName());
+  static final private org.slf4j.Logger LOG = LoggerFactory.getLogger(HadoopJobExecHelper.class.getName());
 
   protected transient JobConf job;
   protected Task<? extends Serializable> task;
@@ -76,8 +75,8 @@ public class HadoopJobExecHelper {
   protected transient int lastReduceProgress;
 
   public transient JobID jobId;
-  private LogHelper console;
-  private HadoopJobExecHook callBackObj;
+  private final LogHelper console;
+  private final HadoopJobExecHook callBackObj;
 
   /**
    * Update counters relevant to this task.
@@ -186,7 +185,7 @@ public class HadoopJobExecHelper {
           System.err.println("killing job with: " + rj.getID());
           rj.killJob();
         } catch (Exception e) {
-          LOG.warn(e);
+          LOG.warn("Failed to kill job", e);
           System.err.println("Failed to kill job: "+ rj.getID());
           // do nothing
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/serde/pom.xml
----------------------------------------------------------------------
diff --git a/serde/pom.xml b/serde/pom.xml
index 772ce71..4e854c2 100644
--- a/serde/pom.xml
+++ b/serde/pom.xml
@@ -85,7 +85,17 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <optional>true</optional>
-    </dependency>
+          <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>
@@ -124,7 +134,17 @@
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
-    </dependency>
+           <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-hdfs</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/service/pom.xml
----------------------------------------------------------------------
diff --git a/service/pom.xml b/service/pom.xml
index 7095448..22234d9 100644
--- a/service/pom.xml
+++ b/service/pom.xml
@@ -59,7 +59,17 @@
       <groupId>net.sf.jpam</groupId>
       <artifactId>jpam</artifactId>
       <version>${jpam.version}</version>
-    </dependency>
+            <exclusions>
+            <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+        </exclusions>
+   </dependency>
     <!-- used by thrift generated code -->
     <dependency>
       <groupId>commons-lang</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/shims/0.23/pom.xml
----------------------------------------------------------------------
diff --git a/shims/0.23/pom.xml b/shims/0.23/pom.xml
index 17efde8..9009762 100644
--- a/shims/0.23/pom.xml
+++ b/shims/0.23/pom.xml
@@ -51,7 +51,17 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <optional>true</optional>
-    </dependency>
+         <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-hdfs</artifactId>
@@ -70,14 +80,34 @@
       <artifactId>hadoop-mapreduce-client-core</artifactId>
       <version>${hadoop.version}</version>
       <optional>true</optional>
-    </dependency>
+          <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-jobclient</artifactId>
       <version>${hadoop.version}</version>
       <type>test-jar</type>
       <optional>true</optional>
-    </dependency>
+           <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.tez</groupId>
       <artifactId>tez-api</artifactId>
@@ -89,7 +119,17 @@
       <artifactId>tez-runtime-library</artifactId>
       <version>${tez.version}</version>
       <optional>true</optional>
-    </dependency>
+            <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.tez</groupId>
       <artifactId>tez-mapreduce</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/shims/common/pom.xml
----------------------------------------------------------------------
diff --git a/shims/common/pom.xml b/shims/common/pom.xml
index 001c96b..ecfc294 100644
--- a/shims/common/pom.xml
+++ b/shims/common/pom.xml
@@ -37,11 +37,6 @@
     <!-- inter-project -->
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-1.2-api</artifactId>
-      <version>${log4j2.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-slf4j-impl</artifactId>
       <version>${log4j2.version}</version>
     </dependency>
@@ -56,7 +51,11 @@
       <version>${hadoop.version}</version>
       <optional>true</optional>
          <exclusions>
-           <exclusion>
+              <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
             <groupId>commmons-logging</groupId>
             <artifactId>commons-logging</artifactId>
           </exclusion>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/spark-client/pom.xml
----------------------------------------------------------------------
diff --git a/spark-client/pom.xml b/spark-client/pom.xml
index 1d05720..a0bbe56 100644
--- a/spark-client/pom.xml
+++ b/spark-client/pom.xml
@@ -59,7 +59,17 @@
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-core_${scala.binary.version}</artifactId>
       <version>${spark.version}</version>
-    </dependency>
+        <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>junit</groupId>
       <artifactId>junit</artifactId>

http://git-wip-us.apache.org/repos/asf/hive/blob/04d92dd4/storage-api/pom.xml
----------------------------------------------------------------------
diff --git a/storage-api/pom.xml b/storage-api/pom.xml
index 0af0d27..ae9da60 100644
--- a/storage-api/pom.xml
+++ b/storage-api/pom.xml
@@ -38,7 +38,17 @@
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <optional>true</optional>
-    </dependency>
+         <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>junit</groupId>
       <artifactId>junit</artifactId>


[05/43] hive git commit: HIVE-12364 : Distcp job fails when run under Tez (Prasanth J via Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-12364 : Distcp job fails when run under Tez (Prasanth J via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: 3915a33482f074ef5770c65a7ff7095a44ae174a
Parents: 9b263ae
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Mon Nov 9 17:59:37 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:30 2015 -0800

----------------------------------------------------------------------
 itests/qtest/pom.xml                                  |  6 ++++++
 .../src/test/resources/testconfiguration.properties   |  1 +
 .../test/queries/clientpositive/insert_dir_distcp.q   |  9 +++++++++
 .../results/clientpositive/insert_dir_distcp.q.out    | 14 ++++++++++++++
 .../clientpositive/tez/insert_dir_distcp.q.out        | 14 ++++++++++++++
 .../org/apache/hadoop/hive/shims/Hadoop23Shims.java   |  3 +++
 6 files changed, 47 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3915a334/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index 65c3c75..cfa49ba 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -145,6 +145,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>test</scope>
+    </dependency>
+      <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
       <version>${hadoop.version}</version>
       <classifier>tests</classifier>

http://git-wip-us.apache.org/repos/asf/hive/blob/3915a334/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index d16c318..70f96da 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -26,6 +26,7 @@ minimr.query.files=auto_sortmerge_join_16.q,\
   infer_bucket_sort_num_buckets.q,\
   infer_bucket_sort_reducers_power_two.q,\
   input16_cc.q,\
+  insert_dir_distcp.q,\
   join1.q,\
   leftsemijoin_mr.q,\
   list_bucket_dml_10.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/3915a334/ql/src/test/queries/clientpositive/insert_dir_distcp.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/insert_dir_distcp.q b/ql/src/test/queries/clientpositive/insert_dir_distcp.q
new file mode 100644
index 0000000..6582938
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/insert_dir_distcp.q
@@ -0,0 +1,9 @@
+set hive.exec.copyfile.maxsize=400;
+
+set tez.am.log.level=INFO;
+set tez.task.log.level=INFO;
+-- see TEZ-2931 for using INFO logging
+
+insert overwrite directory '/tmp/src' select * from src;
+
+dfs -ls ${hiveconf:hive.metastore.warehouse.dir}/src/;

http://git-wip-us.apache.org/repos/asf/hive/blob/3915a334/ql/src/test/results/clientpositive/insert_dir_distcp.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/insert_dir_distcp.q.out b/ql/src/test/results/clientpositive/insert_dir_distcp.q.out
new file mode 100644
index 0000000..b70fa01
--- /dev/null
+++ b/ql/src/test/results/clientpositive/insert_dir_distcp.q.out
@@ -0,0 +1,14 @@
+PREHOOK: query: -- see TEZ-2931 for using INFO logging
+
+#### A masked pattern was here ####
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: -- see TEZ-2931 for using INFO logging
+
+#### A masked pattern was here ####
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+Found 1 items
+#### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/3915a334/ql/src/test/results/clientpositive/tez/insert_dir_distcp.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/insert_dir_distcp.q.out b/ql/src/test/results/clientpositive/tez/insert_dir_distcp.q.out
new file mode 100644
index 0000000..b70fa01
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/insert_dir_distcp.q.out
@@ -0,0 +1,14 @@
+PREHOOK: query: -- see TEZ-2931 for using INFO logging
+
+#### A masked pattern was here ####
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: -- see TEZ-2931 for using INFO logging
+
+#### A masked pattern was here ####
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+Found 1 items
+#### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/3915a334/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
----------------------------------------------------------------------
diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
index 36282a5..4da98e4 100644
--- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
+++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
@@ -1300,11 +1300,14 @@ public class Hadoop23Shims extends HadoopShimsSecure {
     options.setSkipCRC(true);
     options.preserve(FileAttribute.BLOCKSIZE);
     try {
+      conf.setBoolean("mapred.mapper.new-api", true);
       DistCp distcp = new DistCp(conf, options);
       distcp.execute();
       return true;
     } catch (Exception e) {
       throw new IOException("Cannot execute DistCp process: " + e, e);
+    } finally {
+      conf.setBoolean("mapred.mapper.new-api", false);
     }
   }
 


[42/43] hive git commit: HIVE-12196 NPE when converting bad timestamp value (Aihua Xu, reviewed by Chaoyu Tang)

Posted by om...@apache.org.
HIVE-12196 NPE when converting bad timestamp value (Aihua Xu, reviewed by Chaoyu Tang)


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

Branch: refs/heads/master-fixed
Commit: cf1d1604b902a925f0094fc04c16d37c89e276ba
Parents: a3c02d0
Author: aihuaxu <ai...@apache.org>
Authored: Tue Nov 3 13:49:20 2015 -0500
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:35 2015 -0800

----------------------------------------------------------------------
 .../ql/udf/generic/GenericUDFFromUtcTimestamp.java     | 11 +++++++----
 .../queries/clientpositive/udf_from_utc_timestamp.q    |  5 +++++
 .../clientpositive/udf_from_utc_timestamp.q.out        | 13 +++++++++++++
 3 files changed, 25 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cf1d1604/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFFromUtcTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFFromUtcTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFFromUtcTimestamp.java
index 30ef5ab..331ee6b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFFromUtcTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFFromUtcTimestamp.java
@@ -77,12 +77,15 @@ public class GenericUDFFromUtcTimestamp extends GenericUDF {
       return null;
     }
 
-    String tzStr = textConverter.convert(o1).toString();
-    TimeZone timezone = TimeZone.getTimeZone(tzStr);
+    Object converted_o0 = timestampConverter.convert(o0);
+    if (converted_o0 == null) {
+      return null;
+    }
 
-    Timestamp timestamp = ((TimestampWritable) timestampConverter.convert(o0))
-        .getTimestamp();
+    Timestamp timestamp = ((TimestampWritable) converted_o0).getTimestamp();
 
+    String tzStr = textConverter.convert(o1).toString();
+    TimeZone timezone = TimeZone.getTimeZone(tzStr);
     int offset = timezone.getOffset(timestamp.getTime());
     if (invert()) {
       offset = -offset;

http://git-wip-us.apache.org/repos/asf/hive/blob/cf1d1604/ql/src/test/queries/clientpositive/udf_from_utc_timestamp.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_from_utc_timestamp.q b/ql/src/test/queries/clientpositive/udf_from_utc_timestamp.q
index b113eec..ca0a6a8 100644
--- a/ql/src/test/queries/clientpositive/udf_from_utc_timestamp.q
+++ b/ql/src/test/queries/clientpositive/udf_from_utc_timestamp.q
@@ -22,3 +22,8 @@ from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), ''),
 from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), '---'),
 from_utc_timestamp(cast(null as timestamp), 'PST'),
 from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), cast(null as string));
+
+select
+from_utc_timestamp('2012-02-11-04:30:00', 'UTC'),
+from_utc_timestamp('2012-02-11-04:30:00', 'PST');
+

http://git-wip-us.apache.org/repos/asf/hive/blob/cf1d1604/ql/src/test/results/clientpositive/udf_from_utc_timestamp.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_from_utc_timestamp.q.out b/ql/src/test/results/clientpositive/udf_from_utc_timestamp.q.out
index 5b4bc35..be2cb1a 100644
--- a/ql/src/test/results/clientpositive/udf_from_utc_timestamp.q.out
+++ b/ql/src/test/results/clientpositive/udf_from_utc_timestamp.q.out
@@ -80,3 +80,16 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
 2012-02-10 20:30:00	2012-02-11 08:30:00	2012-02-11 12:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	NULL	NULL
+PREHOOK: query: select
+from_utc_timestamp('2012-02-11-04:30:00', 'UTC'),
+from_utc_timestamp('2012-02-11-04:30:00', 'PST')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+from_utc_timestamp('2012-02-11-04:30:00', 'UTC'),
+from_utc_timestamp('2012-02-11-04:30:00', 'PST')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+NULL	NULL


[32/43] hive git commit: HIVE-12080: Support auto type widening (int->bigint & float->double) for Parquet table (Mohammad Kamrul Islam, reviewed by Sergio Pena, Ryan Blue)

Posted by om...@apache.org.
HIVE-12080: Support auto type widening (int->bigint & float->double) for Parquet table (Mohammad Kamrul Islam, reviewed by Sergio Pena, Ryan Blue)

Change-Id: I4e78cb2df41953f3192a47abc7e04e1762d2041d


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

Branch: refs/heads/master-fixed
Commit: f4809406fb1fa8ce5878966e73aa3e16b4e319bd
Parents: 6949855
Author: Sergio Pena <se...@cloudera.com>
Authored: Thu Nov 12 10:38:33 2015 -0600
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:33 2015 -0800

----------------------------------------------------------------------
 data/files/parquet_type_promotion.txt           |   4 +
 .../convert/DataWritableRecordConverter.java    |   6 +-
 .../ql/io/parquet/convert/ETypeConverter.java   |  78 ++++---
 .../convert/HiveCollectionConverter.java        |  38 +--
 .../io/parquet/convert/HiveGroupConverter.java  |  21 +-
 .../io/parquet/convert/HiveStructConverter.java |  86 +++++--
 .../hive/ql/io/parquet/convert/Repeated.java    |   9 +-
 .../parquet/read/DataWritableReadSupport.java   |  11 +-
 .../queries/clientpositive/parquet_columnar.q   |   4 +
 .../clientpositive/parquet_type_promotion.q     |  72 ++++++
 .../clientpositive/parquet_columnar.q.out       |  37 +++
 .../clientpositive/parquet_type_promotion.q.out | 232 +++++++++++++++++++
 12 files changed, 522 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/data/files/parquet_type_promotion.txt
----------------------------------------------------------------------
diff --git a/data/files/parquet_type_promotion.txt b/data/files/parquet_type_promotion.txt
new file mode 100644
index 0000000..dc3e13e
--- /dev/null
+++ b/data/files/parquet_type_promotion.txt
@@ -0,0 +1,4 @@
+100|5643|0.3|0.7|k1:11|7,17,22|10,20|k11:4.0|2.3,3.0,5.5|5.7,4.8
+200|5643|0.4|0.8|k2:14|8,17,24|20,20|v11:5.0|3.3,3.1,5.6|5.8,4.7
+300|7643|0.4|0.9|k3:12|9,17,25|30,60|b11:6.0|4.3,3.2,5.7|5.9,4.6
+400|8643|0.4|0.5|k4:15|7,18,27|50,70|d11:8.0|6.3,3.3,5.8|5.0,4.5

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableRecordConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableRecordConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableRecordConverter.java
index 3261e4b..1bbdcec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableRecordConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/DataWritableRecordConverter.java
@@ -14,6 +14,7 @@
 package org.apache.hadoop.hive.ql.io.parquet.convert;
 
 import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.parquet.io.api.GroupConverter;
 import org.apache.parquet.io.api.RecordMaterializer;
@@ -31,9 +32,10 @@ public class DataWritableRecordConverter extends RecordMaterializer<ArrayWritabl
 
   private final HiveStructConverter root;
 
-  public DataWritableRecordConverter(final GroupType requestedSchema, final Map<String, String> metadata) {
+  public DataWritableRecordConverter(final GroupType requestedSchema, final Map<String, String> metadata, TypeInfo hiveTypeInfo) {
     this.root = new HiveStructConverter(requestedSchema,
-      MessageTypeParser.parseMessageType(metadata.get(DataWritableReadSupport.HIVE_TABLE_AS_PARQUET_SCHEMA)), metadata);
+      MessageTypeParser.parseMessageType(metadata.get(DataWritableReadSupport.HIVE_TABLE_AS_PARQUET_SCHEMA)),
+        metadata, hiveTypeInfo);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java
index 17849fa..ec0dd81 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java
@@ -25,6 +25,8 @@ import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.FloatWritable;
@@ -49,7 +51,7 @@ public enum ETypeConverter {
   EDOUBLE_CONVERTER(Double.TYPE) {
     @Override
 
-    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
       return new PrimitiveConverter() {
         @Override
         public void addDouble(final double value) {
@@ -60,7 +62,7 @@ public enum ETypeConverter {
   },
   EBOOLEAN_CONVERTER(Boolean.TYPE) {
     @Override
-    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
       return new PrimitiveConverter() {
         @Override
         public void addBoolean(final boolean value) {
@@ -71,29 +73,47 @@ public enum ETypeConverter {
   },
   EFLOAT_CONVERTER(Float.TYPE) {
     @Override
-    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
-      return new PrimitiveConverter() {
-        @Override
-        public void addFloat(final float value) {
-          parent.set(index, new FloatWritable(value));
-        }
-      };
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
+      if (hiveTypeInfo != null && hiveTypeInfo.equals(TypeInfoFactory.doubleTypeInfo)) {
+        return new PrimitiveConverter() {
+          @Override
+          public void addFloat(final float value) {
+            parent.set(index, new DoubleWritable((double) value));
+          }
+        };
+      } else {
+        return new PrimitiveConverter() {
+          @Override
+          public void addFloat(final float value) {
+            parent.set(index, new FloatWritable(value));
+          }
+        };
+      }
     }
   },
   EINT32_CONVERTER(Integer.TYPE) {
     @Override
-    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
-      return new PrimitiveConverter() {
-        @Override
-        public void addInt(final int value) {
-          parent.set(index, new IntWritable(value));
-        }
-      };
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
+      if (hiveTypeInfo != null && hiveTypeInfo.equals(TypeInfoFactory.longTypeInfo)) {
+        return new PrimitiveConverter() {
+          @Override
+          public void addInt(final int value) {
+            parent.set(index, new LongWritable((long)value));
+          }
+        };
+      } else {
+        return new PrimitiveConverter() {
+          @Override
+          public void addInt(final int value) {
+            parent.set(index, new IntWritable(value));
+          }
+        };
+      }
     }
   },
   EINT64_CONVERTER(Long.TYPE) {
     @Override
-    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
       return new PrimitiveConverter() {
         @Override
         public void addLong(final long value) {
@@ -104,7 +124,7 @@ public enum ETypeConverter {
   },
   EBINARY_CONVERTER(Binary.class) {
     @Override
-    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
       return new BinaryConverter<BytesWritable>(type, parent, index) {
         @Override
         protected BytesWritable convert(Binary binary) {
@@ -115,7 +135,7 @@ public enum ETypeConverter {
   },
   ESTRING_CONVERTER(String.class) {
     @Override
-    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
       return new BinaryConverter<Text>(type, parent, index) {
         @Override
         protected Text convert(Binary binary) {
@@ -126,7 +146,7 @@ public enum ETypeConverter {
   },
   EDECIMAL_CONVERTER(BigDecimal.class) {
     @Override
-    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
       return new BinaryConverter<HiveDecimalWritable>(type, parent, index) {
         @Override
         protected HiveDecimalWritable convert(Binary binary) {
@@ -137,7 +157,7 @@ public enum ETypeConverter {
   },
   ETIMESTAMP_CONVERTER(TimestampWritable.class) {
     @Override
-    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
       return new BinaryConverter<TimestampWritable>(type, parent, index) {
         @Override
         protected TimestampWritable convert(Binary binary) {
@@ -154,7 +174,7 @@ public enum ETypeConverter {
   },
   EDATE_CONVERTER(DateWritable.class) {
     @Override
-    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent) {
+    PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
       return new PrimitiveConverter() {
         @Override
         public void addInt(final int value) {
@@ -174,26 +194,26 @@ public enum ETypeConverter {
     return _type;
   }
 
-  abstract PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent);
+  abstract PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo);
 
   public static PrimitiveConverter getNewConverter(final PrimitiveType type, final int index,
-      final ConverterParent parent) {
+                                                   final ConverterParent parent, TypeInfo hiveTypeInfo) {
     if (type.isPrimitive() && (type.asPrimitiveType().getPrimitiveTypeName().equals(PrimitiveType.PrimitiveTypeName.INT96))) {
       //TODO- cleanup once parquet support Timestamp type annotation.
-      return ETypeConverter.ETIMESTAMP_CONVERTER.getConverter(type, index, parent);
+      return ETypeConverter.ETIMESTAMP_CONVERTER.getConverter(type, index, parent, hiveTypeInfo);
     }
     if (OriginalType.DECIMAL == type.getOriginalType()) {
-      return EDECIMAL_CONVERTER.getConverter(type, index, parent);
+      return EDECIMAL_CONVERTER.getConverter(type, index, parent, hiveTypeInfo);
     } else if (OriginalType.UTF8 == type.getOriginalType()) {
-      return ESTRING_CONVERTER.getConverter(type, index, parent);
+      return ESTRING_CONVERTER.getConverter(type, index, parent, hiveTypeInfo);
     } else if (OriginalType.DATE == type.getOriginalType()) {
-      return EDATE_CONVERTER.getConverter(type, index, parent);
+      return EDATE_CONVERTER.getConverter(type, index, parent, hiveTypeInfo);
     }
 
     Class<?> javaType = type.getPrimitiveTypeName().javaType;
     for (final ETypeConverter eConverter : values()) {
       if (eConverter.getType() == javaType) {
-        return eConverter.getConverter(type, index, parent);
+        return eConverter.getConverter(type, index, parent, hiveTypeInfo);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveCollectionConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveCollectionConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveCollectionConverter.java
index 06f3d32..5fb58c9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveCollectionConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveCollectionConverter.java
@@ -22,6 +22,9 @@ import com.google.common.base.Preconditions;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.parquet.io.api.Converter;
@@ -37,21 +40,21 @@ public class HiveCollectionConverter extends HiveGroupConverter {
 
   public static HiveGroupConverter forMap(GroupType mapType,
                                           ConverterParent parent,
-                                          int index) {
+                                          int index, TypeInfo hiveTypeInfo) {
     return new HiveCollectionConverter(
-        mapType, parent, index, true /* its a map */ );
+        mapType, parent, index, true /* its a map */, hiveTypeInfo );
   }
 
   public static HiveGroupConverter forList(GroupType listType,
                                            ConverterParent parent,
-                                           int index) {
+                                           int index, TypeInfo hiveTypeInfo) {
     return new HiveCollectionConverter(
-      listType, parent, index, false /* not a map */);
+      listType, parent, index, false /* nUnknown hive type infoot a map */, hiveTypeInfo);
   }
 
   private HiveCollectionConverter(GroupType collectionType,
                                   ConverterParent parent,
-                                  int index, boolean isMap) {
+                                  int index, boolean isMap, TypeInfo hiveTypeInfo) {
     setMetadata(parent.getMetadata());
     this.collectionType = collectionType;
     this.parent = parent;
@@ -59,12 +62,21 @@ public class HiveCollectionConverter extends HiveGroupConverter {
     Type repeatedType = collectionType.getType(0);
     if (isMap) {
       this.innerConverter = new KeyValueConverter(
-          repeatedType.asGroupType(), this);
+          repeatedType.asGroupType(), this, hiveTypeInfo);
     } else if (isElementType(repeatedType, collectionType.getName())) {
-      this.innerConverter = getConverterFromDescription(repeatedType, 0, this);
+      this.innerConverter = getConverterFromDescription(repeatedType, 0, this, extractListCompatibleType(hiveTypeInfo));
     } else {
       this.innerConverter = new ElementConverter(
-          repeatedType.asGroupType(), this);
+          repeatedType.asGroupType(), this,  extractListCompatibleType(hiveTypeInfo));
+    }
+  }
+
+  private TypeInfo extractListCompatibleType(TypeInfo hiveTypeInfo) {
+    if (hiveTypeInfo !=  null && hiveTypeInfo instanceof ListTypeInfo) {
+      return ((ListTypeInfo) hiveTypeInfo).getListElementTypeInfo();
+    } else {
+      return hiveTypeInfo; //to handle map can read list of struct data (i.e. list<struct<key, value>> --> map<key,
+      // value>)
     }
   }
 
@@ -97,13 +109,13 @@ public class HiveCollectionConverter extends HiveGroupConverter {
     private final Converter valueConverter;
     private Writable[] keyValue = null;
 
-    public KeyValueConverter(GroupType keyValueType, HiveGroupConverter parent) {
+    public KeyValueConverter(GroupType keyValueType, HiveGroupConverter parent, TypeInfo hiveTypeInfo) {
       setMetadata(parent.getMetadata());
       this.parent = parent;
       this.keyConverter = getConverterFromDescription(
-          keyValueType.getType(0), 0, this);
+          keyValueType.getType(0), 0, this, hiveTypeInfo == null ? null : ((MapTypeInfo) hiveTypeInfo).getMapKeyTypeInfo());
       this.valueConverter = getConverterFromDescription(
-          keyValueType.getType(1), 1, this);
+          keyValueType.getType(1), 1, this, hiveTypeInfo == null ? null : ((MapTypeInfo) hiveTypeInfo).getMapValueTypeInfo());
     }
 
     @Override
@@ -140,11 +152,11 @@ public class HiveCollectionConverter extends HiveGroupConverter {
     private final Converter elementConverter;
     private Writable element = null;
 
-    public ElementConverter(GroupType repeatedType, HiveGroupConverter parent) {
+    public ElementConverter(GroupType repeatedType, HiveGroupConverter parent, TypeInfo hiveTypeInfo) {
       setMetadata(parent.getMetadata());
       this.parent = parent;
       this.elementConverter = getConverterFromDescription(
-          repeatedType.getType(0), 0, this);
+          repeatedType.getType(0), 0, this, hiveTypeInfo);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveGroupConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveGroupConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveGroupConverter.java
index b1ca85a..e732f4a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveGroupConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveGroupConverter.java
@@ -13,6 +13,7 @@
  */
 package org.apache.hadoop.hive.ql.io.parquet.convert;
 
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.io.Writable;
 import org.apache.parquet.io.api.Converter;
 import org.apache.parquet.io.api.GroupConverter;
@@ -36,39 +37,41 @@ public abstract class HiveGroupConverter extends GroupConverter implements Conve
     return metadata;
   }
 
-  protected static PrimitiveConverter getConverterFromDescription(PrimitiveType type, int index, ConverterParent parent) {
+  protected static PrimitiveConverter getConverterFromDescription(PrimitiveType type, int index, ConverterParent
+      parent, TypeInfo hiveTypeInfo) {
     if (type == null) {
       return null;
     }
 
-    return ETypeConverter.getNewConverter(type, index, parent);
+    return ETypeConverter.getNewConverter(type, index, parent, hiveTypeInfo);
   }
 
-  protected static HiveGroupConverter getConverterFromDescription(GroupType type, int index, ConverterParent parent) {
+  protected static HiveGroupConverter getConverterFromDescription(GroupType type, int index, ConverterParent parent,
+                                                                  TypeInfo hiveTypeInfo) {
     if (type == null) {
       return null;
     }
 
     OriginalType annotation = type.getOriginalType();
     if (annotation == OriginalType.LIST) {
-      return HiveCollectionConverter.forList(type, parent, index);
+      return HiveCollectionConverter.forList(type, parent, index, hiveTypeInfo);
     } else if (annotation == OriginalType.MAP || annotation == OriginalType.MAP_KEY_VALUE) {
-      return HiveCollectionConverter.forMap(type, parent, index);
+      return HiveCollectionConverter.forMap(type, parent, index, hiveTypeInfo);
     }
 
-    return new HiveStructConverter(type, parent, index);
+    return new HiveStructConverter(type, parent, index, hiveTypeInfo);
   }
 
-  protected static Converter getConverterFromDescription(Type type, int index, ConverterParent parent) {
+  protected static Converter getConverterFromDescription(Type type, int index, ConverterParent parent, TypeInfo hiveTypeInfo) {
     if (type == null) {
       return null;
     }
 
     if (type.isPrimitive()) {
-      return getConverterFromDescription(type.asPrimitiveType(), index, parent);
+      return getConverterFromDescription(type.asPrimitiveType(), index, parent, hiveTypeInfo);
     }
 
-    return getConverterFromDescription(type.asGroupType(), index, parent);
+    return getConverterFromDescription(type.asGroupType(), index, parent, hiveTypeInfo);
   }
 
   public abstract void set(int index, Writable value);

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java
index 9c35a9f..1b43dd9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java
@@ -17,6 +17,9 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.*;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.parquet.io.api.Converter;
@@ -31,44 +34,61 @@ import org.apache.parquet.schema.Type;
 public class HiveStructConverter extends HiveGroupConverter {
 
   private final int totalFieldCount;
-  private final Converter[] converters;
+  private Converter[] converters;
   private final ConverterParent parent;
   private final int index;
   private Writable[] writables;
-  private final List<Repeated> repeatedConverters;
+  private List<Repeated> repeatedConverters;
   private boolean reuseWritableArray = false;
+  private List<String> hiveFieldNames;
+  private List<TypeInfo> hiveFieldTypeInfos;
 
-  public HiveStructConverter(final GroupType requestedSchema, final GroupType tableSchema, Map<String, String> metadata) {
-    this(requestedSchema, null, 0, tableSchema);
+  public HiveStructConverter(final GroupType requestedSchema, final GroupType tableSchema,
+                             Map<String, String> metadata, TypeInfo hiveTypeInfo) {
     setMetadata(metadata);
     this.reuseWritableArray = true;
     this.writables = new Writable[tableSchema.getFieldCount()];
+    this.parent = null;
+    this.index = 0;
+    this.totalFieldCount = tableSchema.getFieldCount();
+    init(requestedSchema, null, 0, tableSchema, hiveTypeInfo);
   }
 
   public HiveStructConverter(final GroupType groupType, final ConverterParent parent,
-                             final int index) {
-    this(groupType, parent, index, groupType);
+                             final int index, TypeInfo hiveTypeInfo) {
+    this(groupType, parent, index, groupType, hiveTypeInfo);
   }
 
   public HiveStructConverter(final GroupType selectedGroupType,
-                             final ConverterParent parent, final int index, final GroupType containingGroupType) {
-    if (parent != null) {
-      setMetadata(parent.getMetadata());
-    }
+                             final ConverterParent parent, final int index, final GroupType containingGroupType, TypeInfo hiveTypeInfo) {
     this.parent = parent;
     this.index = index;
     this.totalFieldCount = containingGroupType.getFieldCount();
+    init(selectedGroupType, parent, index, containingGroupType, hiveTypeInfo);
+  }
+
+  private void init(final GroupType selectedGroupType,
+                    final ConverterParent parent, final int index, final GroupType containingGroupType, TypeInfo hiveTypeInfo) {
+    if (parent != null) {
+      setMetadata(parent.getMetadata());
+    }
     final int selectedFieldCount = selectedGroupType.getFieldCount();
 
     converters = new Converter[selectedFieldCount];
     this.repeatedConverters = new ArrayList<Repeated>();
 
+    if (hiveTypeInfo != null && hiveTypeInfo.getCategory().equals(ObjectInspector.Category.STRUCT)) {
+      this.hiveFieldNames = ((StructTypeInfo) hiveTypeInfo).getAllStructFieldNames();
+      this.hiveFieldTypeInfos = ((StructTypeInfo) hiveTypeInfo).getAllStructFieldTypeInfos();
+    }
+
     List<Type> selectedFields = selectedGroupType.getFields();
     for (int i = 0; i < selectedFieldCount; i++) {
       Type subtype = selectedFields.get(i);
       if (containingGroupType.getFields().contains(subtype)) {
         int fieldIndex = containingGroupType.getFieldIndex(subtype.getName());
-        converters[i] = getFieldConverter(subtype, fieldIndex);
+        TypeInfo _hiveTypeInfo = getFieldTypeIgnoreCase(hiveTypeInfo, subtype.getName(), fieldIndex);
+        converters[i] = getFieldConverter(subtype, fieldIndex, _hiveTypeInfo);
       } else {
         throw new IllegalStateException("Group type [" + containingGroupType +
             "] does not contain requested field: " + subtype);
@@ -76,20 +96,56 @@ public class HiveStructConverter extends HiveGroupConverter {
     }
   }
 
-  private Converter getFieldConverter(Type type, int fieldIndex) {
+  private TypeInfo getFieldTypeIgnoreCase(TypeInfo hiveTypeInfo, String fieldName, int fieldIndex) {
+    if (hiveTypeInfo == null) {
+      return null;
+    } else if (hiveTypeInfo.getCategory().equals(ObjectInspector.Category.STRUCT)) {
+      return getStructFieldTypeInfo(fieldName, fieldIndex);
+    } else if (hiveTypeInfo.getCategory().equals(ObjectInspector.Category.MAP)) {
+      //This cover the case where hive table may have map<key, value> but the data file is
+      // of type array<struct<value1, value2>>
+      //Using index in place of type name.
+      if (fieldIndex == 0) {
+        return ((MapTypeInfo) hiveTypeInfo).getMapKeyTypeInfo();
+      } else if (fieldIndex == 1) {
+        return ((MapTypeInfo) hiveTypeInfo).getMapValueTypeInfo();
+      } else {//Other fields are skipped for this case
+        return null;
+      }
+    }
+    throw new RuntimeException("Unknown hive type info " + hiveTypeInfo + " when searching for field " + fieldName);
+  }
+
+  private TypeInfo getStructFieldTypeInfo(String field, int fieldIndex) {
+    String fieldLowerCase = field.toLowerCase();
+    if (Boolean.valueOf(getMetadata().get(DataWritableReadSupport.PARQUET_COLUMN_INDEX_ACCESS))
+        && fieldIndex < hiveFieldNames.size()) {
+      return hiveFieldTypeInfos.get(fieldIndex);
+    }
+    for (int i = 0; i < hiveFieldNames.size(); i++) {
+      if (fieldLowerCase.equalsIgnoreCase(hiveFieldNames.get(i))) {
+        return hiveFieldTypeInfos.get(i);
+      }
+    }
+    throw new RuntimeException("cannot find field " + field
+        + " in " + hiveFieldNames);
+  }
+
+  private Converter getFieldConverter(Type type, int fieldIndex, TypeInfo hiveTypeInfo) {
     Converter converter;
     if (type.isRepetition(Type.Repetition.REPEATED)) {
       if (type.isPrimitive()) {
         converter = new Repeated.RepeatedPrimitiveConverter(
-            type.asPrimitiveType(), this, fieldIndex);
+            type.asPrimitiveType(), this, fieldIndex, hiveTypeInfo);
       } else {
         converter = new Repeated.RepeatedGroupConverter(
-            type.asGroupType(), this, fieldIndex);
+            type.asGroupType(), this, fieldIndex, hiveTypeInfo == null ? null : ((ListTypeInfo) hiveTypeInfo)
+            .getListElementTypeInfo());
       }
 
       repeatedConverters.add((Repeated) converter);
     } else {
-      converter = getConverterFromDescription(type, fieldIndex, this);
+      converter = getConverterFromDescription(type, fieldIndex, this, hiveTypeInfo);
     }
 
     return converter;

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java
index c0af291..a7fad71 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java
@@ -23,6 +23,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.parquet.column.Dictionary;
@@ -65,12 +66,12 @@ public interface Repeated extends ConverterParent {
     private final int index;
     private final List<Writable> list = new ArrayList<Writable>();
 
-    public RepeatedPrimitiveConverter(PrimitiveType primitiveType, ConverterParent parent, int index) {
+    public RepeatedPrimitiveConverter(PrimitiveType primitiveType, ConverterParent parent, int index, TypeInfo hiveTypeInfo) {
       setMetadata(parent.getMetadata());
       this.primitiveType = primitiveType;
       this.parent = parent;
       this.index = index;
-      this.wrapped = HiveGroupConverter.getConverterFromDescription(primitiveType, 0, this);
+      this.wrapped = HiveGroupConverter.getConverterFromDescription(primitiveType, 0, this, hiveTypeInfo);
     }
 
     @Override
@@ -149,12 +150,12 @@ public interface Repeated extends ConverterParent {
     private final Map<String, String> metadata = new HashMap<String, String>();
 
 
-    public RepeatedGroupConverter(GroupType groupType, ConverterParent parent, int index) {
+    public RepeatedGroupConverter(GroupType groupType, ConverterParent parent, int index, TypeInfo hiveTypeInfo) {
       setMetadata(parent.getMetadata());
       this.groupType = groupType;
       this.parent = parent;
       this.index = index;
-      this.wrapped = HiveGroupConverter.getConverterFromDescription(groupType, 0, this);
+      this.wrapped = HiveGroupConverter.getConverterFromDescription(groupType, 0, this, hiveTypeInfo);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
index 97f228f..53f3b72 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 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.io.ArrayWritable;
 import org.apache.hadoop.util.StringUtils;
@@ -52,7 +53,7 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
 
   public static final String HIVE_TABLE_AS_PARQUET_SCHEMA = "HIVE_TABLE_SCHEMA";
   public static final String PARQUET_COLUMN_INDEX_ACCESS = "parquet.column.index.access";
-
+  private TypeInfo hiveTypeInfo;
   /**
    * From a string which columns names (including hive column), return a list
    * of string columns
@@ -204,6 +205,8 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
 
     if (columnNames != null) {
       List<String> columnNamesList = getColumnNames(columnNames);
+      String columnTypes = configuration.get(IOConstants.COLUMNS_TYPES);
+      List<TypeInfo> columnTypesList = getColumnTypes(columnTypes);
 
       MessageType tableSchema;
       if (indexAccess) {
@@ -216,13 +219,13 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
 
         tableSchema = getSchemaByIndex(fileSchema, columnNamesList, indexSequence);
       } else {
-        String columnTypes = configuration.get(IOConstants.COLUMNS_TYPES);
-        List<TypeInfo> columnTypesList = getColumnTypes(columnTypes);
 
         tableSchema = getSchemaByName(fileSchema, columnNamesList, columnTypesList);
       }
 
       contextMetadata.put(HIVE_TABLE_AS_PARQUET_SCHEMA, tableSchema.toString());
+      contextMetadata.put(PARQUET_COLUMN_INDEX_ACCESS, String.valueOf(indexAccess));
+      this.hiveTypeInfo = TypeInfoFactory.getStructTypeInfo(columnNamesList, columnTypesList);
 
       List<Integer> indexColumnsWanted = ColumnProjectionUtils.getReadColumnIDs(configuration);
       if (!ColumnProjectionUtils.isReadAllColumns(configuration) && !indexColumnsWanted.isEmpty()) {
@@ -262,6 +265,6 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
       metadata.put(key, String.valueOf(HiveConf.getBoolVar(
         configuration, HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION)));
     }
-    return new DataWritableRecordConverter(readContext.getRequestedSchema(), metadata);
+    return new DataWritableRecordConverter(readContext.getRequestedSchema(), metadata, hiveTypeInfo);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/test/queries/clientpositive/parquet_columnar.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_columnar.q b/ql/src/test/queries/clientpositive/parquet_columnar.q
index b5aca1b..d6daeec 100644
--- a/ql/src/test/queries/clientpositive/parquet_columnar.q
+++ b/ql/src/test/queries/clientpositive/parquet_columnar.q
@@ -28,3 +28,7 @@ SELECT * FROM parquet_columnar_access;
 ALTER TABLE parquet_columnar_access REPLACE COLUMNS (s1 string, x1 int, y1 int, f1 float);
 
 SELECT * FROM parquet_columnar_access;
+
+ALTER TABLE parquet_columnar_access REPLACE COLUMNS (s1 string, x1 bigint, y1 int, f1 double);
+
+SELECT * FROM parquet_columnar_access;

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/test/queries/clientpositive/parquet_type_promotion.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_type_promotion.q b/ql/src/test/queries/clientpositive/parquet_type_promotion.q
new file mode 100644
index 0000000..71abe3b
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/parquet_type_promotion.q
@@ -0,0 +1,72 @@
+DROP TABLE parquet_type_promotion_staging;
+DROP TABLE parquet_type_promotion;
+
+SET hive.metastore.disallow.incompatible.col.type.changes=false;
+
+CREATE TABLE parquet_type_promotion_staging (
+  cint int,
+  clong bigint,
+  cfloat float,
+  cdouble double,
+  m1 map<string, int>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:int>,
+  fm1 map<string, float>,
+  fl1 array<float>,
+  fst1 struct<c1:float, c2:float>
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':';
+
+LOAD DATA LOCAL INPATH '../../data/files/parquet_type_promotion.txt' OVERWRITE INTO TABLE parquet_type_promotion_staging;
+
+SELECT * FROM parquet_type_promotion_staging;
+
+CREATE TABLE parquet_type_promotion (
+  cint int,
+  clong bigint,
+  cfloat float,
+  cdouble double,
+  m1 map<string, int>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:int>,
+  fm1 map<string, float>,
+  fl1 array<float>,
+  fst1 struct<c1:float, c2:float>
+
+) STORED AS PARQUET;
+
+INSERT OVERWRITE TABLE parquet_type_promotion
+   SELECT * FROM parquet_type_promotion_staging;
+
+SELECT * FROM parquet_type_promotion;
+
+ALTER TABLE  parquet_type_promotion REPLACE COLUMNS(
+  cint bigint,
+  clong bigint,
+  cfloat double,
+  cdouble double,
+  m1 map<string, bigint>,
+  l1 array<bigint>,
+  st1 struct<c1:int, c2:bigint>,
+  fm1 map<string, double>,
+  fl1 array<double>,
+  fst1 struct<c1:double, c2:float>
+);
+
+SELECT * FROM parquet_type_promotion;
+
+-- This test covers the case where array<struct<f1,f2,..>> data
+-- can be retrieved useing map<f1,f2>.
+-- This also test if there are more than 2 fields in array_of_struct
+-- which can be read as  map as well.
+
+DROP TABLE arrays_of_struct_to_map;
+CREATE TABLE arrays_of_struct_to_map (locations1 array<struct<c1:int,c2:int>>, locations2 array<struct<f1:int,
+f2:int,f3:int>>) STORED AS PARQUET;
+INSERT INTO TABLE arrays_of_struct_to_map select array(named_struct("c1",1,"c2",2)), array(named_struct("f1",
+77,"f2",88,"f3",99)) FROM parquet_type_promotion LIMIT 1;
+SELECT * FROM arrays_of_struct_to_map;
+ALTER TABLE arrays_of_struct_to_map REPLACE COLUMNS (locations1 map<int,bigint>, locations2 map<bigint,int>);
+SELECT * FROM arrays_of_struct_to_map;

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/test/results/clientpositive/parquet_columnar.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_columnar.q.out b/ql/src/test/results/clientpositive/parquet_columnar.q.out
index 934333b..fd28f5c 100644
--- a/ql/src/test/results/clientpositive/parquet_columnar.q.out
+++ b/ql/src/test/results/clientpositive/parquet_columnar.q.out
@@ -137,3 +137,40 @@ POSTHOOK: Input: default@parquet_columnar_access
 1cde18	1	2	1.3
 1fgh19	2	3	1.4
 1ijk20	3	4	1.0
+PREHOOK: query: ALTER TABLE parquet_columnar_access REPLACE COLUMNS (s1 string, x1 bigint, y1 int, f1 double)
+PREHOOK: type: ALTERTABLE_REPLACECOLS
+PREHOOK: Input: default@parquet_columnar_access
+PREHOOK: Output: default@parquet_columnar_access
+POSTHOOK: query: ALTER TABLE parquet_columnar_access REPLACE COLUMNS (s1 string, x1 bigint, y1 int, f1 double)
+POSTHOOK: type: ALTERTABLE_REPLACECOLS
+POSTHOOK: Input: default@parquet_columnar_access
+POSTHOOK: Output: default@parquet_columnar_access
+PREHOOK: query: SELECT * FROM parquet_columnar_access
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_columnar_access
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM parquet_columnar_access
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_columnar_access
+#### A masked pattern was here ####
+1abc00	1	2	1.0
+1def01	2	3	1.100000023841858
+1ghi02	3	4	1.2000000476837158
+1jkl03	1	2	1.2999999523162842
+1mno04	2	3	1.399999976158142
+1pqr05	3	4	1.0
+1stu06	1	2	1.100000023841858
+1vwx07	2	3	1.2000000476837158
+1yza08	3	4	1.2999999523162842
+1bcd09	1	2	1.399999976158142
+1efg10	2	3	1.0
+1hij11	3	4	1.100000023841858
+1klm12	1	2	1.2000000476837158
+1nop13	2	3	1.2999999523162842
+1qrs14	3	4	1.399999976158142
+1tuv15	1	2	1.0
+1wxy16	2	3	1.100000023841858
+1zab17	3	4	1.2000000476837158
+1cde18	1	2	1.2999999523162842
+1fgh19	2	3	1.399999976158142
+1ijk20	3	4	1.0

http://git-wip-us.apache.org/repos/asf/hive/blob/f4809406/ql/src/test/results/clientpositive/parquet_type_promotion.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_type_promotion.q.out b/ql/src/test/results/clientpositive/parquet_type_promotion.q.out
new file mode 100644
index 0000000..15a29e3
--- /dev/null
+++ b/ql/src/test/results/clientpositive/parquet_type_promotion.q.out
@@ -0,0 +1,232 @@
+PREHOOK: query: DROP TABLE parquet_type_promotion_staging
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE parquet_type_promotion_staging
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: DROP TABLE parquet_type_promotion
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE parquet_type_promotion
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE parquet_type_promotion_staging (
+  cint int,
+  clong bigint,
+  cfloat float,
+  cdouble double,
+  m1 map<string, int>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:int>,
+  fm1 map<string, float>,
+  fl1 array<float>,
+  fst1 struct<c1:float, c2:float>
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':'
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_type_promotion_staging
+POSTHOOK: query: CREATE TABLE parquet_type_promotion_staging (
+  cint int,
+  clong bigint,
+  cfloat float,
+  cdouble double,
+  m1 map<string, int>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:int>,
+  fm1 map<string, float>,
+  fl1 array<float>,
+  fst1 struct<c1:float, c2:float>
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_type_promotion_staging
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/parquet_type_promotion.txt' OVERWRITE INTO TABLE parquet_type_promotion_staging
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@parquet_type_promotion_staging
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/parquet_type_promotion.txt' OVERWRITE INTO TABLE parquet_type_promotion_staging
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@parquet_type_promotion_staging
+PREHOOK: query: SELECT * FROM parquet_type_promotion_staging
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_type_promotion_staging
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM parquet_type_promotion_staging
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_type_promotion_staging
+#### A masked pattern was here ####
+100	5643	0.3	0.7	{"k1":11}	[7,17,22]	{"c1":10,"c2":20}	{"k11":4.0}	[2.3,3.0,5.5]	{"c1":5.7,"c2":4.8}
+200	5643	0.4	0.8	{"k2":14}	[8,17,24]	{"c1":20,"c2":20}	{"v11":5.0}	[3.3,3.1,5.6]	{"c1":5.8,"c2":4.7}
+300	7643	0.4	0.9	{"k3":12}	[9,17,25]	{"c1":30,"c2":60}	{"b11":6.0}	[4.3,3.2,5.7]	{"c1":5.9,"c2":4.6}
+400	8643	0.4	0.5	{"k4":15}	[7,18,27]	{"c1":50,"c2":70}	{"d11":8.0}	[6.3,3.3,5.8]	{"c1":5.0,"c2":4.5}
+PREHOOK: query: CREATE TABLE parquet_type_promotion (
+  cint int,
+  clong bigint,
+  cfloat float,
+  cdouble double,
+  m1 map<string, int>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:int>,
+  fm1 map<string, float>,
+  fl1 array<float>,
+  fst1 struct<c1:float, c2:float>
+
+) STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_type_promotion
+POSTHOOK: query: CREATE TABLE parquet_type_promotion (
+  cint int,
+  clong bigint,
+  cfloat float,
+  cdouble double,
+  m1 map<string, int>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:int>,
+  fm1 map<string, float>,
+  fl1 array<float>,
+  fst1 struct<c1:float, c2:float>
+
+) STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_type_promotion
+PREHOOK: query: INSERT OVERWRITE TABLE parquet_type_promotion
+   SELECT * FROM parquet_type_promotion_staging
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_type_promotion_staging
+PREHOOK: Output: default@parquet_type_promotion
+POSTHOOK: query: INSERT OVERWRITE TABLE parquet_type_promotion
+   SELECT * FROM parquet_type_promotion_staging
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_type_promotion_staging
+POSTHOOK: Output: default@parquet_type_promotion
+POSTHOOK: Lineage: parquet_type_promotion.cdouble SIMPLE [(parquet_type_promotion_staging)parquet_type_promotion_staging.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: parquet_type_promotion.cfloat SIMPLE [(parquet_type_promotion_staging)parquet_type_promotion_staging.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: parquet_type_promotion.cint SIMPLE [(parquet_type_promotion_staging)parquet_type_promotion_staging.FieldSchema(name:cint, type:int, comment:null), ]
+POSTHOOK: Lineage: parquet_type_promotion.clong SIMPLE [(parquet_type_promotion_staging)parquet_type_promotion_staging.FieldSchema(name:clong, type:bigint, comment:null), ]
+POSTHOOK: Lineage: parquet_type_promotion.fl1 SIMPLE [(parquet_type_promotion_staging)parquet_type_promotion_staging.FieldSchema(name:fl1, type:array<float>, comment:null), ]
+POSTHOOK: Lineage: parquet_type_promotion.fm1 SIMPLE [(parquet_type_promotion_staging)parquet_type_promotion_staging.FieldSchema(name:fm1, type:map<string,float>, comment:null), ]
+POSTHOOK: Lineage: parquet_type_promotion.fst1 SIMPLE [(parquet_type_promotion_staging)parquet_type_promotion_staging.FieldSchema(name:fst1, type:struct<c1:float,c2:float>, comment:null), ]
+POSTHOOK: Lineage: parquet_type_promotion.l1 SIMPLE [(parquet_type_promotion_staging)parquet_type_promotion_staging.FieldSchema(name:l1, type:array<int>, comment:null), ]
+POSTHOOK: Lineage: parquet_type_promotion.m1 SIMPLE [(parquet_type_promotion_staging)parquet_type_promotion_staging.FieldSchema(name:m1, type:map<string,int>, comment:null), ]
+POSTHOOK: Lineage: parquet_type_promotion.st1 SIMPLE [(parquet_type_promotion_staging)parquet_type_promotion_staging.FieldSchema(name:st1, type:struct<c1:int,c2:int>, comment:null), ]
+PREHOOK: query: SELECT * FROM parquet_type_promotion
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_type_promotion
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM parquet_type_promotion
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_type_promotion
+#### A masked pattern was here ####
+100	5643	0.3	0.7	{"k1":11}	[7,17,22]	{"c1":10,"c2":20}	{"k11":4.0}	[2.3,3.0,5.5]	{"c1":5.7,"c2":4.8}
+200	5643	0.4	0.8	{"k2":14}	[8,17,24]	{"c1":20,"c2":20}	{"v11":5.0}	[3.3,3.1,5.6]	{"c1":5.8,"c2":4.7}
+300	7643	0.4	0.9	{"k3":12}	[9,17,25]	{"c1":30,"c2":60}	{"b11":6.0}	[4.3,3.2,5.7]	{"c1":5.9,"c2":4.6}
+400	8643	0.4	0.5	{"k4":15}	[7,18,27]	{"c1":50,"c2":70}	{"d11":8.0}	[6.3,3.3,5.8]	{"c1":5.0,"c2":4.5}
+PREHOOK: query: ALTER TABLE  parquet_type_promotion REPLACE COLUMNS(
+  cint bigint,
+  clong bigint,
+  cfloat double,
+  cdouble double,
+  m1 map<string, bigint>,
+  l1 array<bigint>,
+  st1 struct<c1:int, c2:bigint>,
+  fm1 map<string, double>,
+  fl1 array<double>,
+  fst1 struct<c1:double, c2:float>
+)
+PREHOOK: type: ALTERTABLE_REPLACECOLS
+PREHOOK: Input: default@parquet_type_promotion
+PREHOOK: Output: default@parquet_type_promotion
+POSTHOOK: query: ALTER TABLE  parquet_type_promotion REPLACE COLUMNS(
+  cint bigint,
+  clong bigint,
+  cfloat double,
+  cdouble double,
+  m1 map<string, bigint>,
+  l1 array<bigint>,
+  st1 struct<c1:int, c2:bigint>,
+  fm1 map<string, double>,
+  fl1 array<double>,
+  fst1 struct<c1:double, c2:float>
+)
+POSTHOOK: type: ALTERTABLE_REPLACECOLS
+POSTHOOK: Input: default@parquet_type_promotion
+POSTHOOK: Output: default@parquet_type_promotion
+PREHOOK: query: SELECT * FROM parquet_type_promotion
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_type_promotion
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM parquet_type_promotion
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_type_promotion
+#### A masked pattern was here ####
+100	5643	0.30000001192092896	0.7	{"k1":11}	[7,17,22]	{"c1":10,"c2":20}	{"k11":4.0}	[2.299999952316284,3.0,5.5]	{"c1":5.699999809265137,"c2":4.8}
+200	5643	0.4000000059604645	0.8	{"k2":14}	[8,17,24]	{"c1":20,"c2":20}	{"v11":5.0}	[3.299999952316284,3.0999999046325684,5.599999904632568]	{"c1":5.800000190734863,"c2":4.7}
+300	7643	0.4000000059604645	0.9	{"k3":12}	[9,17,25]	{"c1":30,"c2":60}	{"b11":6.0}	[4.300000190734863,3.200000047683716,5.699999809265137]	{"c1":5.900000095367432,"c2":4.6}
+400	8643	0.4000000059604645	0.5	{"k4":15}	[7,18,27]	{"c1":50,"c2":70}	{"d11":8.0}	[6.300000190734863,3.299999952316284,5.800000190734863]	{"c1":5.0,"c2":4.5}
+PREHOOK: query: -- This test covers the case where array<struct<f1,f2,..>> data
+-- can be retrieved useing map<f1,f2>.
+-- This also test if there are more than 2 fields in array_of_struct
+-- which can be read as  map as well.
+
+DROP TABLE arrays_of_struct_to_map
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: -- This test covers the case where array<struct<f1,f2,..>> data
+-- can be retrieved useing map<f1,f2>.
+-- This also test if there are more than 2 fields in array_of_struct
+-- which can be read as  map as well.
+
+DROP TABLE arrays_of_struct_to_map
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE arrays_of_struct_to_map (locations1 array<struct<c1:int,c2:int>>, locations2 array<struct<f1:int,
+f2:int,f3:int>>) STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@arrays_of_struct_to_map
+POSTHOOK: query: CREATE TABLE arrays_of_struct_to_map (locations1 array<struct<c1:int,c2:int>>, locations2 array<struct<f1:int,
+f2:int,f3:int>>) STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@arrays_of_struct_to_map
+PREHOOK: query: INSERT INTO TABLE arrays_of_struct_to_map select array(named_struct("c1",1,"c2",2)), array(named_struct("f1",
+77,"f2",88,"f3",99)) FROM parquet_type_promotion LIMIT 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_type_promotion
+PREHOOK: Output: default@arrays_of_struct_to_map
+POSTHOOK: query: INSERT INTO TABLE arrays_of_struct_to_map select array(named_struct("c1",1,"c2",2)), array(named_struct("f1",
+77,"f2",88,"f3",99)) FROM parquet_type_promotion LIMIT 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_type_promotion
+POSTHOOK: Output: default@arrays_of_struct_to_map
+POSTHOOK: Lineage: arrays_of_struct_to_map.locations1 EXPRESSION []
+POSTHOOK: Lineage: arrays_of_struct_to_map.locations2 EXPRESSION []
+PREHOOK: query: SELECT * FROM arrays_of_struct_to_map
+PREHOOK: type: QUERY
+PREHOOK: Input: default@arrays_of_struct_to_map
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM arrays_of_struct_to_map
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@arrays_of_struct_to_map
+#### A masked pattern was here ####
+[{"c1":1,"c2":2}]	[{"f1":77,"f2":88,"f3":99}]
+PREHOOK: query: ALTER TABLE arrays_of_struct_to_map REPLACE COLUMNS (locations1 map<int,bigint>, locations2 map<bigint,int>)
+PREHOOK: type: ALTERTABLE_REPLACECOLS
+PREHOOK: Input: default@arrays_of_struct_to_map
+PREHOOK: Output: default@arrays_of_struct_to_map
+POSTHOOK: query: ALTER TABLE arrays_of_struct_to_map REPLACE COLUMNS (locations1 map<int,bigint>, locations2 map<bigint,int>)
+POSTHOOK: type: ALTERTABLE_REPLACECOLS
+POSTHOOK: Input: default@arrays_of_struct_to_map
+POSTHOOK: Output: default@arrays_of_struct_to_map
+PREHOOK: query: SELECT * FROM arrays_of_struct_to_map
+PREHOOK: type: QUERY
+PREHOOK: Input: default@arrays_of_struct_to_map
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM arrays_of_struct_to_map
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@arrays_of_struct_to_map
+#### A masked pattern was here ####
+{1:2}	{77:88}


[33/43] hive git commit: HIVE-12407: Check fetch property to determine if a SortLimit contains a limit operation (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-12407: Check fetch property to determine if a SortLimit contains a limit operation (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master-fixed
Commit: 87ceba6bb082dcb9ca1d6f67dd2d536659a2dd9d
Parents: f759223
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Sat Nov 14 08:46:18 2015 +0100
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:34 2015 -0800

----------------------------------------------------------------------
 .../ql/optimizer/calcite/HiveCalciteUtil.java   | 20 ++++++++++++++------
 .../translator/PlanModifierForASTConv.java      | 10 +++++-----
 2 files changed, 19 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/87ceba6b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
index 90c2067..e1b60b0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
@@ -32,7 +32,6 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
 import org.apache.calcite.rel.core.Sort;
-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;
@@ -45,7 +44,6 @@ import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexRangeRef;
-import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlKind;
@@ -54,8 +52,6 @@ import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
@@ -65,6 +61,8 @@ import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.ParseUtils;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
@@ -566,16 +564,26 @@ public class HiveCalciteUtil {
     }
   }
 
+  public static boolean pureLimitRelNode(RelNode rel) {
+    return limitRelNode(rel) && !orderRelNode(rel);
+  }
+
+  public static boolean pureOrderRelNode(RelNode rel) {
+    return !limitRelNode(rel) && orderRelNode(rel);
+  }
+
   public static boolean limitRelNode(RelNode rel) {
-    if ((rel instanceof Sort) && ((Sort) rel).getCollation().getFieldCollations().isEmpty())
+    if ((rel instanceof Sort) && ((Sort) rel).fetch != null) {
       return true;
+    }
 
     return false;
   }
 
   public static boolean orderRelNode(RelNode rel) {
-    if ((rel instanceof Sort) && !((Sort) rel).getCollation().getFieldCollations().isEmpty())
+    if ((rel instanceof Sort) && !((Sort) rel).getCollation().getFieldCollations().isEmpty()) {
       return true;
+    }
 
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/87ceba6b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
index 29e08f8..b77beb8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
@@ -301,9 +301,10 @@ public class PlanModifierForASTConv {
   private static boolean validSortParent(RelNode sortNode, RelNode parent) {
     boolean validParent = true;
 
-    if (parent != null && !(parent instanceof Project)
-        && !((parent instanceof Sort) || HiveCalciteUtil.orderRelNode(parent)))
+    if (parent != null && !(parent instanceof Project) &&
+        !(HiveCalciteUtil.pureLimitRelNode(parent) && HiveCalciteUtil.pureOrderRelNode(sortNode))) {
       validParent = false;
+    }
 
     return validParent;
   }
@@ -312,9 +313,8 @@ public class PlanModifierForASTConv {
     boolean validChild = true;
     RelNode child = sortNode.getInput();
 
-    if (!(HiveCalciteUtil.limitRelNode(sortNode) && HiveCalciteUtil.orderRelNode(child)
-            && HiveCalciteUtil.limitRelNode(child))
-        && !(child instanceof Project)) {
+    if (!(child instanceof Project) &&
+        !(HiveCalciteUtil.pureLimitRelNode(sortNode) && HiveCalciteUtil.pureOrderRelNode(child))) {
       validChild = false;
     }
 


[11/43] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 a100e9f..9a1d159 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
@@ -220,6 +221,12 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+      throws MetaException {
+    return objectStore.getTableMeta(dbNames, tableNames, tableTypes);
+  }
+
+  @Override
   public List<Table> getTableObjectsByName(String dbName, List<String> tableNames)
       throws MetaException, UnknownDBException {
     return objectStore.getTableObjectsByName(dbName, tableNames);

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 f6100e6..8dde0af 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
@@ -224,6 +225,12 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
+  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+      throws MetaException {
+    return Collections.emptyList();
+  }
+
+  @Override
   public List<Table> getTableObjectsByName(String dbname, List<String> tableNames)
       throws MetaException, UnknownDBException {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index 7af9d85..581a919 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -161,11 +162,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     Matcher matcher = pattern.matcher("");
     Set<String> combinedTableNames = new HashSet<String>();
     for (String tableName : tables.keySet()) {
-      if (matcher == null) {
-        matcher = pattern.matcher(tableName);
-      } else {
-        matcher.reset(tableName);
-      }
+      matcher.reset(tableName);
       if (matcher.matches()) {
         combinedTableNames.add(tableName);
       }
@@ -177,6 +174,55 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     Collections.sort(tableNames);
     return tableNames;
   }
+  
+  @Override
+  public List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
+      throws MetaException {
+    List<TableMeta> tableMetas = super.getTableMeta(dbPatterns, tablePatterns, tableTypes);
+    Map<String, Map<String, Table>> tmpTables = getTempTables();
+    if (tmpTables.isEmpty()) {
+      return tableMetas;
+    }
+
+    List<Matcher> dbPatternList = new ArrayList<>();
+    for (String element : dbPatterns.split("\\|")) {
+      dbPatternList.add(Pattern.compile(element.replaceAll("\\*", ".*")).matcher(""));
+    }
+    List<Matcher> tblPatternList = new ArrayList<>();
+    for (String element : tablePatterns.split("\\|")) {
+      tblPatternList.add(Pattern.compile(element.replaceAll("\\*", ".*")).matcher(""));
+    }
+    StringBuilder builder = new StringBuilder();
+    for (Map.Entry<String, Map<String, Table>> outer : tmpTables.entrySet()) {
+      if (!matchesAny(outer.getKey(), dbPatternList)) {
+        continue;
+      }
+      for (Map.Entry<String, Table> inner : outer.getValue().entrySet()) {
+        Table table = inner.getValue();
+        String tableName = table.getTableName();
+        String typeString = table.getTableType().name();
+        if (tableTypes != null && !tableTypes.contains(typeString)) {
+          continue;
+        }
+        if (!matchesAny(inner.getKey(), tblPatternList)) {
+          continue;
+        }
+        TableMeta tableMeta = new TableMeta(table.getDbName(), tableName, typeString);
+        tableMeta.setComments(table.getProperty("comment"));
+        tableMetas.add(tableMeta);
+      }
+    }
+    return tableMetas;
+  }
+  
+  private boolean matchesAny(String string, List<Matcher> matchers) {
+    for (Matcher matcher : matchers) {
+      if (matcher.reset(string).matches()) {
+        return true;
+      }
+    }
+    return matchers.isEmpty();
+  }
 
   @Override
   public List<org.apache.hadoop.hive.metastore.api.Table> getTableObjectsByName(String dbName,
@@ -508,12 +554,16 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
   }
 
   public static Map<String, Table> getTempTablesForDatabase(String dbName) {
+    return getTempTables().get(dbName);
+  }
+  
+  public static Map<String, Map<String, Table>> getTempTables() {
     SessionState ss = SessionState.get();
     if (ss == null) {
       LOG.debug("No current SessionState, skipping temp tables");
-      return null;
+      return Collections.emptyMap();
     }
-    return ss.getTempTables().get(dbName);
+    return ss.getTempTables();
   }
 
   private Map<String, ColumnStatisticsObj> getTempTableColumnStatsForTable(String dbName,

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
index 296280f..65bbc1c 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
@@ -22,14 +22,11 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.ql.metadata.TableIterable;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils;
-import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationState;
@@ -48,7 +45,7 @@ public class GetTablesOperation extends MetadataOperation {
   private final String catalogName;
   private final String schemaName;
   private final String tableName;
-  private final List<String> tableTypes = new ArrayList<String>();
+  private final List<String> tableTypeList;
   private final RowSet rowSet;
   private final TableTypeMapping tableTypeMapping;
 
@@ -58,7 +55,14 @@ public class GetTablesOperation extends MetadataOperation {
   .addStringColumn("TABLE_SCHEM", "Schema name.")
   .addStringColumn("TABLE_NAME", "Table name.")
   .addStringColumn("TABLE_TYPE", "The table type, e.g. \"TABLE\", \"VIEW\", etc.")
-  .addStringColumn("REMARKS", "Comments about the table.");
+  .addStringColumn("REMARKS", "Comments about the table.")
+  .addStringColumn("TYPE_CAT", "The types catalog.")
+  .addStringColumn("TYPE_SCHEM", "The types schema.")
+  .addStringColumn("TYPE_NAME", "Type name.")
+  .addStringColumn("SELF_REFERENCING_COL_NAME", 
+      "Name of the designated \"identifier\" column of a typed table.")
+  .addStringColumn("REF_GENERATION", 
+      "Specifies how values in SELF_REFERENCING_COL_NAME are created.");
 
   protected GetTablesOperation(HiveSession parentSession,
       String catalogName, String schemaName, String tableName,
@@ -72,7 +76,12 @@ public class GetTablesOperation extends MetadataOperation {
     tableTypeMapping =
         TableTypeMappingFactory.getTableTypeMapping(tableMappingStr);
     if (tableTypes != null) {
-      this.tableTypes.addAll(tableTypes);
+      tableTypeList = new ArrayList<String>();
+      for (String tableType : tableTypes) {
+        tableTypeList.add(tableTypeMapping.mapToHiveType(tableType.trim()));
+      }
+    } else {
+      tableTypeList = null;
     }
     this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion());
   }
@@ -91,23 +100,17 @@ public class GetTablesOperation extends MetadataOperation {
       }
 
       String tablePattern = convertIdentifierPattern(tableName, true);
-      int maxBatchSize = SessionState.get().getConf().getIntVar(ConfVars.METASTORE_BATCH_RETRIEVE_MAX);
 
-      for (String dbName : metastoreClient.getDatabases(schemaPattern)) {
-        List<String> tableNames = metastoreClient.getTables(dbName, tablePattern);
-        for (Table table : new TableIterable(metastoreClient, dbName, tableNames, maxBatchSize)) {
-          Object[] rowData = new Object[] {
+      for (TableMeta tableMeta : 
+          metastoreClient.getTableMeta(schemaPattern, tablePattern, tableTypeList)) {
+        rowSet.addRow(new Object[] {
               DEFAULT_HIVE_CATALOG,
-              table.getDbName(),
-              table.getTableName(),
-              tableTypeMapping.mapToClientType(table.getTableType()),
-              table.getParameters().get("comment")
-              };
-          if (tableTypes.isEmpty() || tableTypes.contains(
-                tableTypeMapping.mapToClientType(table.getTableType()))) {
-            rowSet.addRow(rowData);
-          }
-        }
+              tableMeta.getDbName(),
+              tableMeta.getTableName(),
+              tableTypeMapping.mapToClientType(tableMeta.getTableType()),
+              tableMeta.getComments(),
+              null, null, null, null, null
+              });
       }
       setState(OperationState.FINISHED);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
index 4595ef5..285b4f9 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
@@ -18,7 +18,6 @@
 
 package org.apache.hive.service.cli.operation;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -95,16 +94,30 @@ public abstract class MetadataOperation extends Operation {
    * other hand is done locally inside the hive code and that requires the regex wildchar
    * format '.*'  This is driven by the datanucleusFormat flag.
    */
-  private String convertPattern(final String pattern, boolean datanucleusFormat) {
+  private String convertPattern(String pattern, boolean datanucleusFormat) {
     String wStr;
     if (datanucleusFormat) {
       wStr = "*";
     } else {
       wStr = ".*";
     }
-    return pattern
-        .replaceAll("([^\\\\])%", "$1" + wStr).replaceAll("\\\\%", "%").replaceAll("^%", wStr)
-        .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", ".");
+    pattern = replaceAll(pattern, "([^\\\\])%", "$1" + wStr);
+    pattern = replaceAll(pattern, "\\\\%", "%");
+    pattern = replaceAll(pattern, "^%", wStr);
+    pattern = replaceAll(pattern, "([^\\\\])_", "$1.");
+    pattern = replaceAll(pattern, "\\\\_", "_");
+    pattern = replaceAll(pattern, "^_", ".");
+    return pattern;
+  }
+  
+  private String replaceAll(String input, final String pattern, final String replace) {
+    while (true) {
+      String replaced = input.replaceAll(pattern, replace);
+      if (replaced.equals(input)) {
+        return replaced;
+      }
+      input = replaced;
+    }
   }
 
   protected boolean isAuthV2Enabled(){


[21/43] hive git commit: HIVE-12309 : TableScan should colStats when available for better data size estimate (Ashutosh Chauhan via Prasanth J)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/dcaf0c63/ql/src/test/results/clientpositive/tez/llapdecider.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/llapdecider.q.out b/ql/src/test/results/clientpositive/tez/llapdecider.q.out
index 676a0e4..fd33181 100644
--- a/ql/src/test/results/clientpositive/tez/llapdecider.q.out
+++ b/ql/src/test/results/clientpositive/tez/llapdecider.q.out
@@ -20,11 +20,11 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: key, value
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(value)
                       keys: key (type: string)
@@ -251,11 +251,11 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: key, value
-                    Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count(value)
                       keys: key (type: string)
@@ -324,7 +324,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -343,7 +343,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -412,7 +412,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -431,7 +431,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -503,7 +503,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -522,7 +522,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -593,7 +593,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -611,7 +611,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -680,7 +680,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -698,7 +698,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -767,7 +767,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -786,7 +786,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -856,7 +856,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -874,7 +874,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -943,7 +943,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -962,7 +962,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1039,7 +1039,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (UDFToInteger(key) > 1) (type: boolean)
                     Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1098,7 +1098,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (UDFToInteger(key) > 1) (type: boolean)
                     Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1155,7 +1155,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src_orc
-                  Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (UDFToInteger(GenericUDFTestGetJavaString(key)) > 1) (type: boolean)
                     Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE


[31/43] hive git commit: HIVE-11525: Tez Bucket pruning (Gopal V, reviewed by Sergey Shelukhin)

Posted by om...@apache.org.
HIVE-11525: Tez Bucket pruning (Gopal V, reviewed by Sergey Shelukhin)

Signed-off-by: Gopal V <go...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: 619ff6e99ab1814a31da52b743c4a7fc5d9b938a
Parents: 04d92dd
Author: Gopal V <go...@apache.org>
Authored: Thu Nov 12 18:35:50 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:33 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    5 +
 .../apache/hadoop/hive/ql/exec/Utilities.java   |   28 +
 .../hive/ql/exec/tez/HiveSplitGenerator.java    |   28 +-
 .../optimizer/FixedBucketPruningOptimizer.java  |  319 +++
 .../hadoop/hive/ql/optimizer/Optimizer.java     |    7 +
 .../hadoop/hive/ql/parse/GenTezUtils.java       |    4 +
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |   14 +
 .../hadoop/hive/ql/plan/TableScanDesc.java      |   39 +
 .../queries/clientpositive/bucketpruning1.q     |   97 +
 .../results/clientpositive/bucketpruning1.q.out | 2282 +++++++++++++++++
 .../clientpositive/tez/bucketpruning1.q.out     | 2360 ++++++++++++++++++
 11 files changed, 5181 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/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 7a8517b..01cd731 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2277,6 +2277,11 @@ public class HiveConf extends Configuration {
     TEZ_MIN_PARTITION_FACTOR("hive.tez.min.partition.factor", 0.25f,
         "When auto reducer parallelism is enabled this factor will be used to put a lower limit to the number\n" +
         "of reducers that tez specifies."),
+    TEZ_OPTIMIZE_BUCKET_PRUNING(
+        "hive.tez.bucket.pruning", false,
+         "When pruning is enabled, filters on bucket columns will be processed by \n" +
+         "filtering the splits against a bitset of included buckets. This needs predicates \n"+
+         "produced by hive.optimize.ppd and hive.optimize.index.filters."),
     TEZ_DYNAMIC_PARTITION_PRUNING(
         "hive.tez.dynamic.partition.pruning", true,
         "When dynamic pruning is enabled, joins on partition keys will be processed by sending\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/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 fc04f18..de2eb98 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
@@ -184,7 +184,9 @@ import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
@@ -1716,6 +1718,11 @@ public final class Utilities {
       Pattern.compile("^(.*?\\(.*\\))?([0-9]+)$");
 
   /**
+   * This breaks a prefixed bucket number out into a single integer
+   */
+  private static final Pattern PREFIXED_BUCKET_ID_REGEX =
+      Pattern.compile("^(0*([0-9]+))_([0-9]+).*");
+  /**
    * Get the task id from the filename. It is assumed that the filename is derived from the output
    * of getTaskId
    *
@@ -2138,6 +2145,27 @@ public final class Utilities {
     }
   }
 
+  /* compute bucket id from from Split */
+  public static int parseSplitBucket(InputSplit split) {
+    if (split instanceof FileSplit) {
+      return getBucketIdFromFile(((FileSplit) split).getPath().getName());
+    }
+    // cannot get this for combined splits
+    return -1;
+  }
+
+  public static int getBucketIdFromFile(String bucketName) {
+    Matcher m = PREFIXED_BUCKET_ID_REGEX.matcher(bucketName);
+    if (m.matches()) {
+      if (m.group(2).isEmpty()) {
+        // all zeros
+        return m.group(1).isEmpty() ? -1 : 0;
+      }
+      return Integer.parseInt(m.group(2));
+    }
+    return -1;
+  }
+
   public static String getNameMessage(Exception e) {
     return e.getClass().getName() + "(" + e.getMessage() + ")";
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
index 2ab3328..c370381 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
@@ -19,11 +19,14 @@
 package org.apache.hadoop.hive.ql.exec.tez;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.BitSet;
 import java.util.Comparator;
 import java.util.List;
 
 import com.google.common.base.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -163,6 +166,10 @@ public class HiveSplitGenerator extends InputInitializer {
         LOG.info("Number of input splits: " + splits.length + ". " + availableSlots
             + " available slots, " + waves + " waves. Input format is: " + realInputFormatName);
 
+        if (work.getIncludedBuckets() != null) {
+          splits = pruneBuckets(work, splits);
+        }
+
         Multimap<Integer, InputSplit> groupedSplits =
             splitGrouper.generateGroupedSplits(jobConf, conf, splits, waves, availableSlots);
         // And finally return them in a flat array
@@ -190,8 +197,25 @@ public class HiveSplitGenerator extends InputInitializer {
     }
   }
 
-
-
+  private InputSplit[] pruneBuckets(MapWork work, InputSplit[] splits) {
+    final BitSet buckets = work.getIncludedBuckets();
+    final String bucketIn = buckets.toString();
+    List<InputSplit> filteredSplits = new ArrayList<InputSplit>(splits.length / 2);
+    for (InputSplit split : splits) {
+      final int bucket = Utilities.parseSplitBucket(split);
+      if (bucket < 0 || buckets.get(bucket)) {
+        // match or UNKNOWN
+        filteredSplits.add(split);
+      } else {
+        LOG.info("Pruning with IN ({}) - removing {}", bucketIn, split);
+      }
+    }
+    if (filteredSplits.size() < splits.length) {
+      // reallocate only if any filters pruned
+      splits = filteredSplits.toArray(new InputSplit[filteredSplits.size()]);
+    }
+    return splits;
+  }
 
   private List<Event> createEventList(boolean sendSerializedEvents, InputSplitInfoMem inputSplitInfo) {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FixedBucketPruningOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FixedBucketPruningOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FixedBucketPruningOptimizer.java
new file mode 100644
index 0000000..c036db0
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FixedBucketPruningOptimizer.java
@@ -0,0 +1,319 @@
+/**
+ * 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;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.Stack;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree.Operator;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.optimizer.PrunerOperatorFactory.FilterPruner;
+import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Fixed bucket pruning optimizer goes through all the table scans and annotates them
+ * with a bucketing inclusion bit-set.
+ */
+public class FixedBucketPruningOptimizer implements Transform {
+
+  private static final Log LOG = LogFactory
+      .getLog(FixedBucketPruningOptimizer.class.getName());
+
+  public class NoopWalker implements NodeProcessor {
+    @Override
+    public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
+        Object... nodeOutputs) throws SemanticException {
+      // do nothing
+      return null;
+    }
+  }
+
+  public class FixedBucketPartitionWalker extends FilterPruner {
+
+    @Override
+    protected void generatePredicate(NodeProcessorCtx procCtx,
+        FilterOperator fop, TableScanOperator top) throws SemanticException,
+        UDFArgumentException {
+      FixedBucketPruningOptimizerCtxt ctxt = ((FixedBucketPruningOptimizerCtxt) procCtx);
+      Table tbl = top.getConf().getTableMetadata();
+      if (tbl.getNumBuckets() > 0) {
+        final int nbuckets = tbl.getNumBuckets();
+        ctxt.setNumBuckets(nbuckets);
+        ctxt.setBucketCols(tbl.getBucketCols());
+        ctxt.setSchema(tbl.getFields());
+        if (tbl.isPartitioned()) {
+          // Run partition pruner to get partitions
+          ParseContext parseCtx = ctxt.pctx;
+          PrunedPartitionList prunedPartList;
+          try {
+            String alias = (String) parseCtx.getTopOps().keySet().toArray()[0];
+            prunedPartList = PartitionPruner.prune(top, parseCtx, alias);
+          } catch (HiveException e) {
+            throw new SemanticException(e.getMessage(), e);
+          }
+          if (prunedPartList != null) {
+            ctxt.setPartitions(prunedPartList);
+            for (Partition p : prunedPartList.getPartitions()) {
+              if (nbuckets != p.getBucketCount()) {
+                // disable feature
+                ctxt.setNumBuckets(-1);
+                break;
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  public static class BucketBitsetGenerator extends FilterPruner {
+
+    @Override
+    protected void generatePredicate(NodeProcessorCtx procCtx,
+        FilterOperator fop, TableScanOperator top) throws SemanticException,
+        UDFArgumentException {
+      FixedBucketPruningOptimizerCtxt ctxt = ((FixedBucketPruningOptimizerCtxt) procCtx);
+      if (ctxt.getNumBuckets() <= 0 || ctxt.getBucketCols().size() != 1) {
+        // bucketing isn't consistent or there are >1 bucket columns
+        // optimizer does not extract multiple column predicates for this
+        return;
+      }
+      ExprNodeGenericFuncDesc filter = top.getConf().getFilterExpr();
+      if (filter == null) {
+        return;
+      }
+      // the sargs are closely tied to hive.optimize.index.filter
+      SearchArgument sarg = ConvertAstToSearchArg.create(filter);
+      if (sarg == null) {
+        return;
+      }
+      final String bucketCol = ctxt.getBucketCols().get(0);
+      StructField bucketField = null;
+      for (StructField fs : ctxt.getSchema()) {
+        if(fs.getFieldName().equals(bucketCol)) {
+          bucketField = fs;
+        }
+      }
+      Preconditions.checkArgument(bucketField != null);
+      List<Object> literals = new ArrayList<Object>();
+      List<PredicateLeaf> leaves = sarg.getLeaves();
+      Set<PredicateLeaf> bucketLeaves = new HashSet<PredicateLeaf>();
+      for (PredicateLeaf l : leaves) {
+        if (bucketCol.equals(l.getColumnName())) {
+          switch (l.getOperator()) {
+          case EQUALS:
+          case IN:
+            // supported
+            break;
+          case IS_NULL:
+            // TODO: (a = 1) and NOT (a is NULL) can be potentially folded earlier into a NO-OP
+            // fall through
+          case BETWEEN:
+            // TODO: for ordinal types you can produce a range (BETWEEN 1444442100 1444442107)
+            // fall through
+          default:
+            // cannot optimize any others
+            return;
+          }
+          bucketLeaves.add(l);
+        }
+      }
+      if (bucketLeaves.size() == 0) {
+        return;
+      }
+      // TODO: Add support for AND clauses under OR clauses
+      // first-cut takes a known minimal tree and no others.
+      // $expr = (a=1)
+      //         (a=1 or a=2)
+      //         (a in (1,2))
+      //         ($expr and *)
+      //         (* and $expr)
+      ExpressionTree expr = sarg.getExpression();
+      if (expr.getOperator() == Operator.LEAF) {
+        PredicateLeaf l = leaves.get(expr.getLeaf());
+        if (!addLiteral(literals, l)) {
+          return;
+        }
+      } else if (expr.getOperator() == Operator.AND) {
+        boolean found = false;
+        for (ExpressionTree subExpr : expr.getChildren()) {
+          if (subExpr.getOperator() != Operator.LEAF) {
+            return;
+          }
+          // one of the branches is definitely a bucket-leaf
+          PredicateLeaf l = leaves.get(subExpr.getLeaf());
+          if (bucketLeaves.contains(l)) {
+            if (!addLiteral(literals, l)) {
+              return;
+            }
+            found = true;
+          }
+        }
+        if (!found) {
+          return;
+        }
+      } else if (expr.getOperator() == Operator.OR) {
+        for (ExpressionTree subExpr : expr.getChildren()) {
+          if (subExpr.getOperator() != Operator.LEAF) {
+            return;
+          }
+          PredicateLeaf l = leaves.get(subExpr.getLeaf());
+          if (bucketLeaves.contains(l)) {
+            if (!addLiteral(literals, l)) {
+              return;
+            }
+          } else {
+            // all of the OR branches need to be bucket-leaves
+            return;
+          }
+        }
+      }
+      // invariant: bucket-col IN literals of type bucketField
+      BitSet bs = new BitSet(ctxt.getNumBuckets());
+      bs.clear();
+      PrimitiveObjectInspector bucketOI = (PrimitiveObjectInspector)bucketField.getFieldObjectInspector();
+      PrimitiveObjectInspector constOI = PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(bucketOI.getPrimitiveCategory());
+      for (Object literal: literals) {
+        PrimitiveObjectInspector origOI = PrimitiveObjectInspectorFactory.getPrimitiveObjectInspectorFromClass(literal.getClass());
+        Converter conv = ObjectInspectorConverters.getConverter(origOI, constOI);
+        // exact type conversion or get out
+        if (conv == null) {
+          return;
+        }
+        Object convCols[] = new Object[] {conv.convert(literal)};
+        int n = ObjectInspectorUtils.getBucketNumber(convCols, new ObjectInspector[]{constOI}, ctxt.getNumBuckets());
+        bs.set(n);
+      }
+      if (bs.cardinality() < ctxt.getNumBuckets()) {
+        // there is a valid bucket pruning filter
+        top.getConf().setIncludedBuckets(bs);
+        top.getConf().setNumBuckets(ctxt.getNumBuckets());
+      }
+    }
+
+    private boolean addLiteral(List<Object> literals, PredicateLeaf leaf) {
+      switch (leaf.getOperator()) {
+      case EQUALS:
+        return literals.add(leaf.getLiteral());
+      case IN:
+        return literals.addAll(leaf.getLiteralList());
+      default:
+        return false;
+      }
+    }
+  }
+
+  public final class FixedBucketPruningOptimizerCtxt implements
+      NodeProcessorCtx {
+    public final ParseContext pctx;
+    private int numBuckets;
+    private PrunedPartitionList partitions;
+    private List<String> bucketCols;
+    private List<StructField> schema;
+
+    public FixedBucketPruningOptimizerCtxt(ParseContext pctx) {
+      this.pctx = pctx;
+    }
+
+    public void setSchema(ArrayList<StructField> fields) {
+      this.schema = fields;
+    }
+
+    public List<StructField> getSchema() {
+      return this.schema;
+    }
+
+    public void setBucketCols(List<String> bucketCols) {
+      this.bucketCols = bucketCols;
+    }
+
+    public List<String> getBucketCols() {
+      return this.bucketCols;
+    }
+
+    public void setPartitions(PrunedPartitionList partitions) {
+      this.partitions = partitions;
+    }
+
+    public PrunedPartitionList getPartitions() {
+      return this.partitions;
+    }
+
+    public int getNumBuckets() {
+      return numBuckets;
+    }
+
+    public void setNumBuckets(int numBuckets) {
+      this.numBuckets = numBuckets;
+    }
+  }
+
+  @Override
+  public ParseContext transform(ParseContext pctx) throws SemanticException {
+    // create a the context for walking operators
+    FixedBucketPruningOptimizerCtxt opPartWalkerCtx = new FixedBucketPruningOptimizerCtxt(
+        pctx);
+
+    // Retrieve all partitions generated from partition pruner and partition
+    // column pruner
+    PrunerUtils.walkOperatorTree(pctx, opPartWalkerCtx,
+        new FixedBucketPartitionWalker(), new NoopWalker());
+
+    if (opPartWalkerCtx.getNumBuckets() < 0) {
+      // bail out
+      return pctx;
+    } else {
+      // walk operator tree to create expression tree for filter buckets
+      PrunerUtils.walkOperatorTree(pctx, opPartWalkerCtx,
+          new BucketBitsetGenerator(), new NoopWalker());
+    }
+
+    return pctx;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
index 6347872..4a7fc0d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
@@ -166,6 +166,13 @@ public class Optimizer {
       transformations.add(new JoinReorder());
     }
 
+    if (HiveConf.getBoolVar(hiveConf,
+        HiveConf.ConfVars.TEZ_OPTIMIZE_BUCKET_PRUNING)
+        && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD)
+        && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTINDEXFILTER)) {
+      transformations.add(new FixedBucketPruningOptimizer());
+    }
+
     if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.DYNAMICPARTITIONING) &&
         HiveConf.getVar(hiveConf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equals("nonstrict") &&
         HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTSORTDYNAMICPARTITION) &&

http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
index c5f7426..27d7276 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
@@ -184,6 +184,10 @@ public class GenTezUtils {
       mapWork.setDummyTableScan(true);
     }
 
+    if (ts.getConf().getNumBuckets() > 0) {
+      mapWork.setIncludedBuckets(ts.getConf().getIncludedBuckets());
+    }
+
     // add new item to the tez work
     tezWork.add(mapWork);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/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 d349934..f4e5873 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.BitSet;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -131,6 +132,10 @@ public class MapWork extends BaseWork {
 
   private boolean doSplitsGrouping = true;
 
+  // bitsets can't be correctly serialized by Kryo's default serializer
+  // BitSet::wordsInUse is transient, so force dumping into a lower form
+  private byte[] includedBuckets;
+
   /** Whether LLAP IO will be used for inputs. */
   private String llapIoDesc;
 
@@ -617,4 +622,13 @@ public class MapWork extends BaseWork {
   public void setMapAliases(List<String> mapAliases) {
     this.mapAliases = mapAliases;
   }
+
+  public BitSet getIncludedBuckets() {
+    return includedBuckets != null ? BitSet.valueOf(includedBuckets) : null;
+  }
+
+  public void setIncludedBuckets(BitSet includedBuckets) {
+    // see comment next to the field
+    this.includedBuckets = includedBuckets.toByteArray();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/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 be7139c..b354f98 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
 
@@ -107,6 +108,10 @@ public class TableScanDesc extends AbstractOperatorDesc {
 
   private transient Table tableMetadata;
 
+  private BitSet includedBuckets;
+
+  private int numBuckets = -1;
+
   public TableScanDesc() {
     this(null, null);
   }
@@ -319,4 +324,38 @@ public class TableScanDesc extends AbstractOperatorDesc {
   public void setSerializedFilterObject(String serializedFilterObject) {
     this.serializedFilterObject = serializedFilterObject;
   }
+
+  public void setIncludedBuckets(BitSet bitset) {
+    this.includedBuckets = bitset;
+  }
+
+  public BitSet getIncludedBuckets() {
+    return this.includedBuckets;
+  }
+
+  @Explain(displayName = "buckets included", explainLevels = { Level.EXTENDED })
+  public String getIncludedBucketExplain() {
+    if (this.includedBuckets == null) {
+      return null;
+    }
+
+    StringBuilder sb = new StringBuilder();
+    sb.append("[");
+    for (int i = 0; i < this.includedBuckets.size(); i++) {
+      if (this.includedBuckets.get(i)) {
+        sb.append(i);
+        sb.append(',');
+      }
+    }
+    sb.append(String.format("] of %d", numBuckets));
+    return sb.toString();
+  }
+
+  public int getNumBuckets() {
+    return numBuckets;
+  }
+
+  public void setNumBuckets(int numBuckets) {
+    this.numBuckets = numBuckets;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/619ff6e9/ql/src/test/queries/clientpositive/bucketpruning1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketpruning1.q b/ql/src/test/queries/clientpositive/bucketpruning1.q
new file mode 100644
index 0000000..be403a5
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/bucketpruning1.q
@@ -0,0 +1,97 @@
+set hive.optimize.ppd=true;
+set hive.optimize.index.filter=true;
+set hive.tez.bucket.pruning=true;
+set hive.explain.user=false;
+set hive.fetch.task.conversion=none;
+
+CREATE TABLE srcbucket_pruned(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 16 BUCKETS STORED AS TEXTFILE;
+
+-- cannot prune 2-key scenarios without a smarter optimizer
+CREATE TABLE srcbucket_unpruned(key int, value string) partitioned by (ds string) CLUSTERED BY (key,value) INTO 16 BUCKETS STORED AS TEXTFILE;
+
+-- good cases
+
+explain extended
+select * from srcbucket_pruned where key = 1;
+
+explain extended
+select * from srcbucket_pruned where key = 16;
+
+explain extended
+select * from srcbucket_pruned where key = 17;
+
+explain extended
+select * from srcbucket_pruned where key = 16+1;
+
+explain extended
+select * from srcbucket_pruned where key = '11';
+
+explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08';
+
+explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' and value='One';
+
+explain extended
+select * from srcbucket_pruned where value='One' and key = 1 and ds='2008-04-08';
+
+explain extended
+select * from srcbucket_pruned where key in (2,3);
+
+explain extended
+select * from srcbucket_pruned where key in (2,3) and ds='2008-04-08';
+
+explain extended
+select * from srcbucket_pruned where key in (2,3) and ds='2008-04-08' and value='One';
+
+explain extended
+select * from srcbucket_pruned where value='One' and key in (2,3) and ds='2008-04-08';
+
+explain extended
+select * from srcbucket_pruned where (key=1 or key=2) and ds='2008-04-08';
+
+explain extended
+select * from srcbucket_pruned where (key=1 or key=2) and value = 'One' and ds='2008-04-08';
+
+-- valid but irrelevant case (all buckets selected)
+
+explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17);
+
+explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08';
+
+explain extended
+select * from srcbucket_pruned where key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08' and value='One';
+
+explain extended
+select * from srcbucket_pruned where value='One' and key in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17) and ds='2008-04-08';
+
+-- valid, but unimplemented cases
+
+explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' or key = 2;
+
+explain extended
+select * from srcbucket_pruned where key = 1 and ds='2008-04-08' and (value='One' or value = 'Two');
+
+explain extended
+select * from srcbucket_pruned where key = 1 or value = "One" or key = 2 and value = "Two";
+
+-- Invalid cases
+
+explain extended
+select * from srcbucket_pruned where key = 'x11';
+
+explain extended
+select * from srcbucket_pruned where key = 1 or value = "One";
+
+explain extended
+select * from srcbucket_pruned where key = 1 or value = "One" or key = 2;
+
+explain extended
+select * from srcbucket_unpruned where key in (3, 5);
+
+explain extended
+select * from srcbucket_unpruned where key = 1;
+


[37/43] hive git commit: HIVE-12396 : BucketingSortingReduceSinkOptimizer may still throw IOB exception for duplicate columns (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by om...@apache.org.
HIVE-12396 : BucketingSortingReduceSinkOptimizer may still throw IOB exception for duplicate columns (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: 0b8e70984d8ef44fb947101ff52d2d489befcf6f
Parents: fd54e59
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Thu Nov 12 11:28:33 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:34 2015 -0800

----------------------------------------------------------------------
 .../BucketingSortingReduceSinkOptimizer.java    |  3 +
 .../clientpositive/insertoverwrite_bucket.q     | 12 +++
 .../clientpositive/insertoverwrite_bucket.q.out | 87 ++++++++++++++++++++
 3 files changed, 102 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0b8e7098/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
index d5df34c..f59ab4b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
@@ -602,6 +602,9 @@ public class BucketingSortingReduceSinkOptimizer implements Transform {
             }
 
             for (int pos : sortPositions) {
+              if (pos >= selectDesc.getColList().size()) {
+                return null;
+              }
               ExprNodeDesc selectColList = selectDesc.getColList().get(pos);
               if (!(selectColList instanceof ExprNodeColumnDesc)) {
                 return null;

http://git-wip-us.apache.org/repos/asf/hive/blob/0b8e7098/ql/src/test/queries/clientpositive/insertoverwrite_bucket.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/insertoverwrite_bucket.q b/ql/src/test/queries/clientpositive/insertoverwrite_bucket.q
index 5a10f94..50f9361 100644
--- a/ql/src/test/queries/clientpositive/insertoverwrite_bucket.q
+++ b/ql/src/test/queries/clientpositive/insertoverwrite_bucket.q
@@ -25,12 +25,24 @@ CREATE TABLE temp1
 )
 CLUSTERED BY (num) SORTED BY (num) INTO 4 BUCKETS;
 explain insert overwrite table temp1 select data, data from bucketinput;
+CREATE TABLE temp2
+(
+    create_ts STRING ,
+    change STRING,
+    num STRING
+)
+CLUSTERED BY (create_ts) SORTED BY (num) INTO 4 BUCKETS;
 
+explain
+INSERT OVERWRITE TABLE temp2
+SELECT change, change,num
+FROM temp1;
 set hive.auto.convert.sortmerge.join=true; 
 set hive.optimize.bucketmapjoin = true; 
 set hive.optimize.bucketmapjoin.sortedmerge = true; 
 select * from bucketoutput1 a join bucketoutput2 b on (a.data=b.data);
 drop table temp1;
+drop table temp2;
 drop table buckettestinput;
 drop table buckettestoutput1;
 drop table buckettestoutput2;

http://git-wip-us.apache.org/repos/asf/hive/blob/0b8e7098/ql/src/test/results/clientpositive/insertoverwrite_bucket.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/insertoverwrite_bucket.q.out b/ql/src/test/results/clientpositive/insertoverwrite_bucket.q.out
index 4add20c..bfbe87b 100644
--- a/ql/src/test/results/clientpositive/insertoverwrite_bucket.q.out
+++ b/ql/src/test/results/clientpositive/insertoverwrite_bucket.q.out
@@ -150,6 +150,85 @@ STAGE PLANS:
   Stage: Stage-2
     Stats-Aggr Operator
 
+PREHOOK: query: CREATE TABLE temp2
+(
+    create_ts STRING ,
+    change STRING,
+    num STRING
+)
+CLUSTERED BY (create_ts) SORTED BY (num) INTO 4 BUCKETS
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@temp2
+POSTHOOK: query: CREATE TABLE temp2
+(
+    create_ts STRING ,
+    change STRING,
+    num STRING
+)
+CLUSTERED BY (create_ts) SORTED BY (num) INTO 4 BUCKETS
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@temp2
+PREHOOK: query: explain
+INSERT OVERWRITE TABLE temp2
+SELECT change, change,num
+FROM temp1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+INSERT OVERWRITE TABLE temp2
+SELECT change, change,num
+FROM temp1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+  Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: temp1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Select Operator
+              expressions: change (type: string), num (type: string)
+              outputColumnNames: _col0, _col2
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col2 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                value expressions: _col0 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string), VALUE._col0 (type: string), KEY.reducesinkkey0 (type: string)
+          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.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                name: default.temp2
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: true
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.temp2
+
+  Stage: Stage-2
+    Stats-Aggr Operator
+
 PREHOOK: query: select * from bucketoutput1 a join bucketoutput2 b on (a.data=b.data)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@bucketoutput1
@@ -168,6 +247,14 @@ POSTHOOK: query: drop table temp1
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@temp1
 POSTHOOK: Output: default@temp1
+PREHOOK: query: drop table temp2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@temp2
+PREHOOK: Output: default@temp2
+POSTHOOK: query: drop table temp2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@temp2
+POSTHOOK: Output: default@temp2
 PREHOOK: query: drop table buckettestinput
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: drop table buckettestinput


[02/43] hive git commit: HIVE-12263 : Hive SchemaTool does not tolerate leading spaces in JDBC url (Chen Xin Yu via Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-12263 : Hive SchemaTool does not tolerate leading spaces in JDBC url (Chen Xin Yu via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-fixed
Commit: 1a297a1457335e13612148f0465dc1eb10607a54
Parents: c4abe2e
Author: Chen Xin Yu <qi...@126.com>
Authored: Sun Oct 25 20:19:00 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:29 2015 -0800

----------------------------------------------------------------------
 beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1a297a14/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java b/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
index 7e8cc67..181f0d2 100644
--- a/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
+++ b/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
@@ -86,7 +86,7 @@ public class HiveSchemaHelper {
     if (confVarStr == null || confVarStr.isEmpty()) {
       throw new IOException("Empty " + confVar.varname);
     }
-    return confVarStr;
+    return confVarStr.trim();
   }
 
   public interface NestedScriptParser {


[43/43] hive git commit: HIVE-11948 Investigate TxnHandler and CompactionTxnHandler to see where we improve concurrency(Eugene Koifman, reviewed by Alan Gates)

Posted by om...@apache.org.
HIVE-11948 Investigate TxnHandler and CompactionTxnHandler to see where we improve concurrency(Eugene Koifman, reviewed by Alan Gates)


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

Branch: refs/heads/master-fixed
Commit: d78fea10233cfc1a795d2a019520bc39dffe5036
Parents: 4ca8774
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Tue Nov 17 12:01:44 2015 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:36 2015 -0800

----------------------------------------------------------------------
 .../hive/hcatalog/streaming/TestStreaming.java  |   6 +-
 .../metastore/txn/CompactionTxnHandler.java     | 125 +++--
 .../hadoop/hive/metastore/txn/TxnHandler.java   | 524 ++++++++++++++-----
 .../metastore/txn/TestCompactionTxnHandler.java |  37 --
 .../hive/metastore/txn/TestTxnHandler.java      |  10 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   2 +-
 6 files changed, 464 insertions(+), 240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d78fea10/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
index 58cfbaa..806dbdb 100644
--- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
@@ -618,7 +618,7 @@ public class TestStreaming {
   }
 
   @Test
-  public void testHearbeat() throws Exception {
+  public void testHeartbeat() throws Exception {
     HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName2, tblName2, null);
     DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames2,",", endPt);
     StreamingConnection connection = endPt.newConnection(false, null);
@@ -632,14 +632,14 @@ public class TestStreaming {
     Assert.assertEquals("Wrong nubmer of locks: " + response, 1, response.getLocks().size());
     ShowLocksResponseElement lock = response.getLocks().get(0);
     long acquiredAt = lock.getAcquiredat();
-    long heartbeatAt = lock.getAcquiredat();
+    long heartbeatAt = lock.getLastheartbeat();
     txnBatch.heartbeat();
     response = msClient.showLocks();
     Assert.assertEquals("Wrong number of locks2: " + response, 1, response.getLocks().size());
     lock = response.getLocks().get(0);
     Assert.assertEquals("Acquired timestamp didn't match", acquiredAt, lock.getAcquiredat());
     Assert.assertTrue("Expected new heartbeat (" + lock.getLastheartbeat() +
-      ") > old heartbeat(" + heartbeatAt +")", lock.getLastheartbeat() > heartbeatAt);
+      ") == old heartbeat(" + heartbeatAt +")", lock.getLastheartbeat() == heartbeatAt);
   }
   @Test
   public void testTransactionBatchEmptyAbort() throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/d78fea10/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
index 4d9e8ae..5e4c7be 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
@@ -56,6 +56,7 @@ public class CompactionTxnHandler extends TxnHandler {
     Connection dbConn = null;
     Set<CompactionInfo> response = new HashSet<CompactionInfo>();
     Statement stmt = null;
+    ResultSet rs = null;
     try {
       try {
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
@@ -64,7 +65,7 @@ public class CompactionTxnHandler extends TxnHandler {
         String s = "select distinct ctc_database, ctc_table, " +
           "ctc_partition from COMPLETED_TXN_COMPONENTS";
         LOG.debug("Going to execute query <" + s + ">");
-        ResultSet rs = stmt.executeQuery(s);
+        rs = stmt.executeQuery(s);
         while (rs.next()) {
           CompactionInfo info = new CompactionInfo();
           info.dbname = rs.getString(1);
@@ -72,6 +73,7 @@ public class CompactionTxnHandler extends TxnHandler {
           info.partName = rs.getString(3);
           response.add(info);
         }
+        rs.close();
 
         // Check for aborted txns
         s = "select tc_database, tc_table, tc_partition " +
@@ -97,8 +99,7 @@ public class CompactionTxnHandler extends TxnHandler {
         LOG.error("Unable to connect to transaction database " + e.getMessage());
         checkRetryable(dbConn, e, "findPotentialCompactions(maxAborted:" + maxAborted + ")");
       } finally {
-        closeDbConn(dbConn);
-        closeStmt(stmt);
+        close(rs, stmt, dbConn);
       }
       return response;
     }
@@ -118,7 +119,7 @@ public class CompactionTxnHandler extends TxnHandler {
       Connection dbConn = null;
       Statement stmt = null;
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
         String s = "update COMPACTION_QUEUE set cq_run_as = '" + user + "' where cq_id = " + cq_id;
         LOG.debug("Going to execute update <" + s + ">");
@@ -153,46 +154,58 @@ public class CompactionTxnHandler extends TxnHandler {
   public CompactionInfo findNextToCompact(String workerId) throws MetaException {
     try {
       Connection dbConn = null;
-      CompactionInfo info = new CompactionInfo();
-
       Statement stmt = null;
+      ResultSet rs = null;
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
         String s = "select cq_id, cq_database, cq_table, cq_partition, " +
           "cq_type from COMPACTION_QUEUE where cq_state = '" + INITIATED_STATE + "'";
         LOG.debug("Going to execute query <" + s + ">");
-        ResultSet rs = stmt.executeQuery(s);
+        rs = stmt.executeQuery(s);
         if (!rs.next()) {
           LOG.debug("No compactions found ready to compact");
           dbConn.rollback();
           return null;
         }
-        info.id = rs.getLong(1);
-        info.dbname = rs.getString(2);
-        info.tableName = rs.getString(3);
-        info.partName = rs.getString(4);
-        switch (rs.getString(5).charAt(0)) {
-          case MAJOR_TYPE: info.type = CompactionType.MAJOR; break;
-          case MINOR_TYPE: info.type = CompactionType.MINOR; break;
-          default: throw new MetaException("Unexpected compaction type " + rs.getString(5));
-        }
-
-        // Now, update this record as being worked on by this worker.
-        long now = getDbTime(dbConn);
-        s = "update COMPACTION_QUEUE set cq_worker_id = '" + workerId + "', " +
-          "cq_start = " + now + ", cq_state = '" + WORKING_STATE + "' where cq_id = " + info.id;
-        LOG.debug("Going to execute update <" + s + ">");
-        int updCount = stmt.executeUpdate(s);
-        if (updCount != 1) {
+        do {
+          CompactionInfo info = new CompactionInfo();
+          info.id = rs.getLong(1);
+          info.dbname = rs.getString(2);
+          info.tableName = rs.getString(3);
+          info.partName = rs.getString(4);
+          switch (rs.getString(5).charAt(0)) {
+            case MAJOR_TYPE:
+              info.type = CompactionType.MAJOR;
+              break;
+            case MINOR_TYPE:
+              info.type = CompactionType.MINOR;
+              break;
+            default:
+              throw new MetaException("Unexpected compaction type " + rs.getString(5));
+          }
+          // Now, update this record as being worked on by this worker.
+          long now = getDbTime(dbConn);
+          s = "update COMPACTION_QUEUE set cq_worker_id = '" + workerId + "', " +
+            "cq_start = " + now + ", cq_state = '" + WORKING_STATE + "' where cq_id = " + info.id +
+            " AND cq_state='" + INITIATED_STATE + "'";
+          LOG.debug("Going to execute update <" + s + ">");
+          int updCount = stmt.executeUpdate(s);
+          if(updCount == 1) {
+            dbConn.commit();
+            return info;
+          }
+          if(updCount == 0) {
+            LOG.debug("Another Worker picked up " + info);
+            continue;
+          }
           LOG.error("Unable to set to cq_state=" + WORKING_STATE + " for compaction record: " +
-            info + ". updCnt=" + updCount);
-          LOG.debug("Going to rollback");
+            info + ". updCnt=" + updCount + ".");
           dbConn.rollback();
-        }
-        LOG.debug("Going to commit");
-        dbConn.commit();
-        return info;
+          return null;
+        } while( rs.next());
+        dbConn.rollback();
+        return null;
       } catch (SQLException e) {
         LOG.error("Unable to select next element for compaction, " + e.getMessage());
         LOG.debug("Going to rollback");
@@ -201,8 +214,7 @@ public class CompactionTxnHandler extends TxnHandler {
         throw new MetaException("Unable to connect to transaction database " +
           StringUtils.stringifyException(e));
       } finally {
-        closeDbConn(dbConn);
-        closeStmt(stmt);
+        close(rs, stmt, dbConn);
       }
     } catch (RetryException e) {
       return findNextToCompact(workerId);
@@ -219,7 +231,7 @@ public class CompactionTxnHandler extends TxnHandler {
       Connection dbConn = null;
       Statement stmt = null;
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
         String s = "update COMPACTION_QUEUE set cq_state = '" + READY_FOR_CLEANING + "', " +
           "cq_worker_id = null where cq_id = " + info.id;
@@ -240,8 +252,8 @@ public class CompactionTxnHandler extends TxnHandler {
         throw new MetaException("Unable to connect to transaction database " +
           StringUtils.stringifyException(e));
       } finally {
-        closeDbConn(dbConn);
         closeStmt(stmt);
+        closeDbConn(dbConn);
       }
     } catch (RetryException e) {
       markCompacted(info);
@@ -258,6 +270,7 @@ public class CompactionTxnHandler extends TxnHandler {
     List<CompactionInfo> rc = new ArrayList<CompactionInfo>();
 
     Statement stmt = null;
+    ResultSet rs = null;
     try {
       try {
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
@@ -265,7 +278,7 @@ public class CompactionTxnHandler extends TxnHandler {
         String s = "select cq_id, cq_database, cq_table, cq_partition, " +
           "cq_type, cq_run_as from COMPACTION_QUEUE where cq_state = '" + READY_FOR_CLEANING + "'";
         LOG.debug("Going to execute query <" + s + ">");
-        ResultSet rs = stmt.executeQuery(s);
+        rs = stmt.executeQuery(s);
         while (rs.next()) {
           CompactionInfo info = new CompactionInfo();
           info.id = rs.getLong(1);
@@ -291,8 +304,7 @@ public class CompactionTxnHandler extends TxnHandler {
         throw new MetaException("Unable to connect to transaction database " +
           StringUtils.stringifyException(e));
       } finally {
-        closeDbConn(dbConn);
-        closeStmt(stmt);
+        close(rs, stmt, dbConn);
       }
     } catch (RetryException e) {
       return findReadyToClean();
@@ -303,23 +315,28 @@ public class CompactionTxnHandler extends TxnHandler {
    * This will remove an entry from the queue after
    * it has been compacted.
    * 
-   * todo: possibly a problem?  Worker will start with DB in state X (wrt this partition).
+   * todo: Worker will start with DB in state X (wrt this partition).
    * while it's working more txns will happen, against partition it's compacting.
    * then this will delete state up to X and since then.  There may be new delta files created
    * between compaction starting and cleaning.  These will not be compacted until more
    * transactions happen.  So this ideally should only delete
    * up to TXN_ID that was compacted (i.e. HWM in Worker?)  Then this can also run
-   * at READ_COMMITTED
+   * at READ_COMMITTED.  So this means we'd want to store HWM in COMPACTION_QUEUE when
+   * Worker picks up the job.
    * 
    * Also, by using this method when Worker fails, we prevent future compactions from
-   * running until more data is written to tale or compaction is invoked explicitly
+   * running until more data is written to table or compaction is invoked explicitly
    * @param info info on the compaction entry to remove
    */
   public void markCleaned(CompactionInfo info) throws MetaException {
     try {
       Connection dbConn = null;
       Statement stmt = null;
+      ResultSet rs = null;
       try {
+        //do we need serializable?  Once we have the HWM as above, no.  Before that
+        //it's debatable, but problem described above applies either way
+        //Thus can drop to RC
         dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
         stmt = dbConn.createStatement();
         String s = "delete from COMPACTION_QUEUE where cq_id = " + info.id;
@@ -344,19 +361,20 @@ public class CompactionTxnHandler extends TxnHandler {
             "marking compaction entry as clean!");
         }
 
-
+        //todo: add distinct in query
         s = "select txn_id from TXNS, TXN_COMPONENTS where txn_id = tc_txnid and txn_state = '" +
           TXN_ABORTED + "' and tc_database = '" + info.dbname + "' and tc_table = '" +
           info.tableName + "'";
         if (info.partName != null) s += " and tc_partition = '" + info.partName + "'";
         LOG.debug("Going to execute update <" + s + ">");
-        ResultSet rs = stmt.executeQuery(s);
+        rs = stmt.executeQuery(s);
         Set<Long> txnids = new HashSet<Long>();
         while (rs.next()) txnids.add(rs.getLong(1));
         if (txnids.size() > 0) {
 
           // Remove entries from txn_components, as there may be aborted txn components
           StringBuilder buf = new StringBuilder();
+          //todo: add a safeguard to make sure IN clause is not too large; break up by txn id
           buf.append("delete from TXN_COMPONENTS where tc_txnid in (");
           boolean first = true;
           for (long id : txnids) {
@@ -394,8 +412,7 @@ public class CompactionTxnHandler extends TxnHandler {
         throw new MetaException("Unable to connect to transaction database " +
           StringUtils.stringifyException(e));
       } finally {
-        closeDbConn(dbConn);
-        closeStmt(stmt);
+        close(rs, stmt, dbConn);
       }
     } catch (RetryException e) {
       markCleaned(info);
@@ -409,14 +426,17 @@ public class CompactionTxnHandler extends TxnHandler {
     try {
       Connection dbConn = null;
       Statement stmt = null;
+      ResultSet rs = null;
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        //Aborted is a terminal state, so nothing about the txn can change
+        //after that, so READ COMMITTED is sufficient.
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
         String s = "select txn_id from TXNS where " +
           "txn_id not in (select tc_txnid from TXN_COMPONENTS) and " +
           "txn_state = '" + TXN_ABORTED + "'";
         LOG.debug("Going to execute query <" + s + ">");
-        ResultSet rs = stmt.executeQuery(s);
+        rs = stmt.executeQuery(s);
         Set<Long> txnids = new HashSet<Long>();
         while (rs.next()) txnids.add(rs.getLong(1));
         if (txnids.size() > 0) {
@@ -443,8 +463,7 @@ public class CompactionTxnHandler extends TxnHandler {
         throw new MetaException("Unable to connect to transaction database " +
           StringUtils.stringifyException(e));
       } finally {
-        closeDbConn(dbConn);
-        closeStmt(stmt);
+        close(rs, stmt, dbConn);
       }
     } catch (RetryException e) {
       cleanEmptyAbortedTxns();
@@ -465,7 +484,7 @@ public class CompactionTxnHandler extends TxnHandler {
       Connection dbConn = null;
       Statement stmt = null;
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
         String s = "update COMPACTION_QUEUE set cq_worker_id = null, cq_start = null, cq_state = '"
           + INITIATED_STATE+ "' where cq_state = '" + WORKING_STATE + "' and cq_worker_id like '"
@@ -485,8 +504,8 @@ public class CompactionTxnHandler extends TxnHandler {
         throw new MetaException("Unable to connect to transaction database " +
           StringUtils.stringifyException(e));
       } finally {
-        closeDbConn(dbConn);
         closeStmt(stmt);
+        closeDbConn(dbConn);
       }
     } catch (RetryException e) {
       revokeFromLocalWorkers(hostname);
@@ -507,7 +526,7 @@ public class CompactionTxnHandler extends TxnHandler {
       Connection dbConn = null;
       Statement stmt = null;
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         long latestValidStart = getDbTime(dbConn) - timeout;
         stmt = dbConn.createStatement();
         String s = "update COMPACTION_QUEUE set cq_worker_id = null, cq_start = null, cq_state = '"
@@ -528,8 +547,8 @@ public class CompactionTxnHandler extends TxnHandler {
         throw new MetaException("Unable to connect to transaction database " +
           StringUtils.stringifyException(e));
       } finally {
-        closeDbConn(dbConn);
         closeStmt(stmt);
+        closeDbConn(dbConn);
       }
     } catch (RetryException e) {
       revokeTimedoutWorkers(timeout);

http://git-wip-us.apache.org/repos/asf/hive/blob/d78fea10/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 5c5e6ff..7f8cb71 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
@@ -22,6 +22,7 @@ import com.jolbox.bonecp.BoneCPDataSource;
 import org.apache.commons.dbcp.ConnectionFactory;
 import org.apache.commons.dbcp.DriverManagerConnectionFactory;
 import org.apache.commons.dbcp.PoolableConnectionFactory;
+import org.apache.tools.ant.taskdefs.Java;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.commons.dbcp.PoolingDataSource;
@@ -52,6 +53,11 @@ import java.util.concurrent.TimeUnit;
  * and {@link org.apache.hadoop.hive.common.JavaUtils#lockIdToString(long)} in all messages.
  * The txnid:X and lockid:Y matches how Thrift object toString() methods are generated,
  * so keeping the format consistent makes grep'ing the logs much easier.
+ *
+ * Note on HIVE_LOCKS.hl_last_heartbeat.
+ * For locks that are part of transaction, we set this 0 (would rather set it to NULL but
+ * Currently the DB schema has this NOT NULL) and only update/read heartbeat from corresponding
+ * transaction in TXNS.
  */
 public class TxnHandler {
   // Compactor states
@@ -150,12 +156,20 @@ public class TxnHandler {
       // subsequently shows up in the open list that's ok.
       Connection dbConn = null;
       Statement stmt = null;
+      ResultSet rs = null;
       try {
+        /**
+         * This method can run at READ_COMMITTED as long as long as
+         * {@link #openTxns(org.apache.hadoop.hive.metastore.api.OpenTxnRequest)} is atomic.
+         * More specifically, as long as advancing TransactionID in NEXT_TXN_ID is atomic with
+         * adding corresponding entries into TXNS.  The reason is that any txnid below HWM
+         * is either in TXNS and thus considered open (Open/Aborted) or it's considered Committed.
+         */
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
         String s = "select ntxn_next - 1 from NEXT_TXN_ID";
         LOG.debug("Going to execute query <" + s + ">");
-        ResultSet rs = stmt.executeQuery(s);
+        rs = stmt.executeQuery(s);
         if (!rs.next()) {
           throw new MetaException("Transaction tables not properly " +
             "initialized, no record found in next_txn_id");
@@ -165,7 +179,7 @@ public class TxnHandler {
           throw new MetaException("Transaction tables not properly " +
             "initialized, null record found in next_txn_id");
         }
-
+        close(rs);
         List<TxnInfo> txnInfo = new ArrayList<TxnInfo>();
         //need the WHERE clause below to ensure consistent results with READ_COMMITTED
         s = "select txn_id, txn_state, txn_user, txn_host from TXNS where txn_id <= " + hwm;
@@ -199,8 +213,7 @@ public class TxnHandler {
         throw new MetaException("Unable to select from transaction database: " + getMessage(e)
           + StringUtils.stringifyException(e));
       } finally {
-        closeStmt(stmt);
-        closeDbConn(dbConn);
+        close(rs, stmt, dbConn);
       }
     } catch (RetryException e) {
       return getOpenTxnsInfo();
@@ -215,12 +228,16 @@ public class TxnHandler {
       // subsequently shows up in the open list that's ok.
       Connection dbConn = null;
       Statement stmt = null;
+      ResultSet rs = null;
       try {
+        /**
+         * This runs at READ_COMMITTED for exactly the same reason as {@link #getOpenTxnsInfo()}
+\         */
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
         String s = "select ntxn_next - 1 from NEXT_TXN_ID";
         LOG.debug("Going to execute query <" + s + ">");
-        ResultSet rs = stmt.executeQuery(s);
+        rs = stmt.executeQuery(s);
         if (!rs.next()) {
           throw new MetaException("Transaction tables not properly " +
             "initialized, no record found in next_txn_id");
@@ -230,7 +247,7 @@ public class TxnHandler {
           throw new MetaException("Transaction tables not properly " +
             "initialized, null record found in next_txn_id");
         }
-
+        close(rs);
         Set<Long> openList = new HashSet<Long>();
         //need the WHERE clause below to ensure consistent results with READ_COMMITTED
         s = "select txn_id from TXNS where txn_id <= " + hwm;
@@ -249,8 +266,7 @@ public class TxnHandler {
         throw new MetaException("Unable to select from transaction database, "
           + StringUtils.stringifyException(e));
       } finally {
-        closeStmt(stmt);
-        closeDbConn(dbConn);
+        close(rs, stmt, dbConn);
       }
     } catch (RetryException e) {
       return getOpenTxns();
@@ -284,17 +300,35 @@ public class TxnHandler {
     try {
       Connection dbConn = null;
       Statement stmt = null;
+      ResultSet rs = null;
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        /**
+         * To make {@link #getOpenTxns()}/{@link #getOpenTxnsInfo()} work correctly, this operation must ensure
+         * that advancing the counter in NEXT_TXN_ID and adding appropriate entries to TXNS is atomic.
+         * Also, advancing the counter must work when multiple metastores are running, thus either
+         * SELECT ... FOR UPDATE is used or SERIALIZABLE isolation.  The former is preferred since it prevents
+         * concurrent DB transactions being rolled back due to Write-Write conflict on NEXT_TXN_ID.
+         *
+         * In the current design, there can be several metastore instances running in a given Warehouse.
+         * This makes ideas like reserving a range of IDs to save trips to DB impossible.  For example,
+         * a client may go to MS1 and start a transaction with ID 500 to update a particular row.
+         * Now the same client will start another transaction, except it ends up on MS2 and may get
+         * transaction ID 400 and update the same row.  Now the merge that happens to materialize the snapshot
+         * on read will thing the version of the row from transaction ID 500 is the latest one.
+         *
+         * Longer term we can consider running Active-Passive MS (at least wrt to ACID operations).  This
+         * set could support a write-through cache for added performance.
+         */
+        dbConn = getDbConn(getRequiredIsolationLevel());
         // Make sure the user has not requested an insane amount of txns.
         int maxTxns = HiveConf.getIntVar(conf,
           HiveConf.ConfVars.HIVE_TXN_MAX_OPEN_BATCH);
         if (numTxns > maxTxns) numTxns = maxTxns;
 
         stmt = dbConn.createStatement();
-        String s = "select ntxn_next from NEXT_TXN_ID";
+        String s = addForUpdateClause(dbConn, "select ntxn_next from NEXT_TXN_ID");
         LOG.debug("Going to execute query <" + s + ">");
-        ResultSet rs = stmt.executeQuery(s);
+        rs = stmt.executeQuery(s);
         if (!rs.next()) {
           throw new MetaException("Transaction database not properly " +
             "configured, can't find next transaction id.");
@@ -312,10 +346,11 @@ public class TxnHandler {
         List<Long> txnIds = new ArrayList<Long>(numTxns);
         for (long i = first; i < first + numTxns; i++) {
           ps.setLong(1, i);
+          //todo: this would be more efficient with a single insert with multiple rows in values()
+          //need add a safeguard to not exceed the DB capabilities.
           ps.executeUpdate();
           txnIds.add(i);
         }
-
         LOG.debug("Going to commit");
         dbConn.commit();
         return new OpenTxnsResponse(txnIds);
@@ -326,8 +361,7 @@ public class TxnHandler {
         throw new MetaException("Unable to select from transaction database "
           + StringUtils.stringifyException(e));
       } finally {
-        closeStmt(stmt);
-        closeDbConn(dbConn);
+        close(rs, stmt, dbConn);
       }
     } catch (RetryException e) {
       return openTxns(rqst);
@@ -369,6 +403,24 @@ public class TxnHandler {
       Connection dbConn = null;
       Statement stmt = null;
       try {
+        /**
+         * This has to run at SERIALIZABLE to make no concurrent attempt to acquire locks (insert into HIVE_LOCKS)
+         * can happen.  Otherwise we may end up with orphaned locks.  While lock() and commitTxn() should not
+         * normally run concurrently (for same txn) but could due to bugs in the client which could then
+         * (w/o SERIALIZABLE) corrupt internal transaction manager state.  Also competes with abortTxn()
+         *
+         * Sketch of an improvement:
+         * Introduce a new transaction state in TXNS, state 'c'.  This is a transient Committed state.
+         * commitTxn() would mark the txn 'c' in TXNS in an independent txn.  Other operation like
+         * lock(), heartbeat(), etc would raise errors for txn in 'c' state and getOpenTxns(), etc would
+         * treat 'c' txn as 'open'.  Then this method could run in READ COMMITTED since the
+         * entry for this txn in TXNS in 'c' acts like a monitor.
+         * Since the move to 'c' state is in one txn (to make it visible) and the rest of the
+         * operations in another (could even be made separate txns), there is a possibility of failure
+         * between the 2.  Thus the AcidHouseKeeper logic to timeout txns should apply 'c' state txns.
+         *
+         * Or perhaps Select * TXNS where txn_id = " + txnid; for update
+         */
         dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
         stmt = dbConn.createStatement();
         // Before we do the commit heartbeat the txn.  This is slightly odd in that we're going to
@@ -423,7 +475,7 @@ public class TxnHandler {
       Connection dbConn = null;
       try {
         dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
-        return lock(dbConn, rqst, true);
+        return lock(dbConn, rqst);
       } catch (SQLException e) {
         LOG.debug("Going to rollback");
         rollbackDBConn(dbConn);
@@ -438,48 +490,49 @@ public class TxnHandler {
     }
   }
 
-  public LockResponse lockNoWait(LockRequest rqst)
-    throws NoSuchTxnException,  TxnAbortedException, MetaException {
-    try {
-      Connection dbConn = null;
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
-        return lock(dbConn, rqst, false);
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback");
-        rollbackDBConn(dbConn);
-        checkRetryable(dbConn, e, "lockNoWait(" + rqst + ")");
-        throw new MetaException("Unable to update transaction database " +
-          StringUtils.stringifyException(e));
-      } finally {
-        closeDbConn(dbConn);
-      }
-    } catch (RetryException e) {
-      return lockNoWait(rqst);
-    }
-  }
-
+  /**
+   * Why doesn't this get a txnid as parameter?  The caller should either know the txnid or know there isn't one.
+   * Either way getTxnIdFromLockId() will not be needed.  This would be a Thrift change.
+   *
+   * Also, when lock acquisition returns WAITING, it's retried every 15 seconds (best case, see DbLockManager.backoff(),
+   * in practice more often)
+   * which means this is heartbeating way more often than hive.txn.timeout and creating extra load on DB.
+   *
+   * The clients that operate in blocking mode, can't heartbeat a lock until the lock is acquired.
+   * We should make CheckLockRequest include timestamp or last request to skip unnecessary heartbeats. Thrift change.
+   *
+   * {@link #checkLock(java.sql.Connection, long)}  must run at SERIALIZABLE (make sure some lock we are checking
+   * against doesn't move from W to A in another txn) but this method can heartbeat in
+   * separate txn at READ_COMMITTED.
+   */
   public LockResponse checkLock(CheckLockRequest rqst)
     throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException {
     try {
       Connection dbConn = null;
+      long extLockId = rqst.getLockid();
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
-        long extLockId = rqst.getLockid();
-
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         // Heartbeat on the lockid first, to assure that our lock is still valid.
         // Then look up the lock info (hopefully in the cache).  If these locks
         // are associated with a transaction then heartbeat on that as well.
-        heartbeatLock(dbConn, extLockId);
-        long txnid = getTxnIdFromLockId(dbConn, extLockId);
-        if (txnid > 0)  heartbeatTxn(dbConn, txnid);
-        return checkLock(dbConn, extLockId, true);
+        Long txnid = getTxnIdFromLockId(dbConn, extLockId);
+        if(txnid == null) {
+          throw new NoSuchLockException("No such lock " + JavaUtils.lockIdToString(extLockId));
+        }
+        if (txnid > 0) {
+          heartbeatTxn(dbConn, txnid);
+        }
+        else {
+          heartbeatLock(dbConn, extLockId);
+        }
+        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        return checkLock(dbConn, extLockId);
       } catch (SQLException e) {
         LOG.debug("Going to rollback");
         rollbackDBConn(dbConn);
         checkRetryable(dbConn, e, "checkLock(" + rqst + " )");
         throw new MetaException("Unable to update transaction database " +
-          StringUtils.stringifyException(e));
+          JavaUtils.lockIdToString(extLockId) + " " + StringUtils.stringifyException(e));
       } finally {
         closeDbConn(dbConn);
       }
@@ -489,39 +542,56 @@ public class TxnHandler {
 
   }
 
+  /**
+   * This would have been made simpler if all locks were associated with a txn.  Then only txn needs to
+   * be heartbeated, committed, etc.  no need for client to track individual locks.
+   */
   public void unlock(UnlockRequest rqst)
     throws NoSuchLockException, TxnOpenException, MetaException {
     try {
       Connection dbConn = null;
       Statement stmt = null;
+      long extLockId = rqst.getLockid();
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
-        // Odd as it seems, we need to heartbeat first because this touches the
-        // lock table and assures that our locks our still valid.  If they are
-        // not, this will throw an exception and the heartbeat will fail.
-        long extLockId = rqst.getLockid();
-        heartbeatLock(dbConn, extLockId);
-        long txnid = getTxnIdFromLockId(dbConn, extLockId);
-        // If there is a valid txnid, throw an exception,
-        // as locks associated with transactions should be unlocked only when the
-        // transaction is committed or aborted.
-        if (txnid > 0) {
-          LOG.debug("Going to rollback");
-          dbConn.rollback();
-          String msg = "Unlocking locks associated with transaction" +
-            " not permitted.  Lockid " + JavaUtils.lockIdToString(extLockId) + " is associated with " +
-            "transaction " + JavaUtils.txnIdToString(txnid);
-          LOG.error(msg);
-          throw new TxnOpenException(msg);
-        }
+        /**
+         * This method is logically like commit for read-only auto commit queries.
+         * READ_COMMITTED since this only has 1 delete statement and no new entries with the
+         * same hl_lock_ext_id can be added, i.e. all rows with a given hl_lock_ext_id are
+         * created in a single atomic operation.
+         * Theoretically, this competes with {@link #lock(org.apache.hadoop.hive.metastore.api.LockRequest)}
+         * but hl_lock_ext_id is not known until that method returns.
+         * Also competes with {@link #checkLock(org.apache.hadoop.hive.metastore.api.CheckLockRequest)}
+         * but using SERIALIZABLE doesn't materially change the interaction.
+         * If "delete" stmt misses, additional logic is best effort to produce meaningful error msg.
+         */
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
-        String s = "delete from HIVE_LOCKS where hl_lock_ext_id = " + extLockId;
+        //hl_txnid <> 0 means it's associated with a transaction
+        String s = "delete from HIVE_LOCKS where hl_lock_ext_id = " + extLockId + " AND hl_txnid = 0";
         LOG.debug("Going to execute update <" + s + ">");
         int rc = stmt.executeUpdate(s);
         if (rc < 1) {
           LOG.debug("Going to rollback");
           dbConn.rollback();
-          throw new NoSuchLockException("No such lock " + JavaUtils.lockIdToString(extLockId));
+          Long txnid = getTxnIdFromLockId(dbConn, extLockId);
+          if(txnid == null) {
+            LOG.error("No lock found for unlock(" + rqst + ")");
+            throw new NoSuchLockException("No such lock " + JavaUtils.lockIdToString(extLockId));
+          }
+          if(txnid != 0) {
+            String msg = "Unlocking locks associated with transaction" +
+              " not permitted.  Lockid " + JavaUtils.lockIdToString(extLockId) + " is associated with " +
+              "transaction " + JavaUtils.txnIdToString(txnid);
+            LOG.error(msg);
+            throw new TxnOpenException(msg);
+          }
+          if(txnid == 0) {
+            //we didn't see this lock when running DELETE stmt above but now it showed up
+            //so should "should never happen" happened...
+            String msg = "Found lock " + JavaUtils.lockIdToString(extLockId) + " with " + JavaUtils.txnIdToString(txnid);
+            LOG.error(msg);
+            throw new MetaException(msg);
+          }
         }
         LOG.debug("Going to commit");
         dbConn.commit();
@@ -530,7 +600,7 @@ public class TxnHandler {
         rollbackDBConn(dbConn);
         checkRetryable(dbConn, e, "unlock(" + rqst + ")");
         throw new MetaException("Unable to update transaction database " +
-          StringUtils.stringifyException(e));
+          JavaUtils.lockIdToString(extLockId) + " " + StringUtils.stringifyException(e));
       } finally {
         closeStmt(stmt);
         closeDbConn(dbConn);
@@ -615,6 +685,10 @@ public class TxnHandler {
     }
   }
 
+  /**
+   * {@code ids} should only have txnid or lockid but not both, ideally.
+   * Currently DBTxnManager.heartbeat() enforces this.
+   */
   public void heartbeat(HeartbeatRequest ids)
     throws NoSuchTxnException,  NoSuchLockException, TxnAbortedException, MetaException {
     try {
@@ -647,9 +721,17 @@ public class TxnHandler {
       rsp.setNosuch(nosuch);
       rsp.setAborted(aborted);
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        /**
+         * READ_COMMITTED is sufficient since {@link #heartbeatTxn(java.sql.Connection, long)}
+         * only has 1 update statement in it and
+         * we only update existing txns, i.e. nothing can add additional txns that this operation
+         * would care about (which would have required SERIALIZABLE)
+         */
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         for (long txn = rqst.getMin(); txn <= rqst.getMax(); txn++) {
           try {
+            //todo: this is expensive call: at least 2 update queries per txn
+            //is this really worth it?
             heartbeatTxn(dbConn, txn);
           } catch (NoSuchTxnException e) {
             nosuch.add(txn);
@@ -678,11 +760,11 @@ public class TxnHandler {
       Connection dbConn = null;
       Statement stmt = null;
       try {
-        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        dbConn = getDbConn(getRequiredIsolationLevel());
         stmt = dbConn.createStatement();
 
         // Get the id for the next entry in the queue
-        String s = "select ncq_next from NEXT_COMPACTION_QUEUE_ID";
+        String s = addForUpdateClause(dbConn, "select ncq_next from NEXT_COMPACTION_QUEUE_ID");
         LOG.debug("going to execute query <" + s + ">");
         ResultSet rs = stmt.executeQuery(s);
         if (!rs.next()) {
@@ -1293,19 +1375,31 @@ public class TxnHandler {
     }
   }
 
+  private int abortTxns(Connection dbConn, List<Long> txnids) throws SQLException {
+    return abortTxns(dbConn, txnids, -1);
+  }
   /**
-   * Abort a group of txns
+   * TODO: expose this as an operation to client.  Useful for streaming API to abort all remaining
+   * trasnactions in a batch on IOExceptions.
    * @param dbConn An active connection
    * @param txnids list of transactions to abort
+   * @param max_heartbeat value used by {@link #performTimeOuts()} to ensure this doesn't Abort txn which were
+   *                      hearbetated after #performTimeOuts() select and this operation.
    * @return Number of aborted transactions
    * @throws SQLException
    */
-  private int abortTxns(Connection dbConn, List<Long> txnids) throws SQLException {
+  private int abortTxns(Connection dbConn, List<Long> txnids, long max_heartbeat) throws SQLException {
     Statement stmt = null;
     int updateCnt = 0;
     if (txnids.isEmpty()) {
       return 0;
     }
+    if(Connection.TRANSACTION_SERIALIZABLE != dbConn.getTransactionIsolation()) {
+      /** Running this at SERIALIZABLE prevents new locks being added for this txnid(s) concurrently
+        * which would cause them to become orphaned.
+        */
+      throw new IllegalStateException("Expected SERIALIZABLE isolation. Found " + dbConn.getTransactionIsolation());
+    }
     try {
       stmt = dbConn.createStatement();
 
@@ -1321,6 +1415,8 @@ public class TxnHandler {
       LOG.debug("Going to execute update <" + buf.toString() + ">");
       stmt.executeUpdate(buf.toString());
 
+      //todo: seems like we should do this first and if it misses, don't bother with
+      //delete from HIVE_LOCKS since it will be rolled back
       buf = new StringBuilder("update TXNS set txn_state = '" + TXN_ABORTED +
         "' where txn_state = '" + TXN_OPEN + "' and txn_id in (");
       first = true;
@@ -1330,6 +1426,9 @@ public class TxnHandler {
         buf.append(id);
       }
       buf.append(')');
+      if(max_heartbeat > 0) {
+        buf.append(" and txn_last_heartbeat < ").append(max_heartbeat);
+      }
       LOG.debug("Going to execute update <" + buf.toString() + ">");
       updateCnt = stmt.executeUpdate(buf.toString());
 
@@ -1340,22 +1439,33 @@ public class TxnHandler {
   }
 
   /**
+   * Isolation Level Notes:
+   * Run at SERIALIZABLE to make sure no one is adding new locks while we are checking conflicts here.
+   * 
+   * Ramblings:
+   * We could perhaps get away with writing to TXN_COMPONENTS + HIVE_LOCKS in 1 txn@RC
+   * since this is just in Wait state.
+   * (Then we'd need to ensure that in !wait case we don't rely on rollback and again in case of
+   * failure, the W locks will timeout if failure does not propagate to client in some way, or it
+   * will and client will Abort).
+   * Actually, whether we can do this depends on what happens when you try to get a lock and notice
+   * a conflicting locks in W mode do we wait in this case?  if so it's a problem because while you
+   * are checking new locks someone may insert new  W locks that you don't see...
+   * On the other hand, this attempts to be 'fair', i.e. process locks in order so could we assume
+   * that additional W locks will have higher IDs????
+   *
+   * We can use Select for Update to generate the next LockID.  In fact we can easily do this in a separate txn.
+   * This avoids contention on NEXT_LOCK_ID.  The rest of the logic will be still need to be done at Serializable, I think,
+   * but it will not be updating the same row from 2 DB.
+   *
    * Request a lock
    * @param dbConn database connection
    * @param rqst lock information
-   * @param wait whether to wait for this lock.  The function will return immediately one way or
-   *             another.  If true and the lock could not be acquired the response will have a
-   *             state of  WAITING.  The caller will then need to poll using
-   *             {@link #checkLock(org.apache.hadoop.hive.metastore.api.CheckLockRequest)}. If
-   *             false and the  lock could not be acquired, then the response will have a state
-   *             of NOT_ACQUIRED.  The caller will need to call
-   *             {@link #lockNoWait(org.apache.hadoop.hive.metastore.api.LockRequest)} again to
-   *             attempt another lock.
    * @return information on whether the lock was acquired.
    * @throws NoSuchTxnException
    * @throws TxnAbortedException
    */
-  private LockResponse lock(Connection dbConn, LockRequest rqst, boolean wait)
+  private LockResponse lock(Connection dbConn, LockRequest rqst)
     throws NoSuchTxnException,  TxnAbortedException, MetaException, SQLException {
     // We want to minimize the number of concurrent lock requests being issued.  If we do not we
     // get a large number of deadlocks in the database, since this method has to both clean
@@ -1368,13 +1478,25 @@ public class TxnHandler {
     // etc.) that should not interfere with this one.
     synchronized (lockLock) {
       Statement stmt = null;
+      ResultSet rs = null;
       try {
+        long txnid = rqst.getTxnid();
+        if (txnid > 0) {
+          // Heartbeat the transaction so we know it is valid and we avoid it timing out while we
+          // are locking.
+          heartbeatTxn(dbConn, txnid);
+        }
         stmt = dbConn.createStatement();
 
-        // Get the next lock id.
-        String s = "select nl_next from NEXT_LOCK_ID";
+       /** Get the next lock id.
+        * This has to be atomic with adding entries to HIVE_LOCK entries (1st add in W state) to prevent a race.
+        * Suppose ID gen is a separate txn and 2 concurrent lock() methods are running.  1st one generates nl_next=7,
+        * 2nd nl_next=8.  Then 8 goes first to insert into HIVE_LOCKS and aquires the locks.  Then 7 unblocks,
+        * and add it's W locks but it won't see locks from 8 since to be 'fair' {@link #checkLock(java.sql.Connection, long)}
+        * doesn't block on locks acquired later than one it's checking*/
+        String s = addForUpdateClause(dbConn, "select nl_next from NEXT_LOCK_ID");
         LOG.debug("Going to execute query <" + s + ">");
-        ResultSet rs = stmt.executeQuery(s);
+        rs = stmt.executeQuery(s);
         if (!rs.next()) {
           LOG.debug("Going to rollback");
           dbConn.rollback();
@@ -1385,18 +1507,19 @@ public class TxnHandler {
         s = "update NEXT_LOCK_ID set nl_next = " + (extLockId + 1);
         LOG.debug("Going to execute update <" + s + ">");
         stmt.executeUpdate(s);
-        LOG.debug("Going to commit.");
-        dbConn.commit();
 
-        long txnid = rqst.getTxnid();
         if (txnid > 0) {
-          // Heartbeat the transaction so we know it is valid and we avoid it timing out while we
-          // are locking.
-          heartbeatTxn(dbConn, txnid);
-
           // For each component in this lock request,
           // add an entry to the txn_components table
           // This must be done before HIVE_LOCKS is accessed
+          
+          //Isolation note:
+          //the !wait option is not actually used anywhere.  W/o that,
+          // if we make CompactionTxnHandler.markCleaned() not delete anything above certain txn_id
+          //then there is not reason why this insert into TXN_COMPONENTS needs to run at Serializable.
+          //
+          // Again, w/o the !wait option, insert into HIVE_LOCKS should be OK at READ_COMMITTED as long
+          //as check lock is at serializable (or any other way to make sure it's exclusive)
           for (LockComponent lc : rqst.getComponent()) {
             String dbName = lc.getDbname();
             String tblName = lc.getTablename();
@@ -1429,34 +1552,42 @@ public class TxnHandler {
             " (hl_lock_ext_id, hl_lock_int_id, hl_txnid, hl_db, hl_table, " +
             "hl_partition, hl_lock_state, hl_lock_type, hl_last_heartbeat, hl_user, hl_host)" +
             " values (" + extLockId + ", " +
-            + intLockId + "," + (txnid >= 0 ? txnid : "null") + ", '" +
+            + intLockId + "," + txnid + ", '" +
             dbName + "', " + (tblName == null ? "null" : "'" + tblName + "'" )
             + ", " + (partName == null ? "null" : "'" + partName + "'") +
-            ", '" + LOCK_WAITING + "', " +  "'" + lockChar + "', " + now + ", '" +
+            ", '" + LOCK_WAITING + "', " +  "'" + lockChar + "', " +
+            //for locks associated with a txn, we always heartbeat txn and timeout based on that
+            (isValidTxn(txnid) ? 0 : now) + ", '" +
             rqst.getUser() + "', '" + rqst.getHostname() + "')";
           LOG.debug("Going to execute update <" + s + ">");
           stmt.executeUpdate(s);
         }
-        LockResponse rsp = checkLock(dbConn, extLockId, wait);
-        if (!wait && rsp.getState() != LockState.ACQUIRED) {
-          LOG.debug("Lock not acquired, going to rollback");
-          dbConn.rollback();
-          rsp = new LockResponse();
-          rsp.setState(LockState.NOT_ACQUIRED);
-        }
-        return rsp;
+        /**to make txns shorter we could commit here and start a new txn for checkLock.  This would
+         * require moving checkRetryable() down into here.  Could we then run the part before this
+         * commit are READ_COMMITTED?*/
+        return checkLock(dbConn, extLockId);
       } catch (NoSuchLockException e) {
         // This should never happen, as we just added the lock id
         throw new MetaException("Couldn't find a lock we just created!");
       } finally {
+        close(rs);
         closeStmt(stmt);
       }
     }
   }
-
+  private static boolean isValidTxn(long txnId) {
+    return txnId != 0;
+  }
+  /**
+   * Note: this calls acquire() for (extLockId,intLockId) but extLockId is the same and we either take
+   * all locks for given extLockId or none.  Would be more efficient to update state on all locks
+   * at once.  Semantics are the same since this is all part of the same txn@serializable.
+   *
+   * Lock acquisition is meant to be fair, so every lock can only block on some lock with smaller
+   * hl_lock_ext_id by only checking earlier locks.
+   */
   private LockResponse checkLock(Connection dbConn,
-                                 long extLockId,
-                                 boolean alwaysCommit)
+                                 long extLockId)
     throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, MetaException, SQLException {
     List<LockInfo> locksBeingChecked = getLockInfoFromLockId(dbConn, extLockId);//being acquired now
     LockResponse response = new LockResponse();
@@ -1609,19 +1740,15 @@ public class TxnHandler {
             case WAIT:
               if(!ignoreConflict(info, locks[i])) {
                 wait(dbConn, save);
-                if (alwaysCommit) {
-                  // In the case where lockNoWait has been called we don't want to commit because
-                  // it's going to roll everything back. In every other case we want to commit here.
-                  LOG.debug("Going to commit");
-                  dbConn.commit();
-                }
+                LOG.debug("Going to commit");
+                dbConn.commit();
                 response.setState(LockState.WAITING);
                 LOG.debug("Lock(" + info + ") waiting for Lock(" + locks[i] + ")");
                 return response;
               }
               //fall through to ACQUIRE
             case ACQUIRE:
-              acquire(dbConn, stmt, extLockId, info.intLockId);
+              acquire(dbConn, stmt, extLockId, info);
               acquired = true;
               break;
             case KEEP_LOOKING:
@@ -1633,7 +1760,7 @@ public class TxnHandler {
 
         // If we've arrived here and we have not already acquired, it means there's nothing in the
         // way of the lock, so acquire the lock.
-        if (!acquired) acquire(dbConn, stmt, extLockId, info.intLockId);
+        if (!acquired) acquire(dbConn, stmt, extLockId, info);
       }
 
       // We acquired all of the locks, so commit and return acquired.
@@ -1677,26 +1804,31 @@ public class TxnHandler {
     dbConn.rollback(save);
   }
 
-  private void acquire(Connection dbConn, Statement stmt, long extLockId, long intLockId)
+  private void acquire(Connection dbConn, Statement stmt, long extLockId, LockInfo lockInfo)
     throws SQLException, NoSuchLockException, MetaException {
     long now = getDbTime(dbConn);
     String s = "update HIVE_LOCKS set hl_lock_state = '" + LOCK_ACQUIRED + "', " +
-      "hl_last_heartbeat = " + now + ", hl_acquired_at = " + now + " where hl_lock_ext_id = " +
-      extLockId + " and hl_lock_int_id = " + intLockId;
+      //if lock is part of txn, heartbeat info is in txn record
+      "hl_last_heartbeat = " + (isValidTxn(lockInfo.txnId) ? 0 : now) +
+    ", hl_acquired_at = " + now + " where hl_lock_ext_id = " +
+      extLockId + " and hl_lock_int_id = " + lockInfo.intLockId;
     LOG.debug("Going to execute update <" + s + ">");
     int rc = stmt.executeUpdate(s);
     if (rc < 1) {
       LOG.debug("Going to rollback");
       dbConn.rollback();
       throw new NoSuchLockException("No such lock: (" + JavaUtils.lockIdToString(extLockId) + "," +
-        + intLockId + ")");
+        + lockInfo.intLockId + ") " + JavaUtils.txnIdToString(lockInfo.txnId));
     }
     // We update the database, but we don't commit because there may be other
     // locks together with this, and we only want to acquire one if we can
     // acquire all.
   }
 
-  // Heartbeats on the lock table.  This commits, so do not enter it with any state
+  /**
+   * Heartbeats on the lock table.  This commits, so do not enter it with any state.
+   * Should not be called on a lock that belongs to transaction.
+   */
   private void heartbeatLock(Connection dbConn, long extLockId)
     throws NoSuchLockException, SQLException, MetaException {
     // If the lock id is 0, then there are no locks in this heartbeat
@@ -1731,7 +1863,6 @@ public class TxnHandler {
     try {
       stmt = dbConn.createStatement();
       long now = getDbTime(dbConn);
-      ensureValidTxn(dbConn, txnid, stmt);
       String s = "update TXNS set txn_last_heartbeat = " + now +
         " where txn_id = " + txnid + " and txn_state = '" + TXN_OPEN + "'";
       LOG.debug("Going to execute update <" + s + ">");
@@ -1742,10 +1873,6 @@ public class TxnHandler {
         dbConn.rollback();
         throw new NoSuchTxnException("No such txn: " + txnid);
       }
-      //update locks for this txn to the same heartbeat
-      s = "update HIVE_LOCKS set hl_last_heartbeat = " + now + " where hl_txnid = " + txnid;
-      LOG.debug("Going to execute update <" + s + ">");
-      stmt.executeUpdate(s);
       LOG.debug("Going to commit");
       dbConn.commit();
     } finally {
@@ -1760,6 +1887,7 @@ public class TxnHandler {
     LOG.debug("Going to execute query <" + s + ">");
     ResultSet rs = stmt.executeQuery(s);
     if (!rs.next()) {
+      //todo: add LIMIT 1 instead of count - should be more efficient
       s = "select count(*) from COMPLETED_TXN_COMPONENTS where CTC_TXNID = " + txnid;
       ResultSet rs2 = stmt.executeQuery(s);
       boolean alreadyCommitted = rs2.next() && rs2.getInt(1) > 0;
@@ -1775,28 +1903,28 @@ public class TxnHandler {
       LOG.debug("Going to rollback");
       dbConn.rollback();
       throw new TxnAbortedException("Transaction " + JavaUtils.txnIdToString(txnid) +
-        " already aborted");//todo: add time of abort, which is not currently tracked
+        " already aborted");//todo: add time of abort, which is not currently tracked.  Requires schema change
     }
   }
 
-  // NEVER call this function without first calling heartbeat(long, long)
-  private long getTxnIdFromLockId(Connection dbConn, long extLockId)
+  private Long getTxnIdFromLockId(Connection dbConn, long extLockId)
     throws NoSuchLockException, MetaException, SQLException {
     Statement stmt = null;
+    ResultSet rs = null;
     try {
       stmt = dbConn.createStatement();
       String s = "select hl_txnid from HIVE_LOCKS where hl_lock_ext_id = " +
         extLockId;
       LOG.debug("Going to execute query <" + s + ">");
-      ResultSet rs = stmt.executeQuery(s);
+      rs = stmt.executeQuery(s);
       if (!rs.next()) {
-        throw new MetaException("This should never happen!  We already " +
-          "checked the lock existed but now we can't find it!");
+        return null;
       }
       long txnid = rs.getLong(1);
-      LOG.debug("Return " + JavaUtils.txnIdToString(rs.wasNull() ? -1 : txnid));
-      return (rs.wasNull() ? -1 : txnid);
+      LOG.debug("getTxnIdFromLockId(" + extLockId + ") Return " + JavaUtils.txnIdToString(txnid));
+      return txnid;
     } finally {
+      close(rs);
       closeStmt(stmt);
     }
   }
@@ -1832,14 +1960,13 @@ public class TxnHandler {
   // for read-only autoCommit=true statements.  This does a commit,
   // and thus should be done before any calls to heartbeat that will leave
   // open transactions.
-  private void timeOutLocks(Connection dbConn) {
+  private void timeOutLocks(Connection dbConn, long now) {
     Statement stmt = null;
     try {
-      long now = getDbTime(dbConn);
       stmt = dbConn.createStatement();
       // Remove any timed out locks from the table.
       String s = "delete from HIVE_LOCKS where hl_last_heartbeat < " +
-        (now - timeout) + " and (hl_txnid = 0 or hl_txnid is NULL)";//when txnid is > 0, the lock is
+        (now - timeout) + " and hl_txnid = 0";//when txnid is > 0, the lock is
       //associated with a txn and is handled by performTimeOuts()
       //want to avoid expiring locks for a txn w/o expiring the txn itself
       LOG.debug("Going to execute update <" + s + ">");
@@ -1891,6 +2018,8 @@ public class TxnHandler {
     }
   }
   /**
+   * Isolation Level Notes
+   * Plain: RC is OK
    * This will find transactions that have timed out and abort them.
    * Will also delete locks which are not associated with a transaction and have timed out
    * Tries to keep transactions (against metastore db) small to reduce lock contention.
@@ -1900,9 +2029,19 @@ public class TxnHandler {
     Statement stmt = null;
     ResultSet rs = null;
     try {
-      dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+      //We currently commit after selecting the TXNS to abort.  So whether SERIALIZABLE
+      //READ_COMMITTED, the effect is the same.  We could use FOR UPDATE on Select from TXNS
+      //and do the whole performTimeOuts() in a single huge transaction, but the only benefit
+      //would be to make sure someone cannot heartbeat one of these txns at the same time.
+      //The attempt to heartbeat would block and fail immediately after it's unblocked.
+      //With current (RC + multiple txns) implementation it is possible for someone to send
+      //heartbeat at the very end of the expire interval, and just after the Select from TXNS
+      //is made, in which case heartbeat will succeed but txn will still be Aborted.
+      //Solving this corner case is not worth the perf penalty.  The client should heartbeat in a
+      //timely way.
       long now = getDbTime(dbConn);
-      timeOutLocks(dbConn);
+      timeOutLocks(dbConn, now);
       while(true) {
         stmt = dbConn.createStatement();
         String s = " txn_id from TXNS where txn_state = '" + TXN_OPEN +
@@ -1923,16 +2062,26 @@ public class TxnHandler {
             timedOutTxns.add(currentBatch);
           }
         } while(rs.next());
-        close(rs, stmt, null);
         dbConn.commit();
+        close(rs, stmt, dbConn);
+        dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+        int numTxnsAborted = 0;
         for(List<Long> batchToAbort : timedOutTxns) {
-          abortTxns(dbConn, batchToAbort);
-          dbConn.commit();
-          //todo: add TXNS.COMMENT filed and set it to 'aborted by system due to timeout'
-          LOG.info("Aborted the following transactions due to timeout: " + batchToAbort.toString());
+          if(abortTxns(dbConn, batchToAbort, now - timeout) == batchToAbort.size()) {
+            dbConn.commit();
+            numTxnsAborted += batchToAbort.size();
+            //todo: add TXNS.COMMENT filed and set it to 'aborted by system due to timeout'
+            LOG.info("Aborted the following transactions due to timeout: " + batchToAbort.toString());
+          }
+          else {
+            //could not abort all txns in this batch - this may happen because in parallel with this
+            //operation there was activity on one of the txns in this batch (commit/abort/heartbeat)
+            //This is not likely but may happen if client experiences long pause between heartbeats or
+            //unusually long/extreme pauses between heartbeat() calls and other logic in checkLock(),
+            //lock(), etc.
+            dbConn.rollback();
+          }
         }
-        int numTxnsAborted = (timedOutTxns.size() - 1) * TIMED_OUT_TXN_ABORT_BATCH_SIZE +
-          timedOutTxns.get(timedOutTxns.size() - 1).size();
         LOG.info("Aborted " + numTxnsAborted + " transactions due to timeout");
       }
     } catch (SQLException ex) {
@@ -2110,4 +2259,97 @@ public class TxnHandler {
   private static String getMessage(SQLException ex) {
     return ex.getMessage() + "(SQLState=" + ex.getSQLState() + ",ErrorCode=" + ex.getErrorCode() + ")";
   }
+  /**
+   * Returns one of {@link java.sql.Connection#TRANSACTION_SERIALIZABLE} TRANSACTION_READ_COMMITTED, etc.
+   * Different DBs support different concurrency management options.  This class relies on SELECT ... FOR UPDATE
+   * functionality.  Where that is not available, SERIALIZABLE isolation is used.
+   * This method must always agree with {@link #addForUpdateClause(java.sql.Connection, String)}, in that
+   * if FOR UPDATE is not available, must run operation at SERIALIZABLE.
+   */
+  private int getRequiredIsolationLevel() throws MetaException, SQLException {
+    if(dbProduct == null) {
+      Connection tmp = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+      determineDatabaseProduct(tmp);
+      closeDbConn(tmp);
+    }
+    switch (dbProduct) {
+      case DERBY:
+        return Connection.TRANSACTION_SERIALIZABLE;
+      case MYSQL:
+      case ORACLE:
+      case POSTGRES:
+      case SQLSERVER:
+        return Connection.TRANSACTION_READ_COMMITTED;
+      default:
+        String msg = "Unrecognized database product name <" + dbProduct + ">";
+        LOG.error(msg);
+        throw new MetaException(msg);
+    }
+  }
+  /**
+   * Given a {@code selectStatement}, decorated it with FOR UPDATE or semantically equivalent
+   * construct.  If the DB doesn't support, return original select.  This method must always
+   * agree with {@link #getRequiredIsolationLevel()}
+   */
+  private String addForUpdateClause(Connection dbConn, String selectStatement) throws MetaException {
+    DatabaseProduct prod = determineDatabaseProduct(dbConn);
+    switch (prod) {
+      case DERBY:
+        //https://db.apache.org/derby/docs/10.1/ref/rrefsqlj31783.html
+        //sadly in Derby, FOR UPDATE doesn't meant what it should
+        return selectStatement;
+      case MYSQL:
+        //http://dev.mysql.com/doc/refman/5.7/en/select.html
+      case ORACLE:
+        //https://docs.oracle.com/cd/E17952_01/refman-5.6-en/select.html
+      case POSTGRES:
+        //http://www.postgresql.org/docs/9.0/static/sql-select.html
+        return selectStatement + " for update";
+      case SQLSERVER:
+        //https://msdn.microsoft.com/en-us/library/ms189499.aspx
+        //https://msdn.microsoft.com/en-us/library/ms187373.aspx
+        return selectStatement + " with(updlock)";
+      default:
+        String msg = "Unrecognized database product name <" + prod + ">";
+        LOG.error(msg);
+        throw new MetaException(msg);
+    }
+  }
+  /**
+   * the caller is expected to retry if this fails
+   *
+   * @return
+   * @throws SQLException
+   * @throws MetaException
+   */
+  private long generateNewExtLockId() throws SQLException, MetaException {
+    Connection dbConn = null;
+    Statement stmt = null;
+    ResultSet rs = null;
+    try {
+      dbConn = getDbConn(getRequiredIsolationLevel());
+      stmt = dbConn.createStatement();
+
+      // Get the next lock id.
+      String s = addForUpdateClause(dbConn, "select nl_next from NEXT_LOCK_ID");
+      LOG.debug("Going to execute query <" + s + ">");
+      rs = stmt.executeQuery(s);
+      if (!rs.next()) {
+        LOG.debug("Going to rollback");
+        dbConn.rollback();
+        throw new MetaException("Transaction tables not properly " +
+          "initialized, no record found in next_lock_id");
+      }
+      long extLockId = rs.getLong(1);
+      s = "update NEXT_LOCK_ID set nl_next = " + (extLockId + 1);
+      LOG.debug("Going to execute update <" + s + ">");
+      stmt.executeUpdate(s);
+      LOG.debug("Going to commit.");
+      dbConn.commit();
+      return extLockId;
+    }
+    finally {
+      close(rs, stmt, dbConn);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/d78fea10/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
index df42f1a..06e0932 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
@@ -280,43 +280,6 @@ public class TestCompactionTxnHandler {
   }
 
   @Test
-  public void testLockNoWait() throws Exception {
-    // Test that we can acquire the lock alone
-     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB,
-        "mydb");
-    comp.setTablename("mytable");
-    comp.setPartitionname("mypartition");
-    List<LockComponent> components = new ArrayList<LockComponent>(1);
-    components.add(comp);
-    LockRequest req = new LockRequest(components, "me", "localhost");
-    LockResponse res = txnHandler.lockNoWait(req);
-    assertTrue(res.getState() == LockState.ACQUIRED);
-    txnHandler.unlock(new UnlockRequest(res.getLockid()));
-
-    // test that another lock blocks it
-    comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB,
-        "mydb");
-    comp.setTablename("mytable");
-    comp.setPartitionname("mypartition");
-    components.clear();
-    components.add(comp);
-    req = new LockRequest(components, "me", "localhost");
-    res = txnHandler.lock(req);
-    assertEquals(LockState.ACQUIRED, res.getState());
-
-    comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB,
-        "mydb");
-    comp.setTablename("mytable");
-    comp.setPartitionname("mypartition");
-    components.clear();
-    components.add(comp);
-    req = new LockRequest(components, "me", "localhost");
-    res = txnHandler.lockNoWait(req);
-    assertEquals(LockState.NOT_ACQUIRED, res.getState());
-    assertEquals(1, TxnDbUtil.findNumCurrentLocks());
-  }
-
-  @Test
   public void testFindPotentialCompactions() throws Exception {
     // Test that committing unlocks
     long txnid = openTxn();

http://git-wip-us.apache.org/repos/asf/hive/blob/d78fea10/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
index 59114fe..4debd04 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
@@ -1107,8 +1107,8 @@ public class TestTxnHandler {
         assertNull(lock.getPartname());
         assertEquals(LockState.ACQUIRED, lock.getState());
         assertEquals(LockType.EXCLUSIVE, lock.getType());
-        assertTrue(begining <= lock.getLastheartbeat() &&
-            System.currentTimeMillis() >= lock.getLastheartbeat());
+        assertTrue(lock.toString(), 0 == lock.getLastheartbeat() &&
+            lock.getTxnid() != 0);
         assertTrue("Expected acquired at " + lock.getAcquiredat() + " to be between " + begining
             + " and " + System.currentTimeMillis(),
             begining <= lock.getAcquiredat() && System.currentTimeMillis() >= lock.getAcquiredat());
@@ -1122,8 +1122,8 @@ public class TestTxnHandler {
         assertNull(lock.getPartname());
         assertEquals(LockState.WAITING, lock.getState());
         assertEquals(LockType.SHARED_READ, lock.getType());
-        assertTrue(begining <= lock.getLastheartbeat() &&
-            System.currentTimeMillis() >= lock.getLastheartbeat());
+        assertTrue(lock.toString(), 0 == lock.getLastheartbeat() &&
+          lock.getTxnid() != 0);
         assertEquals(0, lock.getAcquiredat());
         assertEquals("me", lock.getUser());
         assertEquals("localhost", lock.getHostname());
@@ -1135,7 +1135,7 @@ public class TestTxnHandler {
         assertEquals("yourpartition", lock.getPartname());
         assertEquals(LockState.ACQUIRED, lock.getState());
         assertEquals(LockType.SHARED_WRITE, lock.getType());
-        assertTrue(begining <= lock.getLastheartbeat() &&
+        assertTrue(lock.toString(), begining <= lock.getLastheartbeat() &&
             System.currentTimeMillis() >= lock.getLastheartbeat());
         assertTrue(begining <= lock.getAcquiredat() &&
             System.currentTimeMillis() >= lock.getAcquiredat());

http://git-wip-us.apache.org/repos/asf/hive/blob/d78fea10/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 9a5495b..ceaafe1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -1046,7 +1046,7 @@ public class Driver implements CommandProcessor {
         // don't update it after that until txn completes.  Thus the check for {@code initiatingTransaction}
         //For autoCommit=true, Read-only statements, txn is implicit, i.e. lock in the snapshot
         //for each statement.
-        recordValidTxns();
+        recordValidTxns();//todo: we should only need to do this for RO query if it has ACID resources in it.
       }
 
       return 0;


[12/43] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 9873810..5c72a27 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -183,6 +183,15 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_table_meta(self, db_patterns, tbl_patterns, tbl_types):
+    """
+    Parameters:
+     - db_patterns
+     - tbl_patterns
+     - tbl_types
+    """
+    pass
+
   def get_all_tables(self, db_name):
     """
     Parameters:
@@ -1830,6 +1839,43 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o1
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_tables failed: unknown result")
 
+  def get_table_meta(self, db_patterns, tbl_patterns, tbl_types):
+    """
+    Parameters:
+     - db_patterns
+     - tbl_patterns
+     - tbl_types
+    """
+    self.send_get_table_meta(db_patterns, tbl_patterns, tbl_types)
+    return self.recv_get_table_meta()
+
+  def send_get_table_meta(self, db_patterns, tbl_patterns, tbl_types):
+    self._oprot.writeMessageBegin('get_table_meta', TMessageType.CALL, self._seqid)
+    args = get_table_meta_args()
+    args.db_patterns = db_patterns
+    args.tbl_patterns = tbl_patterns
+    args.tbl_types = tbl_types
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_table_meta(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_table_meta_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_table_meta failed: unknown result")
+
   def get_all_tables(self, db_name):
     """
     Parameters:
@@ -5852,6 +5898,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["drop_table"] = Processor.process_drop_table
     self._processMap["drop_table_with_environment_context"] = Processor.process_drop_table_with_environment_context
     self._processMap["get_tables"] = Processor.process_get_tables
+    self._processMap["get_table_meta"] = Processor.process_get_table_meta
     self._processMap["get_all_tables"] = Processor.process_get_all_tables
     self._processMap["get_table"] = Processor.process_get_table
     self._processMap["get_table_objects_by_name"] = Processor.process_get_table_objects_by_name
@@ -6517,6 +6564,28 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_table_meta(self, seqid, iprot, oprot):
+    args = get_table_meta_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_table_meta_result()
+    try:
+      result.success = self._handler.get_table_meta(args.db_patterns, args.tbl_patterns, args.tbl_types)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except MetaException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_table_meta", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_get_all_tables(self, seqid, iprot, oprot):
     args = get_all_tables_args()
     args.read(iprot)
@@ -12752,6 +12821,192 @@ class get_tables_result:
   def __ne__(self, other):
     return not (self == other)
 
+class get_table_meta_args:
+  """
+  Attributes:
+   - db_patterns
+   - tbl_patterns
+   - tbl_types
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'db_patterns', None, None, ), # 1
+    (2, TType.STRING, 'tbl_patterns', None, None, ), # 2
+    (3, TType.LIST, 'tbl_types', (TType.STRING,None), None, ), # 3
+  )
+
+  def __init__(self, db_patterns=None, tbl_patterns=None, tbl_types=None,):
+    self.db_patterns = db_patterns
+    self.tbl_patterns = tbl_patterns
+    self.tbl_types = tbl_types
+
+  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_patterns = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.tbl_patterns = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.LIST:
+          self.tbl_types = []
+          (_etype583, _size580) = iprot.readListBegin()
+          for _i584 in xrange(_size580):
+            _elem585 = iprot.readString()
+            self.tbl_types.append(_elem585)
+          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('get_table_meta_args')
+    if self.db_patterns is not None:
+      oprot.writeFieldBegin('db_patterns', TType.STRING, 1)
+      oprot.writeString(self.db_patterns)
+      oprot.writeFieldEnd()
+    if self.tbl_patterns is not None:
+      oprot.writeFieldBegin('tbl_patterns', TType.STRING, 2)
+      oprot.writeString(self.tbl_patterns)
+      oprot.writeFieldEnd()
+    if self.tbl_types is not None:
+      oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
+      oprot.writeListBegin(TType.STRING, len(self.tbl_types))
+      for iter586 in self.tbl_types:
+        oprot.writeString(iter586)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.db_patterns)
+    value = (value * 31) ^ hash(self.tbl_patterns)
+    value = (value * 31) ^ hash(self.tbl_types)
+    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_table_meta_result:
+  """
+  Attributes:
+   - success
+   - o1
+  """
+
+  thrift_spec = (
+    (0, TType.LIST, 'success', (TType.STRUCT,(TableMeta, TableMeta.thrift_spec)), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, o1=None,):
+    self.success = success
+    self.o1 = o1
+
+  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 = []
+          (_etype590, _size587) = iprot.readListBegin()
+          for _i591 in xrange(_size587):
+            _elem592 = TableMeta()
+            _elem592.read(iprot)
+            self.success.append(_elem592)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.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_table_meta_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.LIST, 0)
+      oprot.writeListBegin(TType.STRUCT, len(self.success))
+      for iter593 in self.success:
+        iter593.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.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)
+    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_all_tables_args:
   """
   Attributes:
@@ -12845,10 +13100,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype583, _size580) = iprot.readListBegin()
-          for _i584 in xrange(_size580):
-            _elem585 = iprot.readString()
-            self.success.append(_elem585)
+          (_etype597, _size594) = iprot.readListBegin()
+          for _i598 in xrange(_size594):
+            _elem599 = iprot.readString()
+            self.success.append(_elem599)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12871,8 +13126,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 iter586 in self.success:
-        oprot.writeString(iter586)
+      for iter600 in self.success:
+        oprot.writeString(iter600)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13108,10 +13363,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype590, _size587) = iprot.readListBegin()
-          for _i591 in xrange(_size587):
-            _elem592 = iprot.readString()
-            self.tbl_names.append(_elem592)
+          (_etype604, _size601) = iprot.readListBegin()
+          for _i605 in xrange(_size601):
+            _elem606 = iprot.readString()
+            self.tbl_names.append(_elem606)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13132,8 +13387,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 iter593 in self.tbl_names:
-        oprot.writeString(iter593)
+      for iter607 in self.tbl_names:
+        oprot.writeString(iter607)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13194,11 +13449,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype597, _size594) = iprot.readListBegin()
-          for _i598 in xrange(_size594):
-            _elem599 = Table()
-            _elem599.read(iprot)
-            self.success.append(_elem599)
+          (_etype611, _size608) = iprot.readListBegin()
+          for _i612 in xrange(_size608):
+            _elem613 = Table()
+            _elem613.read(iprot)
+            self.success.append(_elem613)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13233,8 +13488,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 iter600 in self.success:
-        iter600.write(oprot)
+      for iter614 in self.success:
+        iter614.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13400,10 +13655,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype604, _size601) = iprot.readListBegin()
-          for _i605 in xrange(_size601):
-            _elem606 = iprot.readString()
-            self.success.append(_elem606)
+          (_etype618, _size615) = iprot.readListBegin()
+          for _i619 in xrange(_size615):
+            _elem620 = iprot.readString()
+            self.success.append(_elem620)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13438,8 +13693,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 iter607 in self.success:
-        oprot.writeString(iter607)
+      for iter621 in self.success:
+        oprot.writeString(iter621)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14409,11 +14664,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype611, _size608) = iprot.readListBegin()
-          for _i612 in xrange(_size608):
-            _elem613 = Partition()
-            _elem613.read(iprot)
-            self.new_parts.append(_elem613)
+          (_etype625, _size622) = iprot.readListBegin()
+          for _i626 in xrange(_size622):
+            _elem627 = Partition()
+            _elem627.read(iprot)
+            self.new_parts.append(_elem627)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14430,8 +14685,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 iter614 in self.new_parts:
-        iter614.write(oprot)
+      for iter628 in self.new_parts:
+        iter628.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14589,11 +14844,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype618, _size615) = iprot.readListBegin()
-          for _i619 in xrange(_size615):
-            _elem620 = PartitionSpec()
-            _elem620.read(iprot)
-            self.new_parts.append(_elem620)
+          (_etype632, _size629) = iprot.readListBegin()
+          for _i633 in xrange(_size629):
+            _elem634 = PartitionSpec()
+            _elem634.read(iprot)
+            self.new_parts.append(_elem634)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14610,8 +14865,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 iter621 in self.new_parts:
-        iter621.write(oprot)
+      for iter635 in self.new_parts:
+        iter635.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14785,10 +15040,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype625, _size622) = iprot.readListBegin()
-          for _i626 in xrange(_size622):
-            _elem627 = iprot.readString()
-            self.part_vals.append(_elem627)
+          (_etype639, _size636) = iprot.readListBegin()
+          for _i640 in xrange(_size636):
+            _elem641 = iprot.readString()
+            self.part_vals.append(_elem641)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14813,8 +15068,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 iter628 in self.part_vals:
-        oprot.writeString(iter628)
+      for iter642 in self.part_vals:
+        oprot.writeString(iter642)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15167,10 +15422,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype632, _size629) = iprot.readListBegin()
-          for _i633 in xrange(_size629):
-            _elem634 = iprot.readString()
-            self.part_vals.append(_elem634)
+          (_etype646, _size643) = iprot.readListBegin()
+          for _i647 in xrange(_size643):
+            _elem648 = iprot.readString()
+            self.part_vals.append(_elem648)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15201,8 +15456,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 iter635 in self.part_vals:
-        oprot.writeString(iter635)
+      for iter649 in self.part_vals:
+        oprot.writeString(iter649)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -15797,10 +16052,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype639, _size636) = iprot.readListBegin()
-          for _i640 in xrange(_size636):
-            _elem641 = iprot.readString()
-            self.part_vals.append(_elem641)
+          (_etype653, _size650) = iprot.readListBegin()
+          for _i654 in xrange(_size650):
+            _elem655 = iprot.readString()
+            self.part_vals.append(_elem655)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15830,8 +16085,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 iter642 in self.part_vals:
-        oprot.writeString(iter642)
+      for iter656 in self.part_vals:
+        oprot.writeString(iter656)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -16004,10 +16259,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype646, _size643) = iprot.readListBegin()
-          for _i647 in xrange(_size643):
-            _elem648 = iprot.readString()
-            self.part_vals.append(_elem648)
+          (_etype660, _size657) = iprot.readListBegin()
+          for _i661 in xrange(_size657):
+            _elem662 = iprot.readString()
+            self.part_vals.append(_elem662)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16043,8 +16298,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 iter649 in self.part_vals:
-        oprot.writeString(iter649)
+      for iter663 in self.part_vals:
+        oprot.writeString(iter663)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -16781,10 +17036,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype653, _size650) = iprot.readListBegin()
-          for _i654 in xrange(_size650):
-            _elem655 = iprot.readString()
-            self.part_vals.append(_elem655)
+          (_etype667, _size664) = iprot.readListBegin()
+          for _i668 in xrange(_size664):
+            _elem669 = iprot.readString()
+            self.part_vals.append(_elem669)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16809,8 +17064,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 iter656 in self.part_vals:
-        oprot.writeString(iter656)
+      for iter670 in self.part_vals:
+        oprot.writeString(iter670)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16969,11 +17224,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype658, _vtype659, _size657 ) = iprot.readMapBegin()
-          for _i661 in xrange(_size657):
-            _key662 = iprot.readString()
-            _val663 = iprot.readString()
-            self.partitionSpecs[_key662] = _val663
+          (_ktype672, _vtype673, _size671 ) = iprot.readMapBegin()
+          for _i675 in xrange(_size671):
+            _key676 = iprot.readString()
+            _val677 = iprot.readString()
+            self.partitionSpecs[_key676] = _val677
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17010,9 +17265,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 kiter664,viter665 in self.partitionSpecs.items():
-        oprot.writeString(kiter664)
-        oprot.writeString(viter665)
+      for kiter678,viter679 in self.partitionSpecs.items():
+        oprot.writeString(kiter678)
+        oprot.writeString(viter679)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -17217,11 +17472,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype667, _vtype668, _size666 ) = iprot.readMapBegin()
-          for _i670 in xrange(_size666):
-            _key671 = iprot.readString()
-            _val672 = iprot.readString()
-            self.partitionSpecs[_key671] = _val672
+          (_ktype681, _vtype682, _size680 ) = iprot.readMapBegin()
+          for _i684 in xrange(_size680):
+            _key685 = iprot.readString()
+            _val686 = iprot.readString()
+            self.partitionSpecs[_key685] = _val686
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17258,9 +17513,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 kiter673,viter674 in self.partitionSpecs.items():
-        oprot.writeString(kiter673)
-        oprot.writeString(viter674)
+      for kiter687,viter688 in self.partitionSpecs.items():
+        oprot.writeString(kiter687)
+        oprot.writeString(viter688)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -17343,11 +17598,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype678, _size675) = iprot.readListBegin()
-          for _i679 in xrange(_size675):
-            _elem680 = Partition()
-            _elem680.read(iprot)
-            self.success.append(_elem680)
+          (_etype692, _size689) = iprot.readListBegin()
+          for _i693 in xrange(_size689):
+            _elem694 = Partition()
+            _elem694.read(iprot)
+            self.success.append(_elem694)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17388,8 +17643,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 iter681 in self.success:
-        iter681.write(oprot)
+      for iter695 in self.success:
+        iter695.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17483,10 +17738,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype685, _size682) = iprot.readListBegin()
-          for _i686 in xrange(_size682):
-            _elem687 = iprot.readString()
-            self.part_vals.append(_elem687)
+          (_etype699, _size696) = iprot.readListBegin()
+          for _i700 in xrange(_size696):
+            _elem701 = iprot.readString()
+            self.part_vals.append(_elem701)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17498,10 +17753,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype691, _size688) = iprot.readListBegin()
-          for _i692 in xrange(_size688):
-            _elem693 = iprot.readString()
-            self.group_names.append(_elem693)
+          (_etype705, _size702) = iprot.readListBegin()
+          for _i706 in xrange(_size702):
+            _elem707 = iprot.readString()
+            self.group_names.append(_elem707)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17526,8 +17781,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 iter694 in self.part_vals:
-        oprot.writeString(iter694)
+      for iter708 in self.part_vals:
+        oprot.writeString(iter708)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -17537,8 +17792,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 iter695 in self.group_names:
-        oprot.writeString(iter695)
+      for iter709 in self.group_names:
+        oprot.writeString(iter709)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17967,11 +18222,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype699, _size696) = iprot.readListBegin()
-          for _i700 in xrange(_size696):
-            _elem701 = Partition()
-            _elem701.read(iprot)
-            self.success.append(_elem701)
+          (_etype713, _size710) = iprot.readListBegin()
+          for _i714 in xrange(_size710):
+            _elem715 = Partition()
+            _elem715.read(iprot)
+            self.success.append(_elem715)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18000,8 +18255,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 iter702 in self.success:
-        iter702.write(oprot)
+      for iter716 in self.success:
+        iter716.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18095,10 +18350,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype706, _size703) = iprot.readListBegin()
-          for _i707 in xrange(_size703):
-            _elem708 = iprot.readString()
-            self.group_names.append(_elem708)
+          (_etype720, _size717) = iprot.readListBegin()
+          for _i721 in xrange(_size717):
+            _elem722 = iprot.readString()
+            self.group_names.append(_elem722)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18131,8 +18386,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 iter709 in self.group_names:
-        oprot.writeString(iter709)
+      for iter723 in self.group_names:
+        oprot.writeString(iter723)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18193,11 +18448,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype713, _size710) = iprot.readListBegin()
-          for _i714 in xrange(_size710):
-            _elem715 = Partition()
-            _elem715.read(iprot)
-            self.success.append(_elem715)
+          (_etype727, _size724) = iprot.readListBegin()
+          for _i728 in xrange(_size724):
+            _elem729 = Partition()
+            _elem729.read(iprot)
+            self.success.append(_elem729)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18226,8 +18481,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 iter716 in self.success:
-        iter716.write(oprot)
+      for iter730 in self.success:
+        iter730.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18385,11 +18640,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype720, _size717) = iprot.readListBegin()
-          for _i721 in xrange(_size717):
-            _elem722 = PartitionSpec()
-            _elem722.read(iprot)
-            self.success.append(_elem722)
+          (_etype734, _size731) = iprot.readListBegin()
+          for _i735 in xrange(_size731):
+            _elem736 = PartitionSpec()
+            _elem736.read(iprot)
+            self.success.append(_elem736)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18418,8 +18673,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 iter723 in self.success:
-        iter723.write(oprot)
+      for iter737 in self.success:
+        iter737.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18574,10 +18829,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype727, _size724) = iprot.readListBegin()
-          for _i728 in xrange(_size724):
-            _elem729 = iprot.readString()
-            self.success.append(_elem729)
+          (_etype741, _size738) = iprot.readListBegin()
+          for _i742 in xrange(_size738):
+            _elem743 = iprot.readString()
+            self.success.append(_elem743)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18600,8 +18855,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 iter730 in self.success:
-        oprot.writeString(iter730)
+      for iter744 in self.success:
+        oprot.writeString(iter744)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -18677,10 +18932,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype734, _size731) = iprot.readListBegin()
-          for _i735 in xrange(_size731):
-            _elem736 = iprot.readString()
-            self.part_vals.append(_elem736)
+          (_etype748, _size745) = iprot.readListBegin()
+          for _i749 in xrange(_size745):
+            _elem750 = iprot.readString()
+            self.part_vals.append(_elem750)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18710,8 +18965,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 iter737 in self.part_vals:
-        oprot.writeString(iter737)
+      for iter751 in self.part_vals:
+        oprot.writeString(iter751)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -18775,11 +19030,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype741, _size738) = iprot.readListBegin()
-          for _i742 in xrange(_size738):
-            _elem743 = Partition()
-            _elem743.read(iprot)
-            self.success.append(_elem743)
+          (_etype755, _size752) = iprot.readListBegin()
+          for _i756 in xrange(_size752):
+            _elem757 = Partition()
+            _elem757.read(iprot)
+            self.success.append(_elem757)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18808,8 +19063,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 iter744 in self.success:
-        iter744.write(oprot)
+      for iter758 in self.success:
+        iter758.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18896,10 +19151,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype748, _size745) = iprot.readListBegin()
-          for _i749 in xrange(_size745):
-            _elem750 = iprot.readString()
-            self.part_vals.append(_elem750)
+          (_etype762, _size759) = iprot.readListBegin()
+          for _i763 in xrange(_size759):
+            _elem764 = iprot.readString()
+            self.part_vals.append(_elem764)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18916,10 +19171,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype754, _size751) = iprot.readListBegin()
-          for _i755 in xrange(_size751):
-            _elem756 = iprot.readString()
-            self.group_names.append(_elem756)
+          (_etype768, _size765) = iprot.readListBegin()
+          for _i769 in xrange(_size765):
+            _elem770 = iprot.readString()
+            self.group_names.append(_elem770)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18944,8 +19199,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 iter757 in self.part_vals:
-        oprot.writeString(iter757)
+      for iter771 in self.part_vals:
+        oprot.writeString(iter771)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -18959,8 +19214,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 iter758 in self.group_names:
-        oprot.writeString(iter758)
+      for iter772 in self.group_names:
+        oprot.writeString(iter772)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19022,11 +19277,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype762, _size759) = iprot.readListBegin()
-          for _i763 in xrange(_size759):
-            _elem764 = Partition()
-            _elem764.read(iprot)
-            self.success.append(_elem764)
+          (_etype776, _size773) = iprot.readListBegin()
+          for _i777 in xrange(_size773):
+            _elem778 = Partition()
+            _elem778.read(iprot)
+            self.success.append(_elem778)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19055,8 +19310,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 iter765 in self.success:
-        iter765.write(oprot)
+      for iter779 in self.success:
+        iter779.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19137,10 +19392,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype769, _size766) = iprot.readListBegin()
-          for _i770 in xrange(_size766):
-            _elem771 = iprot.readString()
-            self.part_vals.append(_elem771)
+          (_etype783, _size780) = iprot.readListBegin()
+          for _i784 in xrange(_size780):
+            _elem785 = iprot.readString()
+            self.part_vals.append(_elem785)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19170,8 +19425,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 iter772 in self.part_vals:
-        oprot.writeString(iter772)
+      for iter786 in self.part_vals:
+        oprot.writeString(iter786)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -19235,10 +19490,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype776, _size773) = iprot.readListBegin()
-          for _i777 in xrange(_size773):
-            _elem778 = iprot.readString()
-            self.success.append(_elem778)
+          (_etype790, _size787) = iprot.readListBegin()
+          for _i791 in xrange(_size787):
+            _elem792 = iprot.readString()
+            self.success.append(_elem792)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19267,8 +19522,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 iter779 in self.success:
-        oprot.writeString(iter779)
+      for iter793 in self.success:
+        oprot.writeString(iter793)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19439,11 +19694,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype783, _size780) = iprot.readListBegin()
-          for _i784 in xrange(_size780):
-            _elem785 = Partition()
-            _elem785.read(iprot)
-            self.success.append(_elem785)
+          (_etype797, _size794) = iprot.readListBegin()
+          for _i798 in xrange(_size794):
+            _elem799 = Partition()
+            _elem799.read(iprot)
+            self.success.append(_elem799)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19472,8 +19727,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 iter786 in self.success:
-        iter786.write(oprot)
+      for iter800 in self.success:
+        iter800.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19644,11 +19899,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype790, _size787) = iprot.readListBegin()
-          for _i791 in xrange(_size787):
-            _elem792 = PartitionSpec()
-            _elem792.read(iprot)
-            self.success.append(_elem792)
+          (_etype804, _size801) = iprot.readListBegin()
+          for _i805 in xrange(_size801):
+            _elem806 = PartitionSpec()
+            _elem806.read(iprot)
+            self.success.append(_elem806)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19677,8 +19932,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 iter793 in self.success:
-        iter793.write(oprot)
+      for iter807 in self.success:
+        iter807.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19915,10 +20170,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype797, _size794) = iprot.readListBegin()
-          for _i798 in xrange(_size794):
-            _elem799 = iprot.readString()
-            self.names.append(_elem799)
+          (_etype811, _size808) = iprot.readListBegin()
+          for _i812 in xrange(_size808):
+            _elem813 = iprot.readString()
+            self.names.append(_elem813)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19943,8 +20198,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 iter800 in self.names:
-        oprot.writeString(iter800)
+      for iter814 in self.names:
+        oprot.writeString(iter814)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20003,11 +20258,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype804, _size801) = iprot.readListBegin()
-          for _i805 in xrange(_size801):
-            _elem806 = Partition()
-            _elem806.read(iprot)
-            self.success.append(_elem806)
+          (_etype818, _size815) = iprot.readListBegin()
+          for _i819 in xrange(_size815):
+            _elem820 = Partition()
+            _elem820.read(iprot)
+            self.success.append(_elem820)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20036,8 +20291,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 iter807 in self.success:
-        iter807.write(oprot)
+      for iter821 in self.success:
+        iter821.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20287,11 +20542,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype811, _size808) = iprot.readListBegin()
-          for _i812 in xrange(_size808):
-            _elem813 = Partition()
-            _elem813.read(iprot)
-            self.new_parts.append(_elem813)
+          (_etype825, _size822) = iprot.readListBegin()
+          for _i826 in xrange(_size822):
+            _elem827 = Partition()
+            _elem827.read(iprot)
+            self.new_parts.append(_elem827)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20316,8 +20571,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 iter814 in self.new_parts:
-        iter814.write(oprot)
+      for iter828 in self.new_parts:
+        iter828.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20656,10 +20911,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype818, _size815) = iprot.readListBegin()
-          for _i819 in xrange(_size815):
-            _elem820 = iprot.readString()
-            self.part_vals.append(_elem820)
+          (_etype832, _size829) = iprot.readListBegin()
+          for _i833 in xrange(_size829):
+            _elem834 = iprot.readString()
+            self.part_vals.append(_elem834)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20690,8 +20945,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 iter821 in self.part_vals:
-        oprot.writeString(iter821)
+      for iter835 in self.part_vals:
+        oprot.writeString(iter835)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -20833,10 +21088,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype825, _size822) = iprot.readListBegin()
-          for _i826 in xrange(_size822):
-            _elem827 = iprot.readString()
-            self.part_vals.append(_elem827)
+          (_etype839, _size836) = iprot.readListBegin()
+          for _i840 in xrange(_size836):
+            _elem841 = iprot.readString()
+            self.part_vals.append(_elem841)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20858,8 +21113,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 iter828 in self.part_vals:
-        oprot.writeString(iter828)
+      for iter842 in self.part_vals:
+        oprot.writeString(iter842)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -21217,10 +21472,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype832, _size829) = iprot.readListBegin()
-          for _i833 in xrange(_size829):
-            _elem834 = iprot.readString()
-            self.success.append(_elem834)
+          (_etype846, _size843) = iprot.readListBegin()
+          for _i847 in xrange(_size843):
+            _elem848 = iprot.readString()
+            self.success.append(_elem848)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21243,8 +21498,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 iter835 in self.success:
-        oprot.writeString(iter835)
+      for iter849 in self.success:
+        oprot.writeString(iter849)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21368,11 +21623,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype837, _vtype838, _size836 ) = iprot.readMapBegin()
-          for _i840 in xrange(_size836):
-            _key841 = iprot.readString()
-            _val842 = iprot.readString()
-            self.success[_key841] = _val842
+          (_ktype851, _vtype852, _size850 ) = iprot.readMapBegin()
+          for _i854 in xrange(_size850):
+            _key855 = iprot.readString()
+            _val856 = iprot.readString()
+            self.success[_key855] = _val856
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21395,9 +21650,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 kiter843,viter844 in self.success.items():
-        oprot.writeString(kiter843)
-        oprot.writeString(viter844)
+      for kiter857,viter858 in self.success.items():
+        oprot.writeString(kiter857)
+        oprot.writeString(viter858)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21473,11 +21728,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype846, _vtype847, _size845 ) = iprot.readMapBegin()
-          for _i849 in xrange(_size845):
-            _key850 = iprot.readString()
-            _val851 = iprot.readString()
-            self.part_vals[_key850] = _val851
+          (_ktype860, _vtype861, _size859 ) = iprot.readMapBegin()
+          for _i863 in xrange(_size859):
+            _key864 = iprot.readString()
+            _val865 = iprot.readString()
+            self.part_vals[_key864] = _val865
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21507,9 +21762,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 kiter852,viter853 in self.part_vals.items():
-        oprot.writeString(kiter852)
-        oprot.writeString(viter853)
+      for kiter866,viter867 in self.part_vals.items():
+        oprot.writeString(kiter866)
+        oprot.writeString(viter867)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -21723,11 +21978,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype855, _vtype856, _size854 ) = iprot.readMapBegin()
-          for _i858 in xrange(_size854):
-            _key859 = iprot.readString()
-            _val860 = iprot.readString()
-            self.part_vals[_key859] = _val860
+          (_ktype869, _vtype870, _size868 ) = iprot.readMapBegin()
+          for _i872 in xrange(_size868):
+            _key873 = iprot.readString()
+            _val874 = iprot.readString()
+            self.part_vals[_key873] = _val874
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21757,9 +22012,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 kiter861,viter862 in self.part_vals.items():
-        oprot.writeString(kiter861)
-        oprot.writeString(viter862)
+      for kiter875,viter876 in self.part_vals.items():
+        oprot.writeString(kiter875)
+        oprot.writeString(viter876)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -22814,11 +23069,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype866, _size863) = iprot.readListBegin()
-          for _i867 in xrange(_size863):
-            _elem868 = Index()
-            _elem868.read(iprot)
-            self.success.append(_elem868)
+          (_etype880, _size877) = iprot.readListBegin()
+          for _i881 in xrange(_size877):
+            _elem882 = Index()
+            _elem882.read(iprot)
+            self.success.append(_elem882)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22847,8 +23102,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 iter869 in self.success:
-        iter869.write(oprot)
+      for iter883 in self.success:
+        iter883.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23003,10 +23258,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype873, _size870) = iprot.readListBegin()
-          for _i874 in xrange(_size870):
-            _elem875 = iprot.readString()
-            self.success.append(_elem875)
+          (_etype887, _size884) = iprot.readListBegin()
+          for _i888 in xrange(_size884):
+            _elem889 = iprot.readString()
+            self.success.append(_elem889)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23029,8 +23284,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 iter876 in self.success:
-        oprot.writeString(iter876)
+      for iter890 in self.success:
+        oprot.writeString(iter890)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -25578,10 +25833,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype880, _size877) = iprot.readListBegin()
-          for _i881 in xrange(_size877):
-            _elem882 = iprot.readString()
-            self.success.append(_elem882)
+          (_etype894, _size891) = iprot.readListBegin()
+          for _i895 in xrange(_size891):
+            _elem896 = iprot.readString()
+            self.success.append(_elem896)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25604,8 +25859,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 iter883 in self.success:
-        oprot.writeString(iter883)
+      for iter897 in self.success:
+        oprot.writeString(iter897)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26293,10 +26548,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype887, _size884) = iprot.readListBegin()
-          for _i888 in xrange(_size884):
-            _elem889 = iprot.readString()
-            self.success.append(_elem889)
+          (_etype901, _size898) = iprot.readListBegin()
+          for _i902 in xrange(_size898):
+            _elem903 = iprot.readString()
+            self.success.append(_elem903)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26319,8 +26574,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 iter890 in self.success:
-        oprot.writeString(iter890)
+      for iter904 in self.success:
+        oprot.writeString(iter904)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26834,11 +27089,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype894, _size891) = iprot.readListBegin()
-          for _i895 in xrange(_size891):
-            _elem896 = Role()
-            _elem896.read(iprot)
-            self.success.append(_elem896)
+          (_etype908, _size905) = iprot.readListBegin()
+          for _i909 in xrange(_size905):
+            _elem910 = Role()
+            _elem910.read(iprot)
+            self.success.append(_elem910)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26861,8 +27116,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 iter897 in self.success:
-        iter897.write(oprot)
+      for iter911 in self.success:
+        iter911.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27371,10 +27626,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype901, _size898) = iprot.readListBegin()
-          for _i902 in xrange(_size898):
-            _elem903 = iprot.readString()
-            self.group_names.append(_elem903)
+          (_etype915, _size912) = iprot.readListBegin()
+          for _i916 in xrange(_size912):
+            _elem917 = iprot.readString()
+            self.group_names.append(_elem917)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27399,8 +27654,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 iter904 in self.group_names:
-        oprot.writeString(iter904)
+      for iter918 in self.group_names:
+        oprot.writeString(iter918)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27627,11 +27882,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype908, _size905) = iprot.readListBegin()
-          for _i909 in xrange(_size905):
-            _elem910 = HiveObjectPrivilege()
-            _elem910.read(iprot)
-            self.success.append(_elem910)
+          (_etype922, _size919) = iprot.readListBegin()
+          for _i923 in xrange(_size919):
+            _elem924 = HiveObjectPrivilege()
+            _elem924.read(iprot)
+            self.success.append(_elem924)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27654,8 +27909,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 iter911 in self.success:
-        iter911.write(oprot)
+      for iter925 in self.success:
+        iter925.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28153,10 +28408,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype915, _size912) = iprot.readListBegin()
-          for _i916 in xrange(_size912):
-            _elem917 = iprot.readString()
-            self.group_names.append(_elem917)
+          (_etype929, _size926) = iprot.readListBegin()
+          for _i930 in xrange(_size926):
+            _elem931 = iprot.readString()
+            self.group_names.append(_elem931)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28177,8 +28432,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 iter918 in self.group_names:
-        oprot.writeString(iter918)
+      for iter932 in self.group_names:
+        oprot.writeString(iter932)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -28233,10 +28488,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype922, _size919) = iprot.readListBegin()
-          for _i923 in xrange(_size919):
-            _elem924 = iprot.readString()
-            self.success.append(_elem924)
+          (_etype936, _size933) = iprot.readListBegin()
+          for _i937 in xrange(_size933):
+            _elem938 = iprot.readString()
+            self.success.append(_elem938)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28259,8 +28514,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 iter925 in self.success:
-        oprot.writeString(iter925)
+      for iter939 in self.success:
+        oprot.writeString(iter939)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 8940dff..ba525ed 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -10554,6 +10554,116 @@ class GetAllFunctionsResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class TableMeta:
+  """
+  Attributes:
+   - dbName
+   - tableName
+   - tableType
+   - comments
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'dbName', None, None, ), # 1
+    (2, TType.STRING, 'tableName', None, None, ), # 2
+    (3, TType.STRING, 'tableType', None, None, ), # 3
+    (4, TType.STRING, 'comments', None, None, ), # 4
+  )
+
+  def __init__(self, dbName=None, tableName=None, tableType=None, comments=None,):
+    self.dbName = dbName
+    self.tableName = tableName
+    self.tableType = tableType
+    self.comments = comments
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.tableName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.tableType = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.comments = 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('TableMeta')
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 1)
+      oprot.writeString(self.dbName)
+      oprot.writeFieldEnd()
+    if self.tableName is not None:
+      oprot.writeFieldBegin('tableName', TType.STRING, 2)
+      oprot.writeString(self.tableName)
+      oprot.writeFieldEnd()
+    if self.tableType is not None:
+      oprot.writeFieldBegin('tableType', TType.STRING, 3)
+      oprot.writeString(self.tableType)
+      oprot.writeFieldEnd()
+    if self.comments is not None:
+      oprot.writeFieldBegin('comments', TType.STRING, 4)
+      oprot.writeString(self.comments)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.dbName is None:
+      raise TProtocol.TProtocolException(message='Required field dbName is unset!')
+    if self.tableName is None:
+      raise TProtocol.TProtocolException(message='Required field tableName is unset!')
+    if self.tableType is None:
+      raise TProtocol.TProtocolException(message='Required field tableType is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.tableName)
+    value = (value * 31) ^ hash(self.tableType)
+    value = (value * 31) ^ hash(self.comments)
+    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 MetaException(TException):
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 08b9b06..f943f2d 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2422,6 +2422,31 @@ class GetAllFunctionsResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class TableMeta
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DBNAME = 1
+  TABLENAME = 2
+  TABLETYPE = 3
+  COMMENTS = 4
+
+  FIELDS = {
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
+    TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
+    TABLETYPE => {:type => ::Thrift::Types::STRING, :name => 'tableType'},
+    COMMENTS => {:type => ::Thrift::Types::STRING, :name => 'comments', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbName is unset!') unless @dbName
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tableName is unset!') unless @tableName
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tableType is unset!') unless @tableType
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class MetaException < ::Thrift::Exception
   include ::Thrift::Struct, ::Thrift::Struct_Union
   def initialize(message=nil)

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 c613e4b..5fe54b5 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -366,6 +366,22 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_tables failed: unknown result')
     end
 
+    def get_table_meta(db_patterns, tbl_patterns, tbl_types)
+      send_get_table_meta(db_patterns, tbl_patterns, tbl_types)
+      return recv_get_table_meta()
+    end
+
+    def send_get_table_meta(db_patterns, tbl_patterns, tbl_types)
+      send_message('get_table_meta', Get_table_meta_args, :db_patterns => db_patterns, :tbl_patterns => tbl_patterns, :tbl_types => tbl_types)
+    end
+
+    def recv_get_table_meta()
+      result = receive_message(Get_table_meta_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_table_meta failed: unknown result')
+    end
+
     def get_all_tables(db_name)
       send_get_all_tables(db_name)
       return recv_get_all_tables()
@@ -2471,6 +2487,17 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'get_tables', seqid)
     end
 
+    def process_get_table_meta(seqid, iprot, oprot)
+      args = read_args(iprot, Get_table_meta_args)
+      result = Get_table_meta_result.new()
+      begin
+        result.success = @handler.get_table_meta(args.db_patterns, args.tbl_patterns, args.tbl_types)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'get_table_meta', seqid)
+    end
+
     def process_get_all_tables(seqid, iprot, oprot)
       args = read_args(iprot, Get_all_tables_args)
       result = Get_all_tables_result.new()
@@ -4613,6 +4640,44 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_table_meta_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    DB_PATTERNS = 1
+    TBL_PATTERNS = 2
+    TBL_TYPES = 3
+
+    FIELDS = {
+      DB_PATTERNS => {:type => ::Thrift::Types::STRING, :name => 'db_patterns'},
+      TBL_PATTERNS => {:type => ::Thrift::Types::STRING, :name => 'tbl_patterns'},
+      TBL_TYPES => {:type => ::Thrift::Types::LIST, :name => 'tbl_types', :element => {:type => ::Thrift::Types::STRING}}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_table_meta_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::LIST, :name => 'success', :element => {:type => ::Thrift::Types::STRUCT, :class => ::TableMeta}},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Get_all_tables_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     DB_NAME = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 2e9afaf..3c40d6e 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -110,7 +110,6 @@ import org.apache.thrift.transport.TTransportFactory;
 import javax.jdo.JDOException;
 
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.nio.ByteBuffer;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
@@ -1710,6 +1709,23 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return t;
     }
 
+    @Override
+    public List<TableMeta> get_table_meta(String dbnames, String tblNames, List<String> tblTypes)
+        throws MetaException, NoSuchObjectException {
+      List<TableMeta> t = null;
+      startTableFunction("get_table_metas", dbnames, tblNames);
+      Exception ex = null;
+      try {
+        t = getMS().getTableMeta(dbnames, tblNames, tblTypes);
+      } catch (Exception e) {
+        ex = e;
+        throw newMetaException(e);
+      } finally {
+        endFunction("get_table_metas", t != null, ex);
+      }
+      return t;
+    }
+
     /**
      * Equivalent of get_table, but does not log audits and fire pre-event listener.
      * Meant to be used for calls made by other hive classes, that are not using the
@@ -5249,6 +5265,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     private static MetaException newMetaException(Exception e) {
+      if (e instanceof MetaException) {
+        return (MetaException)e;
+      }
       MetaException me = new MetaException(e.toString());
       me.initCause(e);
       return me;

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 f86ec45..c5e7a5f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -131,6 +131,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.TableStatsRequest;
 import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
@@ -1314,6 +1315,37 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     return null;
   }
 
+  @Override
+  public List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
+      throws MetaException {
+    try {
+      return filterNames(client.get_table_meta(dbPatterns, tablePatterns, tableTypes));
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
+  private List<TableMeta> filterNames(List<TableMeta> metas) throws MetaException {
+    Map<String, TableMeta> sources = new LinkedHashMap<>();
+    Map<String, List<String>> dbTables = new LinkedHashMap<>();
+    for (TableMeta meta : metas) {
+      sources.put(meta.getDbName() + "." + meta.getTableName(), meta);
+      List<String> tables = dbTables.get(meta.getDbName());
+      if (tables == null) {
+        dbTables.put(meta.getDbName(), tables = new ArrayList<String>());
+      }
+      tables.add(meta.getTableName());
+    }
+    List<TableMeta> filtered = new ArrayList<>();
+    for (Map.Entry<String, List<String>> entry : dbTables.entrySet()) {
+      for (String table : filterHook.filterTableNames(entry.getKey(), entry.getValue())) {
+        filtered.add(sources.get(entry.getKey() + "." + table));
+      }
+    }
+    return filtered;
+  }
+
   /** {@inheritDoc} */
   @Override
   public List<String> getAllTables(String dbname) throws MetaException {

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 9279cf5..aa96f77 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.TxnOpenException;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
@@ -161,6 +162,12 @@ public interface IMetaStoreClient {
       throws MetaException, TException, UnknownDBException;
 
   /**
+   * For quick GetTablesOperation
+   */
+  List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
+      throws MetaException, TException, UnknownDBException;
+
+  /**
    * Get the names of all tables in the specified database.
    * @param dbName
    * @return List of table names.

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 9f2f5f4..803c6e7 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -55,8 +55,6 @@ import javax.jdo.datastore.DataStoreCache;
 import javax.jdo.identity.IntIdentity;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configurable;
@@ -109,6 +107,7 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
@@ -143,12 +142,7 @@ import org.apache.hadoop.hive.metastore.model.MTablePrivilege;
 import org.apache.hadoop.hive.metastore.model.MType;
 import org.apache.hadoop.hive.metastore.model.MVersionTable;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
-import org.apache.hadoop.hive.metastore.parser.ExpressionTree.ANTLRNoCaseStringStream;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree.FilterBuilder;
-import org.apache.hadoop.hive.metastore.parser.ExpressionTree.LeafNode;
-import org.apache.hadoop.hive.metastore.parser.ExpressionTree.Operator;
-import org.apache.hadoop.hive.metastore.parser.FilterLexer;
-import org.apache.hadoop.hive.metastore.parser.FilterParser;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.shims.ShimLoader;
@@ -729,6 +723,9 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public List<String> getDatabases(String pattern) throws MetaException {
+    if (pattern == null || pattern.equals("*")) {
+      return getAllDatabases();
+    }
     boolean commited = false;
     List<String> databases = null;
     Query query = null;
@@ -770,7 +767,28 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public List<String> getAllDatabases() throws MetaException {
-    return getDatabases(".*");
+    boolean commited = false;
+    List<String> databases = null;
+
+    String queryStr = "select name from org.apache.hadoop.hive.metastore.model.MDatabase";
+    Query query = null;
+    
+    openTransaction();
+    try {
+      query = pm.newQuery(queryStr);
+      query.setResult("name");
+      databases = new ArrayList<String>((Collection<String>) query.execute());
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    Collections.sort(databases);
+    return databases;
   }
 
   private MType getMType(Type type) {
@@ -1050,6 +1068,84 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
+  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+      throws MetaException {
+
+    boolean commited = false;
+    Query query = null;
+    List<TableMeta> metas = new ArrayList<TableMeta>();
+    try {
+      openTransaction();
+      // Take the pattern and split it on the | to get all the composing
+      // patterns
+      StringBuilder builder = new StringBuilder();
+      if (dbNames != null && !dbNames.equals("*")) {
+        appendPatternCondition(builder, "database.name", dbNames);
+      }
+      if (tableNames != null && !tableNames.equals("*")) {
+        appendPatternCondition(builder, "tableName", tableNames);
+      }
+      if (tableTypes != null && !tableTypes.isEmpty()) {
+        appendSimpleCondition(builder, "tableType", tableTypes.toArray(new String[0]));
+      }
+
+      query = pm.newQuery(MTable.class, builder.toString());
+      Collection<MTable> tables = (Collection<MTable>) query.execute();
+      for (MTable table : tables) {
+        TableMeta metaData = new TableMeta(
+            table.getDatabase().getName(), table.getTableName(), table.getTableType());
+        metaData.setComments(table.getParameters().get("comment"));
+        metas.add(metaData);
+      }
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    return metas;
+  }
+
+  private StringBuilder appendPatternCondition(StringBuilder builder,
+      String fieldName, String elements) {
+      elements = HiveStringUtils.normalizeIdentifier(elements);
+    return appendCondition(builder, fieldName, elements.split("\\|"), true);
+  }
+
+  private StringBuilder appendSimpleCondition(StringBuilder builder,
+      String fieldName, String[] elements) {
+    return appendCondition(builder, fieldName, elements, false);
+  }
+
+  private StringBuilder appendCondition(StringBuilder builder,
+      String fieldName, String[] elements, boolean pattern) {
+    if (builder.length() > 0) {
+      builder.append(" && ");
+    }
+    builder.append(" (");
+    int length = builder.length();
+    for (String element : elements) {
+      if (pattern) {
+        element = "(?i)" + element.replaceAll("\\*", ".*");
+      }
+      if (builder.length() > length) {
+        builder.append(" || ");
+      }
+      builder.append(fieldName);
+      if (pattern) {
+        builder.append(".matches(\"").append(element).append("\")");
+      } else {
+        builder.append(" == \"").append(element).append("\"");
+      }
+    }
+    builder.append(" )");
+    return builder;
+  }
+
+  @Override
   public List<String> getAllTables(String dbName) throws MetaException {
     return getTables(dbName, ".*");
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 4aa17a5..5b36b03 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
@@ -151,6 +152,9 @@ public interface RawStore extends Configurable {
   public List<String> getTables(String dbName, String pattern)
       throws MetaException;
 
+  public List<TableMeta> getTableMeta(
+      String dbNames, String tableNames, List<String> tableTypes) throws MetaException;
+
   /**
    * @param dbname
    *        The name of the database from which to retrieve the tables

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 5cc7c30..98e6c75 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
@@ -62,6 +62,7 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
@@ -71,7 +72,6 @@ import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.PlanResult;
 import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.ScanPlan;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hive.common.util.HiveStringUtils;
@@ -487,10 +487,7 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      List<Table> tables = getHBase().scanTables(HiveStringUtils.normalizeIdentifier(dbName),
-          pattern==null?null:HiveStringUtils.normalizeIdentifier(likeToRegex(pattern)));
-      List<String> tableNames = new ArrayList<String>(tables.size());
-      for (Table table : tables) tableNames.add(table.getTableName());
+      List<String> tableNames = getTableNamesInTx(dbName, pattern);
       commit = true;
       return tableNames;
     } catch (IOException e) {
@@ -501,6 +498,41 @@ public class HBaseStore implements RawStore {
     }
   }
 
+  private List<String> getTableNamesInTx(String dbName, String pattern) throws IOException {
+    List<Table> tables = getHBase().scanTables(HiveStringUtils.normalizeIdentifier(dbName),
+        pattern==null?null:HiveStringUtils.normalizeIdentifier(likeToRegex(pattern)));
+    List<String> tableNames = new ArrayList<String>(tables.size());
+    for (Table table : tables) tableNames.add(table.getTableName());
+    return tableNames;
+  }
+
+  @Override
+  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+      throws MetaException {
+    boolean commit = false;
+    openTransaction();
+    try {
+      List<TableMeta> metas = new ArrayList<>();
+      for (String dbName : getDatabases(dbNames)) {
+        for (Table table : getTableObjectsByName(dbName, getTableNamesInTx(dbName, tableNames))) {
+          if (tableTypes == null || tableTypes.contains(table.getTableType())) {
+            TableMeta metaData = new TableMeta(
+              table.getDbName(), table.getTableName(), table.getTableType());
+            metaData.setComments(table.getParameters().get("comment"));
+            metas.add(metaData);
+          }
+        }
+      }
+      commit = true;
+      return metas;
+    } catch (Exception e) {
+      LOG.error("Unable to get tables ", e);
+      throw new MetaException("Unable to get tables, " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
+  }
+
   @Override
   public List<Table> getTableObjectsByName(String dbname, List<String> tableNames) throws
       MetaException, UnknownDBException {
@@ -1660,7 +1692,7 @@ public class HBaseStore implements RawStore {
     openTransaction();
     try {
       List<ColumnStatistics> cs =
-          getHBase().getPartitionStatistics(dbName, tblName, partNames,  partVals, colNames);
+          getHBase().getPartitionStatistics(dbName, tblName, partNames, partVals, colNames);
       commit = true;
       return cs;
     } catch (IOException e) {


[34/43] hive git commit: HIVE-11120: Generic interface for file format validation (Prasanth Jayachandran reviewed by Xuefu Zhang)

Posted by om...@apache.org.
HIVE-11120: Generic interface for file format validation (Prasanth Jayachandran reviewed by Xuefu Zhang)


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

Branch: refs/heads/master-fixed
Commit: f759223cf41ca2a241b6eadfff92db5776ad6511
Parents: 0b8e709
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Fri Nov 13 14:34:48 2015 -0600
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:34 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |  63 ++++---
 .../hadoop/hive/ql/io/HiveFileFormatUtils.java  | 166 +++++++++----------
 .../ql/io/SequenceFileInputFormatChecker.java   |   3 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   4 +
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |  49 +++---
 .../clientnegative/archive_corrupt.q.out        |  14 +-
 .../clientnegative/load_orc_negative1.q.out     |   2 +-
 .../clientnegative/load_orc_negative2.q.out     |   2 +-
 .../clientnegative/load_orc_negative3.q.out     |   2 +-
 .../clientnegative/load_orc_negative_part.q.out |   2 +-
 .../clientnegative/load_wrong_fileformat.q.out  |   7 +-
 .../load_wrong_fileformat_rc_seq.q.out          |   7 +-
 .../load_wrong_fileformat_txt_seq.q.out         |   7 +-
 13 files changed, 153 insertions(+), 175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/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 5ac523a..e9cd450 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
@@ -18,8 +18,16 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.io.Serializable;
+import java.security.AccessControlException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -62,17 +70,8 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.util.StringUtils;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.security.AccessControlException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * MoveTask implementation.
@@ -295,13 +294,39 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
             throw new HiveException(
                 "addFiles: filesystem error in check phase", e);
           }
+
+          // handle file format check for table level
           if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVECHECKFILEFORMAT)) {
-            // Check if the file format of the file matches that of the table.
-            boolean flag = HiveFileFormatUtils.checkInputFormat(
-                srcFs, conf, tbd.getTable().getInputFileFormatClass(), files);
-            if (!flag) {
-              throw new HiveException(
-                  "Wrong file format. Please check the file's format.");
+            boolean flag = true;
+            // work.checkFileFormat is set to true only for Load Task, so assumption here is
+            // dynamic partition context is null
+            if (tbd.getDPCtx() == null) {
+              if (tbd.getPartitionSpec() == null || tbd.getPartitionSpec().isEmpty()) {
+                // Check if the file format of the file matches that of the table.
+                flag = HiveFileFormatUtils.checkInputFormat(
+                    srcFs, conf, tbd.getTable().getInputFileFormatClass(), files);
+              } else {
+                // Check if the file format of the file matches that of the partition
+                Partition oldPart = db.getPartition(table, tbd.getPartitionSpec(), false);
+                if (oldPart == null) {
+                  // this means we have just created a table and are specifying partition in the
+                  // load statement (without pre-creating the partition), in which case lets use
+                  // table input format class. inheritTableSpecs defaults to true so when a new
+                  // partition is created later it will automatically inherit input format
+                  // from table object
+                  flag = HiveFileFormatUtils.checkInputFormat(
+                      srcFs, conf, tbd.getTable().getInputFileFormatClass(), files);
+                } else {
+                  flag = HiveFileFormatUtils.checkInputFormat(
+                      srcFs, conf, oldPart.getInputFormatClass(), files);
+                }
+              }
+              if (!flag) {
+                throw new HiveException(
+                    "Wrong file format. Please check the file's format.");
+              }
+            } else {
+              LOG.warn("Skipping file format check as dpCtx is not null");
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java
index 0328a23..bc13862 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java
@@ -31,10 +31,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -44,7 +41,7 @@ import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
 import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -68,6 +65,12 @@ import org.apache.hadoop.mapred.TaskAttemptContext;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.util.Shell;
 import org.apache.hive.common.util.ReflectionUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.collect.ImmutableMap;
 
 /**
  * An util class for various Hive file format tasks.
@@ -79,30 +82,68 @@ import org.apache.hive.common.util.ReflectionUtil;
 public final class HiveFileFormatUtils {
   private static final Logger LOG = LoggerFactory.getLogger(HiveFileFormatUtils.class);
 
-  static {
-    outputFormatSubstituteMap =
-        new ConcurrentHashMap<Class<?>, Class<? extends OutputFormat>>();
-    HiveFileFormatUtils.registerOutputFormatSubstitute(
-        IgnoreKeyTextOutputFormat.class, HiveIgnoreKeyTextOutputFormat.class);
-    HiveFileFormatUtils.registerOutputFormatSubstitute(
-        SequenceFileOutputFormat.class, HiveSequenceFileOutputFormat.class);
-  }
+  public static class FileChecker {
+    // we don't have many file formats that implement InputFormatChecker. We won't be holding
+    // multiple instances of such classes
+    private static int MAX_CACHE_SIZE = 16;
 
-  @SuppressWarnings("unchecked")
-  private static Map<Class<?>, Class<? extends OutputFormat>>
-    outputFormatSubstituteMap;
+    // immutable maps
+    Map<Class<? extends InputFormat>, Class<? extends InputFormatChecker>> inputFormatCheckerMap;
+    Map<Class<?>, Class<? extends OutputFormat>> outputFormatSubstituteMap;
 
-  /**
-   * register a substitute.
-   *
-   * @param origin
-   *          the class that need to be substituted
-   * @param substitute
-   */
-  @SuppressWarnings("unchecked")
-  public static void registerOutputFormatSubstitute(Class<?> origin,
-      Class<? extends HiveOutputFormat> substitute) {
-    outputFormatSubstituteMap.put(origin, substitute);
+    // mutable thread-safe map to store instances
+    Cache<Class<? extends InputFormatChecker>, InputFormatChecker> inputFormatCheckerInstanceCache;
+
+    // classloader invokes this static block when its first loaded (lazy initialization).
+    // Class loading is thread safe.
+    private static class Factory {
+      static final FileChecker INSTANCE = new FileChecker();
+    }
+
+    public static FileChecker getInstance() {
+      return Factory.INSTANCE;
+    }
+
+    private FileChecker() {
+      // read-only maps (initialized once)
+      inputFormatCheckerMap = ImmutableMap
+          .<Class<? extends InputFormat>, Class<? extends InputFormatChecker>>builder()
+          .put(SequenceFileInputFormat.class, SequenceFileInputFormatChecker.class)
+          .put(RCFileInputFormat.class, RCFileInputFormat.class)
+          .put(OrcInputFormat.class, OrcInputFormat.class)
+          .build();
+      outputFormatSubstituteMap = ImmutableMap
+          .<Class<?>, Class<? extends OutputFormat>>builder()
+          .put(IgnoreKeyTextOutputFormat.class, HiveIgnoreKeyTextOutputFormat.class)
+          .put(SequenceFileOutputFormat.class, HiveSequenceFileOutputFormat.class)
+          .build();
+
+      // updatable map that holds instances of the class
+      inputFormatCheckerInstanceCache = CacheBuilder.newBuilder().maximumSize(MAX_CACHE_SIZE)
+          .build();
+    }
+
+    public Set<Class<? extends InputFormat>> registeredClasses() {
+      return inputFormatCheckerMap.keySet();
+    }
+
+    public Class<? extends OutputFormat> getOutputFormatSubstiture(Class<?> origin) {
+      return outputFormatSubstituteMap.get(origin);
+    }
+
+    public Class<? extends InputFormatChecker> getInputFormatCheckerClass(Class<?> inputFormat) {
+      return inputFormatCheckerMap.get(inputFormat);
+    }
+
+    public void putInputFormatCheckerInstance(
+        Class<? extends InputFormatChecker> checkerCls, InputFormatChecker instanceCls) {
+      inputFormatCheckerInstanceCache.put(checkerCls, instanceCls);
+    }
+
+    public InputFormatChecker getInputFormatCheckerInstance(
+        Class<? extends InputFormatChecker> checkerCls) {
+      return inputFormatCheckerInstanceCache.getIfPresent(checkerCls);
+    }
   }
 
   /**
@@ -114,7 +155,8 @@ public final class HiveFileFormatUtils {
     if (origin == null || HiveOutputFormat.class.isAssignableFrom(origin)) {
       return (Class<? extends OutputFormat>) origin;  // hive native
     }
-    Class<? extends OutputFormat> substitute = outputFormatSubstituteMap.get(origin);
+    Class<? extends OutputFormat> substitute = FileChecker.getInstance()
+        .getOutputFormatSubstiture(origin);
     if (substitute != null) {
       return substitute;  // substituted
     }
@@ -122,66 +164,6 @@ public final class HiveFileFormatUtils {
   }
 
   /**
-   * get the final output path of a given FileOutputFormat.
-   *
-   * @param parent
-   *          parent dir of the expected final output path
-   * @param jc
-   *          job configuration
-   * @deprecated
-   */
-  @Deprecated
-  public static Path getOutputFormatFinalPath(Path parent, String taskId, JobConf jc,
-      HiveOutputFormat<?, ?> hiveOutputFormat, boolean isCompressed,
-      Path defaultFinalPath) throws IOException {
-    if (hiveOutputFormat instanceof HiveIgnoreKeyTextOutputFormat) {
-      return new Path(parent, taskId
-          + Utilities.getFileExtension(jc, isCompressed));
-    }
-    return defaultFinalPath;
-  }
-
-  static {
-    inputFormatCheckerMap =
-        new HashMap<Class<? extends InputFormat>, Class<? extends InputFormatChecker>>();
-    HiveFileFormatUtils.registerInputFormatChecker(
-        SequenceFileInputFormat.class, SequenceFileInputFormatChecker.class);
-    HiveFileFormatUtils.registerInputFormatChecker(RCFileInputFormat.class,
-        RCFileInputFormat.class);
-    inputFormatCheckerInstanceCache =
-        new HashMap<Class<? extends InputFormatChecker>, InputFormatChecker>();
-  }
-
-  @SuppressWarnings("unchecked")
-  private static Map<Class<? extends InputFormat>, Class<? extends InputFormatChecker>> inputFormatCheckerMap;
-
-  private static Map<Class<? extends InputFormatChecker>, InputFormatChecker> inputFormatCheckerInstanceCache;
-
-  /**
-   * register an InputFormatChecker for a given InputFormat.
-   *
-   * @param format
-   *          the class that need to be substituted
-   * @param checker
-   */
-  @SuppressWarnings("unchecked")
-  public static synchronized void registerInputFormatChecker(
-      Class<? extends InputFormat> format,
-      Class<? extends InputFormatChecker> checker) {
-    inputFormatCheckerMap.put(format, checker);
-  }
-
-  /**
-   * get an InputFormatChecker for a file format.
-   */
-  public static synchronized Class<? extends InputFormatChecker> getInputFormatChecker(
-      Class<?> inputFormat) {
-    Class<? extends InputFormatChecker> result = inputFormatCheckerMap
-        .get(inputFormat);
-    return result;
-  }
-
-  /**
    * checks if files are in same format as the given input format.
    */
   @SuppressWarnings("unchecked")
@@ -189,7 +171,8 @@ public final class HiveFileFormatUtils {
       Class<? extends InputFormat> inputFormatCls, List<FileStatus> files)
       throws HiveException {
     if (files.isEmpty()) return false;
-    Class<? extends InputFormatChecker> checkerCls = getInputFormatChecker(inputFormatCls);
+    Class<? extends InputFormatChecker> checkerCls = FileChecker.getInstance()
+        .getInputFormatCheckerClass(inputFormatCls);
     if (checkerCls == null
         && inputFormatCls.isAssignableFrom(TextInputFormat.class)) {
       // we get a text input format here, we can not determine a file is text
@@ -200,11 +183,12 @@ public final class HiveFileFormatUtils {
     }
 
     if (checkerCls != null) {
-      InputFormatChecker checkerInstance = inputFormatCheckerInstanceCache.get(checkerCls);
+      InputFormatChecker checkerInstance = FileChecker.getInstance()
+          .getInputFormatCheckerInstance(checkerCls);
       try {
         if (checkerInstance == null) {
           checkerInstance = checkerCls.newInstance();
-          inputFormatCheckerInstanceCache.put(checkerCls, checkerInstance);
+          FileChecker.getInstance().putInputFormatCheckerInstance(checkerCls, checkerInstance);
         }
         return checkerInstance.validateInput(fs, conf, files);
       } catch (Exception e) {
@@ -228,7 +212,7 @@ public final class HiveFileFormatUtils {
       }
     }
     if (files2.isEmpty()) return true;
-    Set<Class<? extends InputFormat>> inputFormatter = inputFormatCheckerMap.keySet();
+    Set<Class<? extends InputFormat>> inputFormatter = FileChecker.getInstance().registeredClasses();
     for (Class<? extends InputFormat> reg : inputFormatter) {
       boolean result = checkInputFormat(fs, conf, reg, files2);
       if (result) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/java/org/apache/hadoop/hive/ql/io/SequenceFileInputFormatChecker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/SequenceFileInputFormatChecker.java b/ql/src/java/org/apache/hadoop/hive/ql/io/SequenceFileInputFormatChecker.java
index 6cb46c9..f59b838 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/SequenceFileInputFormatChecker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/SequenceFileInputFormatChecker.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.io;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.fs.FileStatus;
@@ -49,7 +48,7 @@ public class SequenceFileInputFormatChecker implements InputFormatChecker {
         reader = null;
       } catch (IOException e) {
         return false;
-      }finally{
+      } finally{
         IOUtils.closeStream(reader);
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index af40137..bee0831 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@ -421,6 +421,10 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
       return false;
     }
     for (FileStatus file : files) {
+      // 0 length files cannot be ORC files
+      if (file.getLen() == 0) {
+        return false;
+      }
       try {
         OrcFile.createReader(file.getPath(),
             OrcFile.readerOptions(conf).filesystem(fs));

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
index c488029..b90616f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
@@ -41,9 +41,7 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
-import org.apache.hadoop.hive.ql.io.FileFormatException;
-import org.apache.hadoop.hive.ql.io.orc.OrcFile;
-import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
+import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
@@ -52,6 +50,8 @@ import org.apache.hadoop.hive.ql.plan.MoveWork;
 import org.apache.hadoop.hive.ql.plan.StatsWork;
 import org.apache.hadoop.mapred.InputFormat;
 
+import com.google.common.collect.Lists;
+
 /**
  * LoadSemanticAnalyzer.
  *
@@ -128,7 +128,7 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
     return new URI(fromScheme, fromAuthority, path, null, null);
   }
 
-  private FileStatus[] applyConstraintsAndGetFiles(URI fromURI, URI toURI, Tree ast,
+  private List<FileStatus> applyConstraintsAndGetFiles(URI fromURI, Tree ast,
       boolean isLocal) throws SemanticException {
 
     FileStatus[] srcs = null;
@@ -159,7 +159,7 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException(ErrorMsg.INVALID_PATH.getMsg(ast), e);
     }
 
-    return srcs;
+    return Lists.newArrayList(srcs);
   }
 
   @Override
@@ -209,9 +209,6 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException(ErrorMsg.LOAD_INTO_STORED_AS_DIR.getMsg());
     }
 
-    URI toURI = ((ts.partHandle != null) ? ts.partHandle.getDataLocation()
-        : ts.tableHandle.getDataLocation()).toUri();
-
     List<FieldSchema> parts = ts.tableHandle.getPartitionKeys();
     if ((parts != null && parts.size() > 0)
         && (ts.partSpec == null || ts.partSpec.size() == 0)) {
@@ -219,11 +216,12 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     // make sure the arguments make sense
-    FileStatus[] files = applyConstraintsAndGetFiles(fromURI, toURI, fromTree, isLocal);
+    List<FileStatus> files = applyConstraintsAndGetFiles(fromURI, fromTree, isLocal);
 
     // for managed tables, make sure the file formats match
-    if (TableType.MANAGED_TABLE.equals(ts.tableHandle.getTableType())) {
-      ensureFileFormatsMatch(ts, files);
+    if (TableType.MANAGED_TABLE.equals(ts.tableHandle.getTableType())
+        && conf.getBoolVar(HiveConf.ConfVars.HIVECHECKFILEFORMAT)) {
+      ensureFileFormatsMatch(ts, files, fromURI);
     }
     inputs.add(toReadEntity(new Path(fromURI)));
     Task<? extends Serializable> rTask = null;
@@ -317,7 +315,9 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
-  private void ensureFileFormatsMatch(TableSpec ts, FileStatus[] fileStatuses) throws SemanticException {
+  private void ensureFileFormatsMatch(TableSpec ts, List<FileStatus> fileStatuses,
+      final URI fromURI)
+      throws SemanticException {
     final Class<? extends InputFormat> destInputFormat;
     try {
       if (ts.getPartSpec() == null || ts.getPartSpec().isEmpty()) {
@@ -329,23 +329,16 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException(e);
     }
 
-    // Other file formats should do similar check to make sure file formats match
-    // when doing LOAD DATA .. INTO TABLE
-    if (OrcInputFormat.class.equals(destInputFormat)) {
-      for (FileStatus fileStatus : fileStatuses) {
-        try {
-          Path filePath = fileStatus.getPath();
-          FileSystem fs = FileSystem.get(filePath.toUri(), conf);
-          // just creating orc reader is going to do sanity checks to make sure its valid ORC file
-          OrcFile.createReader(fs, filePath);
-        } catch (FileFormatException e) {
-          throw new SemanticException(ErrorMsg.INVALID_FILE_FORMAT_IN_LOAD.getMsg("Destination" +
-              " table is stored as ORC but the file being loaded is not a valid ORC file."));
-        } catch (IOException e) {
-          throw new SemanticException("Unable to load data to destination table." +
-              " Error: " + e.getMessage());
-        }
+    try {
+      FileSystem fs = FileSystem.get(fromURI, conf);
+      boolean validFormat = HiveFileFormatUtils.checkInputFormat(fs, conf, destInputFormat,
+          fileStatuses);
+      if (!validFormat) {
+        throw new SemanticException(ErrorMsg.INVALID_FILE_FORMAT_IN_LOAD.getMsg());
       }
+    } catch (Exception e) {
+      throw new SemanticException("Unable to load data to destination table." +
+          " Error: " + e.getMessage());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/test/results/clientnegative/archive_corrupt.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/archive_corrupt.q.out b/ql/src/test/results/clientnegative/archive_corrupt.q.out
index 56e8ec4..892fbac 100644
--- a/ql/src/test/results/clientnegative/archive_corrupt.q.out
+++ b/ql/src/test/results/clientnegative/archive_corrupt.q.out
@@ -16,16 +16,4 @@ POSTHOOK: query: create table tstsrcpart like srcpart
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@tstsrcpart
-PREHOOK: query: -- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20)
--- The version of GzipCodec that is provided in Hadoop 0.20 silently ignores
--- file format errors. However, versions of Hadoop that include
--- HADOOP-6835 (e.g. 0.23 and 1.x) cause a Wrong File Format exception
--- to be thrown during the LOAD step. This former behavior is tested
--- in clientpositive/archive_corrupt.q
-
-load data local inpath '../../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11')
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@tstsrcpart
-Failed with exception Wrong file format. Please check the file's format.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MoveTask
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/test/results/clientnegative/load_orc_negative1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_orc_negative1.q.out b/ql/src/test/results/clientnegative/load_orc_negative1.q.out
index ca15a30..d103546 100644
--- a/ql/src/test/results/clientnegative/load_orc_negative1.q.out
+++ b/ql/src/test/results/clientnegative/load_orc_negative1.q.out
@@ -6,4 +6,4 @@ POSTHOOK: query: create table orc_test (userid bigint, string1 string, subtype d
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@orc_test
-FAILED: SemanticException [Error 30019]: The file that you are trying to load does not match the file format of the destination table. Destination table is stored as ORC but the file being loaded is not a valid ORC file.
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/test/results/clientnegative/load_orc_negative2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_orc_negative2.q.out b/ql/src/test/results/clientnegative/load_orc_negative2.q.out
index 77fb50e..9b0cb69 100644
--- a/ql/src/test/results/clientnegative/load_orc_negative2.q.out
+++ b/ql/src/test/results/clientnegative/load_orc_negative2.q.out
@@ -22,4 +22,4 @@ POSTHOOK: query: create table orc_test (userid bigint, string1 string, subtype d
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@orc_test
-FAILED: SemanticException [Error 30019]: The file that you are trying to load does not match the file format of the destination table. Destination table is stored as ORC but the file being loaded is not a valid ORC file.
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/test/results/clientnegative/load_orc_negative3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_orc_negative3.q.out b/ql/src/test/results/clientnegative/load_orc_negative3.q.out
index 77fb50e..9b0cb69 100644
--- a/ql/src/test/results/clientnegative/load_orc_negative3.q.out
+++ b/ql/src/test/results/clientnegative/load_orc_negative3.q.out
@@ -22,4 +22,4 @@ POSTHOOK: query: create table orc_test (userid bigint, string1 string, subtype d
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@orc_test
-FAILED: SemanticException [Error 30019]: The file that you are trying to load does not match the file format of the destination table. Destination table is stored as ORC but the file being loaded is not a valid ORC file.
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/test/results/clientnegative/load_orc_negative_part.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_orc_negative_part.q.out b/ql/src/test/results/clientnegative/load_orc_negative_part.q.out
index 32dd627..2e8068d 100644
--- a/ql/src/test/results/clientnegative/load_orc_negative_part.q.out
+++ b/ql/src/test/results/clientnegative/load_orc_negative_part.q.out
@@ -49,4 +49,4 @@ POSTHOOK: query: alter table orc_test add partition(ds='11')
 POSTHOOK: type: ALTERTABLE_ADDPARTS
 POSTHOOK: Output: default@orc_test
 POSTHOOK: Output: default@orc_test@ds=11
-FAILED: SemanticException [Error 30019]: The file that you are trying to load does not match the file format of the destination table. Destination table is stored as ORC but the file being loaded is not a valid ORC file.
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/test/results/clientnegative/load_wrong_fileformat.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_wrong_fileformat.q.out b/ql/src/test/results/clientnegative/load_wrong_fileformat.q.out
index 732eb22..8ec0058 100644
--- a/ql/src/test/results/clientnegative/load_wrong_fileformat.q.out
+++ b/ql/src/test/results/clientnegative/load_wrong_fileformat.q.out
@@ -14,9 +14,4 @@ CREATE TABLE load_wrong_fileformat_T1(name STRING) STORED AS SEQUENCEFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@load_wrong_fileformat_T1
-PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE load_wrong_fileformat_T1
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@load_wrong_fileformat_t1
-Failed with exception Wrong file format. Please check the file's format.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MoveTask
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/test/results/clientnegative/load_wrong_fileformat_rc_seq.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_wrong_fileformat_rc_seq.q.out b/ql/src/test/results/clientnegative/load_wrong_fileformat_rc_seq.q.out
index b68b8e6..916eca4 100644
--- a/ql/src/test/results/clientnegative/load_wrong_fileformat_rc_seq.q.out
+++ b/ql/src/test/results/clientnegative/load_wrong_fileformat_rc_seq.q.out
@@ -14,9 +14,4 @@ CREATE TABLE T1(name STRING) STORED AS RCFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@T1
-PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@t1
-Failed with exception Wrong file format. Please check the file's format.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MoveTask
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/f759223c/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out b/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out
index 179a654..645ece6 100644
--- a/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out
+++ b/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out
@@ -14,9 +14,4 @@ CREATE TABLE T1(name STRING) STORED AS TEXTFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@T1
-PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@t1
-Failed with exception Wrong file format. Please check the file's format.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MoveTask
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.


[40/43] hive git commit: HIVE-11488: Add sessionId and queryId info to HS2 log (Aihua Xu, reviewed by Szehon Ho)

Posted by om...@apache.org.
HIVE-11488: Add sessionId and queryId info to HS2 log (Aihua Xu, reviewed by Szehon Ho)


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

Branch: refs/heads/master-fixed
Commit: 4ca877456bc4c591821f7b6cc5a4010305ff7cd4
Parents: 0fee737
Author: aihuaxu <ai...@apache.org>
Authored: Tue Nov 10 15:13:22 2015 -0500
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:35 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  1 +
 .../cli/session/TestHiveSessionImpl.java        |  1 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  | 10 ++++++---
 .../hive/service/cli/operation/Operation.java   | 23 ++++++++++++++++++++
 .../service/cli/operation/SQLOperation.java     |  2 ++
 .../service/cli/session/HiveSessionBase.java    |  3 ---
 .../service/cli/session/HiveSessionImpl.java    | 14 ++++++++++++
 7 files changed, 48 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4ca87745/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 425c7d9..838f25c 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3106,6 +3106,7 @@ public class HiveConf extends Configuration {
     ConfVars.OUTPUT_FILE_EXTENSION.varname,
     ConfVars.SHOW_JOB_FAIL_DEBUG_INFO.varname,
     ConfVars.TASKLOG_DEBUG_TIMEOUT.varname,
+    ConfVars.HIVEQUERYID.varname,
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/4ca87745/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java
index 2d9ad03..8a32a07 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java
@@ -74,6 +74,7 @@ public class TestHiveSessionImpl {
     try {
 
       //Running a normal async query with no exceptions,then no need to close opHandle
+      session.open(new HashMap<String, String>());
       session.executeStatementAsync(hql, confOverlay);
       Mockito.verify(operationManager, Mockito.times(0)).closeOperation(opHandle);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ca87745/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 93c7a54..9a5495b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -384,9 +384,13 @@ public class Driver implements CommandProcessor {
     }
     saveSession(queryState);
 
-    // generate new query id
-    String queryId = QueryPlan.makeQueryId();
-    conf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId);
+    // Generate new query id if it's not set for CLI case. If it's session based,
+    // query id is passed in from the client or initialized when the session starts.
+    String queryId = conf.getVar(HiveConf.ConfVars.HIVEQUERYID);
+    if (queryId == null || queryId.isEmpty()) {
+      queryId = QueryPlan.makeQueryId();
+      conf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId);
+    }
 
     SessionState.get().setupQueryCurrentTimestamp();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ca87745/service/src/java/org/apache/hive/service/cli/operation/Operation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/Operation.java b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
index 4ca0561..a851936 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/Operation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
@@ -31,6 +31,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.OperationLog;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationHandle;
@@ -41,8 +42,13 @@ import org.apache.hive.service.cli.RowSet;
 import org.apache.hive.service.cli.TableSchema;
 import org.apache.hive.service.cli.session.HiveSession;
 import org.apache.hive.service.cli.thrift.TProtocolVersion;
+import org.apache.logging.log4j.ThreadContext;
 
 public abstract class Operation {
+  // Constants of the key strings for the log4j ThreadContext.
+  private static final String QUERYID = "QueryId";
+  private static final String SESSIONID = "SessionId";
+
   protected final HiveSession parentSession;
   private OperationState state = OperationState.INITIALIZED;
   private final OperationHandle opHandle;
@@ -238,6 +244,22 @@ public abstract class Operation {
    */
   protected void beforeRun() {
     createOperationLog();
+    registerLoggingContext();
+  }
+
+  /**
+   * Register logging context so that Log4J can print QueryId and/or SessionId for each message
+   */
+  protected void registerLoggingContext() {
+    ThreadContext.put(QUERYID, SessionState.get().getQueryId());
+    ThreadContext.put(SESSIONID, SessionState.get().getSessionId());
+  }
+
+  /**
+   * Unregister logging context
+   */
+  protected void unregisterLoggingContext() {
+    ThreadContext.clearAll();
   }
 
   /**
@@ -245,6 +267,7 @@ public abstract class Operation {
    * Clean up resources, which was set up in beforeRun().
    */
   protected void afterRun() {
+    unregisterLoggingContext();
     unregisterOperationLog();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ca87745/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 1c798eb..8b42265 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -225,12 +225,14 @@ public class SQLOperation extends ExecuteStatementOperation {
               SessionState.setCurrentSessionState(parentSessionState);
               // Set current OperationLog in this async thread for keeping on saving query log.
               registerCurrentOperationLog();
+              registerLoggingContext();
               try {
                 runQuery(opConfig);
               } catch (HiveSQLException e) {
                 setOperationException(e);
                 LOG.error("Error running hive query: ", e);
               } finally {
+                unregisterLoggingContext();
                 unregisterOperationLog();
               }
               return null;

http://git-wip-us.apache.org/repos/asf/hive/blob/4ca87745/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
index 449d4c7..1ab5652 100644
--- a/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
+++ b/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
@@ -18,8 +18,6 @@
 
 package org.apache.hive.service.cli.session;
 
-import java.util.Map;
-
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.cli.SessionHandle;
@@ -27,7 +25,6 @@ import org.apache.hive.service.cli.operation.OperationManager;
 import org.apache.hive.service.cli.thrift.TProtocolVersion;
 
 import java.io.File;
-import java.util.Map;
 
 /**
  * Methods that don't need to be executed under a doAs

http://git-wip-us.apache.org/repos/asf/hive/blob/4ca87745/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 27d11df..2d784f0 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
@@ -24,6 +24,7 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.exec.FetchFormatter;
 import org.apache.hadoop.hive.ql.exec.ListSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -432,6 +434,18 @@ public class HiveSessionImpl implements HiveSession {
           throws HiveSQLException {
     acquire(true);
 
+    // Create the queryId if the client doesn't pass in.
+    // Reuse the client's queryId if exists.
+    if (confOverlay == null) {
+      confOverlay = new HashMap<String, String>();
+    }
+    String queryId = confOverlay.get(HiveConf.ConfVars.HIVEQUERYID.varname);
+    if (queryId == null || queryId.isEmpty()) {
+      queryId = QueryPlan.makeQueryId();
+      confOverlay.put(HiveConf.ConfVars.HIVEQUERYID.varname, queryId);
+      sessionState.getConf().setVar(HiveConf.ConfVars.HIVEQUERYID, queryId);
+    }
+
     OperationManager operationManager = getOperationManager();
     ExecuteStatementOperation operation = operationManager
         .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync);


[38/43] hive git commit: HIVE-12378: Exception on HBaseSerDe.serialize binary field (Yongzhi Chen, reviewed by Jimmy Xiang)

Posted by om...@apache.org.
HIVE-12378: Exception on HBaseSerDe.serialize binary field (Yongzhi Chen, reviewed by Jimmy Xiang)


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

Branch: refs/heads/master-fixed
Commit: 0fee7376e153de84351013d5b13e308ac3689d4a
Parents: cf1d160
Author: Yongzhi Chen <yc...@cloudera.com>
Authored: Thu Nov 12 10:57:07 2015 -0500
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:35 2015 -0800

----------------------------------------------------------------------
 .../test/queries/positive/hbase_binary_binary.q | 12 +++++
 .../results/positive/hbase_binary_binary.q.out  | 54 ++++++++++++++++++++
 .../hadoop/hive/serde2/lazy/LazyBinary.java     |  2 +-
 .../hadoop/hive/serde2/lazy/LazyFactory.java    |  3 ++
 .../hadoop/hive/serde2/lazy/LazyUtils.java      |  6 +++
 .../hive/serde2/lazydio/LazyDioBinary.java      | 51 ++++++++++++++++++
 6 files changed, 127 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0fee7376/hbase-handler/src/test/queries/positive/hbase_binary_binary.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_binary_binary.q b/hbase-handler/src/test/queries/positive/hbase_binary_binary.q
new file mode 100644
index 0000000..388e5aa
--- /dev/null
+++ b/hbase-handler/src/test/queries/positive/hbase_binary_binary.q
@@ -0,0 +1,12 @@
+drop table if exists testhbaseb;
+CREATE TABLE testhbaseb (key int, val binary)
+STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
+WITH SERDEPROPERTIES (
+"hbase.columns.mapping" = ":key,cf:val#b"
+);
+insert into table testhbaseb values(1, 'hello');
+insert into table testhbaseb values(2, 'hi');
+select * from testhbaseb;
+drop table testhbaseb;
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/0fee7376/hbase-handler/src/test/results/positive/hbase_binary_binary.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_binary_binary.q.out b/hbase-handler/src/test/results/positive/hbase_binary_binary.q.out
new file mode 100644
index 0000000..c828fd1
--- /dev/null
+++ b/hbase-handler/src/test/results/positive/hbase_binary_binary.q.out
@@ -0,0 +1,54 @@
+PREHOOK: query: drop table if exists testhbaseb
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists testhbaseb
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE testhbaseb (key int, val binary)
+STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
+WITH SERDEPROPERTIES (
+"hbase.columns.mapping" = ":key,cf:val#b"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@testhbaseb
+POSTHOOK: query: CREATE TABLE testhbaseb (key int, val binary)
+STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
+WITH SERDEPROPERTIES (
+"hbase.columns.mapping" = ":key,cf:val#b"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@testhbaseb
+PREHOOK: query: insert into table testhbaseb values(1, 'hello')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@testhbaseb
+POSTHOOK: query: insert into table testhbaseb values(1, 'hello')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@testhbaseb
+PREHOOK: query: insert into table testhbaseb values(2, 'hi')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__2
+PREHOOK: Output: default@testhbaseb
+POSTHOOK: query: insert into table testhbaseb values(2, 'hi')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__2
+POSTHOOK: Output: default@testhbaseb
+PREHOOK: query: select * from testhbaseb
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testhbaseb
+#### A masked pattern was here ####
+POSTHOOK: query: select * from testhbaseb
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testhbaseb
+#### A masked pattern was here ####
+1	hello
+2	hi
+PREHOOK: query: drop table testhbaseb
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@testhbaseb
+PREHOOK: Output: default@testhbaseb
+POSTHOOK: query: drop table testhbaseb
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@testhbaseb
+POSTHOOK: Output: default@testhbaseb

http://git-wip-us.apache.org/repos/asf/hive/blob/0fee7376/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyBinary.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyBinary.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyBinary.java
index 010f1f9..aa19d09 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyBinary.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyBinary.java
@@ -29,7 +29,7 @@ public class LazyBinary extends LazyPrimitive<LazyBinaryObjectInspector, BytesWr
   private static final Logger LOG = LoggerFactory.getLogger(LazyBinary.class);
   private static final boolean DEBUG_LOG_ENABLED = LOG.isDebugEnabled();
   
-  LazyBinary(LazyBinaryObjectInspector oi) {
+  public LazyBinary(LazyBinaryObjectInspector oi) {
     super(oi);
     data = new BytesWritable();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0fee7376/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
index 7aa0fc7..23dbe6a 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyShortObj
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyStringObjectInspector;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyTimestampObjectInspector;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyVoidObjectInspector;
+import org.apache.hadoop.hive.serde2.lazydio.LazyDioBinary;
 import org.apache.hadoop.hive.serde2.lazydio.LazyDioBoolean;
 import org.apache.hadoop.hive.serde2.lazydio.LazyDioByte;
 import org.apache.hadoop.hive.serde2.lazydio.LazyDioDouble;
@@ -162,6 +163,8 @@ public final class LazyFactory {
       return new LazyDioFloat((LazyFloatObjectInspector) poi);
     case DOUBLE:
       return new LazyDioDouble((LazyDoubleObjectInspector) poi);
+    case BINARY:
+      return new LazyDioBinary((LazyBinaryObjectInspector) poi);
     default:
       throw new RuntimeException("Hive Internal Error: no LazyObject for " + poi);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/0fee7376/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
index 29d6ad8..ee39196 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
@@ -368,6 +368,12 @@ public final class LazyUtils {
         dos.writeDouble(d);
         break;
 
+      case BINARY: {
+        BytesWritable bw = ((BinaryObjectInspector) oi).getPrimitiveWritableObject(o);
+        out.write(bw.getBytes(), 0, bw.getLength());
+        break;
+      }
+
       default:
         throw new RuntimeException("Hive internal error.");
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/0fee7376/serde/src/java/org/apache/hadoop/hive/serde2/lazydio/LazyDioBinary.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazydio/LazyDioBinary.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazydio/LazyDioBinary.java
new file mode 100644
index 0000000..a3e4cac
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazydio/LazyDioBinary.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.serde2.lazydio;
+
+import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
+import org.apache.hadoop.hive.serde2.lazy.LazyBinary;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyBinaryObjectInspector;
+
+public class LazyDioBinary extends LazyBinary {
+
+  public LazyDioBinary(LazyBinaryObjectInspector oi) {
+    super(oi);
+  }
+
+  LazyDioBinary(LazyDioBinary copy) {
+    super(copy);
+  }
+
+  /* (non-Javadoc)
+   * This provides a LazyBinary like class which can be initialized from data stored in a
+   * binary format.
+   *
+   * @see org.apache.hadoop.hive.serde2.lazy.LazyObject#init
+   *        (org.apache.hadoop.hive.serde2.lazy.ByteArrayRef, int, int)
+   */
+  @Override
+  public void init(ByteArrayRef bytes, int start, int length) {
+    if (bytes == null) {
+      throw new RuntimeException("bytes cannot be null!");
+    }
+    isNull = false;
+    byte[] recv = new byte[length];
+    System.arraycopy(bytes.getData(), start, recv, 0, length);
+    data.set(recv, 0, length);
+  }
+}


[35/43] hive git commit: HIVE-12391: SkewJoinOptimizer might not kick in if columns are renamed after TableScanOperator (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by om...@apache.org.
HIVE-12391: SkewJoinOptimizer might not kick in if columns are renamed after TableScanOperator (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master-fixed
Commit: 58e07b17353404b252a8b5bc93cc30f89eebc015
Parents: 619ff6e
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Thu Nov 12 15:31:31 2015 +0100
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Nov 17 12:18:34 2015 -0800

----------------------------------------------------------------------
 .../hive/ql/optimizer/SkewJoinOptimizer.java    |  27 ++-
 .../test/queries/clientpositive/skewjoinopt21.q |  30 +++
 .../results/clientpositive/skewjoinopt21.q.out  | 230 +++++++++++++++++++
 3 files changed, 277 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/58e07b17/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
index e8c7486..64dc48c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
@@ -28,8 +28,6 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.Stack;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorFactory;
@@ -49,12 +47,12 @@ import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
-import org.apache.hadoop.hive.ql.parse.QBJoinTree;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc.ExprNodeDescEqualityWrapper;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.FilterDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -70,6 +68,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * SkewJoinOptimizer.
@@ -283,10 +283,11 @@ public class SkewJoinOptimizer implements Transform {
      * @param op The join operator being optimized
      * @param tableScanOpsForJoin table scan operators which are parents of the join operator
      * @return map<join keys intersection skewedkeys, list of skewed values>.
+     * @throws SemanticException 
      */
     private Map<List<ExprNodeDesc>, List<List<String>>>
       getSkewedValues(
-        Operator<? extends OperatorDesc> op, List<TableScanOperator> tableScanOpsForJoin) {
+        Operator<? extends OperatorDesc> op, List<TableScanOperator> tableScanOpsForJoin) throws SemanticException {
 
       Map <List<ExprNodeDesc>, List<List<String>>> skewDataReturn =
         new HashMap<List<ExprNodeDesc>, List<List<String>>>();
@@ -299,6 +300,7 @@ public class SkewJoinOptimizer implements Transform {
         ReduceSinkDesc rsDesc = ((ReduceSinkOperator) reduceSinkOp).getConf();
 
         if (rsDesc.getKeyCols() != null) {
+          TableScanOperator tableScanOp = null;
           Table table = null;
           // Find the skew information corresponding to the table
           List<String> skewedColumns = null;
@@ -321,7 +323,9 @@ public class SkewJoinOptimizer implements Transform {
             if (keyColDesc instanceof ExprNodeColumnDesc) {
               keyCol = (ExprNodeColumnDesc) keyColDesc;
               if (table == null) {
-                table = getTable(parseContext, reduceSinkOp, tableScanOpsForJoin);
+                tableScanOp = getTableScanOperator(parseContext, reduceSinkOp, tableScanOpsForJoin);
+                table =
+                  tableScanOp == null ? null : tableScanOp.getConf().getTableMetadata();
                 skewedColumns =
                   table == null ? null : table.getSkewedColNames();
                 // No skew on the table to take care of
@@ -332,10 +336,13 @@ public class SkewJoinOptimizer implements Transform {
                 skewedValueList =
                   table == null ? null : table.getSkewedColValues();
               }
-              int pos = skewedColumns.indexOf(keyCol.getColumn());
+              ExprNodeDesc keyColOrigin = ExprNodeDescUtils.backtrack(keyCol,
+                      reduceSinkOp, tableScanOp);
+              int pos = keyColOrigin == null || !(keyColOrigin instanceof ExprNodeColumnDesc) ?
+                      -1 : skewedColumns.indexOf(((ExprNodeColumnDesc)keyColOrigin).getColumn());
               if ((pos >= 0) && (!positionSkewedKeys.contains(pos))) {
                 positionSkewedKeys.add(pos);
-                ExprNodeColumnDesc keyColClone = (ExprNodeColumnDesc) keyCol.clone();
+                ExprNodeColumnDesc keyColClone = (ExprNodeColumnDesc) keyColOrigin.clone();
                 keyColClone.setTabAlias(null);
                 joinKeysSkewedCols.add(new ExprNodeDescEqualityWrapper(keyColClone));
               }
@@ -386,9 +393,9 @@ public class SkewJoinOptimizer implements Transform {
     }
 
     /**
-     * Get the table alias from the candidate table scans.
+     * Get the table scan.
      */
-    private Table getTable(
+    private TableScanOperator getTableScanOperator(
       ParseContext parseContext,
       Operator<? extends OperatorDesc> op,
       List<TableScanOperator> tableScanOpsForJoin) {
@@ -396,7 +403,7 @@ public class SkewJoinOptimizer implements Transform {
         if (op instanceof TableScanOperator) {
           TableScanOperator tsOp = (TableScanOperator)op;
           if (tableScanOpsForJoin.contains(tsOp)) {
-            return tsOp.getConf().getTableMetadata();
+            return tsOp;
           }
         }
         if ((op.getParentOperators() == null) || (op.getParentOperators().isEmpty()) || 

http://git-wip-us.apache.org/repos/asf/hive/blob/58e07b17/ql/src/test/queries/clientpositive/skewjoinopt21.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/skewjoinopt21.q b/ql/src/test/queries/clientpositive/skewjoinopt21.q
new file mode 100644
index 0000000..76dde57
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/skewjoinopt21.q
@@ -0,0 +1,30 @@
+set hive.optimize.skewjoin.compiletime = true;
+
+CREATE TABLE T1(key STRING, val STRING)
+SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+
+CREATE TABLE T2(key STRING, val STRING)
+SKEWED BY (key) ON ((3)) STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
+
+-- a simple join query with skew on both the tables on the join key
+-- adding a order by at the end to make the results deterministic
+
+EXPLAIN
+SELECT a.*, b.*
+FROM 
+  (SELECT key as k, val as v FROM T1) a
+  JOIN
+  (SELECT key as k, val as v FROM T2) b
+ON a.k = b.k;
+
+SELECT a.*, b.*
+FROM 
+  (SELECT key as k, val as v FROM T1) a
+  JOIN
+  (SELECT key as k, val as v FROM T2) b
+ON a.k = b.k
+ORDER BY a.k, b.k, a.v, b.v;

http://git-wip-us.apache.org/repos/asf/hive/blob/58e07b17/ql/src/test/results/clientpositive/skewjoinopt21.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/skewjoinopt21.q.out b/ql/src/test/results/clientpositive/skewjoinopt21.q.out
new file mode 100644
index 0000000..d58d694
--- /dev/null
+++ b/ql/src/test/results/clientpositive/skewjoinopt21.q.out
@@ -0,0 +1,230 @@
+PREHOOK: query: CREATE TABLE T1(key STRING, val STRING)
+SKEWED BY (key) ON ((2)) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@T1
+POSTHOOK: query: CREATE TABLE T1(key STRING, val STRING)
+SKEWED BY (key) ON ((2)) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@T1
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@t1
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@t1
+PREHOOK: query: CREATE TABLE T2(key STRING, val STRING)
+SKEWED BY (key) ON ((3)) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@T2
+POSTHOOK: query: CREATE TABLE T2(key STRING, val STRING)
+SKEWED BY (key) ON ((3)) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@T2
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@t2
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@t2
+PREHOOK: query: -- a simple join query with skew on both the tables on the join key
+-- adding a order by at the end to make the results deterministic
+
+EXPLAIN
+SELECT a.*, b.*
+FROM 
+  (SELECT key as k, val as v FROM T1) a
+  JOIN
+  (SELECT key as k, val as v FROM T2) b
+ON a.k = b.k
+PREHOOK: type: QUERY
+POSTHOOK: query: -- a simple join query with skew on both the tables on the join key
+-- adding a order by at the end to make the results deterministic
+
+EXPLAIN
+SELECT a.*, b.*
+FROM 
+  (SELECT key as k, val as v FROM T1) a
+  JOIN
+  (SELECT key as k, val as v FROM T2) b
+ON a.k = b.k
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1, Stage-4
+  Stage-4 is a root stage
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t1
+            Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (key is not null and ((key = '2') or (key = '3'))) (type: boolean)
+              Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), val (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: string)
+          TableScan
+            alias: t2
+            Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (key is not null and ((key = '2') or (key = '3'))) (type: boolean)
+              Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), val (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Union
+              Statistics: Num rows: 2 Data size: 66 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 2 Data size: 66 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          TableScan
+            Union
+              Statistics: Num rows: 2 Data size: 66 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 2 Data size: 66 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t1
+            Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (key is not null and (not ((key = '2') or (key = '3')))) (type: boolean)
+              Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), val (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: string)
+          TableScan
+            alias: t2
+            Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (key is not null and (not ((key = '2') or (key = '3')))) (type: boolean)
+              Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), val (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 1 Data size: 33 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT a.*, b.*
+FROM 
+  (SELECT key as k, val as v FROM T1) a
+  JOIN
+  (SELECT key as k, val as v FROM T2) b
+ON a.k = b.k
+ORDER BY a.k, b.k, a.v, b.v
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT a.*, b.*
+FROM 
+  (SELECT key as k, val as v FROM T1) a
+  JOIN
+  (SELECT key as k, val as v FROM T2) b
+ON a.k = b.k
+ORDER BY a.k, b.k, a.v, b.v
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+#### A masked pattern was here ####
+2	12	2	22
+3	13	3	13
+8	18	8	18
+8	18	8	18
+8	28	8	18
+8	28	8	18


[13/43] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 a6862be..34c2205 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -181,6 +181,14 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function get_tables($db_name, $pattern);
   /**
+   * @param string $db_patterns
+   * @param string $tbl_patterns
+   * @param string[] $tbl_types
+   * @return \metastore\TableMeta[]
+   * @throws \metastore\MetaException
+   */
+  public function get_table_meta($db_patterns, $tbl_patterns, array $tbl_types);
+  /**
    * @param string $db_name
    * @return string[]
    * @throws \metastore\MetaException
@@ -2254,6 +2262,62 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_tables failed: unknown result");
   }
 
+  public function get_table_meta($db_patterns, $tbl_patterns, array $tbl_types)
+  {
+    $this->send_get_table_meta($db_patterns, $tbl_patterns, $tbl_types);
+    return $this->recv_get_table_meta();
+  }
+
+  public function send_get_table_meta($db_patterns, $tbl_patterns, array $tbl_types)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_table_meta_args();
+    $args->db_patterns = $db_patterns;
+    $args->tbl_patterns = $tbl_patterns;
+    $args->tbl_types = $tbl_types;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_table_meta', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_table_meta', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_table_meta()
+  {
+    $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_table_meta_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_table_meta_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    throw new \Exception("get_table_meta failed: unknown result");
+  }
+
   public function get_all_tables($db_name)
   {
     $this->send_get_all_tables($db_name);
@@ -13287,6 +13351,281 @@ class ThriftHiveMetastore_get_tables_result {
 
 }
 
+class ThriftHiveMetastore_get_table_meta_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $db_patterns = null;
+  /**
+   * @var string
+   */
+  public $tbl_patterns = null;
+  /**
+   * @var string[]
+   */
+  public $tbl_types = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'db_patterns',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tbl_patterns',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'tbl_types',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['db_patterns'])) {
+        $this->db_patterns = $vals['db_patterns'];
+      }
+      if (isset($vals['tbl_patterns'])) {
+        $this->tbl_patterns = $vals['tbl_patterns'];
+      }
+      if (isset($vals['tbl_types'])) {
+        $this->tbl_types = $vals['tbl_types'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_table_meta_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->db_patterns);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tbl_patterns);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::LST) {
+            $this->tbl_types = array();
+            $_size583 = 0;
+            $_etype586 = 0;
+            $xfer += $input->readListBegin($_etype586, $_size583);
+            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
+            {
+              $elem588 = null;
+              $xfer += $input->readString($elem588);
+              $this->tbl_types []= $elem588;
+            }
+            $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('ThriftHiveMetastore_get_table_meta_args');
+    if ($this->db_patterns !== null) {
+      $xfer += $output->writeFieldBegin('db_patterns', TType::STRING, 1);
+      $xfer += $output->writeString($this->db_patterns);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tbl_patterns !== null) {
+      $xfer += $output->writeFieldBegin('tbl_patterns', TType::STRING, 2);
+      $xfer += $output->writeString($this->tbl_patterns);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tbl_types !== null) {
+      if (!is_array($this->tbl_types)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('tbl_types', TType::LST, 3);
+      {
+        $output->writeListBegin(TType::STRING, count($this->tbl_types));
+        {
+          foreach ($this->tbl_types as $iter589)
+          {
+            $xfer += $output->writeString($iter589);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_table_meta_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\TableMeta[]
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\TableMeta',
+            ),
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_table_meta_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::LST) {
+            $this->success = array();
+            $_size590 = 0;
+            $_etype593 = 0;
+            $xfer += $input->readListBegin($_etype593, $_size590);
+            for ($_i594 = 0; $_i594 < $_size590; ++$_i594)
+            {
+              $elem595 = null;
+              $elem595 = new \metastore\TableMeta();
+              $xfer += $elem595->read($input);
+              $this->success []= $elem595;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_meta_result');
+    if ($this->success !== null) {
+      if (!is_array($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->success));
+        {
+          foreach ($this->success as $iter596)
+          {
+            $xfer += $iter596->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ThriftHiveMetastore_get_all_tables_args {
   static $_TSPEC;
 
@@ -13424,14 +13763,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size583 = 0;
-            $_etype586 = 0;
-            $xfer += $input->readListBegin($_etype586, $_size583);
-            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
+            $_size597 = 0;
+            $_etype600 = 0;
+            $xfer += $input->readListBegin($_etype600, $_size597);
+            for ($_i601 = 0; $_i601 < $_size597; ++$_i601)
             {
-              $elem588 = null;
-              $xfer += $input->readString($elem588);
-              $this->success []= $elem588;
+              $elem602 = null;
+              $xfer += $input->readString($elem602);
+              $this->success []= $elem602;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13467,9 +13806,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter589)
+          foreach ($this->success as $iter603)
           {
-            $xfer += $output->writeString($iter589);
+            $xfer += $output->writeString($iter603);
           }
         }
         $output->writeListEnd();
@@ -13784,14 +14123,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size590 = 0;
-            $_etype593 = 0;
-            $xfer += $input->readListBegin($_etype593, $_size590);
-            for ($_i594 = 0; $_i594 < $_size590; ++$_i594)
+            $_size604 = 0;
+            $_etype607 = 0;
+            $xfer += $input->readListBegin($_etype607, $_size604);
+            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
             {
-              $elem595 = null;
-              $xfer += $input->readString($elem595);
-              $this->tbl_names []= $elem595;
+              $elem609 = null;
+              $xfer += $input->readString($elem609);
+              $this->tbl_names []= $elem609;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13824,9 +14163,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter596)
+          foreach ($this->tbl_names as $iter610)
           {
-            $xfer += $output->writeString($iter596);
+            $xfer += $output->writeString($iter610);
           }
         }
         $output->writeListEnd();
@@ -13927,15 +14266,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size597 = 0;
-            $_etype600 = 0;
-            $xfer += $input->readListBegin($_etype600, $_size597);
-            for ($_i601 = 0; $_i601 < $_size597; ++$_i601)
+            $_size611 = 0;
+            $_etype614 = 0;
+            $xfer += $input->readListBegin($_etype614, $_size611);
+            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
             {
-              $elem602 = null;
-              $elem602 = new \metastore\Table();
-              $xfer += $elem602->read($input);
-              $this->success []= $elem602;
+              $elem616 = null;
+              $elem616 = new \metastore\Table();
+              $xfer += $elem616->read($input);
+              $this->success []= $elem616;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13987,9 +14326,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter603)
+          foreach ($this->success as $iter617)
           {
-            $xfer += $iter603->write($output);
+            $xfer += $iter617->write($output);
           }
         }
         $output->writeListEnd();
@@ -14225,14 +14564,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size604 = 0;
-            $_etype607 = 0;
-            $xfer += $input->readListBegin($_etype607, $_size604);
-            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
+            $_size618 = 0;
+            $_etype621 = 0;
+            $xfer += $input->readListBegin($_etype621, $_size618);
+            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
             {
-              $elem609 = null;
-              $xfer += $input->readString($elem609);
-              $this->success []= $elem609;
+              $elem623 = null;
+              $xfer += $input->readString($elem623);
+              $this->success []= $elem623;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14284,9 +14623,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter610)
+          foreach ($this->success as $iter624)
           {
-            $xfer += $output->writeString($iter610);
+            $xfer += $output->writeString($iter624);
           }
         }
         $output->writeListEnd();
@@ -15599,15 +15938,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size611 = 0;
-            $_etype614 = 0;
-            $xfer += $input->readListBegin($_etype614, $_size611);
-            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
+            $_size625 = 0;
+            $_etype628 = 0;
+            $xfer += $input->readListBegin($_etype628, $_size625);
+            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
             {
-              $elem616 = null;
-              $elem616 = new \metastore\Partition();
-              $xfer += $elem616->read($input);
-              $this->new_parts []= $elem616;
+              $elem630 = null;
+              $elem630 = new \metastore\Partition();
+              $xfer += $elem630->read($input);
+              $this->new_parts []= $elem630;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15635,9 +15974,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter617)
+          foreach ($this->new_parts as $iter631)
           {
-            $xfer += $iter617->write($output);
+            $xfer += $iter631->write($output);
           }
         }
         $output->writeListEnd();
@@ -15852,15 +16191,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size618 = 0;
-            $_etype621 = 0;
-            $xfer += $input->readListBegin($_etype621, $_size618);
-            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
+            $_size632 = 0;
+            $_etype635 = 0;
+            $xfer += $input->readListBegin($_etype635, $_size632);
+            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
             {
-              $elem623 = null;
-              $elem623 = new \metastore\PartitionSpec();
-              $xfer += $elem623->read($input);
-              $this->new_parts []= $elem623;
+              $elem637 = null;
+              $elem637 = new \metastore\PartitionSpec();
+              $xfer += $elem637->read($input);
+              $this->new_parts []= $elem637;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15888,9 +16227,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter624)
+          foreach ($this->new_parts as $iter638)
           {
-            $xfer += $iter624->write($output);
+            $xfer += $iter638->write($output);
           }
         }
         $output->writeListEnd();
@@ -16140,14 +16479,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size625 = 0;
-            $_etype628 = 0;
-            $xfer += $input->readListBegin($_etype628, $_size625);
-            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
+            $_size639 = 0;
+            $_etype642 = 0;
+            $xfer += $input->readListBegin($_etype642, $_size639);
+            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
             {
-              $elem630 = null;
-              $xfer += $input->readString($elem630);
-              $this->part_vals []= $elem630;
+              $elem644 = null;
+              $xfer += $input->readString($elem644);
+              $this->part_vals []= $elem644;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16185,9 +16524,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter631)
+          foreach ($this->part_vals as $iter645)
           {
-            $xfer += $output->writeString($iter631);
+            $xfer += $output->writeString($iter645);
           }
         }
         $output->writeListEnd();
@@ -16689,14 +17028,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size632 = 0;
-            $_etype635 = 0;
-            $xfer += $input->readListBegin($_etype635, $_size632);
-            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
+            $_size646 = 0;
+            $_etype649 = 0;
+            $xfer += $input->readListBegin($_etype649, $_size646);
+            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
             {
-              $elem637 = null;
-              $xfer += $input->readString($elem637);
-              $this->part_vals []= $elem637;
+              $elem651 = null;
+              $xfer += $input->readString($elem651);
+              $this->part_vals []= $elem651;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16742,9 +17081,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter638)
+          foreach ($this->part_vals as $iter652)
           {
-            $xfer += $output->writeString($iter638);
+            $xfer += $output->writeString($iter652);
           }
         }
         $output->writeListEnd();
@@ -17598,14 +17937,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size639 = 0;
-            $_etype642 = 0;
-            $xfer += $input->readListBegin($_etype642, $_size639);
-            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
+            $_size653 = 0;
+            $_etype656 = 0;
+            $xfer += $input->readListBegin($_etype656, $_size653);
+            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
             {
-              $elem644 = null;
-              $xfer += $input->readString($elem644);
-              $this->part_vals []= $elem644;
+              $elem658 = null;
+              $xfer += $input->readString($elem658);
+              $this->part_vals []= $elem658;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17650,9 +17989,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter645)
+          foreach ($this->part_vals as $iter659)
           {
-            $xfer += $output->writeString($iter645);
+            $xfer += $output->writeString($iter659);
           }
         }
         $output->writeListEnd();
@@ -17905,14 +18244,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size646 = 0;
-            $_etype649 = 0;
-            $xfer += $input->readListBegin($_etype649, $_size646);
-            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
+            $_size660 = 0;
+            $_etype663 = 0;
+            $xfer += $input->readListBegin($_etype663, $_size660);
+            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
             {
-              $elem651 = null;
-              $xfer += $input->readString($elem651);
-              $this->part_vals []= $elem651;
+              $elem665 = null;
+              $xfer += $input->readString($elem665);
+              $this->part_vals []= $elem665;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17965,9 +18304,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter652)
+          foreach ($this->part_vals as $iter666)
           {
-            $xfer += $output->writeString($iter652);
+            $xfer += $output->writeString($iter666);
           }
         }
         $output->writeListEnd();
@@ -18981,14 +19320,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size653 = 0;
-            $_etype656 = 0;
-            $xfer += $input->readListBegin($_etype656, $_size653);
-            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
+            $_size667 = 0;
+            $_etype670 = 0;
+            $xfer += $input->readListBegin($_etype670, $_size667);
+            for ($_i671 = 0; $_i671 < $_size667; ++$_i671)
             {
-              $elem658 = null;
-              $xfer += $input->readString($elem658);
-              $this->part_vals []= $elem658;
+              $elem672 = null;
+              $xfer += $input->readString($elem672);
+              $this->part_vals []= $elem672;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19026,9 +19365,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter659)
+          foreach ($this->part_vals as $iter673)
           {
-            $xfer += $output->writeString($iter659);
+            $xfer += $output->writeString($iter673);
           }
         }
         $output->writeListEnd();
@@ -19270,17 +19609,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size660 = 0;
-            $_ktype661 = 0;
-            $_vtype662 = 0;
-            $xfer += $input->readMapBegin($_ktype661, $_vtype662, $_size660);
-            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
+            $_size674 = 0;
+            $_ktype675 = 0;
+            $_vtype676 = 0;
+            $xfer += $input->readMapBegin($_ktype675, $_vtype676, $_size674);
+            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
             {
-              $key665 = '';
-              $val666 = '';
-              $xfer += $input->readString($key665);
-              $xfer += $input->readString($val666);
-              $this->partitionSpecs[$key665] = $val666;
+              $key679 = '';
+              $val680 = '';
+              $xfer += $input->readString($key679);
+              $xfer += $input->readString($val680);
+              $this->partitionSpecs[$key679] = $val680;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19336,10 +19675,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter667 => $viter668)
+          foreach ($this->partitionSpecs as $kiter681 => $viter682)
           {
-            $xfer += $output->writeString($kiter667);
-            $xfer += $output->writeString($viter668);
+            $xfer += $output->writeString($kiter681);
+            $xfer += $output->writeString($viter682);
           }
         }
         $output->writeMapEnd();
@@ -19651,17 +19990,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size669 = 0;
-            $_ktype670 = 0;
-            $_vtype671 = 0;
-            $xfer += $input->readMapBegin($_ktype670, $_vtype671, $_size669);
-            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
+            $_size683 = 0;
+            $_ktype684 = 0;
+            $_vtype685 = 0;
+            $xfer += $input->readMapBegin($_ktype684, $_vtype685, $_size683);
+            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
             {
-              $key674 = '';
-              $val675 = '';
-              $xfer += $input->readString($key674);
-              $xfer += $input->readString($val675);
-              $this->partitionSpecs[$key674] = $val675;
+              $key688 = '';
+              $val689 = '';
+              $xfer += $input->readString($key688);
+              $xfer += $input->readString($val689);
+              $this->partitionSpecs[$key688] = $val689;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19717,10 +20056,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter676 => $viter677)
+          foreach ($this->partitionSpecs as $kiter690 => $viter691)
           {
-            $xfer += $output->writeString($kiter676);
-            $xfer += $output->writeString($viter677);
+            $xfer += $output->writeString($kiter690);
+            $xfer += $output->writeString($viter691);
           }
         }
         $output->writeMapEnd();
@@ -19853,15 +20192,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size678 = 0;
-            $_etype681 = 0;
-            $xfer += $input->readListBegin($_etype681, $_size678);
-            for ($_i682 = 0; $_i682 < $_size678; ++$_i682)
+            $_size692 = 0;
+            $_etype695 = 0;
+            $xfer += $input->readListBegin($_etype695, $_size692);
+            for ($_i696 = 0; $_i696 < $_size692; ++$_i696)
             {
-              $elem683 = null;
-              $elem683 = new \metastore\Partition();
-              $xfer += $elem683->read($input);
-              $this->success []= $elem683;
+              $elem697 = null;
+              $elem697 = new \metastore\Partition();
+              $xfer += $elem697->read($input);
+              $this->success []= $elem697;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19921,9 +20260,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter684)
+          foreach ($this->success as $iter698)
           {
-            $xfer += $iter684->write($output);
+            $xfer += $iter698->write($output);
           }
         }
         $output->writeListEnd();
@@ -20069,14 +20408,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size685 = 0;
-            $_etype688 = 0;
-            $xfer += $input->readListBegin($_etype688, $_size685);
-            for ($_i689 = 0; $_i689 < $_size685; ++$_i689)
+            $_size699 = 0;
+            $_etype702 = 0;
+            $xfer += $input->readListBegin($_etype702, $_size699);
+            for ($_i703 = 0; $_i703 < $_size699; ++$_i703)
             {
-              $elem690 = null;
-              $xfer += $input->readString($elem690);
-              $this->part_vals []= $elem690;
+              $elem704 = null;
+              $xfer += $input->readString($elem704);
+              $this->part_vals []= $elem704;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20093,14 +20432,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size691 = 0;
-            $_etype694 = 0;
-            $xfer += $input->readListBegin($_etype694, $_size691);
-            for ($_i695 = 0; $_i695 < $_size691; ++$_i695)
+            $_size705 = 0;
+            $_etype708 = 0;
+            $xfer += $input->readListBegin($_etype708, $_size705);
+            for ($_i709 = 0; $_i709 < $_size705; ++$_i709)
             {
-              $elem696 = null;
-              $xfer += $input->readString($elem696);
-              $this->group_names []= $elem696;
+              $elem710 = null;
+              $xfer += $input->readString($elem710);
+              $this->group_names []= $elem710;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20138,9 +20477,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter697)
+          foreach ($this->part_vals as $iter711)
           {
-            $xfer += $output->writeString($iter697);
+            $xfer += $output->writeString($iter711);
           }
         }
         $output->writeListEnd();
@@ -20160,9 +20499,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter698)
+          foreach ($this->group_names as $iter712)
           {
-            $xfer += $output->writeString($iter698);
+            $xfer += $output->writeString($iter712);
           }
         }
         $output->writeListEnd();
@@ -20753,15 +21092,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size699 = 0;
-            $_etype702 = 0;
-            $xfer += $input->readListBegin($_etype702, $_size699);
-            for ($_i703 = 0; $_i703 < $_size699; ++$_i703)
+            $_size713 = 0;
+            $_etype716 = 0;
+            $xfer += $input->readListBegin($_etype716, $_size713);
+            for ($_i717 = 0; $_i717 < $_size713; ++$_i717)
             {
-              $elem704 = null;
-              $elem704 = new \metastore\Partition();
-              $xfer += $elem704->read($input);
-              $this->success []= $elem704;
+              $elem718 = null;
+              $elem718 = new \metastore\Partition();
+              $xfer += $elem718->read($input);
+              $this->success []= $elem718;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20805,9 +21144,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter705)
+          foreach ($this->success as $iter719)
           {
-            $xfer += $iter705->write($output);
+            $xfer += $iter719->write($output);
           }
         }
         $output->writeListEnd();
@@ -20953,14 +21292,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size706 = 0;
-            $_etype709 = 0;
-            $xfer += $input->readListBegin($_etype709, $_size706);
-            for ($_i710 = 0; $_i710 < $_size706; ++$_i710)
+            $_size720 = 0;
+            $_etype723 = 0;
+            $xfer += $input->readListBegin($_etype723, $_size720);
+            for ($_i724 = 0; $_i724 < $_size720; ++$_i724)
             {
-              $elem711 = null;
-              $xfer += $input->readString($elem711);
-              $this->group_names []= $elem711;
+              $elem725 = null;
+              $xfer += $input->readString($elem725);
+              $this->group_names []= $elem725;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21008,9 +21347,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter712)
+          foreach ($this->group_names as $iter726)
           {
-            $xfer += $output->writeString($iter712);
+            $xfer += $output->writeString($iter726);
           }
         }
         $output->writeListEnd();
@@ -21099,15 +21438,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size713 = 0;
-            $_etype716 = 0;
-            $xfer += $input->readListBegin($_etype716, $_size713);
-            for ($_i717 = 0; $_i717 < $_size713; ++$_i717)
+            $_size727 = 0;
+            $_etype730 = 0;
+            $xfer += $input->readListBegin($_etype730, $_size727);
+            for ($_i731 = 0; $_i731 < $_size727; ++$_i731)
             {
-              $elem718 = null;
-              $elem718 = new \metastore\Partition();
-              $xfer += $elem718->read($input);
-              $this->success []= $elem718;
+              $elem732 = null;
+              $elem732 = new \metastore\Partition();
+              $xfer += $elem732->read($input);
+              $this->success []= $elem732;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21151,9 +21490,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter719)
+          foreach ($this->success as $iter733)
           {
-            $xfer += $iter719->write($output);
+            $xfer += $iter733->write($output);
           }
         }
         $output->writeListEnd();
@@ -21373,15 +21712,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size720 = 0;
-            $_etype723 = 0;
-            $xfer += $input->readListBegin($_etype723, $_size720);
-            for ($_i724 = 0; $_i724 < $_size720; ++$_i724)
+            $_size734 = 0;
+            $_etype737 = 0;
+            $xfer += $input->readListBegin($_etype737, $_size734);
+            for ($_i738 = 0; $_i738 < $_size734; ++$_i738)
             {
-              $elem725 = null;
-              $elem725 = new \metastore\PartitionSpec();
-              $xfer += $elem725->read($input);
-              $this->success []= $elem725;
+              $elem739 = null;
+              $elem739 = new \metastore\PartitionSpec();
+              $xfer += $elem739->read($input);
+              $this->success []= $elem739;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21425,9 +21764,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter726)
+          foreach ($this->success as $iter740)
           {
-            $xfer += $iter726->write($output);
+            $xfer += $iter740->write($output);
           }
         }
         $output->writeListEnd();
@@ -21634,14 +21973,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size727 = 0;
-            $_etype730 = 0;
-            $xfer += $input->readListBegin($_etype730, $_size727);
-            for ($_i731 = 0; $_i731 < $_size727; ++$_i731)
+            $_size741 = 0;
+            $_etype744 = 0;
+            $xfer += $input->readListBegin($_etype744, $_size741);
+            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
             {
-              $elem732 = null;
-              $xfer += $input->readString($elem732);
-              $this->success []= $elem732;
+              $elem746 = null;
+              $xfer += $input->readString($elem746);
+              $this->success []= $elem746;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21677,9 +22016,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter733)
+          foreach ($this->success as $iter747)
           {
-            $xfer += $output->writeString($iter733);
+            $xfer += $output->writeString($iter747);
           }
         }
         $output->writeListEnd();
@@ -21795,14 +22134,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size734 = 0;
-            $_etype737 = 0;
-            $xfer += $input->readListBegin($_etype737, $_size734);
-            for ($_i738 = 0; $_i738 < $_size734; ++$_i738)
+            $_size748 = 0;
+            $_etype751 = 0;
+            $xfer += $input->readListBegin($_etype751, $_size748);
+            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
             {
-              $elem739 = null;
-              $xfer += $input->readString($elem739);
-              $this->part_vals []= $elem739;
+              $elem753 = null;
+              $xfer += $input->readString($elem753);
+              $this->part_vals []= $elem753;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21847,9 +22186,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter740)
+          foreach ($this->part_vals as $iter754)
           {
-            $xfer += $output->writeString($iter740);
+            $xfer += $output->writeString($iter754);
           }
         }
         $output->writeListEnd();
@@ -21943,15 +22282,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size741 = 0;
-            $_etype744 = 0;
-            $xfer += $input->readListBegin($_etype744, $_size741);
-            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
+            $_size755 = 0;
+            $_etype758 = 0;
+            $xfer += $input->readListBegin($_etype758, $_size755);
+            for ($_i759 = 0; $_i759 < $_size755; ++$_i759)
             {
-              $elem746 = null;
-              $elem746 = new \metastore\Partition();
-              $xfer += $elem746->read($input);
-              $this->success []= $elem746;
+              $elem760 = null;
+              $elem760 = new \metastore\Partition();
+              $xfer += $elem760->read($input);
+              $this->success []= $elem760;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21995,9 +22334,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter747)
+          foreach ($this->success as $iter761)
           {
-            $xfer += $iter747->write($output);
+            $xfer += $iter761->write($output);
           }
         }
         $output->writeListEnd();
@@ -22144,14 +22483,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size748 = 0;
-            $_etype751 = 0;
-            $xfer += $input->readListBegin($_etype751, $_size748);
-            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
+            $_size762 = 0;
+            $_etype765 = 0;
+            $xfer += $input->readListBegin($_etype765, $_size762);
+            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
             {
-              $elem753 = null;
-              $xfer += $input->readString($elem753);
-              $this->part_vals []= $elem753;
+              $elem767 = null;
+              $xfer += $input->readString($elem767);
+              $this->part_vals []= $elem767;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22175,14 +22514,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size754 = 0;
-            $_etype757 = 0;
-            $xfer += $input->readListBegin($_etype757, $_size754);
-            for ($_i758 = 0; $_i758 < $_size754; ++$_i758)
+            $_size768 = 0;
+            $_etype771 = 0;
+            $xfer += $input->readListBegin($_etype771, $_size768);
+            for ($_i772 = 0; $_i772 < $_size768; ++$_i772)
             {
-              $elem759 = null;
-              $xfer += $input->readString($elem759);
-              $this->group_names []= $elem759;
+              $elem773 = null;
+              $xfer += $input->readString($elem773);
+              $this->group_names []= $elem773;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22220,9 +22559,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter760)
+          foreach ($this->part_vals as $iter774)
           {
-            $xfer += $output->writeString($iter760);
+            $xfer += $output->writeString($iter774);
           }
         }
         $output->writeListEnd();
@@ -22247,9 +22586,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter761)
+          foreach ($this->group_names as $iter775)
           {
-            $xfer += $output->writeString($iter761);
+            $xfer += $output->writeString($iter775);
           }
         }
         $output->writeListEnd();
@@ -22338,15 +22677,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size762 = 0;
-            $_etype765 = 0;
-            $xfer += $input->readListBegin($_etype765, $_size762);
-            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
+            $_size776 = 0;
+            $_etype779 = 0;
+            $xfer += $input->readListBegin($_etype779, $_size776);
+            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
             {
-              $elem767 = null;
-              $elem767 = new \metastore\Partition();
-              $xfer += $elem767->read($input);
-              $this->success []= $elem767;
+              $elem781 = null;
+              $elem781 = new \metastore\Partition();
+              $xfer += $elem781->read($input);
+              $this->success []= $elem781;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22390,9 +22729,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter768)
+          foreach ($this->success as $iter782)
           {
-            $xfer += $iter768->write($output);
+            $xfer += $iter782->write($output);
           }
         }
         $output->writeListEnd();
@@ -22513,14 +22852,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size769 = 0;
-            $_etype772 = 0;
-            $xfer += $input->readListBegin($_etype772, $_size769);
-            for ($_i773 = 0; $_i773 < $_size769; ++$_i773)
+            $_size783 = 0;
+            $_etype786 = 0;
+            $xfer += $input->readListBegin($_etype786, $_size783);
+            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
             {
-              $elem774 = null;
-              $xfer += $input->readString($elem774);
-              $this->part_vals []= $elem774;
+              $elem788 = null;
+              $xfer += $input->readString($elem788);
+              $this->part_vals []= $elem788;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22565,9 +22904,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter775)
+          foreach ($this->part_vals as $iter789)
           {
-            $xfer += $output->writeString($iter775);
+            $xfer += $output->writeString($iter789);
           }
         }
         $output->writeListEnd();
@@ -22660,14 +22999,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size776 = 0;
-            $_etype779 = 0;
-            $xfer += $input->readListBegin($_etype779, $_size776);
-            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
+            $_size790 = 0;
+            $_etype793 = 0;
+            $xfer += $input->readListBegin($_etype793, $_size790);
+            for ($_i794 = 0; $_i794 < $_size790; ++$_i794)
             {
-              $elem781 = null;
-              $xfer += $input->readString($elem781);
-              $this->success []= $elem781;
+              $elem795 = null;
+              $xfer += $input->readString($elem795);
+              $this->success []= $elem795;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22711,9 +23050,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter782)
+          foreach ($this->success as $iter796)
           {
-            $xfer += $output->writeString($iter782);
+            $xfer += $output->writeString($iter796);
           }
         }
         $output->writeListEnd();
@@ -22956,15 +23295,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size783 = 0;
-            $_etype786 = 0;
-            $xfer += $input->readListBegin($_etype786, $_size783);
-            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
+            $_size797 = 0;
+            $_etype800 = 0;
+            $xfer += $input->readListBegin($_etype800, $_size797);
+            for ($_i801 = 0; $_i801 < $_size797; ++$_i801)
             {
-              $elem788 = null;
-              $elem788 = new \metastore\Partition();
-              $xfer += $elem788->read($input);
-              $this->success []= $elem788;
+              $elem802 = null;
+              $elem802 = new \metastore\Partition();
+              $xfer += $elem802->read($input);
+              $this->success []= $elem802;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23008,9 +23347,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter789)
+          foreach ($this->success as $iter803)
           {
-            $xfer += $iter789->write($output);
+            $xfer += $iter803->write($output);
           }
         }
         $output->writeListEnd();
@@ -23253,15 +23592,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size790 = 0;
-            $_etype793 = 0;
-            $xfer += $input->readListBegin($_etype793, $_size790);
-            for ($_i794 = 0; $_i794 < $_size790; ++$_i794)
+            $_size804 = 0;
+            $_etype807 = 0;
+            $xfer += $input->readListBegin($_etype807, $_size804);
+            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
             {
-              $elem795 = null;
-              $elem795 = new \metastore\PartitionSpec();
-              $xfer += $elem795->read($input);
-              $this->success []= $elem795;
+              $elem809 = null;
+              $elem809 = new \metastore\PartitionSpec();
+              $xfer += $elem809->read($input);
+              $this->success []= $elem809;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23305,9 +23644,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter796)
+          foreach ($this->success as $iter810)
           {
-            $xfer += $iter796->write($output);
+            $xfer += $iter810->write($output);
           }
         }
         $output->writeListEnd();
@@ -23627,14 +23966,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size797 = 0;
-            $_etype800 = 0;
-            $xfer += $input->readListBegin($_etype800, $_size797);
-            for ($_i801 = 0; $_i801 < $_size797; ++$_i801)
+            $_size811 = 0;
+            $_etype814 = 0;
+            $xfer += $input->readListBegin($_etype814, $_size811);
+            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
             {
-              $elem802 = null;
-              $xfer += $input->readString($elem802);
-              $this->names []= $elem802;
+              $elem816 = null;
+              $xfer += $input->readString($elem816);
+              $this->names []= $elem816;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23672,9 +24011,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter803)
+          foreach ($this->names as $iter817)
           {
-            $xfer += $output->writeString($iter803);
+            $xfer += $output->writeString($iter817);
           }
         }
         $output->writeListEnd();
@@ -23763,15 +24102,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size804 = 0;
-            $_etype807 = 0;
-            $xfer += $input->readListBegin($_etype807, $_size804);
-            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
+            $_size818 = 0;
+            $_etype821 = 0;
+            $xfer += $input->readListBegin($_etype821, $_size818);
+            for ($_i822 = 0; $_i822 < $_size818; ++$_i822)
             {
-              $elem809 = null;
-              $elem809 = new \metastore\Partition();
-              $xfer += $elem809->read($input);
-              $this->success []= $elem809;
+              $elem823 = null;
+              $elem823 = new \metastore\Partition();
+              $xfer += $elem823->read($input);
+              $this->success []= $elem823;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23815,9 +24154,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter810)
+          foreach ($this->success as $iter824)
           {
-            $xfer += $iter810->write($output);
+            $xfer += $iter824->write($output);
           }
         }
         $output->writeListEnd();
@@ -24156,15 +24495,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size811 = 0;
-            $_etype814 = 0;
-            $xfer += $input->readListBegin($_etype814, $_size811);
-            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
+            $_size825 = 0;
+            $_etype828 = 0;
+            $xfer += $input->readListBegin($_etype828, $_size825);
+            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
             {
-              $elem816 = null;
-              $elem816 = new \metastore\Partition();
-              $xfer += $elem816->read($input);
-              $this->new_parts []= $elem816;
+              $elem830 = null;
+              $elem830 = new \metastore\Partition();
+              $xfer += $elem830->read($input);
+              $this->new_parts []= $elem830;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24202,9 +24541,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter817)
+          foreach ($this->new_parts as $iter831)
           {
-            $xfer += $iter817->write($output);
+            $xfer += $iter831->write($output);
           }
         }
         $output->writeListEnd();
@@ -24674,14 +25013,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size818 = 0;
-            $_etype821 = 0;
-            $xfer += $input->readListBegin($_etype821, $_size818);
-            for ($_i822 = 0; $_i822 < $_size818; ++$_i822)
+            $_size832 = 0;
+            $_etype835 = 0;
+            $xfer += $input->readListBegin($_etype835, $_size832);
+            for ($_i836 = 0; $_i836 < $_size832; ++$_i836)
             {
-              $elem823 = null;
-              $xfer += $input->readString($elem823);
-              $this->part_vals []= $elem823;
+              $elem837 = null;
+              $xfer += $input->readString($elem837);
+              $this->part_vals []= $elem837;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24727,9 +25066,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter824)
+          foreach ($this->part_vals as $iter838)
           {
-            $xfer += $output->writeString($iter824);
+            $xfer += $output->writeString($iter838);
           }
         }
         $output->writeListEnd();
@@ -24914,14 +25253,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size825 = 0;
-            $_etype828 = 0;
-            $xfer += $input->readListBegin($_etype828, $_size825);
-            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
+            $_size839 = 0;
+            $_etype842 = 0;
+            $xfer += $input->readListBegin($_etype842, $_size839);
+            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
             {
-              $elem830 = null;
-              $xfer += $input->readString($elem830);
-              $this->part_vals []= $elem830;
+              $elem844 = null;
+              $xfer += $input->readString($elem844);
+              $this->part_vals []= $elem844;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24956,9 +25295,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter831)
+          foreach ($this->part_vals as $iter845)
           {
-            $xfer += $output->writeString($iter831);
+            $xfer += $output->writeString($iter845);
           }
         }
         $output->writeListEnd();
@@ -25412,14 +25751,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size832 = 0;
-            $_etype835 = 0;
-            $xfer += $input->readListBegin($_etype835, $_size832);
-            for ($_i836 = 0; $_i836 < $_size832; ++$_i836)
+            $_size846 = 0;
+            $_etype849 = 0;
+            $xfer += $input->readListBegin($_etype849, $_size846);
+            for ($_i850 = 0; $_i850 < $_size846; ++$_i850)
             {
-              $elem837 = null;
-              $xfer += $input->readString($elem837);
-              $this->success []= $elem837;
+              $elem851 = null;
+              $xfer += $input->readString($elem851);
+              $this->success []= $elem851;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25455,9 +25794,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter838)
+          foreach ($this->success as $iter852)
           {
-            $xfer += $output->writeString($iter838);
+            $xfer += $output->writeString($iter852);
           }
         }
         $output->writeListEnd();
@@ -25617,17 +25956,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size839 = 0;
-            $_ktype840 = 0;
-            $_vtype841 = 0;
-            $xfer += $input->readMapBegin($_ktype840, $_vtype841, $_size839);
-            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
+            $_size853 = 0;
+            $_ktype854 = 0;
+            $_vtype855 = 0;
+            $xfer += $input->readMapBegin($_ktype854, $_vtype855, $_size853);
+            for ($_i857 = 0; $_i857 < $_size853; ++$_i857)
             {
-              $key844 = '';
-              $val845 = '';
-              $xfer += $input->readString($key844);
-              $xfer += $input->readString($val845);
-              $this->success[$key844] = $val845;
+              $key858 = '';
+              $val859 = '';
+              $xfer += $input->readString($key858);
+              $xfer += $input->readString($val859);
+              $this->success[$key858] = $val859;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25663,10 +26002,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter846 => $viter847)
+          foreach ($this->success as $kiter860 => $viter861)
           {
-            $xfer += $output->writeString($kiter846);
-            $xfer += $output->writeString($viter847);
+            $xfer += $output->writeString($kiter860);
+            $xfer += $output->writeString($viter861);
           }
         }
         $output->writeMapEnd();
@@ -25786,17 +26125,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size848 = 0;
-            $_ktype849 = 0;
-            $_vtype850 = 0;
-            $xfer += $input->readMapBegin($_ktype849, $_vtype850, $_size848);
-            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
+            $_size862 = 0;
+            $_ktype863 = 0;
+            $_vtype864 = 0;
+            $xfer += $input->readMapBegin($_ktype863, $_vtype864, $_size862);
+            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
             {
-              $key853 = '';
-              $val854 = '';
-              $xfer += $input->readString($key853);
-              $xfer += $input->readString($val854);
-              $this->part_vals[$key853] = $val854;
+              $key867 = '';
+              $val868 = '';
+              $xfer += $input->readString($key867);
+              $xfer += $input->readString($val868);
+              $this->part_vals[$key867] = $val868;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25841,10 +26180,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter855 => $viter856)
+          foreach ($this->part_vals as $kiter869 => $viter870)
           {
-            $xfer += $output->writeString($kiter855);
-            $xfer += $output->writeString($viter856);
+            $xfer += $output->writeString($kiter869);
+            $xfer += $output->writeString($viter870);
           }
         }
         $output->writeMapEnd();
@@ -26166,17 +26505,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size857 = 0;
-            $_ktype858 = 0;
-            $_vtype859 = 0;
-            $xfer += $input->readMapBegin($_ktype858, $_vtype859, $_size857);
-            for ($_i861 = 0; $_i861 < $_size857; ++$_i861)
+            $_size871 = 0;
+            $_ktype872 = 0;
+            $_vtype873 = 0;
+            $xfer += $input->readMapBegin($_ktype872, $_vtype873, $_size871);
+            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
             {
-              $key862 = '';
-              $val863 = '';
-              $xfer += $input->readString($key862);
-              $xfer += $input->readString($val863);
-              $this->part_vals[$key862] = $val863;
+              $key876 = '';
+              $val877 = '';
+              $xfer += $input->readString($key876);
+              $xfer += $input->readString($val877);
+              $this->part_vals[$key876] = $val877;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -26221,10 +26560,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter864 => $viter865)
+          foreach ($this->part_vals as $kiter878 => $viter879)
           {
-            $xfer += $output->writeString($kiter864);
-            $xfer += $output->writeString($viter865);
+            $xfer += $output->writeString($kiter878);
+            $xfer += $output->writeString($viter879);
           }
         }
         $output->writeMapEnd();
@@ -27698,15 +28037,15 @@ class ThriftHiveMetastore_get_indexes_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size866 = 0;
-            $_etype869 = 0;
-            $xfer += $input->readListBegin($_etype869, $_size866);
-            for ($_i870 = 0; $_i870 < $_size866; ++$_i870)
+            $_size880 = 0;
+            $_etype883 = 0;
+            $xfer += $input->readListBegin($_etype883, $_size880);
+            for ($_i884 = 0; $_i884 < $_size880; ++$_i884)
             {
-              $elem871 = null;
-              $elem871 = new \metastore\Index();
-              $xfer += $elem871->read($input);
-              $this->success []= $elem871;
+              $elem885 = null;
+              $elem885 = new \metastore\Index();
+              $xfer += $elem885->read($input);
+              $this->success []= $elem885;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27750,9 +28089,9 @@ class ThriftHiveMetastore_get_indexes_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter872)
+          foreach ($this->success as $iter886)
           {
-            $xfer += $iter872->write($output);
+            $xfer += $iter886->write($output);
           }
         }
         $output->writeListEnd();
@@ -27959,14 +28298,14 @@ class ThriftHiveMetastore_get_index_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size873 = 0;
-            $_etype876 = 0;
-            $xfer += $input->readListBegin($_etype876, $_size873);
-            for ($_i877 = 0; $_i877 < $_size873; ++$_i877)
+            $_size887 = 0;
+            $_etype890 = 0;
+            $xfer += $input->readListBegin($_etype890, $_size887);
+            for ($_i891 = 0; $_i891 < $_size887; ++$_i891)
             {
-              $elem878 = null;
-              $xfer += $input->readString($elem878);
-              $this->success []= $elem878;
+              $elem892 = null;
+              $xfer += $input->readString($elem892);
+              $this->success []= $elem892;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28002,9 +28341,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter879)
+          foreach ($this->success as $iter893)
           {
-            $xfer += $output->writeString($iter879);
+            $xfer += $output->writeString($iter893);
           }
         }
         $output->writeListEnd();
@@ -31478,14 +31817,14 @@ class ThriftHiveMetastore_get_functions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size880 = 0;
-            $_etype883 = 0;
-            $xfer += $input->readListBegin($_etype883, $_size880);
-            for ($_i884 = 0; $_i884 < $_size880; ++$_i884)
+            $_size894 = 0;
+            $_etype897 = 0;
+            $xfer += $input->readListBegin($_etype897, $_size894);
+            for ($_i898 = 0; $_i898 < $_size894; ++$_i898)
             {
-              $elem885 = null;
-              $xfer += $input->readString($elem885);
-              $this->success []= $elem885;
+              $elem899 = null;
+              $xfer += $input->readString($elem899);
+              $this->success []= $elem899;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31521,9 +31860,9 @@ class ThriftHiveMetastore_get_functions_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter886)
+          foreach ($this->success as $iter900)
           {
-            $xfer += $output->writeString($iter886);
+            $xfer += $output->writeString($iter900);
           }
         }
         $output->writeListEnd();
@@ -32392,14 +32731,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size887 = 0;
-            $_etype890 = 0;
-            $xfer += $input->readListBegin($_etype890, $_size887);
-            for ($_i891 = 0; $_i891 < $_size887; ++$_i891)
+            $_size901 = 0;
+            $_etype904 = 0;
+            $xfer += $input->readListBegin($_etype904, $_size901);
+            for ($_i905 = 0; $_i905 < $_size901; ++$_i905)
             {
-              $elem892 = null;
-              $xfer += $input->readString($elem892);
-              $this->success []= $elem892;
+              $elem906 = null;
+              $xfer += $input->readString($elem906);
+              $this->success []= $elem906;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32435,9 +32774,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter893)
+          foreach ($this->success as $iter907)
           {
-            $xfer += $output->writeString($iter893);
+            $xfer += $output->writeString($iter907);
           }
         }
         $output->writeListEnd();
@@ -33128,15 +33467,15 @@ class ThriftHiveMetastore_list_roles_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size894 = 0;
-            $_etype897 = 0;
-            $xfer += $input->readListBegin($_etype897, $_size894);
-            for ($_i898 = 0; $_i898 < $_size894; ++$_i898)
+            $_size908 = 0;
+            $_etype911 = 0;
+            $xfer += $input->readListBegin($_etype911, $_size908);
+            for ($_i912 = 0; $_i912 < $_size908; ++$_i912)
             {
-              $elem899 = null;
-              $elem899 = new \metastore\Role();
-              $xfer += $elem899->read($input);
-              $this->success []= $elem899;
+              $elem913 = null;
+              $elem913 = new \metastore\Role();
+              $xfer += $elem913->read($input);
+              $this->success []= $elem913;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33172,9 +33511,9 @@ class ThriftHiveMetastore_list_roles_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter900)
+          foreach ($this->success as $iter914)
           {
-            $xfer += $iter900->write($output);
+            $xfer += $iter914->write($output);
           }
         }
         $output->writeListEnd();
@@ -33836,14 +34175,14 @@ class ThriftHiveMetastore_get_privilege_set_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size901 = 0;
-            $_etype904 = 0;
-            $xfer += $input->readListBegin($_etype904, $_size901);
-            for ($_i905 = 0; $_i905 < $_size901; ++$_i905)
+            $_size915 = 0;
+            $_etype918 = 0;
+            $xfer += $input->readListBegin($_etype918, $_size915);
+            for ($_i919 = 0; $_i919 < $_size915; ++$_i919)
             {
-              $elem906 = null;
-              $xfer += $input->readString($elem906);
-              $this->group_names []= $elem906;
+              $elem920 = null;
+              $xfer += $input->readString($elem920);
+              $this->group_names []= $elem920;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33884,9 +34223,9 @@ class ThriftHiveMetastore_get_privilege_set_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter907)
+          foreach ($this->group_names as $iter921)
           {
-            $xfer += $output->writeString($iter907);
+            $xfer += $output->writeString($iter921);
           }
         }
         $output->writeListEnd();
@@ -34194,15 +34533,15 @@ class ThriftHiveMetastore_list_privileges_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size908 = 0;
-            $_etype911 = 0;
-            $xfer += $input->readListBegin($_etype911, $_size908);
-            for ($_i912 = 0; $_i912 < $_size908; ++$_i912)
+            $_size922 = 0;
+            $_etype925 = 0;
+            $xfer += $input->readListBegin($_etype925, $_size922);
+            for ($_i926 = 0; $_i926 < $_size922; ++$_i926)
             {
-              $elem913 = null;
-              $elem913 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem913->read($input);
-              $this->success []= $elem913;
+              $elem927 = null;
+              $elem927 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem927->read($input);
+              $this->success []= $elem927;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34238,9 +34577,9 @@ class ThriftHiveMetastore_list_privileges_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter914)
+          foreach ($this->success as $iter928)
           {
-            $xfer += $iter914->write($output);
+            $xfer += $iter928->write($output);
           }
         }
         $output->writeListEnd();
@@ -34872,14 +35211,14 @@ class ThriftHiveMetastore_set_ugi_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size915 = 0;
-            $_etype918 = 0;
-            $xfer += $input->readListBegin($_etype918, $_size915);
-            for ($_i919 = 0; $_i919 < $_size915; ++$_i919)
+            $_size929 = 0;
+            $_etype932 = 0;
+            $xfer += $input->readListBegin($_etype932, $_size929);
+            for ($_i933 = 0; $_i933 < $_size929; ++$_i933)
             {
-              $elem920 = null;
-              $xfer += $input->readString($elem920);
-              $this->group_names []= $elem920;
+              $elem934 = null;
+              $xfer += $input->readString($elem934);
+              $this->group_names []= $elem934;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34912,9 +35251,9 @@ class ThriftHiveMetastore_set_ugi_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter921)
+          foreach ($this->group_names as $iter935)
           {
-            $xfer += $output->writeString($iter921);
+            $xfer += $output->writeString($iter935);
           }
         }
         $output->writeListEnd();
@@ -34990,14 +35329,14 @@ class ThriftHiveMetastore_set_ugi_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size922 = 0;
-            $_etype925 = 0;
-            $xfer += $input->readListBegin($_etype925, $_size922);
-            for ($_i926 = 0; $_i926 < $_size922; ++$_i926)
+            $_size936 = 0;
+            $_etype939 = 0;
+            $xfer += $input->readListBegin($_etype939, $_size936);
+            for ($_i940 = 0; $_i940 < $_size936; ++$_i940)
             {
-              $elem927 = null;
-              $xfer += $input->readString($elem927);
-              $this->success []= $elem927;
+              $elem941 = null;
+              $xfer += $input->readString($elem941);
+              $this->success []= $elem941;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35033,9 +35372,9 @@ class ThriftHiveMetastore_set_ugi_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter928)
+          foreach ($this->success as $iter942)
           {
-            $xfer += $output->writeString($iter928);
+            $xfer += $output->writeString($iter942);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 e63213d..e43a13d 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -14957,6 +14957,150 @@ class GetAllFunctionsResponse {
 
 }
 
+class TableMeta {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $tableName = null;
+  /**
+   * @var string
+   */
+  public $tableType = null;
+  /**
+   * @var string
+   */
+  public $comments = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tableName',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'tableType',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'comments',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['tableName'])) {
+        $this->tableName = $vals['tableName'];
+      }
+      if (isset($vals['tableType'])) {
+        $this->tableType = $vals['tableType'];
+      }
+      if (isset($vals['comments'])) {
+        $this->comments = $vals['comments'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'TableMeta';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->comments);
+          } 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('TableMeta');
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableName !== null) {
+      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 2);
+      $xfer += $output->writeString($this->tableName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableType !== null) {
+      $xfer += $output->writeFieldBegin('tableType', TType::STRING, 3);
+      $xfer += $output->writeString($this->tableType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->comments !== null) {
+      $xfer += $output->writeFieldBegin('comments', TType::STRING, 4);
+      $xfer += $output->writeString($this->comments);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class MetaException extends TException {
   static $_TSPEC;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/558b35cf/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 65ba10e..22d794f 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -45,6 +45,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void drop_table(string dbname, string name, bool deleteData)')
   print('  void drop_table_with_environment_context(string dbname, string name, bool deleteData, EnvironmentContext environment_context)')
   print('   get_tables(string db_name, string pattern)')
+  print('   get_table_meta(string db_patterns, string tbl_patterns,  tbl_types)')
   print('   get_all_tables(string db_name)')
   print('  Table get_table(string dbname, string tbl_name)')
   print('   get_table_objects_by_name(string dbname,  tbl_names)')
@@ -349,6 +350,12 @@ elif cmd == 'get_tables':
     sys.exit(1)
   pp.pprint(client.get_tables(args[0],args[1],))
 
+elif cmd == 'get_table_meta':
+  if len(args) != 3:
+    print('get_table_meta requires 3 args')
+    sys.exit(1)
+  pp.pprint(client.get_table_meta(args[0],args[1],eval(args[2]),))
+
 elif cmd == 'get_all_tables':
   if len(args) != 1:
     print('get_all_tables requires 1 args')