You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by an...@apache.org on 2017/02/27 20:25:22 UTC

[1/2] incubator-trafodion git commit: TRAFODION-2498 Add support to run hive stmts from traf interface

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master 2abcb0338 -> 63ab72874


TRAFODION-2498 Add support to run hive stmts from traf interface

Syntax:
  process hive statement '<string>';

  <string>: hive statement starting with create/drop/alter/truncate.
      These are the only stmts currently supported.
  Ex:
   >>process hive statement 'create database trafsch';
   will create hive database 'trafsch'

   >>process hive statement 'create table trafsch.t (a int)';
   will create hive table 't' in hive schema 'trafsch'.

  'process hive statment ..' could be issued from any traf interface
   (sqlci/trafci/jdbc...)


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

Branch: refs/heads/master
Commit: e9389fa010c031669e97385f264d758043b9e4ac
Parents: 549850a
Author: Anoop Sharma <an...@esgyn.com>
Authored: Fri Feb 24 22:34:09 2017 +0000
Committer: Anoop Sharma <an...@esgyn.com>
Committed: Fri Feb 24 22:34:09 2017 +0000

----------------------------------------------------------------------
 core/sql/comexe/ComTdb.h             |   1 +
 core/sql/comexe/ComTdbExeUtil.cpp    |  74 +++++++++++++++++
 core/sql/comexe/ComTdbExeUtil.h      |  49 ++++++++++-
 core/sql/executor/ExComTdb.cpp       |   8 ++
 core/sql/executor/ExExeUtil.h        |  96 ++++++++++++++++++++++
 core/sql/executor/ExExeUtilGet.cpp   |  40 +++++++--
 core/sql/executor/ExExeUtilMisc.cpp  | 125 ++++++++++++++++++++++++++++
 core/sql/generator/GenRelExeUtil.cpp |  63 +++++++++++++-
 core/sql/optimizer/RelExeUtil.cpp    |  56 +++++++++++++
 core/sql/optimizer/RelExeUtil.h      |  42 +++++++++-
 core/sql/parser/ParKeyWords.cpp      |   1 +
 core/sql/parser/sqlparser.y          |  29 ++++++-
 core/sql/regress/hive/EXPECTED005    | 132 +++++++++++++++---------------
 core/sql/regress/hive/EXPECTED007    |  86 +++++++++++--------
 core/sql/regress/hive/TEST005        |  20 ++---
 core/sql/regress/hive/TEST007        |  80 ++++++------------
 core/sql/regress/seabase/EXPECTED003 | 114 +++++++++++++-------------
 core/sql/regress/seabase/EXPECTED031 |  39 +++++----
 core/sql/regress/seabase/EXPECTED033 |  21 +++--
 core/sql/regress/seabase/TEST003     |  16 +---
 core/sql/regress/seabase/TEST031     |  10 +--
 core/sql/regress/seabase/TEST033     |  13 ++-
 core/sql/sqlci/sqlci_lex.ll          |   1 +
 core/sql/sqlci/sqlci_yacc.y          |   2 +
 core/sql/sqlcomp/parser.cpp          |   2 +-
 25 files changed, 831 insertions(+), 289 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/comexe/ComTdb.h
----------------------------------------------------------------------
diff --git a/core/sql/comexe/ComTdb.h b/core/sql/comexe/ComTdb.h
index f062bcb..0cdb244 100644
--- a/core/sql/comexe/ComTdb.h
+++ b/core/sql/comexe/ComTdb.h
@@ -322,6 +322,7 @@ public:
     ex_GET_QID = 152,
     ex_HIVE_TRUNCATE = 153,
     ex_LOB_UPDATE_UTIL = 154,
+    ex_HIVE_QUERY = 155,
     ex_LAST = 9999              // not used
   };
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/comexe/ComTdbExeUtil.cpp
----------------------------------------------------------------------
diff --git a/core/sql/comexe/ComTdbExeUtil.cpp b/core/sql/comexe/ComTdbExeUtil.cpp
index a10dff3..2d0e477 100644
--- a/core/sql/comexe/ComTdbExeUtil.cpp
+++ b/core/sql/comexe/ComTdbExeUtil.cpp
@@ -1420,6 +1420,80 @@ void ComTdbExeUtilHiveTruncate::displayContents(Space * space,
 
 ///////////////////////////////////////////////////////////////////////////
 //
+// Methods for class ComTdbExeUtilHiveQuery
+//
+///////////////////////////////////////////////////////////////////////////
+ComTdbExeUtilHiveQuery::ComTdbExeUtilHiveQuery(
+     char * hiveQuery,
+     ULng32 hiveQueryLen,
+     ex_cri_desc * given_cri_desc,
+     ex_cri_desc * returned_cri_desc,
+     queue_index down,
+     queue_index up,
+     Lng32 num_buffers,
+     ULng32 buffer_size)
+     : ComTdbExeUtil(ComTdbExeUtil::HIVE_QUERY_,
+		     NULL, 0, (Int16)SQLCHARSETCODE_UNKNOWN,
+		     NULL, 0,
+		     NULL, 0,
+		     NULL, 0,
+		     NULL,
+                     NULL, 0,
+		     given_cri_desc, returned_cri_desc,
+		     down, up, 
+		     num_buffers, buffer_size),
+       flags_(0),
+       hiveQuery_(hiveQuery),
+       hiveQueryLen_(hiveQueryLen)
+{
+  setNodeType(ComTdb::ex_HIVE_QUERY);
+}
+
+Long ComTdbExeUtilHiveQuery::pack(void * space)
+{
+  if (hiveQuery_)
+    hiveQuery_.pack(space);
+
+  return ComTdbExeUtil::pack(space);
+}
+
+Lng32 ComTdbExeUtilHiveQuery::unpack(void * base, void * reallocator)
+{
+  if(hiveQuery_.unpack(base))
+    return -1;
+
+  return ComTdbExeUtil::unpack(base, reallocator);
+}
+
+void ComTdbExeUtilHiveQuery::displayContents(Space * space,
+					      ULng32 flag)
+{
+  ComTdb::displayContents(space,flag & 0xFFFFFFFE);
+  
+  if(flag & 0x00000008)
+    {
+      char buf[500];
+      str_sprintf(buf, "\nFor ComTdbExeUtilHiveQuery :");
+      space->allocateAndCopyToAlignedSpace(buf, str_len(buf), sizeof(short));
+      
+      if (getHiveQuery() != NULL)
+	{
+	  str_sprintf(buf,"HiveQuery = %s ",getHiveQuery());
+	  space->allocateAndCopyToAlignedSpace(buf, str_len(buf), 
+					       sizeof(short));
+	}
+    }
+  
+  if (flag & 0x00000001)
+    {
+      displayExpression(space,flag);
+      displayChildren(space,flag);
+    }
+
+}
+
+///////////////////////////////////////////////////////////////////////////
+//
 // Methods for class ComTdbExeUtilGetStatistics
 //
 ///////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/comexe/ComTdbExeUtil.h
----------------------------------------------------------------------
diff --git a/core/sql/comexe/ComTdbExeUtil.h b/core/sql/comexe/ComTdbExeUtil.h
index eb0b56d..58c032f 100644
--- a/core/sql/comexe/ComTdbExeUtil.h
+++ b/core/sql/comexe/ComTdbExeUtil.h
@@ -91,7 +91,8 @@ public:
     HBASE_UNLOAD_TASK_       = 34,
     GET_QID_                 = 35,
     HIVE_TRUNCATE_           = 36,
-    LOB_UPDATE_UTIL_         = 37
+    LOB_UPDATE_UTIL_         = 37,
+    HIVE_QUERY_              = 38
   };
 
   ComTdbExeUtil()
@@ -1714,6 +1715,52 @@ private:
   UInt32 flags_;                                 // 44-47
 };
 
+class ComTdbExeUtilHiveQuery : public ComTdbExeUtil
+{
+public:
+  ComTdbExeUtilHiveQuery()
+  : ComTdbExeUtil()
+  {}
+
+  ComTdbExeUtilHiveQuery(char * hiveQuery,
+                         ULng32 hiveQueryLen,
+                         ex_cri_desc * given_cri_desc,
+                         ex_cri_desc * returned_cri_desc,
+                         queue_index down,
+                         queue_index up,
+                         Lng32 num_buffers,
+                         ULng32 buffer_size
+                         );
+  
+  Long pack(void *);
+  Lng32 unpack(void *, void * reallocator);
+
+  // ---------------------------------------------------------------------
+  // Redefine virtual functions required for Versioning.
+  //----------------------------------------------------------------------
+  virtual short getClassSize() {return (short)sizeof(ComTdbExeUtilHiveQuery);}
+
+  virtual const char *getNodeName() const
+  {
+    return "HIVE_QUERY";
+  };
+
+  char * getHiveQuery() const
+  {
+    return hiveQuery_;
+  }
+
+  // ---------------------------------------------------------------------
+  // Used by the internal SHOWPLAN command to get attributes of a TDB.
+  // ---------------------------------------------------------------------
+  NA_EIDPROC void displayContents(Space *space, ULng32 flag);
+
+private:
+  NABasicPtr hiveQuery_;                     // 00-07
+  UInt32 hiveQueryLen_;                      // 08-11
+  UInt32 flags_;                             // 12-15
+};
+
 class ComTdbExeUtilGetStatistics : public ComTdbExeUtil
 {
   friend class ExExeUtilGetStatisticsTcb;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/executor/ExComTdb.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExComTdb.cpp b/core/sql/executor/ExComTdb.cpp
index df54e7b..a68f80f 100644
--- a/core/sql/executor/ExComTdb.cpp
+++ b/core/sql/executor/ExComTdb.cpp
@@ -436,6 +436,14 @@ NA_EIDPROC char *ComTdb::findVTblPtrExe(short classID)
       break;
     }
 
+    case ex_HIVE_QUERY:
+    {
+#pragma nowarn(1506)   // warning elimination 
+      GetVTblPtr(vtblptr,ExExeUtilHiveQueryTdb);
+#pragma warn(1506)  // warning elimination 
+      break;
+    }
+
     case ex_PROCESS_VOLATILE_TABLE:
     {
 #pragma nowarn(1506)   // warning elimination 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/executor/ExExeUtil.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExExeUtil.h b/core/sql/executor/ExExeUtil.h
index 46b2d5d..ead2e42 100755
--- a/core/sql/executor/ExExeUtil.h
+++ b/core/sql/executor/ExExeUtil.h
@@ -87,6 +87,7 @@ class ExExeUtilDisplayExplainTdb;
 class ExExeUtilDisplayExplainComplexTdb;
 class ExExeUtilFastDeleteTdb;
 class ExExeUtilHiveTruncateTdb;
+class ExExeUtilHiveQueryTdb;
 class ExExeUtilSuspendTdb;
 class ExExeUtilSuspendTcb;
 class ExpHbaseInterface;
@@ -435,6 +436,7 @@ class ExExeUtilPrivateState : public ex_tcb_private_state
   friend class ExExeUtilCreateTableAsTcb;
   friend class ExExeUtilFastDeleteTcb;
   friend class ExExeUtilHiveTruncateTcb;
+  friend class ExExeUtilHiveQueryTcb;
   friend class ExExeUtilAQRTcb;
   friend class ExExeUtilHBaseBulkLoadTcb;
   friend class ExExeUtilHBaseBulkUnLoadTcb;
@@ -3486,6 +3488,100 @@ class ExExeUtilHiveTruncatePrivateState : public ex_tcb_private_state
  protected:
 };
 
