You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by se...@apache.org on 2018/02/16 20:00:35 UTC

[2/9] trafodion git commit: [TRAFODION-2917] Refactor Trafodion implementation of hdfs scan for text formatted hive tables

[TRAFODION-2917] Refactor Trafodion implementation of hdfs scan for text formatted hive tables

Part-1 changes.

Created a new class org.trafodion.sql.HDFSClient. Any direct HDFS access
will be routed to this class via JNI instead of using libhdfs.

Modified the existing code expect for the following to route the HDFS request via this class
1. LOB access
2. Direct HDFS scan of the table
3. Sample data creation during update stats

Added a new class org.trafodio.sql.HdfsScan for scanning one or many ranges of a Hive
text formatted table. This class will be used for Direct HDFS scan in near future.


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

Branch: refs/heads/master
Commit: 60db153329d1ee7088f6805ef3c5eb9eb8b600de
Parents: 03f705b
Author: selvaganesang <se...@esgyn.com>
Authored: Fri Jan 26 16:40:37 2018 +0000
Committer: selvaganesang <se...@esgyn.com>
Committed: Fri Jan 26 16:40:37 2018 +0000

----------------------------------------------------------------------
 core/sql/executor/ExExeUtil.h                   |   3 +-
 core/sql/executor/ExExeUtilLoad.cpp             |  43 +-
 core/sql/executor/ExFastTransport.cpp           | 102 +++--
 core/sql/executor/ExFastTransport.h             |   5 +-
 core/sql/executor/ExHbaseAccess.cpp             |  40 +-
 core/sql/executor/ExHbaseAccess.h               |  16 +-
 core/sql/executor/ExHbaseIUD.cpp                |  12 +-
 core/sql/executor/ExHdfsScan.cpp                |  66 ++-
 core/sql/executor/ExHdfsScan.h                  |   8 +-
 core/sql/executor/HBaseClient_JNI.cpp           | 126 ------
 core/sql/executor/HBaseClient_JNI.h             |  11 -
 core/sql/executor/HdfsClient_JNI.cpp            | 452 +++++++++++++++++++
 core/sql/executor/HdfsClient_JNI.h              | 146 ++++++
 core/sql/executor/SequenceFileReader.cpp        | 298 ------------
 core/sql/executor/SequenceFileReader.h          |  30 --
 core/sql/exp/ExpHbaseInterface.cpp              |  73 ---
 core/sql/exp/ExpHbaseInterface.h                |   6 -
 core/sql/nskgmake/executor/Makefile             |   1 +
 .../main/java/org/trafodion/sql/HDFSClient.java | 319 +++++++++++++
 .../main/java/org/trafodion/sql/HdfsScan.java   | 248 ++++++++++
 .../org/trafodion/sql/SequenceFileWriter.java   | 154 -------
 .../org/trafodion/sql/TrafConfiguration.java    |   1 +
 22 files changed, 1368 insertions(+), 792 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/ExExeUtil.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExExeUtil.h b/core/sql/executor/ExExeUtil.h
index d26f660..2431bcd 100644
--- a/core/sql/executor/ExExeUtil.h
+++ b/core/sql/executor/ExExeUtil.h
@@ -54,6 +54,7 @@ class ExSqlComp;
 class ExProcessStats;
 
 class ExpHbaseInterface;
+class HdfsClient;
 
 //class FILE_STREAM;
 #include "ComAnsiNamePart.h"
@@ -3886,11 +3887,11 @@ class ExExeUtilHBaseBulkUnLoadTcb : public ExExeUtilTcb
   Int64 endTime_;
   Int64 rowsAffected_;
   char statusMsgBuf_[BUFFER_SIZE];
-  SequenceFileWriter* sequenceFileWriter_;
   NAList<struct snapshotStruct *> * snapshotsList_;
   NABoolean emptyTarget_;
   NABoolean oneFile_;
   ExpHbaseInterface * ehi_;
+  HdfsClient *hdfsClient_;
 };
 
 class ExExeUtilHbaseUnLoadPrivateState : public ex_tcb_private_state

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/ExExeUtilLoad.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExExeUtilLoad.cpp b/core/sql/executor/ExExeUtilLoad.cpp
index ab1580c..819b3b1 100644
--- a/core/sql/executor/ExExeUtilLoad.cpp
+++ b/core/sql/executor/ExExeUtilLoad.cpp
@@ -68,6 +68,7 @@ using std::ofstream;
 #include "ExpHbaseInterface.h"
 #include "ExHbaseAccess.h"
 #include "ExpErrorEnums.h"
+#include "HdfsClient_JNI.h"
 
 ///////////////////////////////////////////////////////////////////
 ex_tcb * ExExeUtilCreateTableAsTdb::build(ex_globals * glob)
@@ -1958,10 +1959,10 @@ ex_tcb * ExExeUtilHBaseBulkUnLoadTdb::build(ex_globals * glob)
 
   return (exe_util_tcb);
 }