+// -----------------------------------------------------------------------
+// ExExeUtilHiveQueryTdb
+// -----------------------------------------------------------------------
+class ExExeUtilHiveQueryTdb : public ComTdbExeUtilHiveQuery
+{
+ public:
+
+  // ---------------------------------------------------------------------
+  // Constructor is only called to instantiate an object used for
+  // retrieval of the virtual table function pointer of the class while
+  // unpacking. An empty constructor is enough.
+  // ---------------------------------------------------------------------
+  NA_EIDPROC ExExeUtilHiveQueryTdb()
+    {}
+
+  NA_EIDPROC virtual ~ExExeUtilHiveQueryTdb()
+    {}
+
+  // ---------------------------------------------------------------------
+  // Build a TCB for this TDB. Redefined in the Executor project.
+  // ---------------------------------------------------------------------
+  NA_EIDPROC virtual ex_tcb *build(ex_globals *globals);
+
+ private:
+  // ---------------------------------------------------------------------
+  // !!!!!!! IMPORTANT -- NO DATA MEMBERS ALLOWED IN EXECUTOR TDB !!!!!!!!
+  // *********************************************************************
+  // The Executor TDB's are only used for the sole purpose of providing a
+  // way to supplement the Compiler TDB's (in comexe) with methods whose
+  // implementation depends on Executor objects. This is done so as to
+  // decouple the Compiler from linking in Executor objects unnecessarily.
+  //
+  // When a Compiler generated TDB arrives at the Executor, the same data
+  // image is "cast" as an Executor TDB after unpacking. Therefore, it is
+  // a requirement that a Compiler TDB has the same object layout as its
+  // corresponding Executor TDB. As a result of this, all Executor TDB's
+  // must have absolutely NO data members, but only member functions. So,
+  // if you reach here with an intention to add data members to a TDB, ask
+  // yourself two questions:
+  //
+  // 1. Are those data members Compiler-generated?
+  //    If yes, put them in the ComTdbDLL instead.
+  //    If no, they should probably belong to someplace else (like TCB).
+  //
+  // 2. Are the classes those data members belong defined in the executor
+  //    project?
+  //    If your answer to both questions is yes, you might need to move
+  //    the classes to the comexe project.
+  // ---------------------------------------------------------------------
+};
+
+///////////////////////////////////////////////////////////////
+// ExExeUtilHiveQueryTcb
+///////////////////////////////////////////////////////////////
+class ExExeUtilHiveQueryTcb : public ExExeUtilTcb
+{
+ public:
+  // Constructor
+  ExExeUtilHiveQueryTcb(const ComTdbExeUtilHiveQuery & exe_util_tdb,
+                        ex_globals * glob = 0);
+
+  ~ExExeUtilHiveQueryTcb();
+
+  virtual short work();
+
+  virtual ex_tcb_private_state * allocatePstates(
+       Lng32 &numElems,      // inout, desired/actual elements
+       Lng32 &pstateLength); // out, length of one element
+
+ private:
+  enum Step
+    {
+      INITIAL_,
+      ERROR_,
+      PROCESS_QUERY_,
+      DONE_
+    };
+
+  ExExeUtilHiveQueryTdb & htTdb() const
+    {return (ExExeUtilHiveQueryTdb &) tdb;};
+
+  Step step_;
+};
+
+class ExExeUtilHiveQueryPrivateState : public ex_tcb_private_state
+{
+  friend class ExExeUtilHiveQueryTcb;
+
+ public:
+  ExExeUtilHiveQueryPrivateState();
+  ~ExExeUtilHiveQueryPrivateState();        // destructor
+ protected:
+};
+
 //////////////////////////////////////////////////////////////////////////
 // -----------------------------------------------------------------------
 // ExExeUtilHbaseLoadTdb

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/executor/ExExeUtilGet.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExExeUtilGet.cpp b/core/sql/executor/ExExeUtilGet.cpp
index 969a55b..bba6ce7 100644
--- a/core/sql/executor/ExExeUtilGet.cpp
+++ b/core/sql/executor/ExExeUtilGet.cpp
@@ -2840,9 +2840,15 @@ short ExExeUtilGetMetadataInfoComplexTcb::work()
 
 	    if (fetchAllRows(infoList_, queryBuf_, 1, FALSE, retcode) < 0)
 	      {
-		step_ = HANDLE_ERROR_;
+                // if error is 4222 (command not supported), ignore it.
+                if (getDiagsArea()->mainSQLCODE() != -4222)
+                  {
+                    step_ = HANDLE_ERROR_;
+                    
+                    break;
+                  }
 
-		break;
+                getDiagsArea()->clear();
 	      }
 
 	    // insert a NULL entry, this will cause a blank row to be returned
@@ -2859,9 +2865,15 @@ short ExExeUtilGetMetadataInfoComplexTcb::work()
 
 	    if (fetchAllRows(infoList_, queryBuf_, 1, FALSE, retcode) < 0)
 	      {
-		step_ = HANDLE_ERROR_;
+                // if error is 4222 (command not supported), ignore it.
+                if (getDiagsArea()->mainSQLCODE() != -4222)
+                  {
+                    step_ = HANDLE_ERROR_;
+                    
+                    break;
+                  }
 
-		break;
+                getDiagsArea()->clear();
 	      }
 
 	    // insert a NULL entry, this will cause a blank row to be returned
@@ -2978,9 +2990,15 @@ short ExExeUtilGetMetadataInfoComplexTcb::work()
 	      {
 		if (fetchAllRows(infoList_, queryBuf_, 1, FALSE, retcode) < 0)
 		  {
-		    step_ = HANDLE_ERROR_;
+                    // if error is 4222 (command not supported), ignore it.
+                    if (getDiagsArea()->mainSQLCODE() != -4222)
+                      {
+                        step_ = HANDLE_ERROR_;
+                        
+                        break;
+                      }
 
-		    break;
+                    getDiagsArea()->clear();
 		  }
 	      }
 
@@ -3001,9 +3019,15 @@ short ExExeUtilGetMetadataInfoComplexTcb::work()
 	      {
 		if (fetchAllRows(infoList_, queryBuf_, 1, FALSE, retcode) < 0)
 		  {
-		    step_ = HANDLE_ERROR_;
+                    // if error is 4222 (command not supported), ignore it.
+                    if (getDiagsArea()->mainSQLCODE() != -4222)
+                      {
+                        step_ = HANDLE_ERROR_;
+                        
+                        break;
+                      }
 
-		    break;
+                    getDiagsArea()->clear();
 		  }
 	      }
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/executor/ExExeUtilMisc.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExExeUtilMisc.cpp b/core/sql/executor/ExExeUtilMisc.cpp
index 946a7b2..dc9808a 100644
--- a/core/sql/executor/ExExeUtilMisc.cpp
+++ b/core/sql/executor/ExExeUtilMisc.cpp
@@ -2509,3 +2509,128 @@ ExExeUtilHiveTruncatePrivateState::ExExeUtilHiveTruncatePrivateState()
 ExExeUtilHiveTruncatePrivateState::~ExExeUtilHiveTruncatePrivateState()
 {
 };
+
+////////////////////////////////////////////////////////////////
+// Constructor for class ExExeUtilHiveQueryTdb
+///////////////////////////////////////////////////////////////
+ex_tcb * ExExeUtilHiveQueryTdb::build(ex_globals * glob)
+{
+  ExExeUtilTcb * exe_util_tcb;
+
+  exe_util_tcb = new(glob->getSpace()) ExExeUtilHiveQueryTcb(*this, glob);
+  exe_util_tcb->registerSubtasks();
+
+  return (exe_util_tcb);
+}
+
+
+////////////////////////////////////////////////////////////////
+// Constructor for class ExExeUtilHiveQueryTcb
+///////////////////////////////////////////////////////////////
+ExExeUtilHiveQueryTcb::ExExeUtilHiveQueryTcb(
+     const ComTdbExeUtilHiveQuery & exe_util_tdb,
+     ex_globals * glob)
+     : ExExeUtilTcb( exe_util_tdb, NULL, glob)
+{
+  // Allocate the private state in each entry of the down queue
+  qparent_.down->allocatePstate(this);
+
+  step_ = INITIAL_;
+}
+
+ExExeUtilHiveQueryTcb::~ExExeUtilHiveQueryTcb()
+{
+}
+
+//////////////////////////////////////////////////////
+// work() for ExExeUtilHiveQueryTsb
+//////////////////////////////////////////////////////
+short ExExeUtilHiveQueryTcb::work()
+{
+  short rc = 0;
+  Lng32 cliRC = 0;
+
+  // if no parent request, return
+  if (qparent_.down->isEmpty())
+    return WORK_OK;
+
+  // if no room in up queue, won't be able to return data/status.
+  // Come back later.
+  if (qparent_.up->isFull())
+    return WORK_OK;
+
+  ex_queue_entry * pentry_down = qparent_.down->getHeadEntry();
+  ExExeUtilPrivateState & pstate = *((ExExeUtilPrivateState*) pentry_down->pstate);
+
+  while (1)
+    {
+      switch (step_)
+        {
+        case INITIAL_:
+          {
+            step_ = PROCESS_QUERY_;
+          }
+          break;
+          
+        case PROCESS_QUERY_:
+          {
+            ComDiagsArea * diags = getDiagsArea();
+
+            if (!CmpCommon::context()->execHiveSQL(htTdb().getHiveQuery(),
+                                                   diags))
+              {
+                step_ = ERROR_;
+                break;
+              }
+
+            step_ = DONE_;
+          }
+          break;
+          
+        case ERROR_:
+          {
+            if (handleError())
+              return WORK_OK;
+
+	    getDiagsArea()->clear();
+            
+            step_ = DONE_;
+          }
+          break;
+          
+        case DONE_:
+          {
+            if (handleDone())
+              return WORK_OK;
+            
+            step_ = INITIAL_;
+            
+            return WORK_OK;
+          }
+          break;
+          
+        } // switch
+    } // while
+  
+}
+
+
+ex_tcb_private_state * ExExeUtilHiveQueryTcb::allocatePstates(
+     Lng32 &numElems,      // inout, desired/actual elements
+     Lng32 &pstateLength)  // out, length of one element
+{
+  PstateAllocator<ExExeUtilHiveQueryPrivateState> pa;
+
+  return pa.allocatePstates(this, numElems, pstateLength);
+}
+
+/////////////////////////////////////////////////////////////////////////////
+// Constructor and destructor for ExeUtil_private_state
+/////////////////////////////////////////////////////////////////////////////
+ExExeUtilHiveQueryPrivateState::ExExeUtilHiveQueryPrivateState()
+{
+}
+
+ExExeUtilHiveQueryPrivateState::~ExExeUtilHiveQueryPrivateState()
+{
+};

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/generator/GenRelExeUtil.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenRelExeUtil.cpp b/core/sql/generator/GenRelExeUtil.cpp
index 6dabe11..5318dc9 100644
--- a/core/sql/generator/GenRelExeUtil.cpp
+++ b/core/sql/generator/GenRelExeUtil.cpp
@@ -3428,6 +3428,67 @@ short ExeUtilHiveTruncate::codeGen(Generator * generator)
   return 0;
 }
 