-void ExExeUtilHBaseBulkUnLoadTcb::createHdfsFileError(Int32 sfwRetCode)
+void ExExeUtilHBaseBulkUnLoadTcb::createHdfsFileError(Int32 hdfsClientRetCode)
 {
   ComDiagsArea * diagsArea = NULL;
-  char* errorMsg = sequenceFileWriter_->getErrorText((SFW_RetCode)sfwRetCode);
+  char* errorMsg = hdfsClient_->getErrorText((HDFS_Client_RetCode)hdfsClientRetCode);
   ExRaiseSqlError(getHeap(), &diagsArea, (ExeErrorCode)(8447), NULL,
                   NULL, NULL, NULL, errorMsg, (char *)GetCliGlobals()->currContext()->getJniErrorStr().data());
   ex_queue_entry *pentry_up = qparent_.up->getTailEntry();
@@ -1981,7 +1982,7 @@ ExExeUtilHBaseBulkUnLoadTcb::ExExeUtilHBaseBulkUnLoadTcb(
        emptyTarget_(FALSE),
        oneFile_(FALSE)
 {
-  sequenceFileWriter_ = NULL;
+  hdfsClient_ = NULL;
   int jniDebugPort = 0;
   int jniDebugTimeout = 0;
   ehi_ = ExpHbaseInterface::newInstance(getGlobals()->getDefaultHeap(),
@@ -2011,10 +2012,10 @@ void ExExeUtilHBaseBulkUnLoadTcb::freeResources()
     snapshotsList_ = NULL;
   }
 
-  if (sequenceFileWriter_)
+  if (hdfsClient_)
   {
-    NADELETE(sequenceFileWriter_, SequenceFileWriter, getMyHeap());
-    sequenceFileWriter_ = NULL;
+    NADELETE(hdfsClient_, HdfsClient, getMyHeap());
+    hdfsClient_ = NULL;
   }
   NADELETE(ehi_, ExpHbaseInterface, getGlobals()->getDefaultHeap());
   ehi_ = NULL;
@@ -2163,7 +2164,7 @@ short ExExeUtilHBaseBulkUnLoadTcb::work()
   Lng32 cliRC = 0;
   Lng32 retcode = 0;
   short rc;
-  SFW_RetCode sfwRetCode = SFW_OK;
+  HDFS_Client_RetCode hdfsClientRetCode = HDFS_CLIENT_OK;
   Lng32 hbcRetCode = HBC_OK;
   // if no parent request, return
   if (qparent_.down->isEmpty())
@@ -2197,14 +2198,12 @@ short ExExeUtilHBaseBulkUnLoadTcb::work()
       }
       setEmptyTarget(hblTdb().getEmptyTarget());
       setOneFile(hblTdb().getOneFile());
-      if (!sequenceFileWriter_)
+      if (!hdfsClient_)
       {
-        sequenceFileWriter_ = new(getMyHeap())
-                           SequenceFileWriter((NAHeap *)getMyHeap());
-        sfwRetCode = sequenceFileWriter_->init();
-        if (sfwRetCode != SFW_OK)
+        hdfsClient_ = HdfsClient::newInstance((NAHeap *)getMyHeap(), hdfsClientRetCode);
+        if (hdfsClientRetCode != HDFS_CLIENT_OK)
         {
-          createHdfsFileError(sfwRetCode);
+          createHdfsFileError(hdfsClientRetCode);
           step_ = UNLOAD_END_ERROR_;
           break;
         }
@@ -2220,10 +2219,10 @@ short ExExeUtilHBaseBulkUnLoadTcb::work()
       if (!hblTdb().getOverwriteMergeFile() &&  hblTdb().getMergePath() != NULL)
       {
         NABoolean exists = FALSE;
-        sfwRetCode = sequenceFileWriter_->hdfsExists( hblTdb().getMergePath(), exists);
-        if (sfwRetCode != SFW_OK)
+        hdfsClientRetCode = hdfsClient_->hdfsExists( hblTdb().getMergePath(), exists);
+        if (hdfsClientRetCode != HDFS_CLIENT_OK)
         {
-          createHdfsFileError(sfwRetCode);
+          createHdfsFileError(hdfsClientRetCode);
           step_ = UNLOAD_END_ERROR_;
           break;
         }
@@ -2305,10 +2304,10 @@ short ExExeUtilHBaseBulkUnLoadTcb::work()
 
       NAString uldPath ( hblTdb().getExtractLocation());
 
-      sfwRetCode = sequenceFileWriter_->hdfsCleanUnloadPath( uldPath);
-      if (sfwRetCode != SFW_OK)
+      hdfsClientRetCode = hdfsClient_->hdfsCleanUnloadPath( uldPath);
+      if (hdfsClientRetCode != HDFS_CLIENT_OK)
       {
-        createHdfsFileError(sfwRetCode);
+        createHdfsFileError(hdfsClientRetCode);
         step_ = UNLOAD_END_ERROR_;
         break;
       }
@@ -2450,10 +2449,10 @@ short ExExeUtilHBaseBulkUnLoadTcb::work()
 
       NAString srcPath ( hblTdb().getExtractLocation());
       NAString dstPath ( hblTdb().getMergePath());
-      sfwRetCode = sequenceFileWriter_->hdfsMergeFiles( srcPath, dstPath);
-      if (sfwRetCode != SFW_OK)
+      hdfsClientRetCode = hdfsClient_->hdfsMergeFiles( srcPath, dstPath);
+      if (hdfsClientRetCode != HDFS_CLIENT_OK)
       {
-        createHdfsFileError(sfwRetCode);
+        createHdfsFileError(hdfsClientRetCode);
         step_ = UNLOAD_END_;
         break;
       }

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/ExFastTransport.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExFastTransport.cpp b/core/sql/executor/ExFastTransport.cpp
index b1166bf..bdde201 100644
--- a/core/sql/executor/ExFastTransport.cpp
+++ b/core/sql/executor/ExFastTransport.cpp
@@ -48,6 +48,7 @@
 #include <pthread.h>
 #include "ComSysUtils.h"
 #include "SequenceFileReader.h" 
+#include "HdfsClient_JNI.h" 
 #include  "cli_stdh.h"
 #include "ComSmallDefs.h"
 
@@ -457,6 +458,7 @@ ExHdfsFastExtractTcb::ExHdfsFastExtractTcb(
       childTcb,
       glob),
     sequenceFileWriter_(NULL)
+  , hdfsClient_(NULL)
 {
 
 } // ExHdfsFastExtractTcb::ExFastExtractTcb
@@ -473,6 +475,12 @@ ExHdfsFastExtractTcb::~ExHdfsFastExtractTcb()
      NADELETE(sequenceFileWriter_, SequenceFileWriter, getHeap());
      sequenceFileWriter_ = NULL;
   }
+
+  if (hdfsClient_ != NULL) {
+     NADELETE(hdfsClient_, HdfsClient, getHeap());
+     hdfsClient_ = NULL;
+  }
+
 } // ExHdfsFastExtractTcb::~ExHdfsFastExtractTcb()
 
 
@@ -599,6 +607,7 @@ ExWorkProcRetcode ExHdfsFastExtractTcb::work()
 {
   Lng32 retcode = 0;
   SFW_RetCode sfwRetCode = SFW_OK;
+  HDFS_Client_RetCode hdfsClientRetCode = HDFS_CLIENT_OK;
   ULng32 recSepLen = strlen(myTdb().getRecordSeparator());
   ULng32 delimLen = strlen(myTdb().getDelimiter());
   ULng32 nullLen = 
@@ -767,7 +776,7 @@ ExWorkProcRetcode ExHdfsFastExtractTcb::work()
           else
             snprintf(fileName_,999, "%s%d-%s-%d", "file", fileNum, pt,rand() % 1000);
 
-          if (!sequenceFileWriter_)
+          if (isSequenceFile() && sequenceFileWriter_ == NULL)
           {
             sequenceFileWriter_ = new(getHeap())
                                      SequenceFileWriter((NAHeap *)getHeap());
@@ -779,20 +788,39 @@ ExWorkProcRetcode ExHdfsFastExtractTcb::work()
               break;
             }
           }
+          else if (!isSequenceFile() && hdfsClient_ == NULL)
+          {
+             hdfsClient_ = HdfsClient::newInstance((NAHeap *)getHeap(), hdfsClientRetCode);
+             if (hdfsClientRetCode != HDFS_CLIENT_OK)
+             {
+                createHdfsClientFileError(hdfsClientRetCode);
+                pstate.step_ = EXTRACT_ERROR;
+                break;
+             }
+          }
 
           strcat(targetLocation_, "//");
           strcat(targetLocation_, fileName_);
           if (isSequenceFile())
+          {
             sfwRetCode = sequenceFileWriter_->open(targetLocation_, SFW_COMP_NONE);
-          else
-            sfwRetCode = sequenceFileWriter_->hdfsCreate(targetLocation_, isHdfsCompressed());
-          if (sfwRetCode != SFW_OK)
+            if (sfwRetCode != SFW_OK)
             {
               createSequenceFileError(sfwRetCode);
               pstate.step_ = EXTRACT_ERROR;
               break;
             }
-            
+          }
+          else
+          {
+            hdfsClientRetCode = hdfsClient_->hdfsCreate(targetLocation_, isHdfsCompressed());
+            if (hdfsClientRetCode != HDFS_CLIENT_OK)
+            {
+              createHdfsClientFileError(hdfsClientRetCode);
+              pstate.step_ = EXTRACT_ERROR;
+              break;
+            }
+          }  
           if (feStats)
           {
             feStats->setPartitionNumber(fileNum);
@@ -1014,10 +1042,10 @@ ExWorkProcRetcode ExHdfsFastExtractTcb::work()
         }
       else
         {
-          sfwRetCode = sequenceFileWriter_->hdfsWrite(currBuffer_->data_, bytesToWrite);
-          if (sfwRetCode != SFW_OK)
+          hdfsClientRetCode = hdfsClient_->hdfsWrite(currBuffer_->data_, bytesToWrite);
+          if (hdfsClientRetCode != HDFS_CLIENT_OK)
           {
-            createSequenceFileError(sfwRetCode);
+            createSequenceFileError(hdfsClientRetCode);
             pstate.step_ = EXTRACT_ERROR;
             break;
           }
@@ -1105,28 +1133,31 @@ ExWorkProcRetcode ExHdfsFastExtractTcb::work()
         return WORK_OK;
 
       if (isSequenceFile())
-        {
-          sfwRetCode = sequenceFileWriter_->close();
-          if (!errorOccurred_ && sfwRetCode != SFW_OK )
-          {
-            createSequenceFileError(sfwRetCode);
-            pstate.step_ = EXTRACT_ERROR;
-            break;
-          }
-        }
+      {
+         if (sequenceFileWriter_) 
+         {
+            sfwRetCode = sequenceFileWriter_->close();
+            if (!errorOccurred_ && sfwRetCode != SFW_OK )
+            {
+               createSequenceFileError(sfwRetCode);
+               pstate.step_ = EXTRACT_ERROR;
+               break;
+            }
+         }
+      }
       else
-        {
-          if (sequenceFileWriter_)
+      {
+         if (hdfsClient_)
+         {
+            hdfsClientRetCode = hdfsClient_->hdfsClose();
+            if (!errorOccurred_ && HDFS_CLIENT_OK != HDFS_CLIENT_OK )
             {
-              sfwRetCode = sequenceFileWriter_->hdfsClose();
-              if (!errorOccurred_ && sfwRetCode != SFW_OK )
-                {
-                  createSequenceFileError(sfwRetCode);
-                  pstate.step_ = EXTRACT_ERROR;
-                  break;
-                }
+               createHdfsClientFileError(hdfsClientRetCode);
+               pstate.step_ = EXTRACT_ERROR;
+               break;
             }
-        }
+         }   
+      }
 
       //insertUpQueueEntry will insert Q_NO_DATA into the up queue and
       //remove the head of the down queue
@@ -1248,6 +1279,23 @@ void ExHdfsFastExtractTcb::createSequenceFileError(Int32 sfwRetCode)
   updateWorkATPDiagsArea(diagsArea);
 }
 
+void ExHdfsFastExtractTcb::createHdfsClientFileError(Int32 hdfsClientRetCode)
+{
+  ContextCli *currContext = GetCliGlobals()->currContext();
+
+  ComDiagsArea * diagsArea = NULL;
+  char* errorMsg = hdfsClient_->getErrorText((HDFS_Client_RetCode)hdfsClientRetCode);
+  ExRaiseSqlError(getHeap(),
+                  &diagsArea,
+                  (ExeErrorCode)(8447),
+                  NULL, NULL, NULL, NULL,
+                  errorMsg,
+                (char *)currContext->getJniErrorStr().data());
+  //ex_queue_entry *pentry_down = qParent_.down->getHeadEntry();
+  //pentry_down->setDiagsArea(diagsArea);
+  updateWorkATPDiagsArea(diagsArea);
+}
+
 ExFastExtractPrivateState::ExFastExtractPrivateState()
 {
   init();

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/ExFastTransport.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExFastTransport.h b/core/sql/executor/ExFastTransport.h
index 22b6cb0..94b091d 100644
--- a/core/sql/executor/ExFastTransport.h
+++ b/core/sql/executor/ExFastTransport.h
@@ -34,7 +34,7 @@
 #include "ex_tcb.h"
 #include "ComSmallDefs.h"
 #include "ExStats.h"
-
+#include "HdfsClient_JNI.h"
 #include "ExpLOBinterface.h"
 #include "ex_exe_stmt_globals.h"
 // -----------------------------------------------------------------------
@@ -43,6 +43,7 @@
 class sql_buffer;
 class ExExeStmtGlobals;
 class SequenceFileWriter;
+class HdfsClient;
 
 // -----------------------------------------------------------------------
 // Classes defined in this file
@@ -407,6 +408,7 @@ protected:
                           
   NABoolean isSequenceFile();
   void createSequenceFileError(Int32 sfwRetCode);
+  void createHdfsClientFileError(Int32 sfwRetCode);
   NABoolean isHdfsCompressed();
   NABoolean getEmptyNullString()
   {
@@ -423,6 +425,7 @@ protected:
   char targetLocation_[1000];
   NABoolean errorOccurred_;
   SequenceFileWriter* sequenceFileWriter_;
+  HdfsClient *hdfsClient_;
 }; // class ExHdfsFastExtractTcb
 
 //----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/ExHbaseAccess.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHbaseAccess.cpp b/core/sql/executor/ExHbaseAccess.cpp
index deababb..42fd86e 100644
--- a/core/sql/executor/ExHbaseAccess.cpp
+++ b/core/sql/executor/ExHbaseAccess.cpp
@@ -41,6 +41,7 @@
 #include "jni.h"
 #include "hdfs.h"
 #include <random>
+#include "HdfsClient_JNI.h"
 
 // forward declare
 Int64 generateUniqueValueFast ();
@@ -241,6 +242,9 @@ ExHbaseAccessTcb::ExHbaseAccessTcb(
   , colValVecSize_(0)
   , colValEntry_(0)
   , loggingErrorDiags_(NULL)
+  , hdfsClient_(NULL)
+  , loggingFileCreated_(FALSE)
+  , loggingFileName_(NULL)
 {
   Space * space = (glob ? glob->getSpace() : NULL);
   CollHeap * heap = (glob ? glob->getDefaultHeap() : NULL);
@@ -502,6 +506,8 @@ void ExHbaseAccessTcb::freeResources()
      NADELETEBASIC(directRowBuffer_, getHeap());
   if (colVal_.val != NULL)
      NADELETEBASIC(colVal_.val, getHeap());
+  if (hdfsClient_ != NULL) 
+     NADELETE(hdfsClient_, HdfsClient, getHeap());
 }
 
 
@@ -3251,30 +3257,28 @@ void ExHbaseAccessTcb::buildLoggingPath(
 void ExHbaseAccessTcb::handleException(NAHeap *heap,
                                     char *logErrorRow,
                                     Lng32 logErrorRowLen,
-                                    ComCondition *errorCond,
-                                    ExpHbaseInterface * ehi,
-                                    NABoolean & LoggingFileCreated,
-                                    char *loggingFileName,
-                                    ComDiagsArea **loggingErrorDiags)
+                                    ComCondition *errorCond)
 {
   Lng32 errorMsgLen = 0;
   charBuf *cBuf = NULL;
   char *errorMsg;
-  Lng32 retcode;
+  HDFS_Client_RetCode hdfsClientRetcode;
 
-  if (*loggingErrorDiags != NULL)
+  if (loggingErrorDiags_ != NULL)
      return;
 
-  if (!LoggingFileCreated) {
-     retcode = ehi->hdfsCreateFile(loggingFileName);
-     if (retcode == HBASE_ACCESS_SUCCESS)
-        LoggingFileCreated = TRUE;
+  if (!loggingFileCreated_) {
+     hdfsClient_ = HdfsClient::newInstance((NAHeap *)getHeap(), hdfsClientRetcode);
+     if (hdfsClientRetcode == HDFS_CLIENT_OK)
+        hdfsClientRetcode = hdfsClient_->hdfsCreate(loggingFileName_, FALSE);
+     if (hdfsClientRetcode == HDFS_CLIENT_OK)
+        loggingFileCreated_ = TRUE;
      else 
         goto logErrorReturn;
   }
   
-  retcode = ehi->hdfsWrite(logErrorRow, logErrorRowLen);
-  if (retcode != HBASE_ACCESS_SUCCESS) 
+  hdfsClientRetcode = hdfsClient_->hdfsWrite(logErrorRow, logErrorRowLen);
+  if (hdfsClientRetcode != HDFS_CLIENT_OK) 
      goto logErrorReturn;
   if (errorCond != NULL) {
      errorMsgLen = errorCond->getMessageLength();
@@ -3289,12 +3293,12 @@ void ExHbaseAccessTcb::handleException(NAHeap *heap,
      errorMsg = (char *)"[UNKNOWN EXCEPTION]\n";
      errorMsgLen = strlen(errorMsg);
   }
-  retcode = ehi->hdfsWrite(errorMsg, errorMsgLen);
+  hdfsClientRetcode = hdfsClient_->hdfsWrite(errorMsg, errorMsgLen);
 logErrorReturn:
-  if (retcode != HBASE_ACCESS_SUCCESS) {
-     *loggingErrorDiags = ComDiagsArea::allocate(heap);
-     **loggingErrorDiags << DgSqlCode(EXE_ERROR_WHILE_LOGGING)
-                 << DgString0(loggingFileName)
+  if (hdfsClientRetcode != HDFS_CLIENT_OK) {
+     loggingErrorDiags_ = ComDiagsArea::allocate(heap);
+     *loggingErrorDiags_ << DgSqlCode(EXE_ERROR_WHILE_LOGGING)
+                 << DgString0(loggingFileName_)
                  << DgString1((char *)GetCliGlobals()->currContext()->getJniErrorStr().data());
   }
   return;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/ExHbaseAccess.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHbaseAccess.h b/core/sql/executor/ExHbaseAccess.h
index 661fa5b..863b5b5 100644
--- a/core/sql/executor/ExHbaseAccess.h
+++ b/core/sql/executor/ExHbaseAccess.h
@@ -52,6 +52,8 @@ class ExHbaseAccessStats;
 class ExpHbaseInterface;
 class ExHbaseAccessSelectTcb;
 class ExHbaseAccessUMDTcb;
+class HdfsClient;
+
 #define INLINE_ROWID_LEN 255
 // -----------------------------------------------------------------------
 // ExHbaseAccessTdb
@@ -163,14 +165,11 @@ public:
   static void getErrorCount( ExpHbaseInterface * ehi,Int64 & totalExceptionCount,
                                const char * tabName, const char * rowId);
 
-  static void handleException(NAHeap *heap,
+  void handleException(NAHeap *heap,
                           char *loggingDdata,
                           Lng32 loggingDataLen,
-                          ComCondition *errorCond,
-                          ExpHbaseInterface * ehi,
-                          NABoolean & LoggingFileCreated,
-                          char * loggingFileName,
-                          ComDiagsArea **loggingErrorDiags);
+                          ComCondition *errorCond);
+
   static void buildLoggingPath(const char * loggingLocation,
                                char *logId,
                                const char *tableName,
@@ -502,6 +501,9 @@ protected:
   NABoolean asyncOperation_;
   Int32 asyncOperationTimeout_;
   ComDiagsArea *loggingErrorDiags_;
+  HdfsClient *hdfsClient_;
+  char *loggingFileName_;
+  NABoolean loggingFileCreated_ ;
 
   // Redefined and used by ExHbaseAccessBulkLoadPrepSQTcb.
 
@@ -939,8 +941,6 @@ class ExHbaseAccessBulkLoadPrepSQTcb: public ExHbaseAccessUpsertVsbbSQTcb
     Text   importLocation_;
     Text   hFileName_;
 
-    char *loggingFileName_;
-    NABoolean LoggingFileCreated_ ;
     ComCondition * lastErrorCnd_;
     std::vector<UInt32> posVec_;
 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/ExHbaseIUD.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHbaseIUD.cpp b/core/sql/executor/ExHbaseIUD.cpp
index bd6307b..3bc1d93 100644
--- a/core/sql/executor/ExHbaseIUD.cpp
+++ b/core/sql/executor/ExHbaseIUD.cpp
@@ -33,6 +33,7 @@
 #include "NLSConversion.h"
 #include "ExHdfsScan.h"
 #include "Context.h"
+#include "HdfsClient_JNI.h"
 
 ExHbaseAccessInsertTcb::ExHbaseAccessInsertTcb(
           const ExHbaseAccessTdb &hbaseAccessTdb, 
@@ -1142,7 +1143,6 @@ ExHbaseAccessBulkLoadPrepSQTcb::ExHbaseAccessBulkLoadPrepSQTcb(
     prevRowId_ (NULL),
     hdfs_(NULL),
     hdfsSampleFile_(NULL),
-    loggingFileName_(NULL),
     lastErrorCnd_(NULL)
 {
    hFileParamsInitialized_ = false;
@@ -1158,7 +1158,7 @@ ExHbaseAccessBulkLoadPrepSQTcb::ExHbaseAccessBulkLoadPrepSQTcb(
                       "traf_upsert_err",
                       fileNum,
                       loggingFileName_);
-   LoggingFileCreated_ = FALSE;
+   loggingFileCreated_ = FALSE;
    loggingRow_ =  new(glob->getDefaultHeap()) char[hbaseAccessTdb.updateRowLen_];
 }
 
@@ -1676,10 +1676,7 @@ ExWorkProcRetcode ExHbaseAccessBulkLoadPrepSQTcb::work()
         createLoggingRow( hbaseAccessTdb().updateTuppIndex_,  updateRow_,
             loggingRow_ , loggingRowLen);
         ExHbaseAccessTcb::handleException((NAHeap *)getHeap(), loggingRow_, loggingRowLen,
-               lastErrorCnd_,
-               ehi_,
-               LoggingFileCreated_,
-               loggingFileName_, &loggingErrorDiags_);
+               lastErrorCnd_);
       }
       if (pentry_down->getDiagsArea())
         pentry_down->getDiagsArea()->clear();
@@ -1771,7 +1768,8 @@ ExWorkProcRetcode ExHbaseAccessBulkLoadPrepSQTcb::work()
           if (eodSeen)
           {
             ehi_->closeHFile(table_);
-            ehi_->hdfsClose();
+            if (hdfsClient_ != NULL)
+               hdfsClient_->hdfsClose();
             hFileParamsInitialized_ = false;
             retcode = ehi_->close();
           }

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/ExHdfsScan.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHdfsScan.cpp b/core/sql/executor/ExHdfsScan.cpp
index f49ad06..90ac737 100644
--- a/core/sql/executor/ExHdfsScan.cpp
+++ b/core/sql/executor/ExHdfsScan.cpp
@@ -45,6 +45,7 @@
 
 #include "ExpORCinterface.h"
 #include "ComSmallDefs.h"
+#include "HdfsClient_JNI.h"
 
 ex_tcb * ExHdfsScanTdb::build(ex_globals * glob)
 {
@@ -118,6 +119,7 @@ ExHdfsScanTcb::ExHdfsScanTcb(
   , dataModCheckDone_(FALSE)
   , loggingErrorDiags_(NULL)
   , loggingFileName_(NULL)
+  , hdfsClient_(NULL)
   , hdfsFileInfoListAsArray_(glob->getDefaultHeap(), hdfsScanTdb.getHdfsFileInfoList()->numEntries())
 {
   Space * space = (glob ? glob->getSpace() : 0);
@@ -192,7 +194,7 @@ ExHdfsScanTcb::ExHdfsScanTcb(
                      "hive_scan_err",
                      fileNum,
                      loggingFileName_);
-  LoggingFileCreated_ = FALSE;
+  loggingFileCreated_ = FALSE;
 
   
   //shoud be move to work method
@@ -283,6 +285,8 @@ void ExHdfsScanTcb::freeResources()
      ExpLOBinterfaceCleanup(lobGlob_, (NAHeap *)getGlobals()->getDefaultHeap());
      lobGlob_ = NULL;
   }
+  if (hdfsClient_ != NULL) 
+     NADELETE(hdfsClient_, HdfsClient, getHeap());
 }
 
 NABoolean ExHdfsScanTcb::needStatsEntry()
@@ -1365,12 +1369,8 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
           if (hdfsScanTdb().getLogErrorRows())
           {
             int loggingRowLen =  hdfsLoggingRowEnd_ - hdfsLoggingRow_ +1;
-            ExHbaseAccessTcb::handleException((NAHeap *)getHeap(), hdfsLoggingRow_,
-                       loggingRowLen, lastErrorCnd_, 
-                       ehi_,
-                       LoggingFileCreated_,
-                       loggingFileName_,
-                       &loggingErrorDiags_);
+            handleException((NAHeap *)getHeap(), hdfsLoggingRow_,
+                       loggingRowLen, lastErrorCnd_ );
 
             
           }
@@ -1511,8 +1511,8 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	  {
 	    if (qparent_.up->isFull())
 	      return WORK_OK;
-            if (ehi_ != NULL)
-               retcode = ehi_->hdfsClose();
+            if (hdfsClient_ != NULL)
+               retcode = hdfsClient_->hdfsClose();
 	    ex_queue_entry *up_entry = qparent_.up->getTailEntry();
 	    up_entry->copyAtp(pentry_down);
 	    up_entry->upState.parentIndex =
@@ -1948,6 +1948,54 @@ short ExHdfsScanTcb::handleDone(ExWorkProcRetcode &rc)
   return 0;
 }
 
+void ExHdfsScanTcb::handleException(NAHeap *heap,
+                                    char *logErrorRow,
+                                    Lng32 logErrorRowLen,
+                                    ComCondition *errorCond)
+{
+  Lng32 errorMsgLen = 0;
+  charBuf *cBuf = NULL;
+  char *errorMsg;
+  HDFS_Client_RetCode hdfsClientRetcode;
+
+  if (loggingErrorDiags_ != NULL)
+     return;
+
+  if (!loggingFileCreated_) {
+     hdfsClient_ = HdfsClient::newInstance((NAHeap *)getHeap(), hdfsClientRetcode);
+     if (hdfsClientRetcode == HDFS_CLIENT_OK)
+        hdfsClientRetcode = hdfsClient_->hdfsCreate(loggingFileName_, FALSE);
+     if (hdfsClientRetcode == HDFS_CLIENT_OK)
+        loggingFileCreated_ = TRUE;
+     else 
+        goto logErrorReturn;
+  }
+  hdfsClientRetcode = hdfsClient_->hdfsWrite(logErrorRow, logErrorRowLen);
+  if (hdfsClientRetcode != HDFS_CLIENT_OK) 
+     goto logErrorReturn;
+  if (errorCond != NULL) {
+     errorMsgLen = errorCond->getMessageLength();
+     const NAWcharBuf wBuf((NAWchar*)errorCond->getMessageText(), errorMsgLen, heap);
+     cBuf = unicodeToISO88591(wBuf, heap, cBuf);
+     errorMsg = (char *)cBuf->data();
+     errorMsgLen = cBuf -> getStrLen();
+     errorMsg[errorMsgLen]='\n';
+     errorMsgLen++;
+  }
+  else {
+     errorMsg = (char *)"[UNKNOWN EXCEPTION]\n";
+     errorMsgLen = strlen(errorMsg);
+  }
+  hdfsClientRetcode = hdfsClient_->hdfsWrite(errorMsg, errorMsgLen);
+logErrorReturn:
+  if (hdfsClientRetcode != HDFS_CLIENT_OK) {
+     loggingErrorDiags_ = ComDiagsArea::allocate(heap);
+     *loggingErrorDiags_ << DgSqlCode(EXE_ERROR_WHILE_LOGGING)
+                 << DgString0(loggingFileName_)
+                 << DgString1((char *)GetCliGlobals()->currContext()->getJniErrorStr().data());
+  }
+}
+
 ////////////////////////////////////////////////////////////////////////
 // ORC files
 ////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/ExHdfsScan.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHdfsScan.h b/core/sql/executor/ExHdfsScan.h
index 27ea27f..984fbb9 100644
--- a/core/sql/executor/ExHdfsScan.h
+++ b/core/sql/executor/ExHdfsScan.h
@@ -207,6 +207,11 @@ protected:
   short handleError(short &rc);
   short handleDone(ExWorkProcRetcode &rc);
 
+  void handleException(NAHeap *heap,
+                          char *loggingDdata,
+                          Lng32 loggingDataLen,
+                          ComCondition *errorCond);
+
   short setupError(Lng32 exeError, Lng32 retcode, 
                    const char * str, const char * str2, const char * str3);
 
@@ -275,7 +280,7 @@ protected:
   char cursorId_[8];
 
   char *loggingFileName_;
-  NABoolean LoggingFileCreated_ ;
+  NABoolean loggingFileCreated_ ;
   char * hdfsLoggingRow_;
   char * hdfsLoggingRowEnd_;
   tupp_descriptor * defragTd_;
@@ -291,6 +296,7 @@ protected:
 
   // this array is populated from the info list stored as Queue.
   HdfsFileInfoArray hdfsFileInfoListAsArray_;
+  HdfsClient *hdfsClient_;
 };
 
 class ExOrcScanTcb  : public ExHdfsScanTcb

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/HBaseClient_JNI.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/HBaseClient_JNI.cpp b/core/sql/executor/HBaseClient_JNI.cpp
index c181b41..fe56d94 100644
--- a/core/sql/executor/HBaseClient_JNI.cpp
+++ b/core/sql/executor/HBaseClient_JNI.cpp
@@ -1871,121 +1871,6 @@ HBLC_RetCode HBulkLoadClient_JNI::bulkLoadCleanup(
 }
 
 
-//////////////////////////////////////////////////////////////////////////////
- //
- //////////////////////////////////////////////////////////////////////////////
-HVC_RetCode  HiveClient_JNI::hdfsCreateFile(const char* path)
-{
-  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HiveClient_JNI::hdfsCreate(%s) called.", path);
-
-  if (initJNIEnv() != JOI_OK)
-     return HVC_ERROR_INIT_PARAM;
-
-   jstring js_path = jenv_->NewStringUTF(path);
-   if (js_path == NULL) {
-     jenv_->PopLocalFrame(NULL);
-     return HVC_ERROR_HDFS_CREATE_PARAM;
-   }
-   jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_CREATE_FILE].methodID, js_path);
-   if (jenv_->ExceptionCheck())
-   {
-     getExceptionDetails();
-     logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-     logError(CAT_SQL_HBASE, "HiveClient_JNI::hdfsCreate()", getLastError());
-     jenv_->PopLocalFrame(NULL);
-     return HVC_ERROR_HDFS_CREATE_EXCEPTION;
-   }
-
-   if (jresult == false)
-   {
-     logError(CAT_SQL_HBASE, "HiveClient_JNI::hdfsCreaten()", getLastError());
-     jenv_->PopLocalFrame(NULL);
-     return HVC_ERROR_HDFS_CREATE_EXCEPTION;
-   }
-
-   jenv_->PopLocalFrame(NULL);
-   return HVC_OK;
- }
-
- //////////////////////////////////////////////////////////////////////////////
- //
- //////////////////////////////////////////////////////////////////////////////
- HVC_RetCode  HiveClient_JNI::hdfsWrite(const char* data, Int64 len)
- {
-   QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HiveClient_JNI::hdfsWrite(%ld) called.", len);
-
-   if (initJNIEnv() != JOI_OK)
-     return HVC_ERROR_INIT_PARAM;
-
-   //Write the requisite bytes into the file
-   jbyteArray jbArray = jenv_->NewByteArray( len);
-   if (!jbArray) {
-     jenv_->PopLocalFrame(NULL);
-     return HVC_ERROR_HDFS_WRITE_PARAM;
-   }
-   jenv_->SetByteArrayRegion(jbArray, 0, len, (const jbyte*)data);
-
-   jlong j_len = len;
-   // String write(java.lang.String);
-   jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_WRITE].methodID,jbArray , j_len);
-
-   if (jenv_->ExceptionCheck())
-   {
-     getExceptionDetails();
-     logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-     logError(CAT_SQL_HBASE, "HiveClient_JNI::hdfsWrite()", getLastError());
-     jenv_->PopLocalFrame(NULL);
-     return HVC_ERROR_HDFS_WRITE_EXCEPTION;
-   }
-
-   if (jresult == false)
-   {
-     logError(CAT_SQL_HBASE, "HiveClient_JNI::hdfsWrite()", getLastError());
-     jenv_->PopLocalFrame(NULL);
-     return HVC_ERROR_HDFS_WRITE_EXCEPTION;
-   }
-   jenv_->PopLocalFrame(NULL);
-   return HVC_OK;
- }
-
- //////////////////////////////////////////////////////////////////////////////
- //
- //////////////////////////////////////////////////////////////////////////////
-HVC_RetCode  HiveClient_JNI::hdfsClose()
-{
-   QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HiveClient_JNI::close() called.");
-   if (initJNIEnv() != JOI_OK)
-      return HVC_ERROR_INIT_PARAM;
-   if (javaObj_ == NULL)
-   {
-     // Maybe there was an initialization error.
-     jenv_->PopLocalFrame(NULL);
-     return HVC_OK;
-   }
-
-   // String close();
-   tsRecentJMFromJNI = JavaMethods_[JM_HDFS_CLOSE].jm_full_name;
-   jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_CLOSE].methodID);
-
-   if (jenv_->ExceptionCheck())
-   {
-     getExceptionDetails();
-     logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-     logError(CAT_SQL_HBASE, "HiveClient_JNI::hdfsClose()", getLastError());
-     jenv_->PopLocalFrame(NULL);
-     return HVC_ERROR_HDFS_CLOSE_EXCEPTION;
-   }
-
-   if (jresult == false)
-   {
-     logError(CAT_SQL_HBASE, "HiveClient_JNI::hdfsClose()", getLastError());
-     jenv_->PopLocalFrame(NULL);
-     return HVC_ERROR_HDFS_CLOSE_EXCEPTION;
-   }
-
-   jenv_->PopLocalFrame(NULL);
-   return HVC_OK;
-}
  //////////////////////////////////////////////////////////////////////////////
  //
  //////////////////////////////////////////////////////////////////////////////
@@ -3817,11 +3702,6 @@ static const char* const hvcErrorEnumStr[] =
  ,"Java exception in getAllSchemas()."
  ,"Preparing parameters for getAllTables()."
  ,"Java exception in getAllTables()."
- ,"preparing parameters for hdfsCreateFile()."
- ,"java exception in hdfsCreateFile()."
- ,"preparing parameters for hdfsWrite()."
- ,"java exception in hdfsWrite()."
- ,"java exception in hdfsclose()."
 };
 
 
@@ -3915,12 +3795,6 @@ HVC_RetCode HiveClient_JNI::init()
     JavaMethods_[JM_GET_ASH     ].jm_signature = "()[Ljava/lang/Object;";
     JavaMethods_[JM_GET_ATL    ].jm_name      = "getAllTables";
     JavaMethods_[JM_GET_ATL    ].jm_signature = "(Ljava/lang/String;)[Ljava/lang/Object;";
-    JavaMethods_[JM_HDFS_CREATE_FILE ].jm_name      = "hdfsCreateFile";
-    JavaMethods_[JM_HDFS_CREATE_FILE ].jm_signature = "(Ljava/lang/String;)Z";
-    JavaMethods_[JM_HDFS_WRITE       ].jm_name      = "hdfsWrite";
-    JavaMethods_[JM_HDFS_WRITE       ].jm_signature = "([BJ)Z";
-    JavaMethods_[JM_HDFS_CLOSE       ].jm_name      = "hdfsClose";
-    JavaMethods_[JM_HDFS_CLOSE       ].jm_signature = "()Z";
     JavaMethods_[JM_EXEC_HIVE_SQL].jm_name = "executeHiveSQL";
     JavaMethods_[JM_EXEC_HIVE_SQL].jm_signature = "(Ljava/lang/String;)V";
     rc = (HVC_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/HBaseClient_JNI.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/HBaseClient_JNI.h b/core/sql/executor/HBaseClient_JNI.h
index d2f5d2b..f6667d9 100644
--- a/core/sql/executor/HBaseClient_JNI.h
+++ b/core/sql/executor/HBaseClient_JNI.h
@@ -631,11 +631,6 @@ typedef enum {
  ,HVC_ERROR_GET_ALLSCH_EXCEPTION
  ,HVC_ERROR_GET_ALLTBL_PARAM
  ,HVC_ERROR_GET_ALLTBL_EXCEPTION
- ,HVC_ERROR_HDFS_CREATE_PARAM
- ,HVC_ERROR_HDFS_CREATE_EXCEPTION
- ,HVC_ERROR_HDFS_WRITE_PARAM
- ,HVC_ERROR_HDFS_WRITE_EXCEPTION
- ,HVC_ERROR_HDFS_CLOSE_EXCEPTION
  ,HVC_LAST
 } HVC_RetCode;
 
@@ -668,9 +663,6 @@ public:
   HVC_RetCode getAllSchemas(LIST(Text *)& schNames);
   HVC_RetCode getAllTables(const char* schName, LIST(Text *)& tblNames);
 
-  HVC_RetCode hdfsCreateFile(const char* path);
-  HVC_RetCode hdfsWrite(const char* data, Int64 len);
-  HVC_RetCode hdfsClose();
   HVC_RetCode executeHiveSQL(const char* hiveSQL);
   // Get the error description.
   virtual char* getErrorText(HVC_RetCode errEnum);
@@ -698,9 +690,6 @@ private:
    ,JM_GET_RDT
    ,JM_GET_ASH
    ,JM_GET_ATL
-   ,JM_HDFS_CREATE_FILE
-   ,JM_HDFS_WRITE
-   ,JM_HDFS_CLOSE
    ,JM_EXEC_HIVE_SQL
    ,JM_LAST
   };

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/HdfsClient_JNI.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/HdfsClient_JNI.cpp b/core/sql/executor/HdfsClient_JNI.cpp
new file mode 100644
index 0000000..a3aef5a
--- /dev/null
+++ b/core/sql/executor/HdfsClient_JNI.cpp
@@ -0,0 +1,452 @@
+//**********************************************************************
+// @@@ START COPYRIGHT @@@
+//
+// 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.
+//
+// @@@ END COPYRIGHT @@@
+// **********************************************************************
+
+#include "QRLogger.h"
+#include "Globals.h"
+#include "jni.h"
+#include "HdfsClient_JNI.h"
+
+// ===========================================================================
+// ===== Class HdfsScan
+// ===========================================================================
+
+JavaMethodInit* HdfsScan::JavaMethods_ = NULL;
+jclass HdfsScan::javaClass_ = 0;
+bool HdfsScan::javaMethodsInitialized_ = false;
+pthread_mutex_t HdfsScan::javaMethodsInitMutex_ = PTHREAD_MUTEX_INITIALIZER;
+
+static const char* const hdfsScanErrorEnumStr[] = 
+{
+};
+
+ 
+//////////////////////////////////////////////////////////////////////////////
+// 
+//////////////////////////////////////////////////////////////////////////////
+HDFS_Scan_RetCode HdfsScan::init()
+{
+  static char className[]="org/trafodion/sql/HdfsScan";
+  HDFS_Scan_RetCode rc; 
+
+  if (javaMethodsInitialized_)
+    return (HDFS_Scan_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_); 
+  else
+  {
+    pthread_mutex_lock(&javaMethodsInitMutex_);
+    if (javaMethodsInitialized_)
+    {
+      pthread_mutex_unlock(&javaMethodsInitMutex_);
+      return (HDFS_Scan_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
+    }
+    JavaMethods_ = new JavaMethodInit[JM_LAST];
+    
+    JavaMethods_[JM_CTOR      ].jm_name      = "<init>";
+    JavaMethods_[JM_CTOR      ].jm_signature = "()V";
+    JavaMethods_[JM_INIT_SCAN_RANGES].jm_name      = "<init>";
+    JavaMethods_[JM_INIT_SCAN_RANGES].jm_signature = "(Ljava/lang/Object;Ljava/lang/Object;[Ljava/lang/String;[J[J)V";
+    JavaMethods_[JM_TRAF_HDFS_READ].jm_name      = "trafHdfsRead";
+    JavaMethods_[JM_TRAF_HDFS_READ].jm_signature = "()[I";
+   
+    rc = (HDFS_Scan_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
+    javaMethodsInitialized_ = TRUE;
+    pthread_mutex_unlock(&javaMethodsInitMutex_);
+  }
+  return rc;
+}
+        
+char* HdfsScan::getErrorText(HDFS_Scan_RetCode errEnum)
+{
+  if (errEnum < (HDFS_Scan_RetCode)JOI_LAST)
+    return JavaObjectInterface::getErrorText((JOI_RetCode)errEnum);
+  else
+    return (char*)hdfsScanErrorEnumStr[errEnum-HDFS_SCAN_FIRST-1];
+}
+//////////////////////////////////////////////////////////////////////////////
+HDFS_Scan_RetCode HdfsScan::initScanRanges()
+{
+   return HDFS_SCAN_OK; 
+}
+
+HDFS_Scan_RetCode HdfsScan::trafHdfsRead()
+{
+   return HDFS_SCAN_OK; 
+}
+
+// ===========================================================================
+// ===== Class HdfsClient
+// ===========================================================================
+
+JavaMethodInit* HdfsClient::JavaMethods_ = NULL;
+jclass HdfsClient::javaClass_ = 0;
+bool HdfsClient::javaMethodsInitialized_ = false;
+pthread_mutex_t HdfsClient::javaMethodsInitMutex_ = PTHREAD_MUTEX_INITIALIZER;
+
+static const char* const hdfsClientErrorEnumStr[] = 
+{
+  "JNI NewStringUTF() in hdfsCreate()."
+ ,"Java exception in hdfsCreate()."
+ ,"JNI NewStringUTF() in hdfsWrite()."
+ ,"Java exception in hdfsWrite()."
+ ,"Java exception in hdfsClose()."
+ ,"JNI NewStringUTF() in hdfsMergeFiles()."
+ ,"Java exception in hdfsMergeFiles()."
+ ,"JNI NewStringUTF() in hdfsCleanUnloadPath()."
+ ,"Java exception in hdfsCleanUnloadPath()."
+ ,"JNI NewStringUTF() in hdfsExists()."
+ ,"Java exception in hdfsExists()."
+ ,"JNI NewStringUTF() in hdfsDeletePath()."
+ ,"Java exception in hdfsDeletePath()."
+};
+
+//////////////////////////////////////////////////////////////////////////////
+// 
+//////////////////////////////////////////////////////////////////////////////
+HdfsClient *HdfsClient::newInstance(NAHeap *heap, HDFS_Client_RetCode &retCode)
+{
+   retCode = HDFS_CLIENT_OK;
+   HdfsClient *hdfsClient = new (heap) HdfsClient(heap);
+   if (hdfsClient != NULL) {
+       retCode = hdfsClient->init();
+       if (retCode != HDFS_CLIENT_OK)
+       {
+          NADELETE(hdfsClient, HdfsClient, heap);
+          hdfsClient = NULL;
+       }
+   }
+   return hdfsClient;
+}
+
+HDFS_Client_RetCode HdfsClient::init()
+{
+  static char className[]="org/trafodion/sql/HDFSClient";
+  HDFS_Client_RetCode rc;
+  
+  if (javaMethodsInitialized_)
+    return (HDFS_Client_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
+  else
+  {
+    pthread_mutex_lock(&javaMethodsInitMutex_);
+    if (javaMethodsInitialized_)
+    {
+      pthread_mutex_unlock(&javaMethodsInitMutex_);
+      return (HDFS_Client_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
+    }
+    JavaMethods_ = new JavaMethodInit[JM_LAST];
+    
+    JavaMethods_[JM_CTOR      ].jm_name      = "<init>";
+    JavaMethods_[JM_CTOR      ].jm_signature = "()V";
+    JavaMethods_[JM_HDFS_CREATE     ].jm_name      = "hdfsCreate";
+    JavaMethods_[JM_HDFS_CREATE     ].jm_signature = "(Ljava/lang/String;Z)Z";
+    JavaMethods_[JM_HDFS_WRITE      ].jm_name      = "hdfsWrite";
+    JavaMethods_[JM_HDFS_WRITE      ].jm_signature = "([BJ)Z";
+    JavaMethods_[JM_HDFS_CLOSE      ].jm_name      = "hdfsClose";
+    JavaMethods_[JM_HDFS_CLOSE      ].jm_signature = "()Z";
+    JavaMethods_[JM_HDFS_MERGE_FILES].jm_name      = "hdfsMergeFiles";
+    JavaMethods_[JM_HDFS_MERGE_FILES].jm_signature = "(Ljava/lang/String;Ljava/lang/String;)Z";
+    JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].jm_name      = "hdfsCleanUnloadPath";
+    JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].jm_signature = "(Ljava/lang/String;)Z";
+    JavaMethods_[JM_HDFS_EXISTS].jm_name      = "hdfsExists";
+    JavaMethods_[JM_HDFS_EXISTS].jm_signature = "(Ljava/lang/String;)Z";
+    JavaMethods_[JM_HDFS_DELETE_PATH].jm_name      = "hdfsDeletePath";
+    JavaMethods_[JM_HDFS_DELETE_PATH].jm_signature = "(Ljava/lang/String;)Z";
+    rc = (HDFS_Client_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
+    javaMethodsInitialized_ = TRUE;
+    pthread_mutex_unlock(&javaMethodsInitMutex_);
+  }
+  return rc;
+}
+
+//////////////////////////////////////////////////////////////////////////////
+//
+//////////////////////////////////////////////////////////////////////////////
+char* HdfsClient::getErrorText(HDFS_Client_RetCode errEnum)
+{
+  if (errEnum < (HDFS_Client_RetCode)HDFS_CLIENT_FIRST)
+    return JavaObjectInterface::getErrorText((JOI_RetCode)errEnum);
+  else
+    return (char*)hdfsClientErrorEnumStr[errEnum-HDFS_CLIENT_FIRST];
+}
+
+HDFS_Client_RetCode HdfsClient::hdfsCreate(const char* path, NABoolean compress)
+{
+  QRLogger::log(CAT_SQL_HDFS_SEQ_FILE_WRITER, LL_DEBUG, "HdfsClient::hdfsCreate(%s) called.", path);
+
+  if (initJNIEnv() != JOI_OK)
+     return HDFS_CLIENT_ERROR_HDFS_CREATE_PARAM;
+
+  jstring js_path = jenv_->NewStringUTF(path);
+  if (js_path == NULL) {
+    GetCliGlobals()->setJniErrorStr(getErrorText(HDFS_CLIENT_ERROR_HDFS_CREATE_PARAM));
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_CREATE_PARAM;
+  }
+
+  jboolean j_compress = compress;
+
+  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_CREATE].jm_full_name;
+  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_CREATE].methodID, js_path, j_compress);
+
+  if (jenv_->ExceptionCheck())
+  {
+    getExceptionDetails();
+    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsCreate()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_CREATE_EXCEPTION;
+  }
+
+  if (jresult == false)
+  {
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsCreate()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_CREATE_PARAM;
+  }
+
+  jenv_->PopLocalFrame(NULL);
+  return HDFS_CLIENT_OK;
+}
+
+//////////////////////////////////////////////////////////////////////////////
+//
+//////////////////////////////////////////////////////////////////////////////
+HDFS_Client_RetCode HdfsClient::hdfsWrite(const char* data, Int64 len)
+{
+  QRLogger::log(CAT_SQL_HDFS_SEQ_FILE_WRITER, LL_DEBUG, "HdfsClient::hdfsWrite(%ld) called.", len);
+
+  if (initJNIEnv() != JOI_OK)
+     return HDFS_CLIENT_ERROR_HDFS_WRITE_EXCEPTION;
+
+  //Write the requisite bytes into the file
+  jbyteArray jbArray = jenv_->NewByteArray( len);
+  if (!jbArray) {
+    GetCliGlobals()->setJniErrorStr(getErrorText(HDFS_CLIENT_ERROR_HDFS_WRITE_PARAM));
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_WRITE_PARAM;
+  }
+  jenv_->SetByteArrayRegion(jbArray, 0, len, (const jbyte*)data);
+
+  jlong j_len = len;
+  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_WRITE].jm_full_name;
+  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_WRITE].methodID,jbArray , j_len);
+
+  if (jenv_->ExceptionCheck())
+  {
+    getExceptionDetails();
+    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsWrite()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_WRITE_EXCEPTION;
+  }
+
+  if (jresult == false)
+  {
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsWrite()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_WRITE_EXCEPTION;
+  }
+
+
+  jenv_->PopLocalFrame(NULL);
+  return HDFS_CLIENT_OK;
+}
+
+//////////////////////////////////////////////////////////////////////////////
+//
+//////////////////////////////////////////////////////////////////////////////
+HDFS_Client_RetCode HdfsClient::hdfsClose()
+{
+  QRLogger::log(CAT_SQL_HDFS_SEQ_FILE_WRITER, LL_DEBUG, "HdfsClient::close() called.");
+
+  if (initJNIEnv() != JOI_OK)
+     return HDFS_CLIENT_ERROR_HDFS_CLOSE_EXCEPTION;
+
+  // String close();
+  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_CLOSE].jm_full_name;
+  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_CLOSE].methodID);
+
+  if (jenv_->ExceptionCheck())
+  {
+    getExceptionDetails();
+    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsClose()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_CLOSE_EXCEPTION;
+  }
+
+  if (jresult == false)
+  {
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsClose()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_CLOSE_EXCEPTION;
+  }
+
+  jenv_->PopLocalFrame(NULL);
+  return HDFS_CLIENT_OK;
+}
+
+HDFS_Client_RetCode HdfsClient::hdfsCleanUnloadPath( const NAString& uldPath)
+{
+  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HdfsClient::hdfsCleanUnloadPath(%s) called.",
+                                                                             uldPath.data());
+  if (initJNIEnv() != JOI_OK)
+     return HDFS_CLIENT_ERROR_HDFS_CLEANUP_PARAM;
+
+  jstring js_UldPath = jenv_->NewStringUTF(uldPath.data());
+  if (js_UldPath == NULL) {
+    GetCliGlobals()->setJniErrorStr(getErrorText(HDFS_CLIENT_ERROR_HDFS_CLEANUP_PARAM));
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_CLEANUP_PARAM;
+  }
+
+  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].jm_full_name;
+  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].methodID, js_UldPath);
+
+  if (jenv_->ExceptionCheck())
+  {
+    getExceptionDetails();
+    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsCleanUnloadPath()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_CLEANUP_EXCEPTION;
+  }
+
+  jenv_->PopLocalFrame(NULL);
+  return HDFS_CLIENT_OK;
+}
+
+HDFS_Client_RetCode HdfsClient::hdfsMergeFiles( const NAString& srcPath,
+                                                const NAString& dstPath)
+{
+  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HdfsClient::hdfsMergeFiles(%s, %s) called.",
+                  srcPath.data(), dstPath.data());
+
+  if (initJNIEnv() != JOI_OK)
+     return HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_EXCEPTION;
+
+  jstring js_SrcPath = jenv_->NewStringUTF(srcPath.data());
+
+  if (js_SrcPath == NULL) {
+     GetCliGlobals()->setJniErrorStr(getErrorText(HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_PARAM));
+     jenv_->PopLocalFrame(NULL);
+     return HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_PARAM;
+  }
+  jstring js_DstPath= jenv_->NewStringUTF(dstPath.data());
+  if (js_DstPath == NULL) {
+     GetCliGlobals()->setJniErrorStr(getErrorText(HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_PARAM));
+     jenv_->PopLocalFrame(NULL);
+     return HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_PARAM;
+  }
+
+
+  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_MERGE_FILES].jm_full_name;
+  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_MERGE_FILES].methodID, js_SrcPath, js_DstPath);
+
+  if (jenv_->ExceptionCheck())
+  {
+    getExceptionDetails();
+    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsMergeFiles()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_EXCEPTION;
+  }
+
+  if (jresult == false)
+  {
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsMergeFiles()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_EXCEPTION;
+  } 
+
+  jenv_->PopLocalFrame(NULL);
+  return HDFS_CLIENT_OK;
+}
+
+HDFS_Client_RetCode HdfsClient::hdfsDeletePath( const NAString& delPath)
+{
+  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HdfsClient::hdfsDeletePath(%s called.",
+                  delPath.data());
+  if (initJNIEnv() != JOI_OK)
+     return HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_EXCEPTION;
+
+  jstring js_delPath = jenv_->NewStringUTF(delPath.data());
+  if (js_delPath == NULL) {
+     GetCliGlobals()->setJniErrorStr(getErrorText(HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_PARAM));
+     jenv_->PopLocalFrame(NULL);
+     return HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_PARAM;
+  }
+
+
+  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_DELETE_PATH].jm_full_name;
+  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_DELETE_PATH].methodID, js_delPath);
+
+  if (jenv_->ExceptionCheck())
+  {
+    getExceptionDetails();
+    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsDeletePath()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_EXCEPTION;
+  }
+
+  if (jresult == false)
+  {
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsDeletePath()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_EXCEPTION;
+  }
+
+  jenv_->PopLocalFrame(NULL);
+  return HDFS_CLIENT_OK;
+}
+
+HDFS_Client_RetCode HdfsClient::hdfsExists( const NAString& uldPath, NABoolean & exist)
+{
+  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HdfsClient::hdfsExists(%s) called.",
+                                                      uldPath.data());
+
+  if (initJNIEnv() != JOI_OK)
+     return HDFS_CLIENT_ERROR_HDFS_EXISTS_EXCEPTION;
+
+  jstring js_UldPath = jenv_->NewStringUTF(uldPath.data());
+  if (js_UldPath == NULL) {
+     jenv_->PopLocalFrame(NULL);
+     return HDFS_CLIENT_ERROR_HDFS_EXISTS_PARAM;
+  }
+
+  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_EXISTS].jm_full_name;
+  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_EXISTS].methodID, js_UldPath);
+
+  exist = jresult;
+
+  if (jenv_->ExceptionCheck())
+  {
+    getExceptionDetails();
+    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
+    logError(CAT_SQL_HBASE, "HdfsClient::hdfsExists()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_EXISTS_EXCEPTION;
+  } 
+
+  jenv_->PopLocalFrame(NULL);
+  return HDFS_CLIENT_OK;
+}

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/HdfsClient_JNI.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/HdfsClient_JNI.h b/core/sql/executor/HdfsClient_JNI.h
new file mode 100644
index 0000000..8adf42f
--- /dev/null
+++ b/core/sql/executor/HdfsClient_JNI.h
@@ -0,0 +1,146 @@
+//**********************************************************************
+// @@@ START COPYRIGHT @@@
+//
+// 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.
+//
+// @@@ END COPYRIGHT @@@
+// **********************************************************************
+#ifndef HDFS_CLIENT_H
+#define HDFS_CLIENT_H
+
+#include "JavaObjectInterface.h"
+
+// ===========================================================================
+// ===== The native HdfsScan class implements access to the Java methods 
+// ===== org.trafodion.sql.HdfsScan class.
+// ===========================================================================
+
+typedef enum {
+   HDFS_SCAN_OK     = JOI_OK
+  ,HDFS_SCAN_FIRST = JOI_LAST
+  ,HDFS_SCAN_LAST
+} HDFS_Scan_RetCode;
+
+class HdfsScan : public JavaObjectInterface
+{
+public:
+  // Default constructor - for creating a new JVM		
+  HdfsScan(NAHeap *heap)
+  :  JavaObjectInterface(heap) 
+  {}
+
+  // Destructor
+  virtual ~HdfsScan();
+
+  // Get the error description.
+  virtual char* getErrorText(HDFS_Scan_RetCode errEnum);
+  
+  // Initialize JVM and all the JNI configuration.
+  // Must be called.
+  HDFS_Scan_RetCode init();
+
+  HDFS_Scan_RetCode initScanRanges();
+
+  HDFS_Scan_RetCode trafHdfsRead();
+
+private:
+  enum JAVA_METHODS {
+    JM_CTOR = 0, 
+    JM_INIT_SCAN_RANGES,
+    JM_TRAF_HDFS_READ,
+    JM_LAST
+  };
+ 
+  static jclass javaClass_;
+  static JavaMethodInit* JavaMethods_;
+  static bool javaMethodsInitialized_;
+  // this mutex protects both JaveMethods_ and javaClass_ initialization
+  static pthread_mutex_t javaMethodsInitMutex_;
+};
+
+// ===========================================================================
+// ===== The native HdfsClient class implements access to the Java 
+// ===== org.trafodion.sql.HdfsClient class.
+// ===========================================================================
+
+typedef enum {
+  HDFS_CLIENT_OK     = JOI_OK
+ ,HDFS_CLIENT_FIRST  = HDFS_SCAN_LAST
+ ,HDFS_CLIENT_ERROR_HDFS_CREATE_PARAM = HDFS_CLIENT_FIRST
+ ,HDFS_CLIENT_ERROR_HDFS_CREATE_EXCEPTION
+ ,HDFS_CLIENT_ERROR_HDFS_WRITE_PARAM
+ ,HDFS_CLIENT_ERROR_HDFS_WRITE_EXCEPTION
+ ,HDFS_CLIENT_ERROR_HDFS_CLOSE_EXCEPTION
+ ,HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_PARAM
+ ,HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_EXCEPTION
+ ,HDFS_CLIENT_ERROR_HDFS_CLEANUP_PARAM
+ ,HDFS_CLIENT_ERROR_HDFS_CLEANUP_EXCEPTION
+ ,HDFS_CLIENT_ERROR_HDFS_EXISTS_PARAM
+ ,HDFS_CLIENT_ERROR_HDFS_EXISTS_EXCEPTION
+ ,HDFS_CLIENT_ERROR_HDFS_EXISTS_FILE_EXISTS
+ ,HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_PARAM
+ ,HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_EXCEPTION
+ ,HDFS_CLIENT_LAST
+} HDFS_Client_RetCode;
+
+class HdfsClient : public JavaObjectInterface
+{
+public:
+  // Default constructor - for creating a new JVM		
+  HdfsClient(NAHeap *heap)
+  :  JavaObjectInterface(heap) 
+  {}
+
+  static HdfsClient *newInstance(NAHeap *heap, HDFS_Client_RetCode &retCode);
+
+  // Get the error description.
+  virtual char* getErrorText(HDFS_Client_RetCode errEnum);
+  
+  // Initialize JVM and all the JNI configuration.
+  // Must be called.
+  HDFS_Client_RetCode    init();
+  HDFS_Client_RetCode    hdfsCreate(const char* path, NABoolean compress);
+  HDFS_Client_RetCode    hdfsWrite(const char* data, Int64 size);
+  HDFS_Client_RetCode    hdfsClose();
+  HDFS_Client_RetCode    hdfsMergeFiles(const NAString& srcPath,
+                                 const NAString& dstPath);
+  HDFS_Client_RetCode    hdfsCleanUnloadPath(const NAString& uldPath );
+  HDFS_Client_RetCode    hdfsExists(const NAString& uldPath,  NABoolean & exists );
+  HDFS_Client_RetCode    hdfsDeletePath(const NAString& delPath);
+
+private:  
+  enum JAVA_METHODS {
+    JM_CTOR = 0, 
+    JM_HDFS_CREATE,
+    JM_HDFS_WRITE,
+    JM_HDFS_CLOSE,
+    JM_HDFS_MERGE_FILES,
+    JM_HDFS_CLEAN_UNLOAD_PATH,
+    JM_HDFS_EXISTS,
+    JM_HDFS_DELETE_PATH,
+    JM_LAST
+  };
+  
+  static jclass javaClass_;
+  static JavaMethodInit* JavaMethods_;
+  static bool javaMethodsInitialized_;
+  // this mutex protects both JaveMethods_ and javaClass_ initialization
+  static pthread_mutex_t javaMethodsInitMutex_;
+};
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/SequenceFileReader.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/SequenceFileReader.cpp b/core/sql/executor/SequenceFileReader.cpp
index 22562df..5bc2f7f 100644
--- a/core/sql/executor/SequenceFileReader.cpp
+++ b/core/sql/executor/SequenceFileReader.cpp
@@ -389,21 +389,6 @@ static const char* const sfwErrorEnumStr[] =
  ,"JNI NewStringUTF() in write()"
  ,"Java exception in write()"
  ,"Java exception in close() after writing."