+/////////////////////////////////////////////////////////
+//
+// ExeUtilHiveQuery::codeGen()
+//
+/////////////////////////////////////////////////////////
+short ExeUtilHiveQuery::codeGen(Generator * generator)
+{
+  ExpGenerator * expGen = generator->getExpGenerator();
+  Space * space = generator->getSpace();
+
+  // allocate a map table for the retrieved columns
+  generator->appendAtEnd();
+
+  ex_cri_desc * givenDesc
+    = generator->getCriDesc(Generator::DOWN);
+
+  ex_cri_desc * returnedDesc
+    = new(space) ex_cri_desc(givenDesc->noTuples() + 1, space);
+
+  ex_cri_desc * workCriDesc = new(space) ex_cri_desc(4, space);
+  const Int32 work_atp = 1;
+  const Int32 exe_util_row_atp_index = 2;
+
+  short rc = processOutputRow(generator, work_atp, exe_util_row_atp_index,
+                              returnedDesc);
+  if (rc)
+    {
+      return -1;
+    }
+
+  char * hive_query = 
+    space->AllocateAndCopyToAlignedSpace (hiveQuery(), 0);
+  Lng32 hive_query_len = hiveQuery().length();
+
+  ComTdbExeUtilHiveQuery * exe_util_tdb = 
+    new(space) 
+    ComTdbExeUtilHiveQuery(hive_query, hive_query_len,
+                           (ex_cri_desc *)(generator->getCriDesc(Generator::DOWN)),
+                           (ex_cri_desc *)(generator->getCriDesc(Generator::DOWN)),
+                           (queue_index)getDefault(GEN_DDL_SIZE_DOWN),
+                           (queue_index)getDefault(GEN_DDL_SIZE_UP),
+                           getDefault(GEN_DDL_NUM_BUFFERS),
+                           getDefault(GEN_DDL_BUFFER_SIZE));
+
+  generator->initTdbFields(exe_util_tdb);
+
+  if(!generator->explainDisabled()) {
+    generator->setExplainTuple(
+       addExplainInfo(exe_util_tdb, 0, 0, generator));
+  }
+
+  // no tupps are returned 
+  generator->setCriDesc((ex_cri_desc *)(generator->getCriDesc(Generator::DOWN)),
+			Generator::UP);
+  generator->setGenObj(this, exe_util_tdb);
+
+  generator->setTransactionFlag(0); // transaction is not needed.
+  
+  return 0;
+}
+
 ////////////////////////////////////////////////////////////////////
 // class ExeUtilRegionStats
 ////////////////////////////////////////////////////////////////////
@@ -3617,7 +3678,7 @@ short ExeUtilLobInfo::codeGen(Generator * generator)
       return -1;
     }
 
- NAString tn = "\"";
+  NAString tn = "\"";
   tn += getTableName().getQualifiedNameObj().getCatalogName();
   tn += "\".";
   tn += getTableName().getQualifiedNameObj().getSchemaName();

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/optimizer/RelExeUtil.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelExeUtil.cpp b/core/sql/optimizer/RelExeUtil.cpp
index f593a9a..e439258 100644
--- a/core/sql/optimizer/RelExeUtil.cpp
+++ b/core/sql/optimizer/RelExeUtil.cpp
@@ -900,6 +900,62 @@ RelExpr * ExeUtilHiveTruncate::copyTopNode(RelExpr *derivedNode, CollHeap* outHe
   return ExeUtilExpr::copyTopNode(result, outHeap);
 }
 
+
+// -----------------------------------------------------------------------
+// Member functions for class ExeUtilHiveQuery
+// -----------------------------------------------------------------------
+RelExpr * ExeUtilHiveQuery::copyTopNode(RelExpr *derivedNode,
+                                        CollHeap* outHeap)
+{
+  ExeUtilHiveQuery *result;
+
+  if (derivedNode == NULL)
+    result = new (outHeap) 
+      ExeUtilHiveQuery(hiveQuery(),
+                       sourceType(),
+                       outHeap);
+  else
+    result = (ExeUtilHiveQuery *) derivedNode;
+
+  return ExeUtilExpr::copyTopNode(result, outHeap);
+}
+
+RelExpr * ExeUtilHiveQuery::bindNode(BindWA *bindWA)
+{
+  if (type_ != FROM_STRING)
+    {
+      // error case
+      *CmpCommon::diags() << DgSqlCode(-3242) << DgString0("DDL can only be specified as a string.");
+      
+      bindWA->setErrStatus();
+      return NULL;
+    }
+
+  // currently supported hive queries must start with:
+  //   create, drop, alter, truncate
+  // Check for it.
+
+  // first strip leading spaces.
+  hiveQuery_ = hiveQuery_.strip(NAString::leading, ' ');
+  if (NOT ((hiveQuery_.index("CREATE", 0, NAString::ignoreCase) == 0) ||
+           (hiveQuery_.index("DROP", 0, NAString::ignoreCase) == 0) ||
+           (hiveQuery_.index("ALTER", 0, NAString::ignoreCase) == 0) ||
+           (hiveQuery_.index("TRUNCATE", 0, NAString::ignoreCase) == 0)))
+    {
+      // error case
+      *CmpCommon::diags() << DgSqlCode(-3242) << DgString0("Only CREATE, DROP, ALTER or TRUNCATE hive DDL statements can be specified.");
+      
+      bindWA->setErrStatus();
+      return NULL;
+    }
+
+  RelExpr * boundExpr = ExeUtilExpr::bindNode(bindWA);
+  if (bindWA->errStatus()) 
+    return NULL;
+  
+  return boundExpr;
+}
+
 // -----------------------------------------------------------------------
 // Member functions for class ExeUtilGetStatistics
 // -----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/optimizer/RelExeUtil.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelExeUtil.h b/core/sql/optimizer/RelExeUtil.h
index ba4ff0b..03630b7 100644
--- a/core/sql/optimizer/RelExeUtil.h
+++ b/core/sql/optimizer/RelExeUtil.h
@@ -577,7 +577,8 @@ public:
     ORC_FAST_AGGR_            = 37,
     GET_QID_                  = 38,
     HIVE_TRUNCATE_            = 39,
-    LOB_UPDATE_UTIL_          = 40
+    LOB_UPDATE_UTIL_          = 40,
+    HIVE_QUERY_               = 41
   };
 
   ExeUtilExpr(ExeUtilType type,
@@ -1167,6 +1168,45 @@ private:
   NABoolean dropTableOnDealloc_;
 };
 