- ,"JNI NewStringUTF() in hdfsCreate()."
- ,"Java exception in hdfsCreate()."
- ,"JNI NewStringUTF() in hdfsWrite()."
- ,"Java exception in hdfsWrite()."
- ,"Java exception in hdfsClose()."
- ,"JNI NewStringUTF() in hdfsMergeFiles()."
- ,"Java exception in hdfsMergeFiles()."
- ,"JNI NewStringUTF() in hdfsCleanUnloadPath()."
- ,"Java exception in hdfsCleanUnloadPath()."
- ,"JNI NewStringUTF() in hdfsExists()."
- ,"Java exception in hdfsExists()."
- ,"file already exists."
- ,"Java exception in release()."
- ,"JNI NewStringUTF() in hdfsDeletePath()."
- ,"Java exception in hdfsDeletePath()."
 };
 
 //////////////////////////////////////////////////////////////////////////////
@@ -453,21 +438,6 @@ SFW_RetCode SequenceFileWriter::init()
     JavaMethods_[JM_WRITE     ].jm_signature = "(Ljava/lang/String;)Ljava/lang/String;";
     JavaMethods_[JM_CLOSE     ].jm_name      = "close";
     JavaMethods_[JM_CLOSE     ].jm_signature = "()Ljava/lang/String;";
-   
-    JavaMethods_[JM_HDFS_CREATE     ].jm_name      = "hdfsCreate";
-    JavaMethods_[JM_HDFS_CREATE     ].jm_signature = "(Ljava/lang/String;Z)Z";
-    JavaMethods_[JM_HDFS_WRITE      ].jm_name      = "hdfsWrite";
-    JavaMethods_[JM_HDFS_WRITE      ].jm_signature = "([BJ)Z";
-    JavaMethods_[JM_HDFS_CLOSE      ].jm_name      = "hdfsClose";
-    JavaMethods_[JM_HDFS_CLOSE      ].jm_signature = "()Z";
-    JavaMethods_[JM_HDFS_MERGE_FILES].jm_name      = "hdfsMergeFiles";
-    JavaMethods_[JM_HDFS_MERGE_FILES].jm_signature = "(Ljava/lang/String;Ljava/lang/String;)Z";
-    JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].jm_name      = "hdfsCleanUnloadPath";
-    JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].jm_signature = "(Ljava/lang/String;)Z";
-    JavaMethods_[JM_HDFS_EXISTS].jm_name      = "hdfsExists";
-    JavaMethods_[JM_HDFS_EXISTS].jm_signature = "(Ljava/lang/String;)Z";
-    JavaMethods_[JM_HDFS_DELETE_PATH].jm_name      = "hdfsDeletePath";
-    JavaMethods_[JM_HDFS_DELETE_PATH].jm_signature = "(Ljava/lang/String;)Z";
 
     rc = (SFW_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
     javaMethodsInitialized_ = TRUE;
@@ -591,271 +561,3 @@ SFW_RetCode SequenceFileWriter::close()
   return SFW_OK;
 }
 