+class ExeUtilHiveQuery : public ExeUtilExpr
+{
+public:
+  enum HiveSourceType
+    {
+      FROM_STRING,
+      FROM_FILE
+    };
+
+  ExeUtilHiveQuery(const NAString &hive_query,
+                   HiveSourceType type,
+                   CollHeap *oHeap = CmpCommon::statementHeap())
+       : ExeUtilExpr(HIVE_QUERY_, CorrName("dummyName"), 
+                     NULL, NULL, 
+                     NULL,
+                     CharInfo::UnknownCharSet, oHeap),
+         type_(type),
+         hiveQuery_(hive_query)
+  { }
+
+  virtual NABoolean isExeUtilQueryType() { return TRUE; }
+
+  virtual RelExpr * copyTopNode(RelExpr *derivedNode = NULL,
+				CollHeap* outHeap = 0);
+
+  virtual RelExpr * bindNode(BindWA *bindWAPtr);
+
+  // method to do code generation
+  virtual short codeGen(Generator*);
+  
+  NAString &hiveQuery() { return hiveQuery_; }
+  const NAString &hiveQuery() const { return hiveQuery_; }
+
+  HiveSourceType sourceType() { return type_;}
+private:
+  HiveSourceType type_;
+  NAString hiveQuery_;
+};
+
 class ExeUtilMaintainObject : public ExeUtilExpr
 {
 public:

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/parser/ParKeyWords.cpp
----------------------------------------------------------------------
diff --git a/core/sql/parser/ParKeyWords.cpp b/core/sql/parser/ParKeyWords.cpp
index 4f190de..19589a8 100644
--- a/core/sql/parser/ParKeyWords.cpp
+++ b/core/sql/parser/ParKeyWords.cpp
@@ -490,6 +490,7 @@ ParKeyWord ParKeyWords::keyWords_[] = {
   ParKeyWord("HEX",                TOK_HEX,         NONRESTOKEN_),
   ParKeyWord("HEXADECIMAL",        TOK_HEXADECIMAL, NONRESTOKEN_),
   ParKeyWord("HIGH_VALUE",         TOK_HIGH_VALUE,  NONRESTOKEN_),
+  ParKeyWord("HIVE",               TOK_HIVE,  NONRESTOKEN_),
   ParKeyWord("HIVEMD",                    TOK_HIVEMD,  NONRESTOKEN_),
   ParKeyWord("HOLD",               TOK_HOLD,        THIRD_|NONRESTOKEN_),
   ParKeyWord("HORIZONTAL",         TOK_HORIZONTAL,  NONRESTOKEN_),

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/parser/sqlparser.y
----------------------------------------------------------------------
diff --git a/core/sql/parser/sqlparser.y b/core/sql/parser/sqlparser.y
index 0c5c723..46db6ed 100755
--- a/core/sql/parser/sqlparser.y
+++ b/core/sql/parser/sqlparser.y
@@ -705,6 +705,7 @@ static void enableMakeQuotedStringISO88591Mechanism()
 %token <tokval> TOK_GROUP_CONCAT
 %token <tokval> TOK_GZIP
 %token <tokval> TOK_HAVING
+%token <tokval> TOK_HIVE
 %token <tokval> TOK_HIVEMD
 %token <tokval> TOK_QUALIFY
 %token <tokval> TOK_HEADER
@@ -2824,6 +2825,7 @@ static void enableMakeQuotedStringISO88591Mechanism()
 %type <relx>                    exe_util_get_uid
 %type <relx>                    exe_util_get_qid
 %type <relx>                    exe_util_get_lob_info
+%type <relx>                    exe_util_hive_query
 %type <relx>                    exe_util_populate_in_memory_statistics
 %type <relx>                    exe_util_lob_extract
 %type <relx>                    exe_util_lob_update
@@ -14737,10 +14739,10 @@ interactive_query_expression:
 				{
 				  $$ = finalize($1);
 				}
-           |  osim_statement
-              {
-                  $$ = finalize($1);
-              }
+              |  osim_statement
+                                {
+                                  $$ = finalize($1);
+                                }
               | set_statement
 				{ 
 				  $$ = finalize($1);
@@ -14942,6 +14944,10 @@ interactive_query_expression:
 				  $$ = finalize($1);
 				}
 
+              | exe_util_hive_query
+                                {
+                                  $$ = finalize($1);
+                                }
               | TOK_SELECT TOK_UUID '(' ')'
 	                        {
 				  NAString * v = new (PARSERHEAP()) NAString("1");
@@ -16827,6 +16833,20 @@ exe_util_get_lob_info : TOK_GET TOK_LOB stats_or_statistics TOK_FOR TOK_TABLE ta
                  $$ = new (PARSERHEAP()) 
                    ExeUtilLobInfo(*$6, FALSE,NULL,  PARSERHEAP());
 	       } 
+
+exe_util_hive_query : TOK_PROCESS TOK_HIVE TOK_STATEMENT QUOTED_STRING
+                      {
+                        $$ = new (PARSERHEAP()) 
+                          ExeUtilHiveQuery(*$4, ExeUtilHiveQuery::FROM_STRING,
+                                           PARSERHEAP());
+                      } 
+                    | TOK_PROCESS TOK_HIVE TOK_STATEMENT TOK_FROM TOK_FILE QUOTED_STRING
+                      {
+                        $$ = new (PARSERHEAP()) 
+                          ExeUtilHiveQuery(*$6, ExeUtilHiveQuery::FROM_FILE,
+                                           PARSERHEAP());
+                      } 
+
      
 /*
  * The purpose of dummy_token_lookahead is to force the lexer to look
@@ -34113,6 +34133,7 @@ nonreserved_func_word:  TOK_ABS
                       | TOK_HASH2PARTFUNC
                       | TOK_HBASE_TIMESTAMP
                       | TOK_HBASE_VERSION
+                      | TOK_HIVE
                       | TOK_HIVEMD
                       | TOK_INET_ATON
                       | TOK_INET_NTOA

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/regress/hive/EXPECTED005
----------------------------------------------------------------------
diff --git a/core/sql/regress/hive/EXPECTED005 b/core/sql/regress/hive/EXPECTED005
index ba76df3..5ad0c0b 100644
--- a/core/sql/regress/hive/EXPECTED005
+++ b/core/sql/regress/hive/EXPECTED005
@@ -72,7 +72,7 @@
 
 *** WARNING[8597] Statement was automatically retried 1 time(s). Delay before each retry was 0 seconds. See next entry for the error that caused this retry. 
 
-*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1485388879, failedModTS = 1485388945, failedLoc = hdfs://localhost:27000/user/hive/exttables/customer_ddl
+*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1487805469, failedModTS = 1487805508, failedLoc = hdfs://localhost:24200/user/hive/exttables/customer_ddl
 
 C_PREFERRED_CUST_FLAG      (EXPR)              
 -------------------------  --------------------
@@ -108,7 +108,7 @@ Y                                          9525
 
 *** WARNING[8597] Statement was automatically retried 1 time(s). Delay before each retry was 0 seconds. See next entry for the error that caused this retry. 
 
-*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1485388879, failedModTS = 1485388945, failedLoc = hdfs://localhost:27000/user/hive/exttables/customer_ddl
+*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1487805469, failedModTS = 1487805508, failedLoc = hdfs://localhost:24200/user/hive/exttables/customer_ddl
 
 C_PREFERRED_CUST_FLAG      (EXPR)              
 -------------------------  --------------------
@@ -177,7 +177,7 @@ Y                                          9525
 
 *** WARNING[8597] Statement was automatically retried 1 time(s). Delay before each retry was 0 seconds. See next entry for the error that caused this retry. 
 
-*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1485388949, failedModTS = 1485388954, failedLoc = hdfs://localhost:27000/user/hive/warehouse/newtable
+*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1487805512, failedModTS = 1487805521, failedLoc = hdfs://localhost:24200/user/hive/warehouse/newtable
 
 A                        
 -------------------------
@@ -227,7 +227,7 @@ xyz
 
 *** WARNING[8597] Statement was automatically retried 1 time(s). Delay before each retry was 0 seconds. See next entry for the error that caused this retry. 
 
-*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1485388945, failedModTS = 1485388975, failedLoc = hdfs://localhost:27000/user/hive/exttables/customer_ddl
+*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1487805508, failedModTS = 1487805540, failedLoc = hdfs://localhost:24200/user/hive/exttables/customer_ddl
 
 C_PREFERRED_CUST_FLAG      (EXPR)              
 -------------------------  --------------------
@@ -242,7 +242,7 @@ Y                                         18984
 
 *** WARNING[8597] Statement was automatically retried 1 time(s). Delay before each retry was 0 seconds. See next entry for the error that caused this retry. 
 
-*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1485388945, failedModTS = 1485388975, failedLoc = hdfs://localhost:27000/user/hive/exttables/customer_ddl
+*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1487805508, failedModTS = 1487805540, failedLoc = hdfs://localhost:24200/user/hive/exttables/customer_ddl
 
 C_PREFERRED_CUST_FLAG      (EXPR)              
 -------------------------  --------------------
@@ -298,7 +298,7 @@ Y                                         18984
 
 *** WARNING[8597] Statement was automatically retried 1 time(s). Delay before each retry was 0 seconds. See next entry for the error that caused this retry. 
 
-*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1485388971, failedModTS = 1485388977, failedLoc = hdfs://localhost:27000/user/hive/warehouse/newtable
+*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1487805537, failedModTS = 1487805547, failedLoc = hdfs://localhost:24200/user/hive/warehouse/newtable
 
 A            B                        
 -----------  -------------------------
@@ -617,18 +617,18 @@ C1           C2                    C3                         C4               C
 --- 1 row(s) selected.
 >>load with continue on error into trafodion.seabase.traf_tbl_bad select * from tbl_bad;
 Task:  LOAD            Status: Started    Object: TRAFODION.SEABASE.TRAF_TBL_BAD
-Task:  CLEANUP         Status: Started    Time: 2017-01-26 00:04:00.936
-Task:  CLEANUP         Status: Ended      Time: 2017-01-26 00:04:00.954
-Task:  CLEANUP         Status: Ended      Elapsed Time:    00:00:00.018
-Task:  LOADING DATA    Status: Started    Time: 2017-01-26 00:04:00.955
+Task:  CLEANUP         Status: Started    Time: 2017-02-22 23:20:03.392
+Task:  CLEANUP         Status: Ended      Time: 2017-02-22 23:20:03.405
+Task:  CLEANUP         Status: Ended      Elapsed Time:    00:00:00.012
+Task:  LOADING DATA    Status: Started    Time: 2017-02-22 23:20:03.405
        Rows Processed: 8 
        Error Rows:     5 
-Task:  LOADING DATA    Status: Ended      Time: 2017-01-26 00:04:01.231
-Task:  LOADING DATA    Status: Ended      Elapsed Time:    00:00:00.276
-Task:  COMPLETION      Status: Started    Time: 2017-01-26 00:04:01.231
+Task:  LOADING DATA    Status: Ended      Time: 2017-02-22 23:20:03.802
+Task:  LOADING DATA    Status: Ended      Elapsed Time:    00:00:00.397
+Task:  COMPLETION      Status: Started    Time: 2017-02-22 23:20:03.802
        Rows Loaded:    3 
-Task:  COMPLETION      Status: Ended      Time: 2017-01-26 00:04:01.663
-Task:  COMPLETION      Status: Ended      Elapsed Time:    00:00:00.432
+Task:  COMPLETION      Status: Ended      Time: 2017-02-22 23:20:04.443
+Task:  COMPLETION      Status: Ended      Elapsed Time:    00:00:00.641
 
 --- 3 row(s) loaded.
 >>select count(*) from trafodion.seabase.traf_tbl_bad;
@@ -644,19 +644,19 @@ Task:  COMPLETION      Status: Ended      Elapsed Time:    00:00:00.432
 --- 3 row(s) deleted.
 >>load with log error rows into trafodion.seabase.traf_tbl_bad select * from tbl_bad;
 Task:  LOAD            Status: Started    Object: TRAFODION.SEABASE.TRAF_TBL_BAD
-Task:  CLEANUP         Status: Started    Time: 2017-01-26 00:04:02.707
-Task:  CLEANUP         Status: Ended      Time: 2017-01-26 00:04:02.714
-Task:  CLEANUP         Status: Ended      Elapsed Time:    00:00:00.007
-       Logging Location: /bulkload/logs/ERR_TRAFODION.SEABASE.TRAF_TBL_BAD_20170126_000402
-Task:  LOADING DATA    Status: Started    Time: 2017-01-26 00:04:02.714
+Task:  CLEANUP         Status: Started    Time: 2017-02-22 23:20:05.530
+Task:  CLEANUP         Status: Ended      Time: 2017-02-22 23:20:05.544
+Task:  CLEANUP         Status: Ended      Elapsed Time:    00:00:00.015
+       Logging Location: /bulkload/logs/ERR_TRAFODION.SEABASE.TRAF_TBL_BAD_20170222_232005
+Task:  LOADING DATA    Status: Started    Time: 2017-02-22 23:20:05.544
        Rows Processed: 8 
        Error Rows:     5 
-Task:  LOADING DATA    Status: Ended      Time: 2017-01-26 00:04:03.117
-Task:  LOADING DATA    Status: Ended      Elapsed Time:    00:00:00.404
-Task:  COMPLETION      Status: Started    Time: 2017-01-26 00:04:03.117
+Task:  LOADING DATA    Status: Ended      Time: 2017-02-22 23:20:06.487
+Task:  LOADING DATA    Status: Ended      Elapsed Time:    00:00:00.504
+Task:  COMPLETION      Status: Started    Time: 2017-02-22 23:20:06.488
        Rows Loaded:    3 
-Task:  COMPLETION      Status: Ended      Time: 2017-01-26 00:04:03.702
-Task:  COMPLETION      Status: Ended      Elapsed Time:    00:00:00.585
+Task:  COMPLETION      Status: Ended      Time: 2017-02-22 23:20:07.504
+Task:  COMPLETION      Status: Ended      Elapsed Time:    00:00:01.455
 
 --- 3 row(s) loaded.
 >>select count(*) from trafodion.seabase.traf_tbl_bad;
@@ -669,19 +669,19 @@ Task:  COMPLETION      Status: Ended      Elapsed Time:    00:00:00.585
 --- 1 row(s) selected.
 >>load with log error rows to '/bulkload/logs/TEST005' into trafodion.seabase.traf_tbl_bad select * from tbl_bad;
 Task:  LOAD            Status: Started    Object: TRAFODION.SEABASE.TRAF_TBL_BAD
-Task:  CLEANUP         Status: Started    Time: 2017-01-26 00:04:04.711
-Task:  CLEANUP         Status: Ended      Time: 2017-01-26 00:04:04.724
-Task:  CLEANUP         Status: Ended      Elapsed Time:    00:00:00.013
-       Logging Location: /bulkload/logs/TEST005/ERR_TRAFODION.SEABASE.TRAF_TBL_BAD_20170126_000404
-Task:  LOADING DATA    Status: Started    Time: 2017-01-26 00:04:04.724
+Task:  CLEANUP         Status: Started    Time: 2017-02-22 23:20:08.497
+Task:  CLEANUP         Status: Ended      Time: 2017-02-22 23:20:08.514
+Task:  CLEANUP         Status: Ended      Elapsed Time:    00:00:00.017
+       Logging Location: /bulkload/logs/TEST005/ERR_TRAFODION.SEABASE.TRAF_TBL_BAD_20170222_232008
+Task:  LOADING DATA    Status: Started    Time: 2017-02-22 23:20:08.514
        Rows Processed: 8 
        Error Rows:     5 
-Task:  LOADING DATA    Status: Ended      Time: 2017-01-26 00:04:05.154
-Task:  LOADING DATA    Status: Ended      Elapsed Time:    00:00:00.430
-Task:  COMPLETION      Status: Started    Time: 2017-01-26 00:04:05.154
+Task:  LOADING DATA    Status: Ended      Time: 2017-02-22 23:20:09.440
+Task:  LOADING DATA    Status: Ended      Elapsed Time:    00:00:00.529
+Task:  COMPLETION      Status: Started    Time: 2017-02-22 23:20:09.440
        Rows Loaded:    3 
-Task:  COMPLETION      Status: Ended      Time: 2017-01-26 00:04:05.752
-Task:  COMPLETION      Status: Ended      Elapsed Time:    00:00:00.599
+Task:  COMPLETION      Status: Ended      Time: 2017-02-22 23:20:09.664
+Task:  COMPLETION      Status: Ended      Elapsed Time:    00:00:00.621
 
 --- 3 row(s) loaded.
 >>select count(*) from trafodion.seabase.traf_tbl_bad;
@@ -697,10 +697,10 @@ Task:  COMPLETION      Status: Ended      Elapsed Time:    00:00:00.599
 --- 6 row(s) deleted.
 >>load with stop after 3 error rows into trafodion.seabase.traf_tbl_bad select * from tbl_bad;
 Task:  LOAD            Status: Started    Object: TRAFODION.SEABASE.TRAF_TBL_BAD
-Task:  CLEANUP         Status: Started    Time: 2017-01-26 00:04:06.808
-Task:  CLEANUP         Status: Ended      Time: 2017-01-26 00:04:06.815
-Task:  CLEANUP         Status: Ended      Elapsed Time:    00:00:00.007
-Task:  LOADING DATA    Status: Started    Time: 2017-01-26 00:04:06.815
+Task:  CLEANUP         Status: Started    Time: 2017-02-22 23:20:10.706
+Task:  CLEANUP         Status: Ended      Time: 2017-02-22 23:20:10.717
+Task:  CLEANUP         Status: Ended      Elapsed Time:    00:00:00.011
+Task:  LOADING DATA    Status: Started    Time: 2017-02-22 23:20:10.717
 
 *** ERROR[8113] The maximum number of error rows is exceeded.
 
@@ -715,11 +715,11 @@ Task:  LOADING DATA    Status: Started    Time: 2017-01-26 00:04:06.815
 --- 1 row(s) selected.
 >>load with log error rows, stop after 3 error rows into trafodion.seabase.traf_tbl_bad select * from tbl_bad;
 Task:  LOAD            Status: Started    Object: TRAFODION.SEABASE.TRAF_TBL_BAD
-Task:  CLEANUP         Status: Started    Time: 2017-01-26 00:04:08.538
-Task:  CLEANUP         Status: Ended      Time: 2017-01-26 00:04:08.668
-Task:  CLEANUP         Status: Ended      Elapsed Time:    00:00:00.013
-       Logging Location: /bulkload/logs/ERR_TRAFODION.SEABASE.TRAF_TBL_BAD_20170126_000408
-Task:  LOADING DATA    Status: Started    Time: 2017-01-26 00:04:08.669
+Task:  CLEANUP         Status: Started    Time: 2017-02-22 23:20:12.677
+Task:  CLEANUP         Status: Ended      Time: 2017-02-22 23:20:12.877
+Task:  CLEANUP         Status: Ended      Elapsed Time:    00:00:00.020
+       Logging Location: /bulkload/logs/ERR_TRAFODION.SEABASE.TRAF_TBL_BAD_20170222_232012
+Task:  LOADING DATA    Status: Started    Time: 2017-02-22 23:20:12.877
 
 *** ERROR[8113] The maximum number of error rows is exceeded.
 
@@ -789,11 +789,12 @@ Task:  LOADING DATA    Status: Started    Time: 2017-01-26 00:04:08.669
 
 --- SQL operation complete.
 >>
->>sh echo "drop table thive;" > TEST005_junk;
->>sh regrhive.ksh -f TEST005_junk;
->>
->>sh echo "create table thive(a int);" > TEST005_junk;
->>sh regrhive.ksh -f TEST005_junk;
+>>process hive statement 'drop table thive';
+
+--- SQL operation complete.
+>>process hive statement 'create table thive(a int)';
+
+--- SQL operation complete.
 >>
 >>select a from hive.hive.thive;
 
@@ -806,7 +807,7 @@ Task:  LOADING DATA    Status: Started    Time: 2017-01-26 00:04:08.669
 
 *** WARNING[8597] Statement was automatically retried 1 time(s). Delay before each retry was 0 seconds. See next entry for the error that caused this retry. 
 
-*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1485389081, failedModTS = 1485389096, failedLoc = hdfs://localhost:27000/user/hive/warehouse/thive
+*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1487805630, failedModTS = 1487805645, failedLoc = hdfs://localhost:24200/user/hive/warehouse/thive
 
 A          
 -----------
@@ -827,11 +828,12 @@ A
 
 --- 2 row(s) selected.
 >>
->>sh echo "drop table thive;" > TEST005_junk;
->>sh regrhive.ksh -f TEST005_junk;
->>
->>sh echo "create table thive(a int, b int);" > TEST005_junk;
->>sh regrhive.ksh -f TEST005_junk;
+>>process hive statement 'drop table thive';
+
+--- SQL operation complete.
+>>process hive statement 'create table thive(a int, b int)';
+
+--- SQL operation complete.
 >>
 >>sh echo "insert into thive values (1,2);" > TEST005_junk;
 >>sh regrhive.ksh -f TEST005_junk;
@@ -840,7 +842,7 @@ A
 
 *** WARNING[8597] Statement was automatically retried 1 time(s). Delay before each retry was 0 seconds. See next entry for the error that caused this retry. 
 
-*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1485389099, failedModTS = 1485389135, failedLoc = hdfs://localhost:27000/user/hive/warehouse/thive
+*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1487805647, failedModTS = 1487805663, failedLoc = hdfs://localhost:24200/user/hive/warehouse/thive
 
 A          
 -----------
@@ -867,7 +869,7 @@ A            B
 
 *** WARNING[8597] Statement was automatically retried 1 time(s). Delay before each retry was 0 seconds. See next entry for the error that caused this retry. 
 
-*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1485389135, failedModTS = 1485389138, failedLoc = hdfs://localhost:27000/user/hive/warehouse/thive
+*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1487805663, failedModTS = 1487805668, failedLoc = hdfs://localhost:24200/user/hive/warehouse/thive
 
 --- 0 row(s) selected.
 >>insert into hive.hive.thive values (10, 20);
@@ -891,15 +893,17 @@ A            B
 
 *** WARNING[8597] Statement was automatically retried 1 time(s). Delay before each retry was 0 seconds. See next entry for the error that caused this retry. 
 
-*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1485389140, failedModTS = 1485389142, failedLoc = hdfs://localhost:27000/user/hive/warehouse/thive
+*** WARNING[8436] Mismatch detected between compiletime and runtime hive table definitions. DataModMismatchDetails: compiledModTS = 1487805672, failedModTS = 1487805674, failedLoc = hdfs://localhost:24200/user/hive/warehouse/thive
 
 --- 0 row(s) selected.
 >>
 >>-- truncate of partitioned hive table
->>sh echo "drop table t005part;" > TEST005_junk;
->>sh regrhive.ksh -f TEST005_junk;
->>sh echo "create table t005part(a int) partitioned by (b int, c int);" > TEST005_junk;
->>sh regrhive.ksh -f TEST005_junk;
+>>process hive statement 'drop table t005part';
+
+--- SQL operation complete.
+>>process hive statement 'create table t005part(a int) partitioned by (b int, c int)';
+
+--- SQL operation complete.
 >>
 >>sh echo "insert into t005part partition (b=10,c=11) values (5);" > TEST005_junk;
 >>sh regrhive.ksh -f TEST005_junk;
@@ -981,7 +985,7 @@ t005part.a	t005part.b	t005part.c
 >>invoke hive.hive.thive_insert_smallint;
 
 -- Definition of hive table THIVE_INSERT_SMALLINT
--- Definition current  Thu Jan 26 00:08:37 2017
+-- Definition current  Wed Feb 22 23:23:48 2017
 
   (
     A                                SMALLINT
@@ -1086,7 +1090,7 @@ A
 >>invoke hive.hive.thive_insert_varchar;
 
 -- Definition of hive table THIVE_INSERT_VARCHAR
--- Definition current  Thu Jan 26 00:08:42 2017
+-- Definition current  Wed Feb 22 23:23:56 2017
 
   (
     A                                VARCHAR(1 CHAR) CHARACTER SET UTF8 COLLATE

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/regress/hive/EXPECTED007
----------------------------------------------------------------------
diff --git a/core/sql/regress/hive/EXPECTED007 b/core/sql/regress/hive/EXPECTED007
index 19ea623..b5962c2 100644
--- a/core/sql/regress/hive/EXPECTED007
+++ b/core/sql/regress/hive/EXPECTED007
@@ -11,17 +11,21 @@
 >>
 >>obey TEST007(tests_traf);
 >>-- tests for views on hive tables
->>sh echo "drop table thive1;" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
->>sh echo "create table thive1(a int, b int);" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'drop table thive1';
+
+--- SQL operation complete.
+>>process hive statement 'create table thive1(a int, b int)';
+
+--- SQL operation complete.
 >>sh echo "insert into thive1 values (1, 2);" > TEST007_junk;
 >>sh regrhive.ksh -f TEST007_junk;
 >>
->>sh echo "drop table thive2;" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
->>sh echo "create table thive2(a int, b int);" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'drop table thive2';
+
+--- SQL operation complete.
+>>process hive statement 'create table thive2(a int, b int)';
+
+--- SQL operation complete.
 >>sh echo "insert into thive2 values (1, 2);" > TEST007_junk;
 >>sh regrhive.ksh -f TEST007_junk;
 >>
@@ -79,7 +83,7 @@ CREATE /*IMPLICIT*/ EXTERNAL TABLE THIVE1
 >>invoke trafodion.sch007.vhivehbase;
 
 -- Definition of Trafodion view TRAFODION.SCH007.VHIVEHBASE
--- Definition current  Wed Feb 22 04:31:49 2017
+-- Definition current  Thu Feb 23 20:03:32 2017
 
   (
     AA                               INT DEFAULT NULL
@@ -323,8 +327,9 @@ A            B
           3            4
 
 --- 2 row(s) selected.
->>sh echo "drop table thive1;" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'drop table thive1';
+
+--- SQL operation complete.
 >>execute s;
 
 *** ERROR[4262] Object HIVE.HIVE.THIVE1 has invalid state and cannot be accessed. It has an external table but the correponding hive table does not exist. Use 'drop external table' command to drop the external table.
@@ -344,10 +349,12 @@ A            B
 >>cqd hive_views_create_external_table 'OFF';
 
 --- SQL operation complete.
->>sh echo "drop table thive3;" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
->>sh echo "create table thive3(a int, b int);" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'drop table thive3';
+
+--- SQL operation complete.
+>>process hive statement 'create table thive3(a int, b int)';
+
+--- SQL operation complete.
 >>create view trafodion.sch007.vhive6 as select * from hive.hive.thive3;
 
 *** WARNING[1071] View usage information for the following hive tables could not be set. Make sure that an external table either already exists or implicit creation has not been disabled. Hive tables: HIVE.HIVE.THIVE3
@@ -374,26 +381,32 @@ CREATE TABLE THIVE3
 >>
 >>
 >>obey TEST007(setup_hive);
->>sh echo "create database hivesch007;" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'create database hivesch007';
+
+--- SQL operation complete.
 >>
->>sh echo "create table hivesch007.thive1 (a int);" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'create table hivesch007.thive1 (a int)';
+
+--- SQL operation complete.
 >>
 >>sh echo "insert into hivesch007.thive1 values (1);" > TEST007_junk;
 >>sh regrhive.ksh -f TEST007_junk;
 >>
->>sh echo "create view hivesch007.vhive1 as select * from hivesch007.thive1 where thive1.a > 0;" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'create view hivesch007.vhive1 as select * from hivesch007.thive1 where thive1.a > 0';
+
+--- SQL operation complete.
 >>
->>sh echo "create view hivesch007.vhive11 as select * from hivesch007.vhive1 where vhive1.a > 0;" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'create view hivesch007.vhive11 as select * from hivesch007.vhive1 where vhive1.a > 0';
+
+--- SQL operation complete.
 >>
->>sh echo "create table thive007 (a int);" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'create table thive007 (a int)';
+
+--- SQL operation complete.
 >>
->>sh echo "create view vhive007 as select * from thive007 where a > 0;" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'create view vhive007 as select * from thive007 where a > 0';
+
+--- SQL operation complete.
 >>
 >>insert into hive.hive.thive007 values (1);
 
@@ -403,7 +416,7 @@ CREATE TABLE THIVE3
 >>invoke hive.hivesch007.vhive1;
 
 -- Definition of native Hive view VHIVE1
--- Definition current  Wed Feb 22 04:35:19 2017
+-- Definition current  Thu Feb 23 20:05:17 2017
 
   (
     A                                INT
@@ -445,7 +458,7 @@ A
 >>invoke hive.hivesch007.vhive11;
 
 -- Definition of native Hive view VHIVE11
--- Definition current  Wed Feb 22 04:35:22 2017
+-- Definition current  Thu Feb 23 20:05:21 2017
 
   (
     A                                INT
@@ -651,8 +664,9 @@ A
 
 --- SQL operation complete.
 >>
->>sh echo "drop table hivesch007.thive1;" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'drop table hivesch007.thive1';
+
+--- SQL operation complete.
 >>
 >>prepare s from select * from hive.hivesch007.vhive11;
 
@@ -665,8 +679,9 @@ A
 *** ERROR[15017] Statement S was not found.
 
 >>
->>sh echo "create table hivesch007.thive1 (a int);" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'create table hivesch007.thive1 (a int)';
+
+--- SQL operation complete.
 >>
 >>prepare s from select * from hive.hivesch007.vhive11;
 
@@ -675,8 +690,9 @@ A
 
 --- 0 row(s) selected.
 >>
->>sh echo "drop table hivesch007.thive1;" > TEST007_junk;
->>sh regrhive.ksh -f TEST007_junk;
+>>process hive statement 'drop table hivesch007.thive1';
+
+--- SQL operation complete.
 >>
 >>execute s;
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/regress/hive/TEST005
----------------------------------------------------------------------
diff --git a/core/sql/regress/hive/TEST005 b/core/sql/regress/hive/TEST005
index 3d6ef81..3c35e47 100644
--- a/core/sql/regress/hive/TEST005
+++ b/core/sql/regress/hive/TEST005
@@ -298,11 +298,8 @@ cqd HIVE_SCAN_SPECIAL_MODE reset;
 -- tests for hive timestamp mismatch check
 cqd auto_query_retry_warnings 'ON';
 
-sh echo "drop table thive;" > TEST005_junk;
-sh regrhive.ksh -f TEST005_junk;
-
-sh echo "create table thive(a int);" > TEST005_junk;
-sh regrhive.ksh -f TEST005_junk;
+process hive statement 'drop table thive';
+process hive statement 'create table thive(a int)';
 
 select a from hive.hive.thive;
 
@@ -313,11 +310,8 @@ select a from hive.hive.thive;
 insert into hive.hive.thive values (2);
 select a from hive.hive.thive;
 
-sh echo "drop table thive;" > TEST005_junk;
-sh regrhive.ksh -f TEST005_junk;
-
-sh echo "create table thive(a int, b int);" > TEST005_junk;
-sh regrhive.ksh -f TEST005_junk;
+process hive statement 'drop table thive';
+process hive statement 'create table thive(a int, b int)';
 
 sh echo "insert into thive values (1,2);" > TEST005_junk;
 sh regrhive.ksh -f TEST005_junk;
@@ -337,10 +331,8 @@ truncate hive.hive.thive;
 select * from hive.hive.thive;
 
 -- truncate of partitioned hive table
-sh echo "drop table t005part;" > TEST005_junk;
-sh regrhive.ksh -f TEST005_junk;
-sh echo "create table t005part(a int) partitioned by (b int, c int);" > TEST005_junk;
-sh regrhive.ksh -f TEST005_junk;
+process hive statement 'drop table t005part';
+process hive statement 'create table t005part(a int) partitioned by (b int, c int)';
 
 sh echo "insert into t005part partition (b=10,c=11) values (5);" > TEST005_junk;
 sh regrhive.ksh -f TEST005_junk;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/regress/hive/TEST007
----------------------------------------------------------------------
diff --git a/core/sql/regress/hive/TEST007 b/core/sql/regress/hive/TEST007
index d1e8fd8..b8fb9e3 100644
--- a/core/sql/regress/hive/TEST007
+++ b/core/sql/regress/hive/TEST007
@@ -50,13 +50,10 @@ drop view trafodion.sch007.vhive6 cascade;
 drop external table thive1 for hive.hive.thive1 cascade;
 cleanup table trafodion."_HV_HIVE_".thive1;
 drop external table thive2 for hive.hive.thive2 cascade;
-sh echo "drop table thive1;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
-sh echo "drop table thive2;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'drop table thive1';
+process hive statement 'drop table thive2';
 drop external table thive3 for hive.hive.thive3;
-sh echo "drop table thive3;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'drop table thive3';
 drop table if exists trafodion.sch007.thbase1 cascade;
 
 ?section setup_traf
@@ -66,17 +63,13 @@ create schema if not exists trafodion.sch007;
 
 ?section tests_traf
 -- tests for views on hive tables
-sh echo "drop table thive1;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
-sh echo "create table thive1(a int, b int);" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'drop table thive1';
+process hive statement 'create table thive1(a int, b int)';
 sh echo "insert into thive1 values (1, 2);" > TEST007_junk;
 sh regrhive.ksh -f TEST007_junk;
 
-sh echo "drop table thive2;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
-sh echo "create table thive2(a int, b int);" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'drop table thive2';
+process hive statement 'create table thive2(a int, b int)';
 sh echo "insert into thive2 values (1, 2);" > TEST007_junk;
 sh regrhive.ksh -f TEST007_junk;
 
@@ -148,67 +141,47 @@ create view vhive5 as select * from thive1;
 -- if underlying hive table is dropped, an error is returned
 prepare s from select * from trafodion.sch007.vhive1;
 execute s;
-sh echo "drop table thive1;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'drop table thive1';
 execute s;
 select * from trafodion.sch007.vhive1;
 
 -- if external table creation has been disabled, then view usage will
 -- not available
 cqd hive_views_create_external_table 'OFF';
-sh echo "drop table thive3;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
-sh echo "create table thive3(a int, b int);" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'drop table thive3';
+process hive statement 'create table thive3(a int, b int)';
 create view trafodion.sch007.vhive6 as select * from hive.hive.thive3;
 showddl hive.hive.thive3;
 get all views on table hive.hive.thive3;
 get all tables in view trafodion.sch007.vhive6;
 
 ?section clean_up_hive
-sh echo "drop table hivesch007.thive1;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
-
-sh echo "drop view hivesch007.vhive11;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
-
-sh echo "drop view hivesch007.vhive1;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
-
-sh echo "drop database hivesch007;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
-
-sh echo "drop table thive007;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
-
-sh echo "drop view vhive007;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'drop table hivesch007.thive1';
+process hive statement 'drop view hivesch007.vhive11';
+process hive statement 'drop view hivesch007.vhive1';
+process hive statement 'drop database hivesch007';
+process hive statement 'drop table thive007';
+process hive statement 'drop view vhive007';
 
 drop view trafodion.sch007.vtrafonhive;
 
 drop schema trafodion.sch007 cascade;
 
 ?section setup_hive
-sh echo "create database hivesch007;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'create database hivesch007';
 
-sh echo "create table hivesch007.thive1 (a int);" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'create table hivesch007.thive1 (a int)';
 
 sh echo "insert into hivesch007.thive1 values (1);" > TEST007_junk;
 sh regrhive.ksh -f TEST007_junk;
 
-sh echo "create view hivesch007.vhive1 as select * from hivesch007.thive1 where thive1.a > 0;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'create view hivesch007.vhive1 as select * from hivesch007.thive1 where thive1.a > 0';
 
-sh echo "create view hivesch007.vhive11 as select * from hivesch007.vhive1 where vhive1.a > 0;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'create view hivesch007.vhive11 as select * from hivesch007.vhive1 where vhive1.a > 0';
 
-sh echo "create table thive007 (a int);" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'create table thive007 (a int)';
 
-sh echo "create view vhive007 as select * from thive007 where a > 0;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'create view vhive007 as select * from thive007 where a > 0';
 
 insert into hive.hive.thive007 values (1);
 
@@ -265,19 +238,16 @@ delete from hive.hivesch007.vhive1;
 -- drop underlying hive table
 cqd auto_query_retry_warnings 'ON';
 
-sh echo "drop table hivesch007.thive1;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'drop table hivesch007.thive1';
 
 prepare s from select * from hive.hivesch007.vhive11;
 execute s;
 
-sh echo "create table hivesch007.thive1 (a int);" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'create table hivesch007.thive1 (a int)';
 
 prepare s from select * from hive.hivesch007.vhive11;
 execute s;
 
-sh echo "drop table hivesch007.thive1;" > TEST007_junk;
-sh regrhive.ksh -f TEST007_junk;
+process hive statement 'drop table hivesch007.thive1';
 
 execute s;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/regress/seabase/EXPECTED003
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/EXPECTED003 b/core/sql/regress/seabase/EXPECTED003
index 268cf66..8ce0a4c 100644
--- a/core/sql/regress/seabase/EXPECTED003
+++ b/core/sql/regress/seabase/EXPECTED003
@@ -31,7 +31,7 @@
 >>invoke t003t1;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T1
--- Definition current  Mon Aug 15 08:53:30 2016
+-- Definition current  Wed Feb 22 23:04:05 2017
 
   (
     A                                TINYINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -50,7 +50,7 @@
 >>invoke t003t1_like;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T1_LIKE
--- Definition current  Mon Aug 15 08:53:34 2016
+-- Definition current  Wed Feb 22 23:04:11 2017
 
   (
     A                                TINYINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -69,7 +69,7 @@
 >>invoke t003t1_as;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T1_AS
--- Definition current  Mon Aug 15 08:53:40 2016
+-- Definition current  Wed Feb 22 23:04:16 2017
 
   (
     A                                TINYINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -88,7 +88,7 @@
 >>invoke t003t1_view;
 
 -- Definition of Trafodion view TRAFODION.SCH.T003T1_VIEW
--- Definition current  Mon Aug 15 08:53:42 2016
+-- Definition current  Wed Feb 22 23:04:19 2017
 
   (
     A                                TINYINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -341,19 +341,19 @@ A     B     C    D
 --- SQL operation complete.
 >>
 >>obey TEST003(hive_tiny);
->>sh echo "drop table ttiny;" > TEST003_junk;
->>sh regrhive.ksh -f TEST003_junk;
->>
->>sh echo "create table ttiny(a tinyint, b tinyint);" > TEST003_junk;
->>sh regrhive.ksh -f TEST003_junk;
->>
+>>process hive statement 'drop table ttiny';
+
+--- SQL operation complete.
+>>process hive statement 'create table ttiny(a tinyint, b tinyint)';
+
+--- SQL operation complete.
 >>sh echo "insert into ttiny values (1, -1);" > TEST003_junk;
 >>sh regrhive.ksh -f TEST003_junk;
 >>
 >>invoke hive.hive.ttiny;
 
 -- Definition of hive table TTINY
--- Definition current  Mon Aug 15 08:54:30 2016
+-- Definition current  Wed Feb 22 23:04:49 2017
 
   (
     A                                TINYINT
@@ -512,7 +512,7 @@ A     B
 >>invoke t003t1;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T1
--- Definition current  Mon Aug 15 08:54:56 2016
+-- Definition current  Wed Feb 22 23:05:19 2017
 
   (
     A                                TINYINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -531,7 +531,7 @@ A     B
 >>invoke t003t1_like;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T1_LIKE
--- Definition current  Mon Aug 15 08:54:59 2016
+-- Definition current  Wed Feb 22 23:05:25 2017
 
   (
     A                                TINYINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -550,7 +550,7 @@ A     B
 >>invoke t003t1_as;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T1_AS
--- Definition current  Mon Aug 15 08:55:03 2016
+-- Definition current  Wed Feb 22 23:05:30 2017
 
   (
     A                                TINYINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -569,7 +569,7 @@ A     B
 >>invoke t003t1_view;
 
 -- Definition of Trafodion view TRAFODION.SCH.T003T1_VIEW
--- Definition current  Mon Aug 15 08:55:05 2016
+-- Definition current  Wed Feb 22 23:05:32 2017
 
   (
     A                                TINYINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -822,19 +822,19 @@ A       B       C      D
 --- SQL operation complete.
 >>
 >>obey TEST003(hive_tiny);
->>sh echo "drop table ttiny;" > TEST003_junk;
->>sh regrhive.ksh -f TEST003_junk;
->>
->>sh echo "create table ttiny(a tinyint, b tinyint);" > TEST003_junk;
->>sh regrhive.ksh -f TEST003_junk;
->>
+>>process hive statement 'drop table ttiny';
+
+--- SQL operation complete.
+>>process hive statement 'create table ttiny(a tinyint, b tinyint)';
+
+--- SQL operation complete.
 >>sh echo "insert into ttiny values (1, -1);" > TEST003_junk;
 >>sh regrhive.ksh -f TEST003_junk;
 >>
 >>invoke hive.hive.ttiny;
 
 -- Definition of hive table TTINY
--- Definition current  Mon Aug 15 08:55:44 2016
+-- Definition current  Wed Feb 22 23:05:57 2017
 
   (
     A                                TINYINT
@@ -978,7 +978,7 @@ A       B
 >>invoke t003t2;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T2
--- Definition current  Mon Aug 15 08:55:51 2016
+-- Definition current  Wed Feb 22 23:06:12 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -995,7 +995,7 @@ A       B
 >>invoke t003t2_like;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T2_LIKE
--- Definition current  Mon Aug 15 08:55:54 2016
+-- Definition current  Wed Feb 22 23:06:18 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1012,7 +1012,7 @@ A       B
 >>invoke t003t2_as;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T2_AS
--- Definition current  Mon Aug 15 08:55:58 2016
+-- Definition current  Wed Feb 22 23:06:22 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1029,7 +1029,7 @@ A       B
 >>invoke t003t2_view;
 
 -- Definition of Trafodion view TRAFODION.SCH.T003T2_VIEW
--- Definition current  Mon Aug 15 08:56:00 2016
+-- Definition current  Wed Feb 22 23:06:25 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1162,8 +1162,8 @@ A                     (EXPR)
 >>
 >>select cast('-9223372036854775808' as largeint) from (values(1)) x(a);
 
-(EXPR)
-----------
+(EXPR)              
+--------------------
 
 -9223372036854775808
 
@@ -1336,7 +1336,7 @@ A                     B
 >>invoke t003t2;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T2
--- Definition current  Mon Aug 15 08:56:27 2016
+-- Definition current  Wed Feb 22 23:06:51 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1353,7 +1353,7 @@ A                     B
 >>invoke t003t2_like;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T2_LIKE
--- Definition current  Mon Aug 15 08:56:30 2016
+-- Definition current  Wed Feb 22 23:06:57 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1370,7 +1370,7 @@ A                     B
 >>invoke t003t2_as;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T2_AS
--- Definition current  Mon Aug 15 08:56:34 2016
+-- Definition current  Wed Feb 22 23:07:03 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1387,7 +1387,7 @@ A                     B
 >>invoke t003t2_view;
 
 -- Definition of Trafodion view TRAFODION.SCH.T003T2_VIEW
--- Definition current  Mon Aug 15 08:56:36 2016
+-- Definition current  Wed Feb 22 23:07:06 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1520,8 +1520,8 @@ A                     (EXPR)
 >>
 >>select cast('-9223372036854775808' as largeint) from (values(1)) x(a);
 
-(EXPR)
-----------
+(EXPR)              
+--------------------
 
 -9223372036854775808
 
@@ -1697,7 +1697,7 @@ A                     B
 >>invoke t003t2;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T2
--- Definition current  Mon Aug 15 08:57:01 2016
+-- Definition current  Wed Feb 22 23:07:37 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1714,7 +1714,7 @@ A                     B
 >>invoke t003t2_like;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T2_LIKE
--- Definition current  Mon Aug 15 08:57:04 2016
+-- Definition current  Wed Feb 22 23:07:42 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1731,7 +1731,7 @@ A                     B
 >>invoke t003t2_as;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T2_AS
--- Definition current  Mon Aug 15 08:57:07 2016
+-- Definition current  Wed Feb 22 23:07:47 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1748,7 +1748,7 @@ A                     B
 >>invoke t003t2_view;
 
 -- Definition of Trafodion view TRAFODION.SCH.T003T2_VIEW
--- Definition current  Mon Aug 15 08:57:09 2016
+-- Definition current  Wed Feb 22 23:07:50 2017
 
   (
     A                                LARGEINT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1881,8 +1881,8 @@ A                      (EXPR)
 >>
 >>select cast('-9223372036854775808' as largeint) from (values(1)) x(a);
 
-(EXPR)
-----------
+(EXPR)              
+--------------------
 
 -9223372036854775808
 
@@ -2058,7 +2058,7 @@ A                      B
 >>invoke t003t3;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T3
--- Definition current  Mon Aug 15 08:57:17 2016
+-- Definition current  Wed Feb 22 23:08:05 2017
 
   (
     A                                BOOLEAN NO DEFAULT NOT NULL NOT DROPPABLE
@@ -2074,7 +2074,7 @@ A                      B
 >>invoke t003t3_like;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T3_LIKE
--- Definition current  Mon Aug 15 08:57:20 2016
+-- Definition current  Wed Feb 22 23:08:11 2017
 
   (
     A                                BOOLEAN NO DEFAULT NOT NULL NOT DROPPABLE
@@ -2090,7 +2090,7 @@ A                      B
 >>invoke t003t3_as;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T3_AS
--- Definition current  Mon Aug 15 08:57:24 2016
+-- Definition current  Wed Feb 22 23:08:16 2017
 
   (
     A                                BOOLEAN NO DEFAULT NOT NULL NOT DROPPABLE
@@ -2107,7 +2107,7 @@ A                      B
 >>invoke t003t3_salt;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T3_SALT
--- Definition current  Mon Aug 15 08:57:26 2016
+-- Definition current  Wed Feb 22 23:08:19 2017
 
   (
     A                                BOOLEAN NO DEFAULT NOT NULL NOT DROPPABLE
@@ -2124,7 +2124,7 @@ A                      B
 >>invoke t003t3_view;
 
 -- Definition of Trafodion view TRAFODION.SCH.T003T3_VIEW
--- Definition current  Mon Aug 15 08:57:28 2016
+-- Definition current  Wed Feb 22 23:08:22 2017
 
   (
     A                                BOOLEAN NO DEFAULT NOT NULL NOT DROPPABLE
@@ -2451,19 +2451,19 @@ TRUE   FALSE
 --- 0 row(s) inserted.
 >>
 >>obey TEST003(hive_bool);
->>sh echo "drop table tbool;" > TEST003_junk;
->>sh regrhive.ksh -f TEST003_junk;
->>
->>sh echo "create table tbool(a boolean);" > TEST003_junk;
->>sh regrhive.ksh -f TEST003_junk;
->>
+>>process hive statement 'drop table tbool';
+
+--- SQL operation complete.
+>>process hive statement 'create table tbool(a boolean)';
+
+--- SQL operation complete.
 >>sh echo "insert into tbool values (true), (false), (NULL);" > TEST003_junk;
 >>sh regrhive.ksh -f TEST003_junk;
 >>
 >>invoke hive.hive.tbool;
 
 -- Definition of hive table TBOOL
--- Definition current  Mon Aug 15 08:58:07 2016
+-- Definition current  Wed Feb 22 23:08:45 2017
 
   (
     A                                BOOLEAN
@@ -2535,7 +2535,7 @@ TRUE
 >>invoke t003t3;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T3
--- Definition current  Mon Aug 15 08:58:30 2016
+-- Definition current  Wed Feb 22 23:09:08 2017
 
   (
     A                                BOOLEAN NO DEFAULT NOT NULL NOT DROPPABLE
@@ -2551,7 +2551,7 @@ TRUE
 >>invoke t003t3_like;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T3_LIKE
--- Definition current  Mon Aug 15 08:58:34 2016
+-- Definition current  Wed Feb 22 23:09:15 2017
 
   (
     A                                BOOLEAN NO DEFAULT NOT NULL NOT DROPPABLE
@@ -2567,7 +2567,7 @@ TRUE
 >>invoke t003t3_as;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T3_AS
--- Definition current  Mon Aug 15 08:58:38 2016
+-- Definition current  Wed Feb 22 23:09:20 2017
 
   (
     A                                BOOLEAN NO DEFAULT NOT NULL NOT DROPPABLE
@@ -2584,7 +2584,7 @@ TRUE
 >>invoke t003t3_salt;
 
 -- Definition of Trafodion table TRAFODION.SCH.T003T3_SALT
--- Definition current  Mon Aug 15 08:58:38 2016
+-- Definition current  Wed Feb 22 23:09:21 2017
 
   (
     A                                BOOLEAN NO DEFAULT NOT NULL NOT DROPPABLE
@@ -2601,7 +2601,7 @@ TRUE
 >>invoke t003t3_view;
 
 -- Definition of Trafodion view TRAFODION.SCH.T003T3_VIEW
--- Definition current  Mon Aug 15 08:58:40 2016
+-- Definition current  Wed Feb 22 23:09:23 2017
 
   (
     A                                BOOLEAN NO DEFAULT NOT NULL NOT DROPPABLE

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/regress/seabase/EXPECTED031
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/EXPECTED031 b/core/sql/regress/seabase/EXPECTED031
index 335fed2..e5c580a 100644
--- a/core/sql/regress/seabase/EXPECTED031
+++ b/core/sql/regress/seabase/EXPECTED031
@@ -416,11 +416,12 @@ Z            Z            (EXPR)  (EXPR)
 
 --- 7 row(s) selected.
 >>
->>sh echo "drop table t031hive;" > TEST031_junk;
->>sh regrhive.ksh -f TEST031_junk;
->>
->>sh echo "create table t031hive(z int, a string, b string);" > TEST031_junk;
->>sh regrhive.ksh -f TEST031_junk;
+>>process hive statement 'drop table t031hive';
+
+--- SQL operation complete.
+>>process hive statement 'create table t031hive(z int, a string, b string)';
+
+--- SQL operation complete.
 >>
 >>cqd hive_max_string_length '1000000';
 
@@ -557,6 +558,14 @@ LC   RC   OP   OPERATOR              OPT       DESCRIPTION           CARD
 
 --- SQL operation complete.
 >>
+>>-- should return error.
+>>process hive statement 'insert into t values (1)';
+
+*** ERROR[3242] This statement is not supported. Reason: Only CREATE, DROP, ALTER or TRUNCATE hive DDL statements can be specified.
+
+*** ERROR[8822] The statement was not prepared.
+
+>>
 >>-- default USER
 >>drop table if exists t031t1;
 
@@ -567,7 +576,7 @@ LC   RC   OP   OPERATOR              OPT       DESCRIPTION           CARD
 >>invoke t031t1;
 
 -- Definition of Trafodion table TRAFODION.SCH.T031T1
--- Definition current  Tue Sep 20 11:39:15 2016
+-- Definition current  Thu Feb 23 00:12:00 2017
 
   (
     SYSKEY                           LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -598,7 +607,7 @@ A            B
 >>invoke t031t1;
 
 -- Definition of Trafodion table TRAFODION.SCH.T031T1
--- Definition current  Tue Sep 20 11:39:18 2016
+-- Definition current  Thu Feb 23 00:12:06 2017
 
   (
     SYSKEY                           LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -649,7 +658,7 @@ A            B                     C
 >>invoke t031t1;
 
 -- Definition of Trafodion table TRAFODION.SCH.T031T1
--- Definition current  Tue Sep 20 11:39:28 2016
+-- Definition current  Thu Feb 23 00:12:14 2017
 
   (
     SYSKEY                           LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -678,7 +687,7 @@ A            B                     C
 >>invoke t031t1;
 
 -- Definition of Trafodion table TRAFODION.SCH.T031T1
--- Definition current  Tue Sep 20 11:39:36 2016
+-- Definition current  Thu Feb 23 00:12:23 2017
 
   (
     SYSKEY                           LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -703,7 +712,7 @@ A            B                     C
 >>invoke t031t1;
 
 -- Definition of Trafodion table TRAFODION.SCH.T031T1
--- Definition current  Tue Sep 20 11:39:44 2016
+-- Definition current  Thu Feb 23 00:12:28 2017
 
   (
     SYSKEY                           LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -723,7 +732,7 @@ A            B                     C
 >>invoke t031t1;
 
 -- Definition of Trafodion table TRAFODION.SCH.T031T1
--- Definition current  Tue Sep 20 11:39:54 2016
+-- Definition current  Thu Feb 23 00:12:35 2017
 
   (
     SYSKEY                           LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -738,7 +747,7 @@ A            B                     C
 >>invoke t031v1;
 
 -- Definition of Trafodion view TRAFODION.SCH.T031V1
--- Definition current  Tue Sep 20 11:39:56 2016
+-- Definition current  Thu Feb 23 00:12:38 2017
 
   (
     A                                INT DEFAULT NULL
@@ -761,7 +770,7 @@ A            B                     C
 >>invoke t031v1;
 
 -- Definition of Trafodion view TRAFODION.SCH.T031V1
--- Definition current  Tue Sep 20 11:40:04 2016
+-- Definition current  Thu Feb 23 00:12:42 2017
 
   (
     A                                INT DEFAULT NULL
@@ -779,7 +788,7 @@ A            B                     C
 >>invoke t031v1;
 
 -- Definition of Trafodion view TRAFODION.SCH.T031V1
--- Definition current  Tue Sep 20 11:40:09 2016
+-- Definition current  Thu Feb 23 00:12:52 2017
 
   (
     A                                INT DEFAULT NULL
@@ -866,7 +875,7 @@ CREATE INDEX T031T1I1 ON TRAFODION.SCH.T031T1
 >>invoke table(index_table t031t1i1);
 
 -- Definition of Trafodion table TRAFODION.SCH.T031T1I1
--- Definition current  Tue Sep 20 11:40:37 2016
+-- Definition current  Thu Feb 23 00:13:14 2017
 
   (
     "A@"                             INT NO DEFAULT

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/regress/seabase/EXPECTED033
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/EXPECTED033 b/core/sql/regress/seabase/EXPECTED033
index af0dc54..46bc3dd 100644
--- a/core/sql/regress/seabase/EXPECTED033
+++ b/core/sql/regress/seabase/EXPECTED033
@@ -663,14 +663,19 @@ LC   RC   OP   OPERATOR              OPT       DESCRIPTION           CARD
 --- SQL operation complete.
 >>
 >>-- group by rollup and grouping on hive tables
->>sh echo "drop table t033hive1;" > TEST033_junk;
->>sh regrhive.ksh -f TEST033_junk;
->>sh echo "create table t033hive1(a int, b int);" > TEST033_junk;
->>sh regrhive.ksh -f TEST033_junk;
->>sh echo "drop table t033hive2;" > TEST033_junk;
->>sh regrhive.ksh -f TEST033_junk;
->>sh echo "create table t033hive2(a int, b int);" > TEST033_junk;
->>sh regrhive.ksh -f TEST033_junk;
+>>process hive statement 'drop table t033hive1';
+
+--- SQL operation complete.
+>>process hive statement 'create table t033hive1(a int, b int)';
+
+--- SQL operation complete.
+>>process hive statement 'drop table t033hive2';
+
+--- SQL operation complete.
+>>process hive statement 'create table t033hive2(a int, b int)';
+
+--- SQL operation complete.
+>>
 >>insert into hive.hive.t033hive1 values (1,2), (null, null);
 
 --- 2 row(s) inserted.

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/regress/seabase/TEST003
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/TEST003 b/core/sql/regress/seabase/TEST003
index 5a39599..4f170b9 100644
--- a/core/sql/regress/seabase/TEST003
+++ b/core/sql/regress/seabase/TEST003
@@ -142,12 +142,8 @@ select * from t003t1;
 rollback work;
 
 ?section hive_tiny
-sh echo "drop table ttiny;" > TEST003_junk;
-sh regrhive.ksh -f TEST003_junk;
-
-sh echo "create table ttiny(a tinyint, b tinyint);" > TEST003_junk;
-sh regrhive.ksh -f TEST003_junk;
-
+process hive statement 'drop table ttiny';
+process hive statement 'create table ttiny(a tinyint, b tinyint)';
 sh echo "insert into ttiny values (1, -1);" > TEST003_junk;
 sh regrhive.ksh -f TEST003_junk;
 
@@ -342,12 +338,8 @@ rollback work;
 select * from t003t3;
 
 ?section hive_bool
-sh echo "drop table tbool;" > TEST003_junk;
-sh regrhive.ksh -f TEST003_junk;
-
-sh echo "create table tbool(a boolean);" > TEST003_junk;
-sh regrhive.ksh -f TEST003_junk;
-
+process hive statement 'drop table tbool';
+process hive statement 'create table tbool(a boolean)';
 sh echo "insert into tbool values (true), (false), (NULL);" > TEST003_junk;
 sh regrhive.ksh -f TEST003_junk;
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/regress/seabase/TEST031
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/TEST031 b/core/sql/regress/seabase/TEST031
index a6788f4..e79b022 100644
--- a/core/sql/regress/seabase/TEST031
+++ b/core/sql/regress/seabase/TEST031
@@ -150,11 +150,8 @@ select x.z, y.z, substring(x.a, 1, 5), substring(y.a,1,5)
   from t031t1 x, t031t2 y where x.a = y.a and x.b = y.b
   order by x.z, y.z;
 
-sh echo "drop table t031hive;" > TEST031_junk;
-sh regrhive.ksh -f TEST031_junk;
-
-sh echo "create table t031hive(z int, a string, b string);" > TEST031_junk;
-sh regrhive.ksh -f TEST031_junk;
+process hive statement 'drop table t031hive';
+process hive statement 'create table t031hive(z int, a string, b string)';
 
 cqd hive_max_string_length '1000000';
 insert into hive.hive.t031hive select * from t031t1;
@@ -188,6 +185,9 @@ control query shape cut ;
 cqd hdfs_io_buffersize reset ;
 cqd hive_min_bytes_per_esp_partition reset;
 
+-- should return error.
+process hive statement 'insert into t values (1)';
+
 -- default USER
 drop table if exists t031t1;
 create table t031t1 (a int, b varchar(20) default user);

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/regress/seabase/TEST033
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/TEST033 b/core/sql/regress/seabase/TEST033
index ce79022..045d93a 100644
--- a/core/sql/regress/seabase/TEST033
+++ b/core/sql/regress/seabase/TEST033
@@ -182,14 +182,11 @@ prepare s from select a, sum(b) from t033t3 group by (a);
 explain options 'f' s;
 
 -- group by rollup and grouping on hive tables
-sh echo "drop table t033hive1;" > TEST033_junk;
-sh regrhive.ksh -f TEST033_junk;
-sh echo "create table t033hive1(a int, b int);" > TEST033_junk;
-sh regrhive.ksh -f TEST033_junk;
-sh echo "drop table t033hive2;" > TEST033_junk;
-sh regrhive.ksh -f TEST033_junk;
-sh echo "create table t033hive2(a int, b int);" > TEST033_junk;
-sh regrhive.ksh -f TEST033_junk;
+process hive statement 'drop table t033hive1';
+process hive statement 'create table t033hive1(a int, b int)';
+process hive statement 'drop table t033hive2';
+process hive statement 'create table t033hive2(a int, b int)';
+
 insert into hive.hive.t033hive1 values (1,2), (null, null);
 insert into hive.hive.t033hive2 values (1,2), (null, null);
 prepare s from select t1.a, t2.b , grouping(t1.a), grouping(t2.b),

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/sqlci/sqlci_lex.ll
----------------------------------------------------------------------
diff --git a/core/sql/sqlci/sqlci_lex.ll b/core/sql/sqlci/sqlci_lex.ll
index 2e518a6..ab7d7e8 100755
--- a/core/sql/sqlci/sqlci_lex.ll
+++ b/core/sql/sqlci/sqlci_lex.ll
@@ -285,6 +285,7 @@ B			[ \t\n]+
 [Oo][Pp][Ee][Nn]                       return_IDENT_or_TOKEN(OPENtoken, 0);
 [Oo][Pp][Tt][Ii][Oo][Nn][Ss]           return_IDENT_or_TOKEN(OPTIONStoken, 0);
 [Pp][Rr][Ee][Pp][Aa][Rr][Ee][Dd]       return_IDENT_or_TOKEN(PREPARED, 0);
+[Pp][Rr][Oo][Cc][Ee][Ss][Ss]           return_IDENT_or_TOKEN(PROCESStoken, 0);
 [Ee][Rr][Rr][Oo][Rr]                   return_IDENT_or_TOKEN(ERRORtoken, 0);
 [Ee][Xx][Ii][Tt] 		       return_IDENT_or_TOKEN(EXIT, 0);
 [Ff][Cc]   		               return_IDENT_or_TOKEN(FC, -1);

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/sqlci/sqlci_yacc.y
----------------------------------------------------------------------
diff --git a/core/sql/sqlci/sqlci_yacc.y b/core/sql/sqlci/sqlci_yacc.y
index 03e6094..eb94003 100644
--- a/core/sql/sqlci/sqlci_yacc.y
+++ b/core/sql/sqlci/sqlci_yacc.y
@@ -543,6 +543,7 @@ static char * FCString (const char *idString, int isFC)
 %token INITIALIZE REINITIALIZE
 %token CATALOG SCHEMA
 %token HIVEtoken
+%token PROCESStoken
 %token IF WHILE
 %token MPLOC
 %token NAMETYPE
@@ -2278,6 +2279,7 @@ dml_type :
         |       DOWNGRADEtoken          {$$ = DML_DDL_TYPE;}
         |       GETtoken                {$$ = DML_DESCRIBE_TYPE;}
  	|       LABEL_ALTER             {$$ = DML_DDL_TYPE;}
+ 	|       PROCESStoken            {$$ = DML_DDL_TYPE;}
 ;
 
 dml_simple_table_type :

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/e9389fa0/core/sql/sqlcomp/parser.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/parser.cpp b/core/sql/sqlcomp/parser.cpp
index 4b032d4..3f48463 100644
--- a/core/sql/sqlcomp/parser.cpp
+++ b/core/sql/sqlcomp/parser.cpp
@@ -1490,7 +1490,7 @@ NABoolean Parser::processSpecialDDL(const char* inputStr, size_t inputStrLen, Ch
       if (nstemp.index("STATISTICS", 0, NAString::ignoreCase) == 0)
 	{
 	  specialDDL = TRUE; // UPDATE STATISTICS
-    ustat = TRUE;
+          ustat = TRUE;
 
 	  // do not start Xn at runtime. 
 	  xnNeeded = FALSE;



[2/2] incubator-trafodion git commit: Merge [TRAFODION-2498] PR-979 Add support to run hive stmts from traf interface

Posted by an...@apache.org.
Merge [TRAFODION-2498] PR-979 Add support to run hive stmts from traf interface


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

Branch: refs/heads/master
Commit: 63ab7287477f7d0648ff0e1c0170b2d37708f93c
Parents: 2abcb03 e9389fa
Author: Anoop Sharma <an...@esgyn.com>
Authored: Mon Feb 27 20:25:06 2017 +0000
Committer: Anoop Sharma <an...@esgyn.com>
Committed: Mon Feb 27 20:25:06 2017 +0000

----------------------------------------------------------------------
 core/sql/comexe/ComTdb.h             |   1 +
 core/sql/comexe/ComTdbExeUtil.cpp    |  74 +++++++++++++++++
 core/sql/comexe/ComTdbExeUtil.h      |  49 ++++++++++-
 core/sql/executor/ExComTdb.cpp       |   8 ++
 core/sql/executor/ExExeUtil.h        |  96 ++++++++++++++++++++++
 core/sql/executor/ExExeUtilGet.cpp   |  40 +++++++--
 core/sql/executor/ExExeUtilMisc.cpp  | 125 ++++++++++++++++++++++++++++
 core/sql/generator/GenRelExeUtil.cpp |  63 +++++++++++++-
 core/sql/optimizer/RelExeUtil.cpp    |  56 +++++++++++++
 core/sql/optimizer/RelExeUtil.h      |  42 +++++++++-
 core/sql/parser/ParKeyWords.cpp      |   1 +
 core/sql/parser/sqlparser.y          |  29 ++++++-
 core/sql/regress/hive/EXPECTED005    | 132 +++++++++++++++---------------
 core/sql/regress/hive/EXPECTED007    |  86 +++++++++++--------
 core/sql/regress/hive/TEST005        |  20 ++---
 core/sql/regress/hive/TEST007        |  80 ++++++------------
 core/sql/regress/seabase/EXPECTED003 | 114 +++++++++++++-------------
 core/sql/regress/seabase/EXPECTED031 |  39 +++++----
 core/sql/regress/seabase/EXPECTED033 |  21 +++--
 core/sql/regress/seabase/TEST003     |  16 +---
 core/sql/regress/seabase/TEST031     |  10 +--
 core/sql/regress/seabase/TEST033     |  13 ++-
 core/sql/sqlci/sqlci_lex.ll          |   1 +
 core/sql/sqlci/sqlci_yacc.y          |   2 +
 core/sql/sqlcomp/parser.cpp          |   2 +-
 25 files changed, 831 insertions(+), 289 deletions(-)
----------------------------------------------------------------------