-
-//////////////////////////////////////////////////////////////////////////////
-//
-//////////////////////////////////////////////////////////////////////////////
-SFW_RetCode SequenceFileWriter::hdfsCreate(const char* path, NABoolean compress)
-{
-  QRLogger::log(CAT_SQL_HDFS_SEQ_FILE_WRITER, LL_DEBUG, "SequenceFileWriter::hdfsCreate(%s) called.", path);
-
-  if (initJNIEnv() != JOI_OK)
-     return SFW_ERROR_HDFS_CREATE_PARAM;
-
-  jstring js_path = jenv_->NewStringUTF(path);
-  if (js_path == NULL) {
-    GetCliGlobals()->setJniErrorStr(getErrorText(SFW_ERROR_HDFS_CREATE_PARAM));
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_CREATE_PARAM;
-  }
-
-  jboolean j_compress = compress;
-
-  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_CREATE].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_CREATE].methodID, js_path, j_compress);
-
-  if (jenv_->ExceptionCheck())
-  {
-    getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsCreate()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_CREATE_EXCEPTION;
-  }
-
-  if (jresult == false)
-  {
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsCreaten()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_CREATE_EXCEPTION;
-  }
-
-  jenv_->PopLocalFrame(NULL);
-  return SFW_OK;
-}
-
-//////////////////////////////////////////////////////////////////////////////
-//
-//////////////////////////////////////////////////////////////////////////////
-SFW_RetCode SequenceFileWriter::hdfsWrite(const char* data, Int64 len)
-{
-  QRLogger::log(CAT_SQL_HDFS_SEQ_FILE_WRITER, LL_DEBUG, "SequenceFileWriter::hdfsWrite(%ld) called.", len);
-
-  if (initJNIEnv() != JOI_OK)
-     return SFW_ERROR_HDFS_WRITE_EXCEPTION;
-
-  //Write the requisite bytes into the file
-  jbyteArray jbArray = jenv_->NewByteArray( len);
-  if (!jbArray) {
-    GetCliGlobals()->setJniErrorStr(getErrorText(SFW_ERROR_HDFS_WRITE_PARAM));
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_WRITE_PARAM;
-  }
-  jenv_->SetByteArrayRegion(jbArray, 0, len, (const jbyte*)data);
-
-  jlong j_len = len;
-  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_WRITE].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_WRITE].methodID,jbArray , j_len);
-
-  if (jenv_->ExceptionCheck())
-  {
-    getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsWrite()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_WRITE_EXCEPTION;
-  }
-
-  if (jresult == false)
-  {
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsWrite()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_WRITE_EXCEPTION;
-  }
-
-
-  jenv_->PopLocalFrame(NULL);
-  return SFW_OK;
-}
-
-//////////////////////////////////////////////////////////////////////////////
-//
-//////////////////////////////////////////////////////////////////////////////
-SFW_RetCode SequenceFileWriter::hdfsClose()
-{
-  QRLogger::log(CAT_SQL_HDFS_SEQ_FILE_WRITER, LL_DEBUG, "SequenceFileWriter::close() called.");
-
-  if (initJNIEnv() != JOI_OK)
-     return SFW_ERROR_HDFS_CLOSE_EXCEPTION;
-
-  // String close();
-  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_CLOSE].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_CLOSE].methodID);
-
-  if (jenv_->ExceptionCheck())
-  {
-    getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsClose()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_CLOSE_EXCEPTION;
-  }
-
-  if (jresult == false)
-  {
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsClose()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_CLOSE_EXCEPTION;
-  }
-
-  jenv_->PopLocalFrame(NULL);
-  return SFW_OK;
-}
-
-SFW_RetCode SequenceFileWriter::hdfsCleanUnloadPath( const NAString& uldPath)
-{
-  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "SequenceFileWriter::hdfsCleanUnloadPath(%s) called.",
-                                                                             uldPath.data());
-  if (initJNIEnv() != JOI_OK)
-     return SFW_ERROR_HDFS_CLEANUP_PARAM;
-
-  jstring js_UldPath = jenv_->NewStringUTF(uldPath.data());
-  if (js_UldPath == NULL) {
-    GetCliGlobals()->setJniErrorStr(getErrorText(SFW_ERROR_HDFS_CLEANUP_PARAM));
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_CLEANUP_PARAM;
-  }
-
-  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].methodID, js_UldPath);
-
-  if (jenv_->ExceptionCheck())
-  {
-    getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsCleanUnloadPath()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_CLEANUP_EXCEPTION;
-  }
-
-  jenv_->PopLocalFrame(NULL);
-  return SFW_OK;
-}
-
-SFW_RetCode SequenceFileWriter::hdfsMergeFiles( const NAString& srcPath,
-                                                const NAString& dstPath)
-{
-  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "SequenceFileWriter::hdfsMergeFiles(%s, %s) called.",
-                  srcPath.data(), dstPath.data());
-
-  if (initJNIEnv() != JOI_OK)
-     return SFW_ERROR_HDFS_MERGE_FILES_EXCEPTION;
-
-  jstring js_SrcPath = jenv_->NewStringUTF(srcPath.data());
-
-  if (js_SrcPath == NULL) {
-     GetCliGlobals()->setJniErrorStr(getErrorText(SFW_ERROR_HDFS_MERGE_FILES_PARAM));
-     jenv_->PopLocalFrame(NULL);
-     return SFW_ERROR_HDFS_MERGE_FILES_PARAM;
-  }
-  jstring js_DstPath= jenv_->NewStringUTF(dstPath.data());
-  if (js_DstPath == NULL) {
-     GetCliGlobals()->setJniErrorStr(getErrorText(SFW_ERROR_HDFS_MERGE_FILES_PARAM));
-     jenv_->PopLocalFrame(NULL);
-     return SFW_ERROR_HDFS_MERGE_FILES_PARAM;
-  }
-
-
-  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_MERGE_FILES].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_MERGE_FILES].methodID, js_SrcPath, js_DstPath);
-
-  if (jenv_->ExceptionCheck())
-  {
-    getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsMergeFiles()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_MERGE_FILES_EXCEPTION;
-  }
-
-  if (jresult == false)
-  {
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsMergeFiles()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_MERGE_FILES_EXCEPTION;
-  } 
-
-  jenv_->PopLocalFrame(NULL);
-  return SFW_OK;
-}
-
-SFW_RetCode SequenceFileWriter::hdfsDeletePath( const NAString& delPath)
-{
-  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "SequenceFileWriter::hdfsDeletePath(%s called.",
-                  delPath.data());
-  if (initJNIEnv() != JOI_OK)
-     return SFW_ERROR_HDFS_DELETE_PATH_EXCEPTION;
-
-  jstring js_delPath = jenv_->NewStringUTF(delPath.data());
-  if (js_delPath == NULL) {
-     GetCliGlobals()->setJniErrorStr(getErrorText(SFW_ERROR_HDFS_DELETE_PATH_PARAM));
-     jenv_->PopLocalFrame(NULL);
-     return SFW_ERROR_HDFS_DELETE_PATH_PARAM;
-  }
-
-
-  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_DELETE_PATH].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_DELETE_PATH].methodID, js_delPath);
-
-  if (jenv_->ExceptionCheck())
-  {
-    getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsDeletePath()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_DELETE_PATH_EXCEPTION;
-  }
-
-  if (jresult == false)
-  {
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsDeletePath()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_DELETE_PATH_EXCEPTION;
-  }
-
-  jenv_->PopLocalFrame(NULL);
-  return SFW_OK;
-}
-
-
-SFW_RetCode SequenceFileWriter::hdfsExists( const NAString& uldPath, NABoolean & exist)
-{
-  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "SequenceFileWriter::hdfsExists(%s) called.",
-                                                      uldPath.data());
-
-  if (initJNIEnv() != JOI_OK)
-     return SFW_ERROR_HDFS_EXISTS_EXCEPTION;
-
-  jstring js_UldPath = jenv_->NewStringUTF(uldPath.data());
-  if (js_UldPath == NULL) {
-     jenv_->PopLocalFrame(NULL);
-     return SFW_ERROR_HDFS_EXISTS_PARAM;
-  }
-
-  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_EXISTS].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_EXISTS].methodID, js_UldPath);
-
-  exist = jresult;
-
-  if (jenv_->ExceptionCheck())
-  {
-    getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "SequenceFileWriter::hdfsExists()", getLastError());
-    jenv_->PopLocalFrame(NULL);
-    return SFW_ERROR_HDFS_EXISTS_EXCEPTION;
-  } 
-
-  jenv_->PopLocalFrame(NULL);
-  return SFW_OK;
-}

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/executor/SequenceFileReader.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/SequenceFileReader.h b/core/sql/executor/SequenceFileReader.h
index d950677..12a68c2 100644
--- a/core/sql/executor/SequenceFileReader.h
+++ b/core/sql/executor/SequenceFileReader.h
@@ -144,20 +144,6 @@ typedef enum {
  ,SFW_ERROR_WRITE_PARAM
  ,SFW_ERROR_WRITE_EXCEPTION
  ,SFW_ERROR_CLOSE_EXCEPTION
- ,SFW_ERROR_HDFS_CREATE_PARAM
- ,SFW_ERROR_HDFS_CREATE_EXCEPTION
- ,SFW_ERROR_HDFS_WRITE_PARAM
- ,SFW_ERROR_HDFS_WRITE_EXCEPTION
- ,SFW_ERROR_HDFS_CLOSE_EXCEPTION
- ,SFW_ERROR_HDFS_MERGE_FILES_PARAM
- ,SFW_ERROR_HDFS_MERGE_FILES_EXCEPTION
- ,SFW_ERROR_HDFS_CLEANUP_PARAM
- ,SFW_ERROR_HDFS_CLEANUP_EXCEPTION
- ,SFW_ERROR_HDFS_EXISTS_PARAM
- ,SFW_ERROR_HDFS_EXISTS_EXCEPTION
- ,SFW_ERROR_HDFS_EXISTS_FILE_EXISTS
- ,SFW_ERROR_HDFS_DELETE_PATH_PARAM
- ,SFW_ERROR_HDFS_DELETE_PATH_EXCEPTION
  ,SFW_LAST
 } SFW_RetCode;
 
@@ -199,15 +185,6 @@ public:
   
   // Close the file.
   SFW_RetCode    close();
-
-  SFW_RetCode    hdfsCreate(const char* path, NABoolean compress);
-  SFW_RetCode    hdfsWrite(const char* data, Int64 size);
-  SFW_RetCode    hdfsMergeFiles(const NAString& srcPath,
-                                 const NAString& dstPath);
-  SFW_RetCode    hdfsDeletePath(const NAString& delPath);
-  SFW_RetCode    hdfsCleanUnloadPath(const NAString& uldPath );
-  SFW_RetCode    hdfsExists(const NAString& uldPath,  NABoolean & exists );
-  SFW_RetCode    hdfsClose();
   SFW_RetCode    release();
 
   virtual char*  getErrorText(SFW_RetCode errEnum);
@@ -222,13 +199,6 @@ private:
     JM_OPEN,
     JM_WRITE,
     JM_CLOSE,
-    JM_HDFS_CREATE,
-    JM_HDFS_WRITE,
-    JM_HDFS_CLOSE,
-    JM_HDFS_MERGE_FILES,
-    JM_HDFS_CLEAN_UNLOAD_PATH,
-    JM_HDFS_EXISTS,
-    JM_HDFS_DELETE_PATH,
     JM_LAST
   };
   

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/exp/ExpHbaseInterface.cpp
----------------------------------------------------------------------
diff --git a/core/sql/exp/ExpHbaseInterface.cpp b/core/sql/exp/ExpHbaseInterface.cpp
index dff16dc..b7746e0 100644
--- a/core/sql/exp/ExpHbaseInterface.cpp
+++ b/core/sql/exp/ExpHbaseInterface.cpp
@@ -1055,26 +1055,6 @@ Lng32 ExpHbaseInterface_JNI::initHFileParams(HbaseStr &tblName,
      return -HBASE_CLEANUP_HFILE_ERROR;
  }
  ///////////////////
-
- //////////////////////////////////////////////////////////////////////////////
- //
- //////////////////////////////////////////////////////////////////////////////
- Lng32 ExpHbaseInterface_JNI::hdfsCreateFile(const char* path)
- {
-   if (hive_ == NULL) {
-      retCode_ = initHive();
-      if (retCode_ != HVC_OK)
-         return retCode_;
-   }
-
-    retCode_ = hive_->hdfsCreateFile( path);
-
-    if (retCode_ == HVC_OK)
-      return HBASE_ACCESS_SUCCESS;
-    else
-      return -HVC_ERROR_HDFS_CREATE_EXCEPTION;
- }
-
  Lng32  ExpHbaseInterface_JNI::incrCounter( const char * tabName, const char * rowId,
                              const char * famName, const char * qualName ,
                              Int64 incr, Int64 & count)
@@ -1107,59 +1087,6 @@ Lng32 ExpHbaseInterface_JNI::initHFileParams(HbaseStr &tblName,
    else
       return -HBC_ERROR_CREATE_COUNTER_EXCEPTION;
  }
- //////////////////////////////////////////////////////////////////////////////
- //
- //////////////////////////////////////////////////////////////////////////////
- Lng32 ExpHbaseInterface_JNI::hdfsWrite(const char* data, Int64 len)
- {
-   if (hive_ == NULL) {
-      retCode_ = initHive();
-      if (retCode_ != HVC_OK)
-         return retCode_;
-   }
-   retCode_ = hive_->hdfsWrite( data, len);
-
-   if (retCode_ == HVC_OK)
-      return HBASE_ACCESS_SUCCESS;
-    else
-      return -HVC_ERROR_HDFS_WRITE_EXCEPTION;
- }
-
- //////////////////////////////////////////////////////////////////////////////
- //
- //////////////////////////////////////////////////////////////////////////////
- Lng32 ExpHbaseInterface_JNI::hdfsClose()
- {
-   if (hive_ == NULL) {
-      retCode_ = initHive();
-      if (retCode_ != HVC_OK)
-         return retCode_;
-   }
-
-   retCode_ = hive_->hdfsClose();
-
-   if (retCode_ == HVC_OK)
-      return HVC_OK;
-    else
-      return -HVC_ERROR_HDFS_CLOSE_EXCEPTION;
- }
-/*
- Lng32 ExpHbaseInterface_JNI::hdfsCleanPath( const std::string& path)
- {
-   if (hblc_ == NULL) {
-      retCode_ = initHBLC();
-      if (retCode_ != HBLC_OK)
-         return -HBASE_ACCESS_ERROR;
-   }
-
-   retCode_ = hblc_->hdfsCleanPath(path);
-
-   if (retCode_ == HBLC_OK)
-      return HBLC_OK;
-    else
-      return -HBLC_ERROR_HDFS_CLOSE_EXCEPTION;
- }
-*/
 
 Lng32 ExpHbaseInterface_JNI::isEmpty(
                                      HbaseStr &tblName)

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/exp/ExpHbaseInterface.h
----------------------------------------------------------------------
diff --git a/core/sql/exp/ExpHbaseInterface.h b/core/sql/exp/ExpHbaseInterface.h
index 0bfda35..f7b23cb 100644
--- a/core/sql/exp/ExpHbaseInterface.h
+++ b/core/sql/exp/ExpHbaseInterface.h
@@ -296,9 +296,6 @@ class ExpHbaseInterface : public NABasicObject
  virtual Lng32 bulkLoadCleanup(HbaseStr &tblName,
                           Text& location) = 0;
 
- virtual Lng32  hdfsCreateFile(const char* path)=0;
- virtual Lng32  hdfsWrite(const char* data, Int64 size)=0;
- virtual Lng32  hdfsClose()=0;
  virtual Lng32  incrCounter( const char * tabName, const char * rowId,
                              const char * famName, const char * qualName ,
                              Int64 incr, Int64 & count)=0;
@@ -609,9 +606,6 @@ virtual Lng32 initHFileParams(HbaseStr &tblName,
  
  virtual Lng32 bulkLoadCleanup(HbaseStr &tblName,
                           Text& location);
- virtual Lng32  hdfsCreateFile(const char* path);
- virtual Lng32  hdfsWrite(const char* data, Int64 size);
- virtual Lng32  hdfsClose();
  virtual Lng32  incrCounter( const char * tabName, const char * rowId,
                              const char * famName, const char * qualName ,
                              Int64 incr, Int64 & count);

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/nskgmake/executor/Makefile
----------------------------------------------------------------------
diff --git a/core/sql/nskgmake/executor/Makefile b/core/sql/nskgmake/executor/Makefile
index ca487fb..c530ab4 100755
--- a/core/sql/nskgmake/executor/Makefile
+++ b/core/sql/nskgmake/executor/Makefile
@@ -112,6 +112,7 @@ CPPSRC := Allocator.cpp \
         JavaObjectInterface.cpp \
         SequenceFileReader.cpp \
         OrcFileReader.cpp \
+        HdfsClient_JNI.cpp \
         HBaseClient_JNI.cpp
 CPPSRC += vers_libexecutor.cpp \
 	ssmpipc.cpp \