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:34 UTC

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

Repository: trafodion
Updated Branches:
  refs/heads/master 087af70db -> de8357677


http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
new file mode 100644
index 0000000..8d2052f
--- /dev/null
+++ b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
@@ -0,0 +1,319 @@
+// @@@ 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 @@@
+
+package org.trafodion.sql;
+
+import org.apache.log4j.PropertyConfigurator;
+import org.apache.log4j.Logger;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.conf.Configuration;
+import java.nio.ByteBuffer;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.util.ReflectionUtils;
+
+public class HDFSClient 
+{
+   static Logger logger_ = Logger.getLogger(HDFSClient.class.getName());
+   private static Configuration config_ = null;
+   private static ExecutorService executorService_ = null;
+   private static FileSystem defaultFs_ = null;
+   private FileSystem fs_ = null;
+   private int bufNo_;
+   private FSDataInputStream fsdis_; 
+   private OutputStream outStream_;
+   private String filename_;
+   private ByteBuffer buf_;
+   private int bufLen_;
+   private int bufOffset_ = 0;
+   private long pos_ = 0;
+   private int len_ = 0;
+   private int lenRemain_ = 0; 
+   private int blockSize_; 
+   private int bytesRead_;
+   private Future future_ = null;
+    
+   static {
+      String confFile = System.getProperty("trafodion.log4j.configFile");
+      System.setProperty("trafodion.root", System.getenv("TRAF_HOME"));
+      if (confFile == null) {
+         confFile = System.getenv("TRAF_CONF") + "/log4j.sql.config";
+      }
+      PropertyConfigurator.configure(confFile);
+      config_ = TrafConfiguration.create(TrafConfiguration.HDFS_CONF);
+      executorService_ = Executors.newCachedThreadPool();
+      try {
+         defaultFs_ = FileSystem.get(config_);
+      }
+      catch (IOException ioe) {
+         throw new RuntimeException("Exception in HDFSClient static block", ioe);
+      }
+   }
+
+   class HDFSRead implements Callable 
+   {
+      int length_;
+
+      HDFSRead(int length) 
+      {
+         length_ = length;
+      }
+ 
+      public Object call() throws IOException 
+      {
+         int bytesRead;
+         if (buf_.hasArray())
+            bytesRead = fsdis_.read(pos_, buf_.array(), bufOffset_, length_);
+         else
+         {
+            buf_.limit(bufOffset_ + length_);
+            bytesRead = fsdis_.read(buf_);
+         }
+         return new Integer(bytesRead);
+      }
+   }
+       
+   public HDFSClient() 
+   {
+   }
+ 
+   public HDFSClient(int bufNo, String filename, ByteBuffer buffer, long position, int length) throws IOException
+   {
+      bufNo_ = bufNo; 
+      filename_ = filename;
+      Path filepath = new Path(filename_);
+      fs_ = FileSystem.get(filepath.toUri(),config_);
+      fsdis_ = fs_.open(filepath);
+      blockSize_ = (int)fs_.getDefaultBlockSize(filepath);
+      buf_  = buffer;
+      bufOffset_ = 0;
+      pos_ = position;
+      len_ = length;
+      if (buffer.hasArray()) 
+         bufLen_ = buffer.array().length;
+      else
+      {
+         bufLen_ = buffer.capacity();
+         buf_.position(0);
+      }
+      lenRemain_ = (len_ > bufLen_) ? bufLen_ : len_;
+      if (lenRemain_ != 0)
+      {
+         int readLength = (lenRemain_ > blockSize_) ? blockSize_ : lenRemain_;
+         future_ = executorService_.submit(new HDFSRead(readLength));
+      }
+   }
+
+   public int trafHdfsRead() throws IOException, InterruptedException, ExecutionException
+   {
+      Integer retObject = 0;
+      int bytesRead;
+      int readLength;
+       
+      if (lenRemain_ == 0)
+         return 0;
+      retObject = (Integer)future_.get();
+      bytesRead = retObject.intValue();
+      if (bytesRead == -1)
+         return -1;
+      bufOffset_ += bytesRead;
+      pos_ += bytesRead;
+      lenRemain_ -= bytesRead;
+      if (bufOffset_ == bufLen_)
+         return bytesRead; 
+      else if (bufOffset_ > bufLen_)
+         throw new IOException("Internal Error in trafHdfsRead ");
+      if (lenRemain_ == 0)
+         return bytesRead; 
+      readLength = (lenRemain_ > blockSize_) ? blockSize_ : lenRemain_;
+      future_ = executorService_.submit(new HDFSRead(readLength));
+      return bytesRead;
+   } 
+
+   public int trafHdfsReadBuffer() throws IOException, InterruptedException, ExecutionException
+   {
+      int bytesRead;
+      int totalBytesRead = 0;
+      while (true) {
+         bytesRead = trafHdfsRead();
+         if (bytesRead == -1 || bytesRead == 0)
+            return totalBytesRead;
+         totalBytesRead += bytesRead;
+         if (totalBytesRead == bufLen_)
+              return totalBytesRead;
+      }  
+   } 
+
+   boolean hdfsCreate(String fname , boolean compress) throws IOException
+   {
+     if (logger_.isDebugEnabled()) 
+        logger_.debug("HDFSClient.hdfsCreate() - started" );
+      Path filePath = null;
+      if (!compress || (compress && fname.endsWith(".gz")))
+        filePath = new Path(fname);
+      else
+        filePath = new Path(fname + ".gz");
+        
+      FileSystem fs = FileSystem.get(filePath.toUri(),config_);
+      FSDataOutputStream fsOut = fs.create(filePath, true);
+      
+      if (compress) {
+        GzipCodec gzipCodec = (GzipCodec) ReflectionUtils.newInstance( GzipCodec.class, config_);
+        Compressor gzipCompressor = CodecPool.getCompressor(gzipCodec);
+        outStream_= gzipCodec.createOutputStream(fsOut, gzipCompressor);
+      }
+      else
+        outStream_ = fsOut;      
+      if (logger_.isDebugEnabled()) 
+         logger_.debug("HDFSClient.hdfsCreate() - compressed output stream created" );
+      return true;
+    }
+    
+    boolean hdfsWrite(byte[] buff, long len) throws IOException
+    {
+
+      if (logger_.isDebugEnabled()) 
+         logger_.debug("HDFSClient.hdfsWrite() - started" );
+      outStream_.write(buff);
+      outStream_.flush();
+      if (logger_.isDebugEnabled()) logger_.debug("HDFSClient.hdfsWrite() - bytes written and flushed:" + len  );
+      return true;
+    }
+    
+    boolean hdfsClose() throws IOException
+    {
+      if (logger_.isDebugEnabled()) logger_.debug("HDFSClient.hdfsClose() - started" );
+      if (outStream_ != null) {
+          outStream_.close();
+          outStream_ = null;
+      }
+      return true;
+    }
+
+    
+    public boolean hdfsMergeFiles(String srcPathStr, String dstPathStr) throws IOException
+    {
+      if (logger_.isDebugEnabled()) logger_.debug("HDFSClient.hdfsMergeFiles() - start");
+      if (logger_.isDebugEnabled()) logger_.debug("HDFSClient.hdfsMergeFiles() - source Path: " + srcPathStr + 
+                                               ", destination File:" + dstPathStr );
+        Path srcPath = new Path(srcPathStr );
+        srcPath = srcPath.makeQualified(srcPath.toUri(), null);
+        FileSystem srcFs = FileSystem.get(srcPath.toUri(),config_);
+  
+        Path dstPath = new Path(dstPathStr);
+        dstPath = dstPath.makeQualified(dstPath.toUri(), null);
+        FileSystem dstFs = FileSystem.get(dstPath.toUri(),config_);
+        
+        if (dstFs.exists(dstPath))
+        {
+          if (logger_.isDebugEnabled()) logger_.debug("HDFSClient.hdfsMergeFiles() - destination files exists" );
+          // for this prototype we just delete the file-- will change in next code drops
+          dstFs.delete(dstPath, false);
+           // The caller should already have checked existence of file-- throw exception 
+           //throw new FileAlreadyExistsException(dstPath.toString());
+        }
+        
+        Path tmpSrcPath = new Path(srcPath, "tmp");
+
+        FileSystem.mkdirs(srcFs, tmpSrcPath,srcFs.getFileStatus(srcPath).getPermission());
+        logger_.debug("HDFSClient.hdfsMergeFiles() - tmp folder created." );
+        Path[] files = FileUtil.stat2Paths(srcFs.listStatus(srcPath));
+        for (Path f : files)
+        {
+          srcFs.rename(f, tmpSrcPath);
+        }
+        // copyMerge and use false for the delete option since it removes the whole directory
+        if (logger_.isDebugEnabled()) logger_.debug("HDFSClient.hdfsMergeFiles() - copyMerge" );
+        FileUtil.copyMerge(srcFs, tmpSrcPath, dstFs, dstPath, false, config_, null);
+        
+        if (logger_.isDebugEnabled()) logger_.debug("HDFSClient.hdfsMergeFiles() - delete intermediate files" );
+        srcFs.delete(tmpSrcPath, true);
+      return true;
+    }
+
+   public boolean hdfsCleanUnloadPath(String uldPathStr
+                         /*, boolean checkExistence, String mergeFileStr*/) throws IOException
+   {
+      if (logger_.isDebugEnabled()) 
+         logger_.debug("HDFSClient.hdfsCleanUnloadPath() - unload Path: " + uldPathStr );
+      
+      Path uldPath = new Path(uldPathStr );
+      FileSystem fs = FileSystem.get(uldPath.toUri(), config_);
+      if (!fs.exists(uldPath))
+      {
+        //unload location does not exist. hdfscreate will create it later
+        //nothing to do 
+        return true;
+      }
+       
+      Path[] files = FileUtil.stat2Paths(fs.listStatus(uldPath));
+      if (logger_.isDebugEnabled()) 
+         logger_.debug("HDFSClient.hdfsCleanUnloadPath() - delete files" );
+      for (Path f : files){
+        fs.delete(f, false);
+      }
+      return true;
+   }
+
+   public boolean hdfsExists(String filePathStr) throws IOException 
+   {
+      if (logger_.isDebugEnabled()) 
+         logger_.debug("HDFSClient.hdfsExists() - Path: " + filePathStr);
+
+      Path filePath = new Path(filePathStr );
+      FileSystem fs = FileSystem.get(filePath.toUri(), config_);
+      if (fs.exists(filePath)) 
+         return true;
+      return false;
+   }
+
+   public boolean hdfsDeletePath(String pathStr) throws IOException
+   {
+      if (logger_.isDebugEnabled()) 
+         logger_.debug("HDFSClient.hdfsDeletePath() - start - Path: " + pathStr);
+      Path delPath = new Path(pathStr );
+      FileSystem fs = FileSystem.get(delPath.toUri(), config_);
+      fs.delete(delPath, true);
+      return true;
+   }
+ 
+   public static void shutdown() throws InterruptedException
+   {
+      executorService_.awaitTermination(100, TimeUnit.MILLISECONDS);
+      executorService_.shutdown();
+   }
+}

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
new file mode 100644
index 0000000..bf81ab0
--- /dev/null
+++ b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
@@ -0,0 +1,248 @@
+// @@@ 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 @@@
+
+package org.trafodion.sql;
+
+// This class implements an efficient mechanism to read hdfs files
+// Trafodion ExHdfsScan operator provides a range of scans to be performed.
+// The range consists of a hdfs filename, offset and length to be read
+// This class takes in two ByteBuffers. These ByteBuffer can be either direct buffers
+// backed up native buffers or indirect buffer backed by java arrays.
+// All the ranges are read alternating between the two buffers using ExecutorService
+// using CachedThreadPool mechanism. 
+// For a given HdfsScan instance, only one thread(IO thread) is scheduled to read
+// the next full or partial buffer while the main thread processes the previously
+// read information from the other buffer
+
+import org.apache.log4j.PropertyConfigurator;
+import org.apache.log4j.Logger;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.conf.Configuration;
+import java.nio.ByteBuffer;
+import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ExecutionException;
+import org.trafodion.sql.HDFSClient;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.fs.FileStatus;
+import java.net.URI;
+
+public class HdfsScan 
+{
+   static Logger logger_ = Logger.getLogger(HdfsScan.class.getName());
+   private ByteBuffer buf_[];
+   private int bufLen_[];
+   private HDFSClient hdfsClient_[];
+   private int currRange_;
+   private long currPos_;
+   private long lenRemain_;
+   private int lastBufCompleted_ = -1;
+   private boolean scanCompleted_;
+   private boolean lastScanRangeScheduled_;
+   
+   class HdfsScanRange 
+   {
+      String filename_;
+      long pos_;
+      long len_;
+      
+      HdfsScanRange(String filename, long pos, long len)
+      {
+         filename_ = filename;
+         pos_ = pos;
+         len_ = len;
+      }
+   }
+   
+   private HdfsScanRange hdfsScanRanges_[];
+    
+   static {
+      String confFile = System.getProperty("trafodion.log4j.configFile");
+      System.setProperty("trafodion.root", System.getenv("TRAF_HOME"));
+   }
+
+   HdfsScan(ByteBuffer buf1, ByteBuffer buf2, String filename[], long pos[], long len[]) throws IOException
+   {
+      buf_ = new ByteBuffer[2];
+      bufLen_ = new int[2];
+
+      buf_[0] = buf1;
+      buf_[1] = buf2;
+
+      for (int i = 0; i < 2 ; i++) {
+          if (buf_[i].hasArray())
+             bufLen_[i] = buf_[i].array().length;
+          else
+             bufLen_[i] = buf_[i].capacity();
+      }
+      hdfsClient_ = new HDFSClient[2];
+      hdfsScanRanges_ = new HdfsScanRange[filename.length]; 
+      for (int i = 0; i < filename.length; i++) {
+         hdfsScanRanges_[i] = new HdfsScanRange(filename[i], pos[i], len[i]);
+      }
+      if (hdfsScanRanges_.length > 0) {
+         currRange_ = 0;
+         currPos_ = hdfsScanRanges_[0].pos_;
+         lenRemain_ = hdfsScanRanges_[0].len_;
+         hdfsScanRange(0);
+      }
+      scanCompleted_ = false;
+      lastScanRangeScheduled_ = false;
+   }
+
+   public void hdfsScanRange(int bufNo) throws IOException
+   {
+      System.out.println (" CurrentRange " + currRange_ + " LenRemain " + lenRemain_ + " BufNo " + bufNo); 
+      int readLength;
+      if (lenRemain_ > bufLen_[bufNo])
+         readLength = bufLen_[bufNo];
+      else
+         readLength = (int)lenRemain_;
+      hdfsClient_[bufNo] = new HDFSClient(bufNo, hdfsScanRanges_[currRange_].filename_, buf_[bufNo], currPos_, readLength);
+      lenRemain_ -= readLength;
+      currPos_ += readLength; 
+      if (lenRemain_ == 0) {
+         if (currRange_  == (hdfsScanRanges_.length-1)) 
+            lastScanRangeScheduled_ = true;
+         else {
+            currRange_++;
+            currPos_ = hdfsScanRanges_[currRange_].pos_;
+            lenRemain_ = hdfsScanRanges_[currRange_].len_; 
+         }
+      } 
+   } 
+   
+   public int[] trafHdfsRead() throws IOException, InterruptedException, ExecutionException
+   {
+      int[] retArray;
+      int byteCompleted;
+      int bufNo;
+ 
+      if (scanCompleted_)
+         return null; 
+      retArray = new int[2];
+      switch (lastBufCompleted_) {
+         case -1:
+         case 1:
+            byteCompleted = hdfsClient_[0].trafHdfsReadBuffer(); 
+            bufNo = 0;
+            break;
+         case 0:
+            byteCompleted = hdfsClient_[1].trafHdfsReadBuffer(); 
+            bufNo = 1;
+            break;
+         default:
+            bufNo = -1;
+            byteCompleted = -1;
+      }    
+      lastBufCompleted_ = bufNo;
+      retArray[0] = byteCompleted;
+      retArray[1] = bufNo;
+      System.out.println (" Buffer No " + retArray[1] + " Bytes Read " + retArray[0]); 
+      lastBufCompleted_ = bufNo;
+      if (lastScanRangeScheduled_) {
+         scanCompleted_ = true;
+         return retArray; 
+      }
+      switch (lastBufCompleted_)
+      {
+         case 0:
+            hdfsScanRange(1);
+            break;
+         case 1:
+            hdfsScanRange(0);
+            break;            
+         default:
+            break;
+      }
+      return retArray;
+   } 
+   
+   public static void shutdown() throws InterruptedException
+   {
+      HDFSClient.shutdown();
+   }
+   public static void main(String[] args) throws Exception
+   {
+
+      if (args.length < 3)
+      {
+         System.out.println("Usage: org.trafodion.sql.HdfsScan <tableName> <buffer_length> <number_of_splits>");
+         return;
+      }
+      String tableName = args[0];
+      int capacity = Integer.parseInt(args[1]) * 1024 *1024;
+      int split = Integer.parseInt(args[2]);
+      HiveConf config = new HiveConf(); 
+      HiveMetaStoreClient hiveMeta = new HiveMetaStoreClient(config); 
+      Table table = hiveMeta.getTable(tableName);
+      StorageDescriptor sd = table.getSd();
+      String location = sd.getLocation();
+      URI uri = new URI(location);
+      Path path = new Path(uri);
+      FileSystem fs = FileSystem.get(config);       
+      FileStatus file_status[] = fs.listStatus(path);
+      ByteBuffer buf1 = ByteBuffer.allocateDirect(capacity);
+      ByteBuffer buf2 = ByteBuffer.allocateDirect(capacity);
+      String fileName[] = new String[file_status.length * split];
+      long pos[] = new long[file_status.length * split];
+      long len[] = new long[file_status.length * split];
+      for (int i = 0 ; i < file_status.length * split; i++) {
+         Path filePath = file_status[i].getPath();
+         long fileLen = file_status[i].getLen(); 
+         long splitLen = fileLen / split;
+         fileName[i] = filePath.toString();
+         System.out.println (" fileName " + fileName[i] + " Length " + fileLen); 
+         long splitPos = 0;
+         for (int j = 0 ; j < split ; j++)
+         { 
+            fileName[i] = filePath.toString();
+            pos[i] = splitPos + (splitLen * j);
+            len[i] = splitLen;
+            if (j == (split-1))
+               len[i] = fileLen - (splitLen *(j));
+            System.out.println ("Range " + i + " Pos " + pos[i] + " Length " + len[i]); 
+            i++;
+         }
+      }
+      long time1 = System.currentTimeMillis();
+      HdfsScan hdfsScan = new HdfsScan(buf1, buf2, fileName, pos, len);
+      int[] retArray;
+      int bytesCompleted;
+      while (true) {
+         retArray = hdfsScan.trafHdfsRead();
+         if (retArray == null)
+            break;
+      }
+      long time2 = System.currentTimeMillis();
+      HdfsScan.shutdown();
+      System.out.println("Time taken in milliSeconds " + (time2-time1) );
+   }
+}

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/src/main/java/org/trafodion/sql/SequenceFileWriter.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/SequenceFileWriter.java b/core/sql/src/main/java/org/trafodion/sql/SequenceFileWriter.java
index 2007005..ff88dd7 100644
--- a/core/sql/src/main/java/org/trafodion/sql/SequenceFileWriter.java
+++ b/core/sql/src/main/java/org/trafodion/sql/SequenceFileWriter.java
@@ -143,158 +143,4 @@ public class SequenceFileWriter {
         }
         return null;
     }
-    
-    boolean hdfsCreate(String fname , boolean compress) throws IOException
-    {
-      if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsCreate() - started" );
-      Path filePath = null;
-      if (!compress || (compress && fname.endsWith(".gz")))
-        filePath = new Path(fname);
-      else
-        filePath = new Path(fname + ".gz");
-        
-      fs = FileSystem.get(filePath.toUri(),conf);
-      fsOut = fs.create(filePath, true);
-      
-      outStream = fsOut;
-      
-      if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsCreate() - file created" );
-      if (compress)
-      {
-        GzipCodec gzipCodec = (GzipCodec) ReflectionUtils.newInstance( GzipCodec.class, conf);
-        Compressor gzipCompressor = CodecPool.getCompressor(gzipCodec);
-        outStream = gzipCodec.createOutputStream(fsOut, gzipCompressor);
-        sameStream = false;
-      }
-      
-      if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsCreate() - compressed output stream created" );
-      return true;
-    }
-    
-    boolean hdfsWrite(byte[] buff, long len) throws IOException
-    {
-
-      if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsWrite() - started" );
-      outStream.write(buff);
-      outStream.flush();
-      if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsWrite() - bytes written and flushed:" + len  );
-      return true;
-    }
-    
-    boolean hdfsClose() throws IOException
-    {
-      if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsClose() - started" );
-      if (sameStream) { 
-         if (outStream != null) {
-            outStream.close();
-            outStream = null;
-         }
-         fsOut = null;
-      }
-      else {
-         if (outStream != null) {
-            outStream.close();
-            outStream = null;
-         }
-         if (fsOut != null) {
-            fsOut.close();
-            fsOut = null;
-         }
-      }
-      return true;
-    }
-
-    
-    public boolean hdfsMergeFiles(String srcPathStr, String dstPathStr) throws IOException
-    {
-      if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsMergeFiles() - start");
-      if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsMergeFiles() - source Path: " + srcPathStr + 
-                                               ", destination File:" + dstPathStr );
-        Path srcPath = new Path(srcPathStr );
-        srcPath = srcPath.makeQualified(srcPath.toUri(), null);
-        FileSystem srcFs = FileSystem.get(srcPath.toUri(),conf);
-  
-        Path dstPath = new Path(dstPathStr);
-        dstPath = dstPath.makeQualified(dstPath.toUri(), null);
-        FileSystem dstFs = FileSystem.get(dstPath.toUri(),conf);
-        
-        if (dstFs.exists(dstPath))
-        {
-          if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsMergeFiles() - destination files exists" );
-          // for this prototype we just delete the file-- will change in next code drops
-          dstFs.delete(dstPath, false);
-           // The caller should already have checked existence of file-- throw exception 
-           //throw new FileAlreadyExistsException(dstPath.toString());
-        }
-        
-        Path tmpSrcPath = new Path(srcPath, "tmp");
-
-        FileSystem.mkdirs(srcFs, tmpSrcPath,srcFs.getFileStatus(srcPath).getPermission());
-        logger.debug("SequenceFileWriter.hdfsMergeFiles() - tmp folder created." );
-        Path[] files = FileUtil.stat2Paths(srcFs.listStatus(srcPath));
-        for (Path f : files)
-        {
-          srcFs.rename(f, tmpSrcPath);
-        }
-        // copyMerge and use false for the delete option since it removes the whole directory
-        if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsMergeFiles() - copyMerge" );
-        FileUtil.copyMerge(srcFs, tmpSrcPath, dstFs, dstPath, false, conf, null);
-        
-        if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsMergeFiles() - delete intermediate files" );
-        srcFs.delete(tmpSrcPath, true);
-      return true;
-    }
-
-    public boolean hdfsCleanUnloadPath(String uldPathStr
-                         /*, boolean checkExistence, String mergeFileStr*/) throws IOException
-    {
-      if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsCleanUnloadPath() - start");
-      logger.debug("SequenceFileWriter.hdfsCleanUnloadPath() - unload Path: " + uldPathStr );
-      
-      Path uldPath = new Path(uldPathStr );
-      uldPath = uldPath.makeQualified(uldPath.toUri(), null);
-      FileSystem srcFs = FileSystem.get(uldPath.toUri(),conf);
-      if (!srcFs.exists(uldPath))
-      {
-        //unload location does not exist. hdfscreate will create it later
-        //nothing to do 
-        logger.debug("SequenceFileWriter.hdfsCleanUnloadPath() -- unload location does not exist." );
-        return true;
-      }
-       
-      Path[] files = FileUtil.stat2Paths(srcFs.listStatus(uldPath));
-      logger.debug("SequenceFileWriter.hdfsCleanUnloadPath() - delete files" );
-      for (Path f : files){
-        srcFs.delete(f, false);
-      }
-      return true;
-    }
-
-  public boolean hdfsExists(String filePathStr) throws IOException 
-  {
-    logger.debug("SequenceFileWriter.hdfsExists() - start");
-    logger.debug("SequenceFileWriter.hdfsExists() - Path: " + filePathStr);
-
-        //check existence of the merge Path
-       Path filePath = new Path(filePathStr );
-       filePath = filePath.makeQualified(filePath.toUri(), null);
-       FileSystem mergeFs = FileSystem.get(filePath.toUri(),conf);
-       if (mergeFs.exists( filePath))
-       {
-       logger.debug("SequenceFileWriter.hdfsExists() - Path: "
-       + filePath + " exists" );
-         return true;
-       }
-    return false;
-  }
-
-  public boolean hdfsDeletePath(String pathStr) throws IOException
-  {
-    if (logger.isDebugEnabled()) logger.debug("SequenceFileWriter.hdfsDeletePath() - start - Path: " + pathStr);
-      Path delPath = new Path(pathStr );
-      delPath = delPath.makeQualified(delPath.toUri(), null);
-      FileSystem fs = FileSystem.get(delPath.toUri(),conf);
-      fs.delete(delPath, true);
-    return true;
-  }
 }

http://git-wip-us.apache.org/repos/asf/trafodion/blob/60db1533/core/sql/src/main/java/org/trafodion/sql/TrafConfiguration.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/TrafConfiguration.java b/core/sql/src/main/java/org/trafodion/sql/TrafConfiguration.java
index 80b9382..cf0cfa1 100644
--- a/core/sql/src/main/java/org/trafodion/sql/TrafConfiguration.java
+++ b/core/sql/src/main/java/org/trafodion/sql/TrafConfiguration.java
@@ -37,6 +37,7 @@ public class TrafConfiguration {
     Configuration lv_conf = new Configuration();
     switch (config) {
        case HBASE_CONF:
+       case HDFS_CONF:
           String trafSiteXml = new String(System.getenv("TRAF_CONF") + "/trafodion-site.xml");
           Path fileRes = new Path(trafSiteXml);
           lv_conf.addResource(fileRes);


[8/9] trafodion git commit: Fix for regression failure of the commit a99ee11ed

Posted by se...@apache.org.
Fix for regression failure of the commit a99ee11ed


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

Branch: refs/heads/master
Commit: cc43432cfa251396ecc44650d4cdbcc250457daa
Parents: a99ee11
Author: selvaganesang <se...@esgyn.com>
Authored: Fri Feb 16 03:39:09 2018 +0000
Committer: selvaganesang <se...@esgyn.com>
Committed: Fri Feb 16 03:39:09 2018 +0000

----------------------------------------------------------------------
 core/sql/executor/ExHbaseIUD.cpp | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/cc43432c/core/sql/executor/ExHbaseIUD.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHbaseIUD.cpp b/core/sql/executor/ExHbaseIUD.cpp
index 2663b24..e8aac8c 100644
--- a/core/sql/executor/ExHbaseIUD.cpp
+++ b/core/sql/executor/ExHbaseIUD.cpp
@@ -1141,7 +1141,8 @@ ExHbaseAccessBulkLoadPrepSQTcb::ExHbaseAccessBulkLoadPrepSQTcb(
           ex_globals * glob ) :
     ExHbaseAccessUpsertVsbbSQTcb( hbaseAccessTdb, glob),
     prevRowId_ (NULL),
-    lastErrorCnd_(NULL)
+    lastErrorCnd_(NULL),
+    sampleFileHdfsClient_(NULL)
 {
    hFileParamsInitialized_ = false;
    //sortedListOfColNames_ = NULL;


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

Posted by se...@apache.org.
[TRAFODION-2917] Refactor Trafodion implementation of hdfs scan for text format hive tables

Part-6 changes

Removed the dependency of libhdfs while assigning ranges for hdfs scan at runtime
and for sample file creation during bulk load


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

Branch: refs/heads/master
Commit: a99ee11ed3617c0d3e623082a7984fb08b371e96
Parents: a187b03
Author: selvaganesang <se...@esgyn.com>
Authored: Thu Feb 15 21:39:41 2018 +0000
Committer: selvaganesang <se...@esgyn.com>
Committed: Thu Feb 15 21:39:41 2018 +0000

----------------------------------------------------------------------
 core/sql/comexe/ComTdbHdfsScan.h                |   2 -
 core/sql/executor/ExExeUtilCli.cpp              |   1 -
 core/sql/executor/ExExeUtilMisc.cpp             |   1 -
 core/sql/executor/ExHbaseAccess.cpp             |  17 +-
 core/sql/executor/ExHbaseAccess.h               |  18 +-
 core/sql/executor/ExHbaseIUD.cpp                |  44 +++--
 core/sql/executor/ExHdfsScan.cpp                |  47 ++---
 core/sql/executor/ExHdfsScan.h                  |   6 +-
 core/sql/executor/HdfsClient_JNI.cpp            | 176 +++++++++++++++++--
 core/sql/executor/HdfsClient_JNI.h              |  44 ++++-
 core/sql/executor/JavaObjectInterface.cpp       |  10 +-
 core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp        |   1 -
 .../main/java/org/trafodion/sql/HDFSClient.java |  83 ++++++++-
 13 files changed, 360 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/comexe/ComTdbHdfsScan.h
----------------------------------------------------------------------
diff --git a/core/sql/comexe/ComTdbHdfsScan.h b/core/sql/comexe/ComTdbHdfsScan.h
index ff692c9..86534be 100755
--- a/core/sql/comexe/ComTdbHdfsScan.h
+++ b/core/sql/comexe/ComTdbHdfsScan.h
@@ -24,7 +24,6 @@
 #define COM_HDFS_SCAN_H
 
 #include "ComTdb.h"
-//#include "hdfs.h"   
 #include "ExpLOBinterface.h"
 #include "ComQueue.h"
 
@@ -108,7 +107,6 @@ class ComTdbHdfsScan : public ComTdb
 
   UInt32 flags_;                                             // 96 - 99
 
-  // hadoop port num. An unsigned short in hdfs.h, subject to change.
   UInt16 port_;                                              // 100 - 101
 
   UInt16 convertSkipListSize_;                               // 102 - 103

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/executor/ExExeUtilCli.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExExeUtilCli.cpp b/core/sql/executor/ExExeUtilCli.cpp
index fdfef44..3effdb6 100644
--- a/core/sql/executor/ExExeUtilCli.cpp
+++ b/core/sql/executor/ExExeUtilCli.cpp
@@ -42,7 +42,6 @@
 #include  "sql_id.h"
 #include  "ComSqlId.h"
 #include  "ExExeUtilCli.h"
-#include  "hdfs.h"
 OutputInfo::OutputInfo(Lng32 numEntries)
   : numEntries_(numEntries)
 {

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/executor/ExExeUtilMisc.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExExeUtilMisc.cpp b/core/sql/executor/ExExeUtilMisc.cpp
index 2ed95fb..b7322ff 100644
--- a/core/sql/executor/ExExeUtilMisc.cpp
+++ b/core/sql/executor/ExExeUtilMisc.cpp
@@ -53,7 +53,6 @@
 #include  "ComRtUtils.h"
 #include  "ExStats.h"
 #include  "ComSmallDefs.h"
-//#include  "hdfs.h" //will replace with LOB interface
 #include <unistd.h>
 
 //////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/executor/ExHbaseAccess.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHbaseAccess.cpp b/core/sql/executor/ExHbaseAccess.cpp
index 2247b9a..2182f9a 100644
--- a/core/sql/executor/ExHbaseAccess.cpp
+++ b/core/sql/executor/ExHbaseAccess.cpp
@@ -39,7 +39,6 @@
 #include  "cli_stdh.h"
 #include "exp_function.h"
 #include "jni.h"
-#include "hdfs.h"
 #include <random>
 #include "HdfsClient_JNI.h"
 
@@ -242,7 +241,7 @@ ExHbaseAccessTcb::ExHbaseAccessTcb(
   , colValVecSize_(0)
   , colValEntry_(0)
   , loggingErrorDiags_(NULL)
-  , hdfsClient_(NULL)
+  , logFileHdfsClient_(NULL)
   , loggingFileCreated_(FALSE)
   , loggingFileName_(NULL)
 {
@@ -500,8 +499,8 @@ void ExHbaseAccessTcb::freeResources()
      NADELETEBASIC(directRowBuffer_, getHeap());
   if (colVal_.val != NULL)
      NADELETEBASIC(colVal_.val, getHeap());
-  if (hdfsClient_ != NULL) 
-     NADELETE(hdfsClient_, HdfsClient, getHeap());
+  if (logFileHdfsClient_ != NULL) 
+     NADELETE(logFileHdfsClient_, HdfsClient, getHeap());
   if (loggingFileName_ != NULL)
      NADELETEBASIC(loggingFileName_, getHeap());
 }
@@ -2992,7 +2991,7 @@ short ExHbaseAccessTcb::createDirectRowBuffer( UInt16 tuppIndex,
     {
       // Overwrite trailing delimiter with newline.
       hiveBuff[hiveBuffInx-1] = '\n';
-      hdfsWrite(getHdfs(), getHdfsSampleFile(), hiveBuff, hiveBuffInx);
+      sampleFileHdfsClient()->hdfsWrite(hiveBuff, hiveBuffInx);
     }
   return 0;
 }
@@ -3264,16 +3263,16 @@ void ExHbaseAccessTcb::handleException(NAHeap *heap,
      return;
 
   if (!loggingFileCreated_) {
-     hdfsClient_ = HdfsClient::newInstance((NAHeap *)getHeap(), hdfsClientRetcode);
+     logFileHdfsClient_ = HdfsClient::newInstance((NAHeap *)getHeap(), hdfsClientRetcode);
      if (hdfsClientRetcode == HDFS_CLIENT_OK)
-        hdfsClientRetcode = hdfsClient_->hdfsCreate(loggingFileName_, FALSE);
+        hdfsClientRetcode = logFileHdfsClient_->hdfsCreate(loggingFileName_, FALSE);
      if (hdfsClientRetcode == HDFS_CLIENT_OK)
         loggingFileCreated_ = TRUE;
      else 
         goto logErrorReturn;
   }
   
-  hdfsClientRetcode = hdfsClient_->hdfsWrite(logErrorRow, logErrorRowLen);
+  hdfsClientRetcode = logFileHdfsClient_->hdfsWrite(logErrorRow, logErrorRowLen);
   if (hdfsClientRetcode != HDFS_CLIENT_OK) 
      goto logErrorReturn;
   if (errorCond != NULL) {
@@ -3289,7 +3288,7 @@ void ExHbaseAccessTcb::handleException(NAHeap *heap,
      errorMsg = (char *)"[UNKNOWN EXCEPTION]\n";
      errorMsgLen = strlen(errorMsg);
   }
-  hdfsClientRetcode = hdfsClient_->hdfsWrite(errorMsg, errorMsgLen);
+  hdfsClientRetcode = logFileHdfsClient_->hdfsWrite(errorMsg, errorMsgLen);
 logErrorReturn:
   if (hdfsClientRetcode != HDFS_CLIENT_OK) {
      loggingErrorDiags_ = ComDiagsArea::allocate(heap);

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/executor/ExHbaseAccess.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHbaseAccess.h b/core/sql/executor/ExHbaseAccess.h
index 863b5b5..7be1551 100644
--- a/core/sql/executor/ExHbaseAccess.h
+++ b/core/sql/executor/ExHbaseAccess.h
@@ -36,7 +36,6 @@
 #include "key_range.h"
 #include "key_single_subset.h"
 #include "ex_mdam.h"
-#include "hdfs.h"
 
 // -----------------------------------------------------------------------
 // Classes defined in this file
@@ -501,14 +500,13 @@ protected:
   NABoolean asyncOperation_;
   Int32 asyncOperationTimeout_;
   ComDiagsArea *loggingErrorDiags_;
-  HdfsClient *hdfsClient_;
+  HdfsClient *logFileHdfsClient_;
   char *loggingFileName_;
   NABoolean loggingFileCreated_ ;
 
   // Redefined and used by ExHbaseAccessBulkLoadPrepSQTcb.
 
-  virtual hdfsFS getHdfs() const { return NULL; }
-  virtual hdfsFile getHdfsSampleFile() const { return NULL; }
+  virtual HdfsClient *sampleFileHdfsClient() const { return NULL; }
 };
 
 class ExHbaseTaskTcb : public ExGod
@@ -921,14 +919,9 @@ class ExHbaseAccessBulkLoadPrepSQTcb: public ExHbaseAccessUpsertVsbbSQTcb
     virtual ExWorkProcRetcode work();
 
   protected:
-    virtual hdfsFS getHdfs() const
+    virtual HdfsClient *sampleFileHdfsClient() const
     {
-      return hdfs_;
-    }
-
-    virtual hdfsFile getHdfsSampleFile() const
-    {
-      return hdfsSampleFile_;
+      return sampleFileHdfsClient_;
     }
 
    private:
@@ -949,8 +942,7 @@ class ExHbaseAccessBulkLoadPrepSQTcb: public ExHbaseAccessUpsertVsbbSQTcb
 
 
     // HDFS file system and output file ptrs used for ustat sample table.
-    hdfsFS hdfs_;
-    hdfsFile hdfsSampleFile_;
+    HdfsClient *sampleFileHdfsClient_;
 };
 // UMD SQ: UpdMergeDel on Trafodion table
 class ExHbaseUMDtrafSubsetTaskTcb  : public ExHbaseTaskTcb

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/executor/ExHbaseIUD.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHbaseIUD.cpp b/core/sql/executor/ExHbaseIUD.cpp
index e8896b2..2663b24 100644
--- a/core/sql/executor/ExHbaseIUD.cpp
+++ b/core/sql/executor/ExHbaseIUD.cpp
@@ -1141,8 +1141,6 @@ ExHbaseAccessBulkLoadPrepSQTcb::ExHbaseAccessBulkLoadPrepSQTcb(
           ex_globals * glob ) :
     ExHbaseAccessUpsertVsbbSQTcb( hbaseAccessTdb, glob),
     prevRowId_ (NULL),
-    hdfs_(NULL),
-    hdfsSampleFile_(NULL),
     lastErrorCnd_(NULL)
 {
    hFileParamsInitialized_ = false;
@@ -1163,17 +1161,8 @@ ExHbaseAccessBulkLoadPrepSQTcb::ExHbaseAccessBulkLoadPrepSQTcb(
 
 ExHbaseAccessBulkLoadPrepSQTcb::~ExHbaseAccessBulkLoadPrepSQTcb()
 {
-  // Flush and close sample file if used
-  if (hdfs_)
-    {
-      if (hdfsSampleFile_)
-        {
-          hdfsFlush(hdfs_, hdfsSampleFile_);
-          hdfsCloseFile(hdfs_, hdfsSampleFile_);
-        }
-     
-    }
-
+  if (sampleFileHdfsClient_ != NULL)
+     NADELETE(sampleFileHdfsClient_, HdfsClient, getHeap()); 
 }
 
 // Given the type information available via the argument, return the name of
@@ -1394,7 +1383,7 @@ ExWorkProcRetcode ExHbaseAccessBulkLoadPrepSQTcb::work()
 
       if (!hFileParamsInitialized_)
       {
-              importLocation_= std::string(((ExHbaseAccessTdb&)hbaseAccessTdb()).getLoadPrepLocation()) +
+        importLocation_= std::string(((ExHbaseAccessTdb&)hbaseAccessTdb()).getLoadPrepLocation()) +
             ((ExHbaseAccessTdb&)hbaseAccessTdb()).getTableName() ;
         familyLocation_ = std::string(importLocation_ + "/#1");
         Lng32 fileNum = getGlobals()->castToExExeStmtGlobals()->getMyInstanceNumber();
@@ -1424,15 +1413,30 @@ ExWorkProcRetcode ExHbaseAccessBulkLoadPrepSQTcb::work()
             // Set up HDFS file for sample table.
            
             ContextCli *currContext = getGlobals()->castToExExeStmtGlobals()->getCliGlobals()->currContext();
-            hdfs_ = currContext->getHdfsServerConnection((char*)"default",0);
             Text samplePath = std::string(((ExHbaseAccessTdb&)hbaseAccessTdb()).getSampleLocation()) +
                                           ((ExHbaseAccessTdb&)hbaseAccessTdb()).getTableName() ;
             char filePart[10];
             sprintf(filePart, "/%d", fileNum);
+            HDFS_Client_RetCode hdfsClientRetcode;
             samplePath.append(filePart);
-            hdfsSampleFile_ = hdfsOpenFile(hdfs_, samplePath.data(), O_WRONLY|O_CREAT, 0, 0, 0);
+            if (sampleFileHdfsClient_ == NULL)
+                sampleFileHdfsClient_ = HdfsClient::newInstance((NAHeap *)getHeap(), hdfsClientRetcode);
+            if (hdfsClientRetcode == HDFS_CLIENT_OK) {
+                hdfsClientRetcode = sampleFileHdfsClient_->hdfsOpen(samplePath.data(), FALSE);
+                if (hdfsClientRetcode != HDFS_CLIENT_OK) {
+                    NADELETE(sampleFileHdfsClient_, HdfsClient, getHeap());
+                    sampleFileHdfsClient_ = NULL;
+                }
+            } 
+            if (hdfsClientRetcode != HDFS_CLIENT_OK) {
+              ComDiagsArea * diagsArea = NULL;
+              ExRaiseSqlError(getHeap(), &diagsArea,
+                              (ExeErrorCode)(8110));
+              pentry_down->setDiagsArea(diagsArea);
+              step_ = HANDLE_ERROR;
+              break;
+            }
           }
-
           posVec_.clear();
           hbaseAccessTdb().listOfUpdatedColNames()->position();
           while (NOT hbaseAccessTdb().listOfUpdatedColNames()->atEnd())
@@ -1763,8 +1767,10 @@ ExWorkProcRetcode ExHbaseAccessBulkLoadPrepSQTcb::work()
           if (eodSeen)
           {
             ehi_->closeHFile(table_);
-            if (hdfsClient_ != NULL)
-               hdfsClient_->hdfsClose();
+            if (logFileHdfsClient_ != NULL)
+               logFileHdfsClient_->hdfsClose();
+            if (sampleFileHdfsClient_ != NULL)
+               sampleFileHdfsClient_->hdfsClose();
             hFileParamsInitialized_ = false;
             retcode = ehi_->close();
           }

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/executor/ExHdfsScan.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHdfsScan.cpp b/core/sql/executor/ExHdfsScan.cpp
index cd95899..c36270e 100644
--- a/core/sql/executor/ExHdfsScan.cpp
+++ b/core/sql/executor/ExHdfsScan.cpp
@@ -119,6 +119,7 @@ ExHdfsScanTcb::ExHdfsScanTcb(
   , dataModCheckDone_(FALSE)
   , loggingErrorDiags_(NULL)
   , loggingFileName_(NULL)
+  , logFileHdfsClient_(NULL)
   , hdfsClient_(NULL)
   , hdfsScan_(NULL)
   , hdfsStats_(NULL)
@@ -226,7 +227,10 @@ ExHdfsScanTcb::ExHdfsScanTcb(
   ehi_ = ExpHbaseInterface::newInstance(glob->getDefaultHeap(),
                                         (char*)"",  //Later replace with server cqd
                                         (char*)"");
-
+  ex_assert(ehi_ != NULL, "Internal error: ehi_ is null in ExHdfsScan");
+  HDFS_Client_RetCode hdfsClientRetcode;
+  hdfsClient_ = HdfsClient::newInstance((NAHeap *)getHeap(), hdfsClientRetcode);
+  ex_assert(hdfsClientRetcode == HDFS_CLIENT_OK, "Internal error: HdfsClient::newInstance returned an error"); 
   // Populate the hdfsInfo list into an array to gain o(1) lookup access
   Queue* hdfsInfoList = hdfsScanTdb.getHdfsFileInfoList();
   if ( hdfsInfoList && hdfsInfoList->numEntries() > 0 )
@@ -308,6 +312,8 @@ void ExHdfsScanTcb::freeResources()
   }
   if (hdfsClient_ != NULL) 
      NADELETE(hdfsClient_, HdfsClient, getHeap());
+  if (logFileHdfsClient_ != NULL) 
+     NADELETE(logFileHdfsClient_, HdfsClient, getHeap());
   if (hdfsScan_ != NULL) 
      NADELETE(hdfsScan_, HdfsScan, getHeap());
 }
@@ -408,8 +414,6 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
   Lng32 openType = 0;
   int changedLen = 0;
   ContextCli *currContext = getGlobals()->castToExExeStmtGlobals()->getCliGlobals()->currContext();
-  hdfsFS hdfs = currContext->getHdfsServerConnection(hdfsScanTdb().hostName_,hdfsScanTdb().port_);
-  hdfsFileInfo *dirInfo = NULL;
   Int32 hdfsErrorDetail = 0;//this is errno returned form underlying hdfsOpenFile call.
   HDFS_Scan_RetCode hdfsScanRetCode;
 
@@ -1075,8 +1079,12 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	  {
             if (useLibhdfsScan_)
 	       step_ = REPOS_HDFS_DATA;
-            else
-               step_ = COPY_TAIL_TO_HEAD;
+            else {
+               if (retArray_[IS_EOF]) 
+                  step_ = TRAF_HDFS_READ;
+               else
+                  step_ = COPY_TAIL_TO_HEAD;
+            }
 	    if (!exception_)
 	      break;
 	  }
@@ -1299,7 +1307,7 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	          if ((pentry_down->downState.request == ex_queue::GET_N) &&
 	              (pentry_down->downState.requestValue == matches_)) {
                      if (useLibhdfsScan_)
-                        step_ = CLOSE_HDFS_CURSOR;
+                        step_ = CLOSE_FILE;
                      else
                         step_ = DONE;
                   }
@@ -1668,8 +1676,8 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	  {
 	    if (qparent_.up->isFull())
 	      return WORK_OK;
-            if (hdfsClient_ != NULL)
-               retcode = hdfsClient_->hdfsClose();
+            if (logFileHdfsClient_ != NULL)
+               retcode = logFileHdfsClient_->hdfsClose();
 	    ex_queue_entry *up_entry = qparent_.up->getTailEntry();
 	    up_entry->copyAtp(pentry_down);
 	    up_entry->upState.parentIndex =
@@ -1693,7 +1701,6 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	    
 	    qparent_.down->removeHead();
 	    step_ = NOT_STARTED;
-            dirInfo = hdfsGetPathInfo(hdfs, "/");
 	    break;
 	  }
 	  
@@ -1900,19 +1907,19 @@ void ExHdfsScanTcb::computeRangesAtRuntime()
   Int64 firstFileStartingOffset = 0;
   Int64 lastFileBytesToRead = -1;
   Int32 numParallelInstances = MAXOF(getGlobals()->getNumOfInstances(),1);
-  hdfsFS fs = ((GetCliGlobals()->currContext())->getHdfsServerConnection(
-                    hdfsScanTdb().hostName_,
-                    hdfsScanTdb().port_));
-  hdfsFileInfo *fileInfos = hdfsListDirectory(fs,
-                                              hdfsScanTdb().hdfsRootDir_,
-                                              &numFiles);
+
+  HDFS_FileInfo *fileInfos;
+  HDFS_Client_RetCode hdfsClientRetcode;
+
+  hdfsClientRetcode = hdfsClient_->hdfsListDirectory(hdfsScanTdb().hdfsRootDir_, &fileInfos, &numFiles); 
+  ex_assert(hdfsClientRetcode == HDFS_CLIENT_OK, "Internal error:hdfsClient->hdfsListDirectory returned an error")
 
   deallocateRuntimeRanges();
 
   // in a first round, count the total number of bytes
   for (int f=0; f<numFiles; f++)
     {
-      ex_assert(fileInfos[f].mKind == kObjectKindFile,
+      ex_assert(fileInfos[f].mKind == HDFS_FILE_KIND,
                 "subdirectories not supported with runtime HDFS ranges");
       totalSize += (Int64) fileInfos[f].mSize;
     }
@@ -2127,15 +2134,15 @@ void ExHdfsScanTcb::handleException(NAHeap *heap,
      return;
 
   if (!loggingFileCreated_) {
-     hdfsClient_ = HdfsClient::newInstance((NAHeap *)getHeap(), hdfsClientRetcode);
+     logFileHdfsClient_ = HdfsClient::newInstance((NAHeap *)getHeap(), hdfsClientRetcode);
      if (hdfsClientRetcode == HDFS_CLIENT_OK)
-        hdfsClientRetcode = hdfsClient_->hdfsCreate(loggingFileName_, FALSE);
+        hdfsClientRetcode = logFileHdfsClient_->hdfsCreate(loggingFileName_, FALSE);
      if (hdfsClientRetcode == HDFS_CLIENT_OK)
         loggingFileCreated_ = TRUE;
      else 
         goto logErrorReturn;
   }
-  hdfsClientRetcode = hdfsClient_->hdfsWrite(logErrorRow, logErrorRowLen);
+  hdfsClientRetcode = logFileHdfsClient_->hdfsWrite(logErrorRow, logErrorRowLen);
   if (hdfsClientRetcode != HDFS_CLIENT_OK) 
      goto logErrorReturn;
   if (errorCond != NULL) {
@@ -2151,7 +2158,7 @@ void ExHdfsScanTcb::handleException(NAHeap *heap,
      errorMsg = (char *)"[UNKNOWN EXCEPTION]\n";
      errorMsgLen = strlen(errorMsg);
   }
-  hdfsClientRetcode = hdfsClient_->hdfsWrite(errorMsg, errorMsgLen);
+  hdfsClientRetcode = logFileHdfsClient_->hdfsWrite(errorMsg, errorMsgLen);
 logErrorReturn:
   if (hdfsClientRetcode != HDFS_CLIENT_OK) {
      loggingErrorDiags_ = ComDiagsArea::allocate(heap);

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/executor/ExHdfsScan.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHdfsScan.h b/core/sql/executor/ExHdfsScan.h
index 2570a58..f4ad7e1 100644
--- a/core/sql/executor/ExHdfsScan.h
+++ b/core/sql/executor/ExHdfsScan.h
@@ -31,9 +31,6 @@
 #include "ExStats.h"
 #include "sql_buffer.h"
 #include "ex_queue.h"
-
-#include "hdfs.h"
-
 #include <time.h>
 #include "ExHbaseAccess.h"
 #include "ExpHbaseInterface.h"
@@ -130,7 +127,7 @@ private:
       another Java thread.
    6. Native layer after processing all the rows in one ByteBuffer, moves the last incomplete row to head room of the
       other ByteBuffer. Then it requests to check if the read is complete. The native layer processes the buffer starting
-      from the copied incomplete row.
+      from the copied partial row.
 */
 
 class ExHdfsScanTcb  : public ex_tcb
@@ -339,6 +336,7 @@ protected:
   // this array is populated from the info list stored as Queue.
   HdfsFileInfoArray hdfsFileInfoListAsArray_;
 
+  HdfsClient *logFileHdfsClient_;
   HdfsClient *hdfsClient_;
   HdfsScan *hdfsScan_;
   NABoolean useLibhdfsScan_;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/executor/HdfsClient_JNI.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/HdfsClient_JNI.cpp b/core/sql/executor/HdfsClient_JNI.cpp
index 8f2845a..2fab571 100644
--- a/core/sql/executor/HdfsClient_JNI.cpp
+++ b/core/sql/executor/HdfsClient_JNI.cpp
@@ -258,6 +258,8 @@ static const char* const hdfsClientErrorEnumStr[] =
 {
   "JNI NewStringUTF() in hdfsCreate()."
  ,"Java exception in hdfsCreate()."
+  "JNI NewStringUTF() in hdfsOpen()."
+ ,"Java exception in hdfsOpen()."
  ,"JNI NewStringUTF() in hdfsWrite()."
  ,"Java exception in hdfsWrite()."
  ,"Java exception in hdfsClose()."
@@ -269,11 +271,32 @@ static const char* const hdfsClientErrorEnumStr[] =
  ,"Java exception in hdfsExists()."
  ,"JNI NewStringUTF() in hdfsDeletePath()."
  ,"Java exception in hdfsDeletePath()."
+ ,"Error in setHdfsFileInfo()."
+ ,"Error in hdfsListDirectory()."
+ ,"Java exception in hdfsListDirectory()."
 };
 
 //////////////////////////////////////////////////////////////////////////////
 // 
 //////////////////////////////////////////////////////////////////////////////
+HdfsClient::~HdfsClient()
+{
+   QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::~HdfsClient() called.");
+   deleteHdfsFileInfo();
+}
+
+void HdfsClient::deleteHdfsFileInfo()
+{
+   for (int i = 0; i < numFiles_ ; i ++) {
+      NADELETEBASIC(hdfsFileInfo_[i].mName, getHeap());
+      NADELETEBASIC(hdfsFileInfo_[i].mOwner, getHeap());
+      NADELETEBASIC(hdfsFileInfo_[i].mGroup, getHeap());
+   }
+   NADELETEBASIC(hdfsFileInfo_, getHeap()); 
+   numFiles_ = 0;
+   hdfsFileInfo_ = NULL;
+}
+
 HdfsClient *HdfsClient::newInstance(NAHeap *heap, HDFS_Client_RetCode &retCode)
 {
    QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::newInstance() called.");
@@ -313,6 +336,8 @@ HDFS_Client_RetCode HdfsClient::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_OPEN       ].jm_name      = "hdfsOpen";
+    JavaMethods_[JM_HDFS_OPEN       ].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";
@@ -325,6 +350,8 @@ HDFS_Client_RetCode HdfsClient::init()
     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";
+    JavaMethods_[JM_HDFS_LIST_DIRECTORY].jm_name      = "hdfsListDirectory";
+    JavaMethods_[JM_HDFS_LIST_DIRECTORY].jm_signature = "(Ljava/lang/String;J)I";
     rc = (HDFS_Client_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
     if (rc == HDFS_CLIENT_OK)
        javaMethodsInitialized_ = TRUE;
@@ -383,9 +410,46 @@ HDFS_Client_RetCode HdfsClient::hdfsCreate(const char* path, NABoolean compress)
   return HDFS_CLIENT_OK;
 }
 
-//////////////////////////////////////////////////////////////////////////////
-//
-//////////////////////////////////////////////////////////////////////////////
+HDFS_Client_RetCode HdfsClient::hdfsOpen(const char* path, NABoolean compress)
+{
+  QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::hdfsOpen(%s) called.", path);
+
+  if (initJNIEnv() != JOI_OK)
+     return HDFS_CLIENT_ERROR_HDFS_OPEN_PARAM;
+
+  jstring js_path = jenv_->NewStringUTF(path);
+  if (js_path == NULL) {
+    GetCliGlobals()->setJniErrorStr(getErrorText(HDFS_CLIENT_ERROR_HDFS_OPEN_PARAM));
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_OPEN_PARAM;
+  }
+
+  jboolean j_compress = compress;
+
+  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_OPEN].jm_full_name;
+  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_OPEN].methodID, js_path, j_compress);
+
+  if (jenv_->ExceptionCheck())
+  {
+    getExceptionDetails();
+    logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsOpen()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_OPEN_EXCEPTION;
+  }
+
+  if (jresult == false)
+  {
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsOpen()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_OPEN_PARAM;
+  }
+
+  jenv_->PopLocalFrame(NULL);
+  return HDFS_CLIENT_OK;
+}
+
+
 HDFS_Client_RetCode HdfsClient::hdfsWrite(const char* data, Int64 len)
 {
   QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::hdfsWrite(%ld) called.", len);
@@ -467,7 +531,6 @@ HDFS_Client_RetCode HdfsClient::hdfsCleanUnloadPath( const NAString& uldPath)
                                                                              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));
@@ -476,7 +539,7 @@ HDFS_Client_RetCode HdfsClient::hdfsCleanUnloadPath( const NAString& uldPath)
   }
 
   tsRecentJMFromJNI = JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].methodID, js_UldPath);
+  jboolean jresult = jenv_->CallStaticBooleanMethod(javaClass_, JavaMethods_[JM_HDFS_CLEAN_UNLOAD_PATH].methodID, js_UldPath);
 
   if (jenv_->ExceptionCheck())
   {
@@ -516,7 +579,7 @@ HDFS_Client_RetCode HdfsClient::hdfsMergeFiles( const NAString& srcPath,
 
 
   tsRecentJMFromJNI = JavaMethods_[JM_HDFS_MERGE_FILES].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_MERGE_FILES].methodID, js_SrcPath, js_DstPath);
+  jboolean jresult = jenv_->CallStaticBooleanMethod(javaClass_, JavaMethods_[JM_HDFS_MERGE_FILES].methodID, js_SrcPath, js_DstPath);
 
   if (jenv_->ExceptionCheck())
   {
@@ -554,7 +617,7 @@ HDFS_Client_RetCode HdfsClient::hdfsDeletePath( const NAString& delPath)
 
 
   tsRecentJMFromJNI = JavaMethods_[JM_HDFS_DELETE_PATH].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_DELETE_PATH].methodID, js_delPath);
+  jboolean jresult = jenv_->CallStaticBooleanMethod(javaClass_, JavaMethods_[JM_HDFS_DELETE_PATH].methodID, js_delPath);
 
   if (jenv_->ExceptionCheck())
   {
@@ -576,6 +639,37 @@ HDFS_Client_RetCode HdfsClient::hdfsDeletePath( const NAString& delPath)
   return HDFS_CLIENT_OK;
 }
 
+HDFS_Client_RetCode HdfsClient::hdfsListDirectory(const char *pathStr, HDFS_FileInfo **hdfsFileInfo, int *numFiles)
+{
+  QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::hdfsListDirectory(%s) called.", pathStr);
+
+  if (initJNIEnv() != JOI_OK)
+     return HDFS_CLIENT_ERROR_HDFS_LIST_DIR_PARAM;
+
+  jstring js_pathStr = jenv_->NewStringUTF(pathStr);
+  if (js_pathStr == NULL) {
+     jenv_->PopLocalFrame(NULL);
+     return HDFS_CLIENT_ERROR_HDFS_LIST_DIR_PARAM;
+  }
+  jlong jniObj = (long)this;
+  tsRecentJMFromJNI = JavaMethods_[JM_HDFS_LIST_DIRECTORY].jm_full_name;
+  
+  jint retNumFiles = jenv_->CallIntMethod(javaObj_, JavaMethods_[JM_HDFS_LIST_DIRECTORY].methodID, 
+          js_pathStr, jniObj);
+  if (jenv_->ExceptionCheck())
+  {
+    getExceptionDetails();
+    logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsListDirectory()", getLastError());
+    jenv_->PopLocalFrame(NULL);
+    return HDFS_CLIENT_ERROR_HDFS_LIST_DIR_EXCEPTION;
+  } 
+  *numFiles = retNumFiles;
+  *hdfsFileInfo = hdfsFileInfo_;
+  jenv_->PopLocalFrame(NULL);
+  return HDFS_CLIENT_OK;
+}
+
 HDFS_Client_RetCode HdfsClient::hdfsExists( const NAString& uldPath, NABoolean & exist)
 {
   QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::hdfsExists(%s) called.",
@@ -591,10 +685,8 @@ HDFS_Client_RetCode HdfsClient::hdfsExists( const NAString& uldPath, NABoolean &
   }
 
   tsRecentJMFromJNI = JavaMethods_[JM_HDFS_EXISTS].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_HDFS_EXISTS].methodID, js_UldPath);
-
+  jboolean jresult = jenv_->CallStaticBooleanMethod(javaClass_, JavaMethods_[JM_HDFS_EXISTS].methodID, js_UldPath);
   exist = jresult;
-
   if (jenv_->ExceptionCheck())
   {
     getExceptionDetails();
@@ -603,7 +695,69 @@ HDFS_Client_RetCode HdfsClient::hdfsExists( const NAString& uldPath, NABoolean &
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_EXISTS_EXCEPTION;
   } 
-
   jenv_->PopLocalFrame(NULL);
   return HDFS_CLIENT_OK;
 }
+
+HDFS_Client_RetCode HdfsClient::setHdfsFileInfo(JNIEnv *jenv, jint numFiles, jint fileNo, jboolean isDir, 
+          jstring filename, jlong modTime, jlong len, jshort numReplicas, jlong blockSize, 
+          jstring owner, jstring group, jshort permissions, jlong accessTime)
+{
+   HDFS_FileInfo *hdfsFileInfo;
+
+   if (fileNo == 0 && hdfsFileInfo_ != NULL)
+      deleteHdfsFileInfo();
+
+   if (hdfsFileInfo_ == NULL) {
+       hdfsFileInfo_ = new (getHeap()) HDFS_FileInfo[numFiles];
+       numFiles_ = numFiles;
+   }
+
+   if (fileNo >= numFiles_)
+      return HDFS_CLIENT_ERROR_SET_HDFSFILEINFO;
+   hdfsFileInfo = &hdfsFileInfo_[fileNo];
+   if (isDir)
+      hdfsFileInfo->mKind = HDFS_DIRECTORY_KIND;
+   else
+      hdfsFileInfo->mKind = HDFS_FILE_KIND;
+   hdfsFileInfo->mLastMod = modTime;
+   hdfsFileInfo->mSize = len;  
+   hdfsFileInfo->mReplication = numReplicas;
+   hdfsFileInfo->mBlockSize = blockSize;
+   hdfsFileInfo->mPermissions = permissions;
+   hdfsFileInfo->mLastAccess = accessTime;
+   jint tempLen = jenv->GetStringUTFLength(filename);
+   hdfsFileInfo->mName = new (getHeap()) char[tempLen+1];   
+   strncpy(hdfsFileInfo->mName, jenv->GetStringUTFChars(filename, NULL), tempLen);
+   hdfsFileInfo->mName[tempLen] = '\0';
+   tempLen = jenv->GetStringUTFLength(owner);
+   hdfsFileInfo->mOwner = new (getHeap()) char[tempLen+1];   
+   strncpy(hdfsFileInfo->mOwner, jenv->GetStringUTFChars(owner, NULL), tempLen);
+   hdfsFileInfo->mOwner[tempLen] = '\0';
+   tempLen = jenv->GetStringUTFLength(group);
+   hdfsFileInfo->mGroup = new (getHeap()) char[tempLen+1];   
+   strncpy(hdfsFileInfo->mGroup, jenv->GetStringUTFChars(group, NULL), tempLen);
+   hdfsFileInfo->mGroup[tempLen] = '\0';
+   return HDFS_CLIENT_OK;
+}
+
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+jint JNICALL Java_org_trafodion_sql_HDFSClient_sendFileStatus
+  (JNIEnv *jenv, jobject j_obj, jlong hdfsClientJniObj, jint numFiles, jint fileNo, jboolean isDir, 
+          jstring filename, jlong modTime, jlong len, jshort numReplicas, jlong blockSize, 
+          jstring owner, jstring group, jshort permissions, jlong accessTime)
+{
+   HDFS_Client_RetCode retcode;
+   HdfsClient *hdfsClient = (HdfsClient *)hdfsClientJniObj;
+   retcode =  hdfsClient->setHdfsFileInfo(jenv, numFiles, fileNo, isDir, filename, modTime, len, numReplicas, blockSize, owner,
+            group, permissions, accessTime);
+   return (jint) retcode;
+}
+
+#ifdef __cplusplus
+}
+#endif

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/executor/HdfsClient_JNI.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/HdfsClient_JNI.h b/core/sql/executor/HdfsClient_JNI.h
index 5854d59..c45d226 100644
--- a/core/sql/executor/HdfsClient_JNI.h
+++ b/core/sql/executor/HdfsClient_JNI.h
@@ -87,10 +87,31 @@ private:
 // ===========================================================================
 
 typedef enum {
+     HDFS_FILE_KIND
+   , HDFS_DIRECTORY_KIND 
+} HDFS_FileType;
+
+typedef struct {
+   HDFS_FileType mKind; /* file or directory */
+   char *mName;         /* the name of the file */
+   Int64 mLastMod;      /* the last modification time for the file in seconds */
+   Int64 mSize;       /* the size of the file in bytes */
+   short mReplication;    /* the count of replicas */
+   Int64 mBlockSize;  /* the block size for the file */
+   char *mOwner;        /* the owner of the file */
+   char *mGroup;        /* the group associated with the file */
+   short mPermissions;  /* the permissions associated with the file */
+   Int64 mLastAccess;    /* the last access time for the file in seconds */
+} HDFS_FileInfo;
+
+
+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_OPEN_PARAM
+ ,HDFS_CLIENT_ERROR_HDFS_OPEN_EXCEPTION
  ,HDFS_CLIENT_ERROR_HDFS_WRITE_PARAM
  ,HDFS_CLIENT_ERROR_HDFS_WRITE_EXCEPTION
  ,HDFS_CLIENT_ERROR_HDFS_CLOSE_EXCEPTION
@@ -103,6 +124,9 @@ typedef enum {
  ,HDFS_CLIENT_ERROR_HDFS_EXISTS_FILE_EXISTS
  ,HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_PARAM
  ,HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_EXCEPTION
+ ,HDFS_CLIENT_ERROR_SET_HDFSFILEINFO
+ ,HDFS_CLIENT_ERROR_HDFS_LIST_DIR_PARAM
+ ,HDFS_CLIENT_ERROR_HDFS_LIST_DIR_EXCEPTION
  ,HDFS_CLIENT_LAST
 } HDFS_Client_RetCode;
 
@@ -112,8 +136,12 @@ public:
   // Default constructor - for creating a new JVM		
   HdfsClient(NAHeap *heap)
   :  JavaObjectInterface(heap) 
-  {}
-
+    , hdfsFileInfo_(NULL) 
+    , numFiles_(0)
+  {
+  }
+ 
+  ~HdfsClient();
   static HdfsClient *newInstance(NAHeap *heap, HDFS_Client_RetCode &retCode);
 
   // Get the error description.
@@ -123,6 +151,7 @@ public:
   // Must be called.
   HDFS_Client_RetCode    init();
   HDFS_Client_RetCode    hdfsCreate(const char* path, NABoolean compress);
+  HDFS_Client_RetCode    hdfsOpen(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,
@@ -130,20 +159,28 @@ public:
   HDFS_Client_RetCode    hdfsCleanUnloadPath(const NAString& uldPath );
   HDFS_Client_RetCode    hdfsExists(const NAString& uldPath,  NABoolean & exists );
   HDFS_Client_RetCode    hdfsDeletePath(const NAString& delPath);
+  HDFS_Client_RetCode    setHdfsFileInfo(JNIEnv *jenv, jint numFiles, jint fileNo, jboolean isDir, 
+          jstring filename, jlong modTime, jlong len, jshort numReplicas, jlong blockSize, 
+          jstring owner, jstring group, jshort permissions, jlong accessTime);
+  HDFS_Client_RetCode    hdfsListDirectory(const char *pathStr, HDFS_FileInfo **hdfsFileInfo, int *numFiles);
+  void	                 deleteHdfsFileInfo();
 
 private:  
   enum JAVA_METHODS {
     JM_CTOR = 0, 
     JM_HDFS_CREATE,
+    JM_HDFS_OPEN,
     JM_HDFS_WRITE,
     JM_HDFS_CLOSE,
     JM_HDFS_MERGE_FILES,
     JM_HDFS_CLEAN_UNLOAD_PATH,
     JM_HDFS_EXISTS,
     JM_HDFS_DELETE_PATH,
+    JM_HDFS_LIST_DIRECTORY,
     JM_LAST
   };
-  
+  HDFS_FileInfo *hdfsFileInfo_; 
+  int numFiles_;
   static jclass javaClass_;
   static JavaMethodInit* JavaMethods_;
   static bool javaMethodsInitialized_;
@@ -151,4 +188,5 @@ private:
   static pthread_mutex_t javaMethodsInitMutex_;
 };
 
+
 #endif

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/executor/JavaObjectInterface.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/JavaObjectInterface.cpp b/core/sql/executor/JavaObjectInterface.cpp
index ecd8c1e..6919866 100644
--- a/core/sql/executor/JavaObjectInterface.cpp
+++ b/core/sql/executor/JavaObjectInterface.cpp
@@ -476,8 +476,14 @@ JOI_RetCode JavaObjectInterface::init(char *className,
         if (JavaMethods[i].methodID == 0 || jenv_->ExceptionCheck())
         { 
           getExceptionDetails();
-          QRLogger::log(CAT_SQL_HDFS_JNI_TOP, LL_ERROR, "Error in GetMethod(%s).", JavaMethods[i].jm_name);
-          return JOI_ERROR_GETMETHOD;
+          JavaMethods[i].methodID = jenv_->GetStaticMethodID(javaClass, 
+                                                     JavaMethods[i].jm_name, 
+                                                     JavaMethods[i].jm_signature);
+          if (JavaMethods[i].methodID == 0 || jenv_->ExceptionCheck()) {
+             getExceptionDetails();
+             QRLogger::log(CAT_SQL_HDFS_JNI_TOP, LL_ERROR, "Error in GetMethod(%s).", JavaMethods[i].jm_name);
+             return JOI_ERROR_GETMETHOD;
+          }
         }      
       }
     }

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp b/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
index 9bad515..e767cec 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
@@ -68,7 +68,6 @@
 #include "ComMisc.h"
 #include "CmpSeabaseDDLmd.h"
 #include "CmpSeabaseDDLroutine.h"
-#include "hdfs.h"
 #include "StmtDDLAlterLibrary.h"
 #include "logmxevent_traf.h"
 #include "exp_clause_derived.h"

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a99ee11e/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
index 5c8c487..fe116d7 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.conf.Configuration;
 import java.nio.ByteBuffer;
 import java.io.IOException;
@@ -84,6 +85,7 @@ public class HDFSClient
       catch (IOException ioe) {
          throw new RuntimeException("Exception in HDFSClient static block", ioe);
       }
+      System.loadLibrary("executor");
    }
 
    class HDFSRead implements Callable 
@@ -202,6 +204,35 @@ public class HDFSClient
          logger_.debug("HDFSClient.hdfsCreate() - compressed output stream created" );
       return true;
     }
+
+    boolean hdfsOpen(String fname , boolean compress) throws IOException
+    {
+      if (logger_.isDebugEnabled()) 
+         logger_.debug("HDFSClient.hdfsOpen() - started" );
+      Path filePath = null;
+      if (!compress || (compress && fname.endsWith(".gz")))
+        filePath = new Path(fname);
+      else
+        filePath = new Path(fname + ".gz");
+        
+      FileSystem fs = FileSystem.get(filePath.toUri(),config_);
+      FSDataOutputStream fsOut;
+      if (fs.exists(filePath))
+         fsOut = fs.append(filePath);
+      else
+         fsOut = fs.create(filePath);
+      
+      if (compress) {
+        GzipCodec gzipCodec = (GzipCodec) ReflectionUtils.newInstance( GzipCodec.class, config_);
+        Compressor gzipCompressor = CodecPool.getCompressor(gzipCodec);
+        outStream_= gzipCodec.createOutputStream(fsOut, gzipCompressor);
+      }
+      else
+        outStream_ = fsOut;      
+      if (logger_.isDebugEnabled()) 
+         logger_.debug("HDFSClient.hdfsCreate() - compressed output stream created" );
+      return true;
+    }
     
     boolean hdfsWrite(byte[] buff, long len) throws IOException
     {
@@ -225,7 +256,7 @@ public class HDFSClient
     }
 
     
-    public boolean hdfsMergeFiles(String srcPathStr, String dstPathStr) throws IOException
+    public static boolean hdfsMergeFiles(String srcPathStr, String dstPathStr) throws IOException
     {
       if (logger_.isDebugEnabled()) logger_.debug("HDFSClient.hdfsMergeFiles() - start");
       if (logger_.isDebugEnabled()) logger_.debug("HDFSClient.hdfsMergeFiles() - source Path: " + srcPathStr + 
@@ -265,7 +296,7 @@ public class HDFSClient
       return true;
     }
 
-   public boolean hdfsCleanUnloadPath(String uldPathStr
+   public static boolean hdfsCleanUnloadPath(String uldPathStr
                          /*, boolean checkExistence, String mergeFileStr*/) throws IOException
    {
       if (logger_.isDebugEnabled()) 
@@ -289,7 +320,7 @@ public class HDFSClient
       return true;
    }
 
-   public boolean hdfsExists(String filePathStr) throws IOException 
+   public static boolean hdfsExists(String filePathStr) throws IOException 
    {
       if (logger_.isDebugEnabled()) 
          logger_.debug("HDFSClient.hdfsExists() - Path: " + filePathStr);
@@ -301,7 +332,7 @@ public class HDFSClient
       return false;
    }
 
-   public boolean hdfsDeletePath(String pathStr) throws IOException
+   public static boolean hdfsDeletePath(String pathStr) throws IOException
    {
       if (logger_.isDebugEnabled()) 
          logger_.debug("HDFSClient.hdfsDeletePath() - start - Path: " + pathStr);
@@ -310,10 +341,54 @@ public class HDFSClient
       fs.delete(delPath, true);
       return true;
    }
+
+   public int hdfsListDirectory(String pathStr, long hdfsClientJniObj) throws IOException
+   {
+      if (logger_.isDebugEnabled()) 
+         logger_.debug("HDFSClient.hdfsListDirectory() - start - Path: " + pathStr);
+      Path listPath = new Path(pathStr );
+      FileSystem fs = FileSystem.get(listPath.toUri(), config_);
+      FileStatus[] fileStatus;
+      if (fs.isDirectory(listPath)) 
+         fileStatus = fs.listStatus(listPath); 
+      else
+         throw new IOException("The path " + listPath + "is not a directory");
+      FileStatus aFileStatus; 
+      int retcode;
+      if (fileStatus != null) {
+         for (int i = 0; i < fileStatus.length; i++)
+         {
+             aFileStatus = fileStatus[i];
+             retcode = sendFileStatus(hdfsClientJniObj, fileStatus.length, 
+                            i,
+                            aFileStatus.isDirectory(),
+                            aFileStatus.getPath().toString(),
+                            aFileStatus.getModificationTime(),
+                            aFileStatus.getLen(),
+                            aFileStatus.getReplication(),
+                            aFileStatus.getBlockSize(),
+                            aFileStatus.getOwner(),
+                            aFileStatus.getGroup(),
+                            aFileStatus.getPermission().toShort(),
+                            aFileStatus.getAccessTime());          
+             if (retcode != 0)
+                throw new IOException("Error " + retcode + " while sending the file status info for file " + aFileStatus.getPath().toString());
+         }
+         return fileStatus.length;
+      }
+      else  
+         return 0;
+   }
  
    public static void shutdown() throws InterruptedException
    {
       executorService_.awaitTermination(100, TimeUnit.MILLISECONDS);
       executorService_.shutdown();
    }
+   
+   private native int sendFileStatus(long jniObj, int numFiles, int fileNo, boolean isDir, 
+                        String filename, long modTime, long len,
+                        short numReplicas, long blockSize, String owner, String group,
+                        short permissions, long accessTime);
+
 }


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

Posted by se...@apache.org.
[TRAFODION-2917] Refactor Trafodion implementation of hdfs scan for text format hive tables

Part-4 changes

Hive scan with ESPs proceses the correct amount of rows now
Hdfs Scan Statistics are populated

Pending changes:
Hdfs Scan with compression needs to be done.


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

Branch: refs/heads/master
Commit: 7066e3e478d0de7c1af5dfea298e4d6a10fb9596
Parents: 202a040
Author: selvaganesang <se...@esgyn.com>
Authored: Tue Feb 6 04:48:01 2018 +0000
Committer: selvaganesang <se...@esgyn.com>
Committed: Tue Feb 6 04:48:01 2018 +0000

----------------------------------------------------------------------
 core/sql/executor/ExHdfsScan.cpp                | 82 ++++++++++----------
 core/sql/executor/HdfsClient_JNI.cpp            | 59 ++++++++++----
 core/sql/executor/HdfsClient_JNI.h              |  8 +-
 .../main/java/org/trafodion/sql/HdfsScan.java   | 16 ++--
 4 files changed, 103 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/7066e3e4/core/sql/executor/ExHdfsScan.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHdfsScan.cpp b/core/sql/executor/ExHdfsScan.cpp
index 730f0dc..f8ec9a1 100644
--- a/core/sql/executor/ExHdfsScan.cpp
+++ b/core/sql/executor/ExHdfsScan.cpp
@@ -121,6 +121,7 @@ ExHdfsScanTcb::ExHdfsScanTcb(
   , loggingFileName_(NULL)
   , hdfsClient_(NULL)
   , hdfsScan_(NULL)
+  , hdfsStats_(NULL)
   , hdfsFileInfoListAsArray_(glob->getDefaultHeap(), hdfsScanTdb.getHdfsFileInfoList()->numEntries())
 {
   Space * space = (glob ? glob->getSpace() : 0);
@@ -333,12 +334,12 @@ ExOperStats * ExHdfsScanTcb::doAllocateStatsEntry(CollHeap *heap,
   else
      ss = getGlobals()->castToExExeStmtGlobals()->castToExMasterStmtGlobals()->getStatement()->getStmtStats(); 
   
-  ExHdfsScanStats *hdfsScanStats = new(heap) ExHdfsScanStats(heap,
+  hdfsStats_ = new(heap) ExHdfsScanStats(heap,
 				   this,
 				   tdb);
   if (ss != NULL) 
-     hdfsScanStats->setQueryId(ss->getQueryId(), ss->getQueryIdLen());
-  return hdfsScanStats;
+     hdfsStats_->setQueryId(ss->getQueryId(), ss->getQueryIdLen());
+  return hdfsStats_;
 }
 
 void ExHdfsScanTcb::registerSubtasks()
@@ -419,12 +420,6 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	  {
 	    matches_ = 0;
 	    
-	    hdfsStats_ = NULL;
-	    if (getStatsEntry())
-	      hdfsStats_ = getStatsEntry()->castToExHdfsScanStats();
-
-	    ex_assert(hdfsStats_, "hdfs stats cannot be null");
-
 	    beginRangeNum_ = -1;
 	    numRanges_ = -1;
 	    hdfsOffset_ = 0;
@@ -557,7 +552,8 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
              if (hdfsScan_ != NULL)
                 NADELETE(hdfsScan_, HdfsScan, getHeap());
              hdfsScan_ = HdfsScan::newInstance((NAHeap *)getHeap(), hdfsScanBuf_, hdfsScanBufMaxSize_, 
-                            &hdfsFileInfoListAsArray_, hdfsScanTdb().rangeTailIOSize_, hdfsScanRetCode);
+                            &hdfsFileInfoListAsArray_, beginRangeNum_, numRanges_, hdfsScanTdb().rangeTailIOSize_, 
+                            hdfsStats_, hdfsScanRetCode);
              if (hdfsScanRetCode != HDFS_SCAN_OK)
              {
                 setupError(EXE_ERROR_HDFS_SCAN, hdfsScanRetCode, "SETUP_HDFS_SCAN", 
@@ -578,7 +574,7 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
           break;
         case TRAF_HDFS_READ:
           {
-             hdfsScanRetCode = hdfsScan_->trafHdfsRead((NAHeap *)getHeap(), retArray_, sizeof(retArray_)/sizeof(int));
+             hdfsScanRetCode = hdfsScan_->trafHdfsRead((NAHeap *)getHeap(), hdfsStats_, retArray_, sizeof(retArray_)/sizeof(int));
              if (hdfsScanRetCode == HDFS_SCAN_EOR) {
                 step_ = DONE;
                 break;
@@ -589,7 +585,6 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
                 step_ = HANDLE_ERROR_AND_DONE;
                 break;
              } 
-             // Assign the starting address of the buffer
              hdfo = hdfsFileInfoListAsArray_.at(retArray_[RANGE_NO]);
              bufEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED];
              if (retArray_[RANGE_NO] != prevRangeNum_) {  
@@ -608,8 +603,12 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
                 extraBytesRead_ = currRangeBytesRead_ - hdfo->getBytesToRead(); 
              else
                 extraBytesRead_ = 0;
+             // headRoom_ is the number of extra bytes read (rangeTailIOSize)
+             // If EOF is reached while reading the range and the extraBytes read
+             // is less than headRoom_, then process all the data till EOF 
+             if (retArray_[IS_EOF] && extraBytesRead_ < headRoom_)
+                extraBytesRead_ = 0;
              bufLogicalEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED] - extraBytesRead_;
- 
              prevRangeNum_ = retArray_[RANGE_NO];
              headRoomCopied_ = 0;
              if (recordSkip_) {
@@ -633,7 +632,7 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
         case COPY_TAIL_TO_HEAD:
           {
              BYTE *headRoomStartAddr;
-             headRoomCopied_ = bufEnd_ - (BYTE *)hdfsBufNextRow_;
+             headRoomCopied_ = bufLogicalEnd_ - (BYTE *)hdfsBufNextRow_;
              if (retArray_[BUF_NO] == 0)
                 headRoomStartAddr = hdfsScanBuf_[1].buf_ - headRoomCopied_;
              else
@@ -1087,9 +1086,11 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	       hdfsBufNextRow_ = startOfNextRow;
              } 
              else {
-                if ((BYTE *)startOfNextRow >= bufLogicalEnd_) {
+                if ((BYTE *)startOfNextRow > bufLogicalEnd_) {
                    step_ = TRAF_HDFS_READ;
                    hdfsBufNextRow_ = NULL;
+	           if (!exception_)
+	               break;
                 }
                 else
 	          hdfsBufNextRow_ = startOfNextRow;
@@ -1223,8 +1224,12 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
                 if (hdfsScanTdb().continueOnError())
                 {
                   if ((pentry_down->downState.request == ex_queue::GET_N) &&
-                      (pentry_down->downState.requestValue == matches_))
-                    step_ = CLOSE_HDFS_CURSOR;
+                      (pentry_down->downState.requestValue == matches_)) {
+                     if (useLibhdfsScan_)
+                        step_ = CLOSE_HDFS_CURSOR;
+                     else
+                        step_ = DONE;
+                  }
                   else
                     step_ = PROCESS_HDFS_ROW;
 
@@ -1290,8 +1295,12 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	        if (hdfsScanTdb().continueOnError())
 	        {
 	          if ((pentry_down->downState.request == ex_queue::GET_N) &&
-	              (pentry_down->downState.requestValue == matches_))
-	            step_ = CLOSE_FILE;
+	              (pentry_down->downState.requestValue == matches_)) {
+                     if (useLibhdfsScan_)
+                        step_ = CLOSE_HDFS_CURSOR;
+                     else
+                        step_ = DONE;
+                  }
 	          else
 	            step_ = PROCESS_HDFS_ROW;
 
@@ -1332,7 +1341,10 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	          pentry_down->getDiagsArea()->
 	              mergeAfter(*up_entry->getDiagsArea());
 	          up_entry->setDiagsArea(NULL);
-	          step_ = HANDLE_ERROR_WITH_CLOSE;
+                  if (useLibhdfsScan_)
+	             step_ = HANDLE_ERROR_WITH_CLOSE;
+	          else
+	             step_ = HANDLE_ERROR;
 	          break;
 	        }
 	      }
@@ -1551,15 +1563,17 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	      }
 	    up_entry->upState.status = ex_queue::Q_SQLERROR;
 	    qparent_.up->insert();
-	    
-            if (step_ == HANDLE_ERROR_WITH_CLOSE)
-               step_ = CLOSE_HDFS_CURSOR;
-            else if (step_ == HANDLE_ERROR_AND_DONE)
-              step_ = DONE;
-            else
-	       step_ = ERROR_CLOSE_FILE;
+          
+            if (useLibhdfsScan_) {
+               if (step_ == HANDLE_ERROR_WITH_CLOSE)
+                  step_ = CLOSE_HDFS_CURSOR;
+               else if (step_ == HANDLE_ERROR_AND_DONE)
+                  step_ = DONE;
+               else
+	          step_ = ERROR_CLOSE_FILE;
+            } else
+               step_ = DONE;
 	    break;
-           
 	  }
 
 	case CLOSE_FILE:
@@ -2280,12 +2294,6 @@ ExWorkProcRetcode ExOrcScanTcb::work()
 	  {
 	    matches_ = 0;
 	    
-	    hdfsStats_ = NULL;
-	    if (getStatsEntry())
-	      hdfsStats_ = getStatsEntry()->castToExHdfsScanStats();
-
-	    ex_assert(hdfsStats_, "hdfs stats cannot be null");
-
 	    beginRangeNum_ = -1;
 	    numRanges_ = -1;
 
@@ -2629,12 +2637,6 @@ ExWorkProcRetcode ExOrcFastAggrTcb::work()
 	  {
 	    matches_ = 0;
 
-	    hdfsStats_ = NULL;
-	    if (getStatsEntry())
-	      hdfsStats_ = getStatsEntry()->castToExHdfsScanStats();
-            
-	    ex_assert(hdfsStats_, "hdfs stats cannot be null");
-
             orcAggrTdb().getHdfsFileInfoList()->position();
 
             rowCount_ = 0;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/7066e3e4/core/sql/executor/HdfsClient_JNI.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/HdfsClient_JNI.cpp b/core/sql/executor/HdfsClient_JNI.cpp
index 63c4ac1..8f2845a 100644
--- a/core/sql/executor/HdfsClient_JNI.cpp
+++ b/core/sql/executor/HdfsClient_JNI.cpp
@@ -68,7 +68,7 @@ HDFS_Scan_RetCode HdfsScan::init()
     JavaMethods_[JM_CTOR      ].jm_name      = "<init>";
     JavaMethods_[JM_CTOR      ].jm_signature = "()V";
     JavaMethods_[JM_SET_SCAN_RANGES].jm_name      = "setScanRanges";
-    JavaMethods_[JM_SET_SCAN_RANGES].jm_signature = "(Ljava/nio/ByteBuffer;Ljava/nio/ByteBuffer;[Ljava/lang/String;[J[J)V";
+    JavaMethods_[JM_SET_SCAN_RANGES].jm_signature = "(Ljava/nio/ByteBuffer;Ljava/nio/ByteBuffer;[Ljava/lang/String;[J[J[I)V";
     JavaMethods_[JM_TRAF_HDFS_READ].jm_name      = "trafHdfsRead";
     JavaMethods_[JM_TRAF_HDFS_READ].jm_signature = "()[I";
    
@@ -90,7 +90,8 @@ char* HdfsScan::getErrorText(HDFS_Scan_RetCode errEnum)
 
 /////////////////////////////////////////////////////////////////////////////
 HDFS_Scan_RetCode HdfsScan::setScanRanges(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN_BUF *hdfsScanBuf,  int scanBufSize,
-      HdfsFileInfoArray *hdfsFileInfoArray, int rangeTailIOSize)
+      HdfsFileInfoArray *hdfsFileInfoArray, Int32 beginRangeNum, Int32 numRanges, int rangeTailIOSize,
+      ExHdfsScanStats *hdfsStats)
 {
    QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsScan::setScanRanges() called.");
 
@@ -113,12 +114,16 @@ HDFS_Scan_RetCode HdfsScan::setScanRanges(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN
    jobjectArray j_filenames = NULL;
    jlongArray j_offsets = NULL;
    jlongArray j_lens = NULL;  
+   jintArray j_rangenums = NULL;
    HdfsFileInfo *hdfo;
    jstring j_obj;
 
    HDFS_Scan_RetCode hdfsScanRetCode =  HDFS_SCAN_ERROR_SET_SCAN_RANGES_PARAM;
    int arrayLen = hdfsFileInfoArray->entries();
-   for (int i = 0; i < arrayLen; i++) {
+   
+   for (Int32 i = beginRangeNum, rangeCount=0; i < arrayLen; i++, rangeCount++) {
+       if (rangeCount >= numRanges)
+           break;
        hdfo = hdfsFileInfoArray->at(i);
        j_obj = jenv_->NewStringUTF(hdfo->fileName());
        if (jenv_->ExceptionCheck()) {
@@ -126,35 +131,54 @@ HDFS_Scan_RetCode HdfsScan::setScanRanges(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN
           return hdfsScanRetCode;
        }
        if (j_filenames == NULL) {
-          j_filenames = jenv_->NewObjectArray(arrayLen, jenv_->GetObjectClass(j_obj), NULL);
+          j_filenames = jenv_->NewObjectArray(numRanges, jenv_->GetObjectClass(j_obj), NULL);
           if (jenv_->ExceptionCheck()) {
              jenv_->PopLocalFrame(NULL);
              return hdfsScanRetCode;
           }
        }
-       jenv_->SetObjectArrayElement(j_filenames, i, (jobject)j_obj);
+       jenv_->SetObjectArrayElement(j_filenames, rangeCount, (jobject)j_obj);
        jenv_->DeleteLocalRef(j_obj);
+
        if (j_offsets == NULL) {
-          j_offsets = jenv_->NewLongArray(arrayLen);
+          j_offsets = jenv_->NewLongArray(numRanges);
           if (jenv_->ExceptionCheck()) {
              jenv_->PopLocalFrame(NULL);
              return hdfsScanRetCode;
           }
        }
        long offset = hdfo->getStartOffset(); 
-       jenv_->SetLongArrayRegion(j_offsets, i, 1, &offset);
+       jenv_->SetLongArrayRegion(j_offsets, rangeCount, 1, &offset);
+
        if (j_lens == NULL) {
-          j_lens = jenv_->NewLongArray(arrayLen);
+          j_lens = jenv_->NewLongArray(numRanges);
           if (jenv_->ExceptionCheck()) {
              jenv_->PopLocalFrame(NULL);
              return hdfsScanRetCode;
           }
        }
        long len = hdfo->getBytesToRead()+rangeTailIOSize;
-       jenv_->SetLongArrayRegion(j_lens, i, 1, &len);
+       jenv_->SetLongArrayRegion(j_lens, rangeCount, 1, &len);
+
+       if (j_rangenums == NULL) {
+          j_rangenums = jenv_->NewIntArray(numRanges);
+          if (jenv_->ExceptionCheck()) {
+             jenv_->PopLocalFrame(NULL);
+             return hdfsScanRetCode;
+          }
+       }
+       jint tdbRangeNum = i;
+       jenv_->SetIntArrayRegion(j_rangenums, rangeCount, 1, &tdbRangeNum);
    } 
 
-   jenv_->CallVoidMethod(javaObj_, JavaMethods_[JM_SET_SCAN_RANGES].methodID, j_buf1, j_buf2, j_filenames, j_offsets, j_lens);
+   if (hdfsStats)
+       hdfsStats->getHdfsTimer().start();
+   tsRecentJMFromJNI = JavaMethods_[JM_SET_SCAN_RANGES].jm_full_name;
+   jenv_->CallVoidMethod(javaObj_, JavaMethods_[JM_SET_SCAN_RANGES].methodID, j_buf1, j_buf2, j_filenames, j_offsets, j_lens, j_rangenums);
+   if (hdfsStats) {
+      hdfsStats->incMaxHdfsIOTime(hdfsStats->getHdfsTimer().stop());
+      hdfsStats->incHdfsCalls();
+   }
 
    if (jenv_->ExceptionCheck()) {
       getExceptionDetails();
@@ -167,7 +191,8 @@ HDFS_Scan_RetCode HdfsScan::setScanRanges(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN
 }
 
 HdfsScan *HdfsScan::newInstance(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN_BUF *hdfsScanBuf,  int scanBufSize,
-      HdfsFileInfoArray *hdfsFileInfoArray, int rangeTailIOSize, HDFS_Scan_RetCode &hdfsScanRetCode)
+      HdfsFileInfoArray *hdfsFileInfoArray, Int32 beginRangeNum, Int32 numRanges, int rangeTailIOSize, 
+      ExHdfsScanStats *hdfsStats, HDFS_Scan_RetCode &hdfsScanRetCode)
 {
    QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsScan::newInstance() called.");
 
@@ -179,7 +204,7 @@ HdfsScan *HdfsScan::newInstance(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN_BUF *hdfs
        hdfsScanRetCode = hdfsScan->init();
        if (hdfsScanRetCode == HDFS_SCAN_OK) 
           hdfsScanRetCode = hdfsScan->setScanRanges(heap, hdfsScanBuf, scanBufSize, 
-                    hdfsFileInfoArray, rangeTailIOSize); 
+                    hdfsFileInfoArray, beginRangeNum, numRanges, rangeTailIOSize, hdfsStats); 
        if (hdfsScanRetCode != HDFS_SCAN_OK) {
           NADELETE(hdfsScan, HdfsScan, heap);
           hdfsScan = NULL;
@@ -189,14 +214,22 @@ HdfsScan *HdfsScan::newInstance(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN_BUF *hdfs
 }
 
 
-HDFS_Scan_RetCode HdfsScan::trafHdfsRead(NAHeap *heap, int retArray[], short arrayLen)
+HDFS_Scan_RetCode HdfsScan::trafHdfsRead(NAHeap *heap, ExHdfsScanStats *hdfsStats, int retArray[], short arrayLen)
 {
    QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsScan::trafHdfsRead() called.");
 
    if (initJNIEnv() != JOI_OK)
      return HDFS_SCAN_ERROR_TRAF_HDFS_READ_PARAM;
 
+   if (hdfsStats)
+       hdfsStats->getHdfsTimer().start();
+   tsRecentJMFromJNI = JavaMethods_[JM_TRAF_HDFS_READ].jm_full_name;
    jintArray j_retArray = (jintArray)jenv_->CallObjectMethod(javaObj_, JavaMethods_[JM_TRAF_HDFS_READ].methodID);
+   if (hdfsStats) {
+      hdfsStats->incMaxHdfsIOTime(hdfsStats->getHdfsTimer().stop());
+      hdfsStats->incHdfsCalls();
+   }
+
    if (jenv_->ExceptionCheck()) {
       getExceptionDetails();
       logError(CAT_SQL_HDFS, __FILE__, __LINE__);

http://git-wip-us.apache.org/repos/asf/trafodion/blob/7066e3e4/core/sql/executor/HdfsClient_JNI.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/HdfsClient_JNI.h b/core/sql/executor/HdfsClient_JNI.h
index 0426ebc..5854d59 100644
--- a/core/sql/executor/HdfsClient_JNI.h
+++ b/core/sql/executor/HdfsClient_JNI.h
@@ -58,12 +58,14 @@ public:
   static char* getErrorText(HDFS_Scan_RetCode errEnum);
 
   static HdfsScan *newInstance(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN_BUF *hdfsScanBuf, int scanBufSize, 
-            HdfsFileInfoArray *hdfsFileInfoArray, int rangeTailIOSize, HDFS_Scan_RetCode &hdfsScanRetCode);
+            HdfsFileInfoArray *hdfsFileInfoArray, Int32 beginRangeNum, Int32 numRanges, int rangeTailIOSize,
+            ExHdfsScanStats *hdfsStats, HDFS_Scan_RetCode &hdfsScanRetCode);
 
   HDFS_Scan_RetCode setScanRanges(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN_BUF *hdfsScanBuf, int scanBufSize, 
-            HdfsFileInfoArray *hdfsFileInfoArray, int rangeTailIOSize);
+            HdfsFileInfoArray *hdfsFileInfoArray, Int32 beginRangeNum, Int32 numRanges, 
+            int rangeTailIOSize, ExHdfsScanStats *hdfsStats);
 
-  HDFS_Scan_RetCode trafHdfsRead(NAHeap *heap, int retArray[], short arrayLen);
+  HDFS_Scan_RetCode trafHdfsRead(NAHeap *heap, ExHdfsScanStats *hdfsStats, int retArray[], short arrayLen);
 
 private:
   enum JAVA_METHODS {

http://git-wip-us.apache.org/repos/asf/trafodion/blob/7066e3e4/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
index 73ceda8..f3d505d 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
@@ -71,12 +71,14 @@ public class HdfsScan
       String filename_;
       long pos_;
       long len_;
+      int tdbRangeNum_;
       
-      HdfsScanRange(String filename, long pos, long len)
+      HdfsScanRange(String filename, long pos, long len, int tdbRangeNum)
       {
          filename_ = filename;
          pos_ = pos;
          len_ = len;
+         tdbRangeNum_ = tdbRangeNum;
       }
    }
    
@@ -91,7 +93,7 @@ public class HdfsScan
    {
    }
 
-   public void setScanRanges(ByteBuffer buf1, ByteBuffer buf2, String filename[], long pos[], long len[]) throws IOException
+   public void setScanRanges(ByteBuffer buf1, ByteBuffer buf2, String filename[], long pos[], long len[], int rangeNum[]) throws IOException
    {
       buf_ = new ByteBuffer[2];
       bufLen_ = new int[2];
@@ -108,7 +110,7 @@ public class HdfsScan
       hdfsClient_ = new HDFSClient[2];
       hdfsScanRanges_ = new HdfsScanRange[filename.length]; 
       for (int i = 0; i < filename.length; i++) {
-         hdfsScanRanges_[i] = new HdfsScanRange(filename[i], pos[i], len[i]);
+         hdfsScanRanges_[i] = new HdfsScanRange(filename[i], pos[i], len[i], rangeNum[i]);
       }
       if (hdfsScanRanges_.length > 0) {
          currRange_ = 0;
@@ -141,8 +143,8 @@ public class HdfsScan
          readLength = (int)lenRemain_;
       if (! scanCompleted_) {
          if (logger_.isDebugEnabled())
-            logger_.debug(" CurrentRange " + currRange_ + " LenRemain " + lenRemain_ + " BufNo " + bufNo); 
-         hdfsClient_[bufNo] = new HDFSClient(bufNo, currRange_, hdfsScanRanges_[currRange_].filename_, buf_[bufNo], currPos_, readLength);
+            logger_.debug(" CurrentRange " + hdfsScanRanges_[currRange_].tdbRangeNum_ + " LenRemain " + lenRemain_ + " BufNo " + bufNo); 
+         hdfsClient_[bufNo] = new HDFSClient(bufNo, hdfsScanRanges_[currRange_].tdbRangeNum_, hdfsScanRanges_[currRange_].filename_, buf_[bufNo], currPos_, readLength);
       }
    } 
    
@@ -233,6 +235,7 @@ public class HdfsScan
       String fileName[] = new String[file_status.length * split];
       long pos[] = new long[file_status.length * split];
       long len[] = new long[file_status.length * split];
+      int range[] = new int[file_status.length * split];
       for (int i = 0 ; i < file_status.length * split; i++) {
          Path filePath = file_status[i].getPath();
          long fileLen = file_status[i].getLen(); 
@@ -245,6 +248,7 @@ public class HdfsScan
             fileName[i] = filePath.toString();
             pos[i] = splitPos + (splitLen * j);
             len[i] = splitLen;
+            range[i] = i;
             if (j == (split-1))
                len[i] = fileLen - (splitLen *(j));
             System.out.println ("Range " + i + " Pos " + pos[i] + " Length " + len[i]); 
@@ -253,7 +257,7 @@ public class HdfsScan
       }
       long time1 = System.currentTimeMillis();
       HdfsScan hdfsScan = new HdfsScan();
-      hdfsScan.setScanRanges(buf1, buf2, fileName, pos, len);
+      hdfsScan.setScanRanges(buf1, buf2, fileName, pos, len, range);
       int[] retArray;
       int bytesCompleted;
       ByteBuffer buf;


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

Posted by se...@apache.org.
[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 \


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

Posted by se...@apache.org.
[TRAFODION-2917] Refactor Trafodion implementation of hdfs scan for text format hive tables

Part-3 changes

Changes to ensure the multiple chunk and ranges of hdfs scan
work fine with refactored code.

Pending issues:
Statistics needs to be populated.
ESP should be assigned the ranges in advance to avoid duplicate scans


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

Branch: refs/heads/master
Commit: 202a040ea9eb8ae3ea79329ee14048c4fe2f082c
Parents: f17e15e
Author: selvaganesang <se...@esgyn.com>
Authored: Fri Feb 2 07:46:48 2018 +0000
Committer: selvaganesang <se...@esgyn.com>
Committed: Fri Feb 2 07:46:48 2018 +0000

----------------------------------------------------------------------
 core/sql/executor/ExHdfsScan.cpp                | 54 ++++++++----
 core/sql/executor/ExHdfsScan.h                  |  5 +-
 .../main/java/org/trafodion/sql/HDFSClient.java | 86 +++++++-------------
 .../main/java/org/trafodion/sql/HdfsScan.java   | 69 +++++++++-------
 4 files changed, 113 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/202a040e/core/sql/executor/ExHdfsScan.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHdfsScan.cpp b/core/sql/executor/ExHdfsScan.cpp
index e29baf6..730f0dc 100644
--- a/core/sql/executor/ExHdfsScan.cpp
+++ b/core/sql/executor/ExHdfsScan.cpp
@@ -143,7 +143,7 @@ ExHdfsScanTcb::ExHdfsScanTcb(
      }
      bufBegin_ = NULL;
      bufEnd_ = NULL;
-     logicalBufEnd_ = NULL;
+     bufLogicalEnd_ = NULL;
      headRoomCopied_ = 0;
      prevRangeNum_ = -1;
      currRangeBytesRead_ = 0;
@@ -567,11 +567,12 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
              } 
              bufBegin_ = NULL;
              bufEnd_ = NULL;
-             logicalBufEnd_ = NULL;
+             bufLogicalEnd_ = NULL;
              headRoomCopied_ = 0;
              prevRangeNum_ = -1;                         
              currRangeBytesRead_ = 0;                   
              recordSkip_ = FALSE;
+             extraBytesRead_ = 0;
              step_ = TRAF_HDFS_READ;
           } 
           break;
@@ -589,25 +590,28 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
                 break;
              } 
              // Assign the starting address of the buffer
+             hdfo = hdfsFileInfoListAsArray_.at(retArray_[RANGE_NO]);
              bufEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED];
-             if (retArray_[IS_EOF])
-                logicalBufEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED];
-             else if (retArray_[BYTES_COMPLETED] < hdfsScanBufMaxSize_) 
-                logicalBufEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED] - headRoom_;   
-             else
-                logicalBufEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED];
-             hdfo_ = getRange(retArray_[RANGE_NO]);
              if (retArray_[RANGE_NO] != prevRangeNum_) {  
+                currRangeBytesRead_ = retArray_[BYTES_COMPLETED];
                 bufBegin_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_;
-                if (hdfo_->getStartOffset() == 0)
+                if (hdfo->getStartOffset() == 0)
                    recordSkip_ = FALSE;
                 else
                    recordSkip_ = TRUE; 
              } else {
+                currRangeBytesRead_ += retArray_[BYTES_COMPLETED];
                 bufBegin_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ - headRoomCopied_;
                 recordSkip_ = FALSE;
-             } 
+             }
+             if (currRangeBytesRead_ > hdfo->getBytesToRead())
+                extraBytesRead_ = currRangeBytesRead_ - hdfo->getBytesToRead(); 
+             else
+                extraBytesRead_ = 0;
+             bufLogicalEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED] - extraBytesRead_;
+ 
              prevRangeNum_ = retArray_[RANGE_NO];
+             headRoomCopied_ = 0;
              if (recordSkip_) {
 		hdfsBufNextRow_ = hdfs_strchr((char *)bufBegin_,
 			      hdfsScanTdb().recordDelimiter_, 
@@ -628,6 +632,13 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
           break;
         case COPY_TAIL_TO_HEAD:
           {
+             BYTE *headRoomStartAddr;
+             headRoomCopied_ = bufEnd_ - (BYTE *)hdfsBufNextRow_;
+             if (retArray_[BUF_NO] == 0)
+                headRoomStartAddr = hdfsScanBuf_[1].buf_ - headRoomCopied_;
+             else
+                headRoomStartAddr = hdfsScanBuf_[0].buf_ - headRoomCopied_;
+             memcpy(headRoomStartAddr, hdfsBufNextRow_, headRoomCopied_);
              step_ = TRAF_HDFS_READ;  
           }
           break;
@@ -1023,6 +1034,10 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 
 	case PROCESS_HDFS_ROW:
 	{
+          if (!useLibhdfsScan_ && hdfsBufNextRow_ == NULL) {
+             step_ = TRAF_HDFS_READ;
+             break;
+          } 
 	  exception_ = FALSE;
 	  nextStep_ = NOT_STARTED;
 	  debugPenultimatePrevRow_ = debugPrevRow_;
@@ -1066,9 +1081,20 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	  }
 	  else
 	  {
-	    numBytesProcessedInRange_ +=
+            if (useLibhdfsScan_) {
+	       numBytesProcessedInRange_ +=
 	        startOfNextRow - hdfsBufNextRow_;
-	    hdfsBufNextRow_ = startOfNextRow;
+	       hdfsBufNextRow_ = startOfNextRow;
+             } 
+             else {
+                if ((BYTE *)startOfNextRow >= bufLogicalEnd_) {
+                   step_ = TRAF_HDFS_READ;
+                   hdfsBufNextRow_ = NULL;
+                }
+                else
+	          hdfsBufNextRow_ = startOfNextRow;
+             }
+           
 	  }
 
 	  if (exception_)
@@ -1691,7 +1717,7 @@ char * ExHdfsScanTcb::extractAndTransformAsciiSourceToSqlRow(int &err,
   }
   else {
      sourceDataEnd = (const char *)bufEnd_;
-     endOfRequestedRange = (const char *)logicalBufEnd_;
+     endOfRequestedRange = NULL;
   }
   hdfsLoggingRow_ = hdfsBufNextRow_;
   if (asciiSourceTD->numAttrs() == 0)

http://git-wip-us.apache.org/repos/asf/trafodion/blob/202a040e/core/sql/executor/ExHdfsScan.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHdfsScan.h b/core/sql/executor/ExHdfsScan.h
index 62bb11e..2570a58 100644
--- a/core/sql/executor/ExHdfsScan.h
+++ b/core/sql/executor/ExHdfsScan.h
@@ -119,7 +119,7 @@ private:
          a) filename
          b) offset
          c) len
-      Java layer always reads more than the len by rangeTailIOSize_ to accomdate the record split 
+      Java layer always reads more than the len by rangeTailIOSize_ to accommdate the record split 
    2. Two ByteBuffer objects are also passsed to HdfsScan object. These ByteBuffers are backed up by
       2 native buffers where the data is fetched. The buffer has a head room of size rangeTailIOSize_ and the 
       data is always read after the head room. 
@@ -347,11 +347,12 @@ protected:
   int retArray_[4];
   BYTE *bufBegin_;
   BYTE *bufEnd_;
-  BYTE *logicalBufEnd_;
+  BYTE *bufLogicalEnd_;
   long currRangeBytesRead_;
   int headRoomCopied_;
   int headRoom_;
   int prevRangeNum_;
+  int extraBytesRead_;
   NABoolean recordSkip_;
 };
 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/202a040e/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
index 1af2c49..3b83c8f 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
@@ -87,31 +87,45 @@ public class HDFSClient
 
    class HDFSRead implements Callable 
    {
-      int length_;
-
-      HDFSRead(int length) 
+      HDFSRead() 
       {
-         length_ = length;
       }
  
       public Object call() throws IOException 
       {
          int bytesRead;
-         if (buf_.hasArray())
-            bytesRead = fsdis_.read(pos_, buf_.array(), bufOffset_, length_);
-         else
+         int totalBytesRead = 0;
+         if (! buf_.hasArray())
+            fsdis_.seek(pos_);
+         do
          {
-            buf_.limit(bufOffset_ + length_);
-            bytesRead = fsdis_.read(buf_);
-         }
-         return new Integer(bytesRead);
+            if (buf_.hasArray())
+               bytesRead = fsdis_.read(pos_, buf_.array(), bufOffset_, lenRemain_);
+            else 
+               bytesRead = fsdis_.read(buf_);
+            if (bytesRead == -1) {
+               isEOF_ = 1;
+               break;
+            }
+            if (bytesRead == 0)
+               break;
+            totalBytesRead += bytesRead;          
+            if (totalBytesRead == bufLen_)
+                break;
+            bufOffset_ += bytesRead;
+            pos_ += bytesRead;
+            lenRemain_ -= bytesRead;
+         } while (lenRemain_ > 0);
+         return new Integer(totalBytesRead);
       }
    }
        
    public HDFSClient() 
    {
    }
- 
+
+   // This constructor enables the hdfs data to be read in another thread while the previously 
+   // read buffer is being processed by the SQL engine 
    public HDFSClient(int bufNo, int rangeNo, String filename, ByteBuffer buffer, long position, int length) throws IOException
    {
       bufNo_ = bufNo; 
@@ -127,44 +141,24 @@ public class HDFSClient
       len_ = length;
       if (buffer.hasArray()) 
          bufLen_ = buffer.array().length;
-      else
-      {
+      else {
          bufLen_ = buffer.capacity();
          buf_.position(0);
       }
       lenRemain_ = (len_ > bufLen_) ? bufLen_ : len_;
-      if (lenRemain_ != 0)
-      {
-         int readLength = (lenRemain_ > blockSize_) ? blockSize_ : lenRemain_;
-         future_ = executorService_.submit(new HDFSRead(readLength));
+      if (lenRemain_ != 0) {
+         future_ = executorService_.submit(new HDFSRead());
       }
    }
 
-   public int trafHdfsRead() throws IOException, InterruptedException, ExecutionException
+   public int trafHdfsReadBuffer() throws IOException, InterruptedException, ExecutionException
    {
       Integer retObject = 0;
       int bytesRead;
-      int readLength;
-       
-      if (lenRemain_ == 0)
-         return 0;
       retObject = (Integer)future_.get();
       bytesRead = retObject.intValue();
-      if (bytesRead == -1)
-         return -1;
-      bufOffset_ += bytesRead;
-      pos_ += bytesRead;
-      lenRemain_ -= bytesRead;
-      if (bufOffset_ == bufLen_)
-         return bytesRead; 
-      else if (bufOffset_ > bufLen_)
-         throw new IOException("Internal Error in trafHdfsRead ");
-      if (lenRemain_ == 0)
-         return bytesRead; 
-      readLength = (lenRemain_ > blockSize_) ? blockSize_ : lenRemain_;
-      future_ = executorService_.submit(new HDFSRead(readLength));
       return bytesRead;
-   } 
+   }
 
    public int getRangeNo()
    {
@@ -176,24 +170,6 @@ public class HDFSClient
       return isEOF_;
    }
 
-   public int trafHdfsReadBuffer() throws IOException, InterruptedException, ExecutionException
-   {
-      int bytesRead;
-      int totalBytesRead = 0;
-      while (true) {
-         bytesRead = trafHdfsRead();
-         if (bytesRead == -1) {
-            isEOF_ = 1;
-            return totalBytesRead;
-         }
-         if (bytesRead == 0)
-            return totalBytesRead;
-         totalBytesRead += bytesRead;
-         if (totalBytesRead == bufLen_)
-              return totalBytesRead;
-      }  
-   } 
-
    boolean hdfsCreate(String fname , boolean compress) throws IOException
    {
      if (logger_.isDebugEnabled()) 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/202a040e/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
index 9fb145e..73ceda8 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
@@ -65,7 +65,6 @@ public class HdfsScan
    private long lenRemain_;
    private int lastBufCompleted_ = -1;
    private boolean scanCompleted_;
-   private boolean lastScanRangeScheduled_;
    
    class HdfsScanRange 
    {
@@ -113,41 +112,44 @@ public class HdfsScan
       }
       if (hdfsScanRanges_.length > 0) {
          currRange_ = 0;
-         currPos_ = hdfsScanRanges_[0].pos_;
-         lenRemain_ = hdfsScanRanges_[0].len_;
-         hdfsScanRange(0);
+         currPos_ = hdfsScanRanges_[currRange_].pos_;
+         lenRemain_ = hdfsScanRanges_[currRange_].len_; 
+         hdfsScanRange(0, 0);
       }
       scanCompleted_ = false;
-      lastScanRangeScheduled_ = false;
    }
 
-   public void hdfsScanRange(int bufNo) throws IOException
+   public void hdfsScanRange(int bufNo, int bytesCompleted) throws IOException
    {
-      if (logger_.isDebugEnabled())
-         logger_.debug(" CurrentRange " + currRange_ + " LenRemain " + lenRemain_ + " BufNo " + bufNo); 
+      lenRemain_ -= bytesCompleted;
+      currPos_ += bytesCompleted; 
       int readLength;
-      if (lenRemain_ > bufLen_[bufNo])
-         readLength = bufLen_[bufNo];
-      else
-         readLength = (int)lenRemain_;
-      hdfsClient_[bufNo] = new HDFSClient(bufNo, currRange_, hdfsScanRanges_[currRange_].filename_, buf_[bufNo], currPos_, readLength);
-      lenRemain_ -= readLength;
-      currPos_ += readLength; 
-      if (lenRemain_ == 0) {
-         if (currRange_  == (hdfsScanRanges_.length-1)) 
-            lastScanRangeScheduled_ = true;
+      if (lenRemain_ <= 0) {
+         if (currRange_  == (hdfsScanRanges_.length-1)) {
+            scanCompleted_ = true;
+            return;
+         }
          else {
             currRange_++;
             currPos_ = hdfsScanRanges_[currRange_].pos_;
             lenRemain_ = hdfsScanRanges_[currRange_].len_; 
          }
       } 
+      if (lenRemain_ > bufLen_[bufNo])
+         readLength = bufLen_[bufNo];
+      else
+         readLength = (int)lenRemain_;
+      if (! scanCompleted_) {
+         if (logger_.isDebugEnabled())
+            logger_.debug(" CurrentRange " + currRange_ + " LenRemain " + lenRemain_ + " BufNo " + bufNo); 
+         hdfsClient_[bufNo] = new HDFSClient(bufNo, currRange_, hdfsScanRanges_[currRange_].filename_, buf_[bufNo], currPos_, readLength);
+      }
    } 
    
    public int[] trafHdfsRead() throws IOException, InterruptedException, ExecutionException
    {
       int[] retArray;
-      int byteCompleted;
+      int bytesRead;
       int bufNo;
       int rangeNo;
       int isEOF;
@@ -160,44 +162,41 @@ public class HdfsScan
       switch (lastBufCompleted_) {
          case -1:
          case 1:
-            byteCompleted = hdfsClient_[0].trafHdfsReadBuffer(); 
+            bytesRead = hdfsClient_[0].trafHdfsReadBuffer(); 
             bufNo = 0;
             rangeNo = hdfsClient_[0].getRangeNo();
             isEOF = hdfsClient_[0].isEOF();
             break;
          case 0:
-            byteCompleted = hdfsClient_[1].trafHdfsReadBuffer(); 
+            bytesRead = hdfsClient_[1].trafHdfsReadBuffer(); 
             bufNo = 1;
             rangeNo = hdfsClient_[1].getRangeNo();
             isEOF = hdfsClient_[1].isEOF();
             break;
          default:
             bufNo = -1;
-            byteCompleted = -1;
+            bytesRead = -1;
             rangeNo = -1;
             isEOF = 0;
       }    
-      lastBufCompleted_ = bufNo;
-      retArray[0] = byteCompleted;
+      retArray[0] = bytesRead;
       retArray[1] = bufNo;
       retArray[2] = rangeNo; 
       retArray[3] = isEOF;
       if (logger_.isDebugEnabled())
          logger_.debug(" Range No " + retArray[2] + " Buffer No " + retArray[1] + " Bytes Read " + retArray[0] + " isEOF " + retArray[3]); 
       lastBufCompleted_ = bufNo;
-      if ((isEOF == 1) && (currRange_ == (hdfsScanRanges_.length-1))) 
-         lastScanRangeScheduled_ = true;
-      if (lastScanRangeScheduled_) {
+      if ((isEOF == 1) && (currRange_ == (hdfsScanRanges_.length-1))) {
          scanCompleted_ = true;
-         return retArray; 
+         return retArray;
       }
       switch (lastBufCompleted_)
       {
          case 0:
-            hdfsScanRange(1);
+            hdfsScanRange(1, bytesRead);
             break;
          case 1:
-            hdfsScanRange(0);
+            hdfsScanRange(0, bytesRead);
             break;            
          default:
             break;
@@ -257,10 +256,20 @@ public class HdfsScan
       hdfsScan.setScanRanges(buf1, buf2, fileName, pos, len);
       int[] retArray;
       int bytesCompleted;
+      ByteBuffer buf;
       while (true) {
          retArray = hdfsScan.trafHdfsRead();
          if (retArray == null)
             break;
+         System.out.println("Range No:" + retArray[2] + " Buf No:" + retArray[1] + " Bytes Completed:" + retArray[0] + " EOF:" + retArray[3]);
+         if (retArray[1] == 0)
+            buf = buf1;
+         else
+            buf = buf2; 
+         buf.position(0);
+         for (int i = 0; i < 50; i++)
+           System.out.print(buf.get());
+         System.out.println("");
       }
       long time2 = System.currentTimeMillis();
       HdfsScan.shutdown();


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

Posted by se...@apache.org.
[TRAFODION-2917] Refactor Trafodion implementation of hdfs scan for text formatted hive tables

Part-2 changes

Introduced a new CQD USE_LIBHDFS_SCAN 'OFF' to switch to the new implementation
The new implementation details are at executor/ExHdfsScan.h

Fixed a bug that was causing unexpected errors in JVM when the JNI object corresponding
to java class is not initialized correctly


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

Branch: refs/heads/master
Commit: f17e15eed741a40a41eec6a1a206dd661589623c
Parents: 60db153
Author: selvaganesang <se...@esgyn.com>
Authored: Wed Jan 31 19:48:49 2018 +0000
Committer: selvaganesang <se...@esgyn.com>
Committed: Wed Jan 31 19:48:49 2018 +0000

----------------------------------------------------------------------
 core/sql/comexe/ComTdbHdfsScan.h                |  10 +-
 core/sql/executor/ExExeUtilGet.cpp              |  16 +-
 core/sql/executor/ExExeUtilLoad.cpp             |  10 +-
 core/sql/executor/ExFastTransport.cpp           |   2 -
 core/sql/executor/ExFastTransport.h             |   2 +-
 core/sql/executor/ExHbaseAccess.cpp             |  10 +-
 core/sql/executor/ExHbaseIUD.cpp                |   5 -
 core/sql/executor/ExHdfsScan.cpp                | 185 ++++++++++++++---
 core/sql/executor/ExHdfsScan.h                  |  56 ++++++
 core/sql/executor/HBaseClient_JNI.cpp           |  21 +-
 core/sql/executor/HBaseClient_JNI.h             |  12 +-
 core/sql/executor/HdfsClient_JNI.cpp            | 198 +++++++++++++++----
 core/sql/executor/HdfsClient_JNI.h              |  28 +--
 core/sql/executor/JavaObjectInterface.cpp       |  14 +-
 core/sql/executor/JavaObjectInterface.h         |  24 +--
 core/sql/executor/OrcFileReader.cpp             |   3 +-
 core/sql/executor/SequenceFileReader.cpp        |   6 +-
 core/sql/exp/ExpErrorEnums.h                    |   1 +
 core/sql/exp/ExpHbaseInterface.cpp              |  20 +-
 core/sql/exp/ExpHbaseInterface.h                |  12 +-
 core/sql/exp/ExpLOBinterface.h                  |  13 --
 core/sql/generator/GenRelScan.cpp               |   3 +
 core/sql/qmscommon/QRLogger.cpp                 |   1 +
 core/sql/qmscommon/QRLogger.h                   |   1 +
 core/sql/sqlcomp/DefaultConstants.h             |   3 +
 core/sql/sqlcomp/nadefaults.cpp                 |   2 +
 .../main/java/org/trafodion/sql/HDFSClient.java |  22 ++-
 .../main/java/org/trafodion/sql/HdfsScan.java   |  35 +++-
 28 files changed, 511 insertions(+), 204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/comexe/ComTdbHdfsScan.h
----------------------------------------------------------------------
diff --git a/core/sql/comexe/ComTdbHdfsScan.h b/core/sql/comexe/ComTdbHdfsScan.h
index 1d65bca..ff692c9 100755
--- a/core/sql/comexe/ComTdbHdfsScan.h
+++ b/core/sql/comexe/ComTdbHdfsScan.h
@@ -24,7 +24,7 @@
 #define COM_HDFS_SCAN_H
 
 #include "ComTdb.h"
-//#include "hdfs.h"  // tPort 
+//#include "hdfs.h"   
 #include "ExpLOBinterface.h"
 #include "ComQueue.h"
 
@@ -54,7 +54,8 @@ class ComTdbHdfsScan : public ComTdb
     // ignore conversion errors and continue reading the next row.
     CONTINUE_ON_ERROR           = 0x0020,
     LOG_ERROR_ROWS              = 0x0040,
-    ASSIGN_RANGES_AT_RUNTIME    = 0x0080
+    ASSIGN_RANGES_AT_RUNTIME    = 0x0080,
+    USE_LIBHDFS_SCAN            = 0x0100
   };
 
   // Expression to filter rows.
@@ -284,6 +285,11 @@ public:
   {(v ? flags_ |= ASSIGN_RANGES_AT_RUNTIME : flags_ &= ~ASSIGN_RANGES_AT_RUNTIME); }
   NABoolean getAssignRangesAtRuntime() const
                                 { return (flags_ & ASSIGN_RANGES_AT_RUNTIME) != 0; }
+
+  void setUseLibhdfsScan(NABoolean v)
+  {(v ? flags_ |= USE_LIBHDFS_SCAN : flags_ &= ~USE_LIBHDFS_SCAN); }
+  NABoolean getUseLibhdfsScan() const
+                                { return (flags_ & USE_LIBHDFS_SCAN) != 0; }
   
   UInt32 getMaxErrorRows() const { return maxErrorRows_;}
   void setMaxErrorRows(UInt32 v ) { maxErrorRows_= v; }

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/ExExeUtilGet.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExExeUtilGet.cpp b/core/sql/executor/ExExeUtilGet.cpp
index 6d675cb..539a8cf 100644
--- a/core/sql/executor/ExExeUtilGet.cpp
+++ b/core/sql/executor/ExExeUtilGet.cpp
@@ -3521,13 +3521,9 @@ ExExeUtilGetHbaseObjectsTcb::ExExeUtilGetHbaseObjectsTcb(
      ex_globals * glob)
      : ExExeUtilGetMetadataInfoTcb( exe_util_tdb, glob)
 {
-  int jniDebugPort = 0;
-  int jniDebugTimeout = 0;
   ehi_ = ExpHbaseInterface::newInstance(glob->getDefaultHeap(),
 					(char*)exe_util_tdb.server(), 
-					(char*)exe_util_tdb.zkPort(),
-                                        jniDebugPort,
-                                        jniDebugTimeout);
+					(char*)exe_util_tdb.zkPort());
 
   hbaseName_ = NULL;
   hbaseNameBuf_ = new(getGlobals()->getDefaultHeap()) 
@@ -6106,9 +6102,7 @@ ExExeUtilRegionStatsTcb::ExExeUtilRegionStatsTcb(
   int jniDebugTimeout = 0;
   ehi_ = ExpHbaseInterface::newInstance(glob->getDefaultHeap(),
 					(char*)"", //exe_util_tdb.server(), 
-					(char*)"", //exe_util_tdb.zkPort(),
-                                        jniDebugPort,
-                                        jniDebugTimeout);
+					(char*)""); //exe_util_tdb.zkPort(),
 
   regionInfoList_ = NULL;
   
@@ -6879,13 +6873,9 @@ ExExeUtilClusterStatsTcb::ExExeUtilClusterStatsTcb(
 
   stats_ = (ComTdbClusterStatsVirtTableColumnStruct*)statsBuf_;
 
-  int jniDebugPort = 0;
-  int jniDebugTimeout = 0;
   ehi_ = ExpHbaseInterface::newInstance(glob->getDefaultHeap(),
 					(char*)"", //exe_util_tdb.server(), 
-					(char*)"", //exe_util_tdb.zkPort(),
-                                        jniDebugPort,
-                                        jniDebugTimeout);
+					(char*)""); //exe_util_tdb.zkPort());
 
   regionInfoList_ = NULL;
   

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/ExExeUtilLoad.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExExeUtilLoad.cpp b/core/sql/executor/ExExeUtilLoad.cpp
index 819b3b1..0ebc65c 100644
--- a/core/sql/executor/ExExeUtilLoad.cpp
+++ b/core/sql/executor/ExExeUtilLoad.cpp
@@ -1245,9 +1245,7 @@ short ExExeUtilHBaseBulkLoadTcb::work()
       int jniDebugTimeout = 0;
       ehi_ = ExpHbaseInterface::newInstance(getGlobals()->getDefaultHeap(),
                                               (char*)"", //Later may need to change to hblTdb.server_,
-                                              (char*)"", //Later may need to change to hblTdb.zkPort_,
-                                              jniDebugPort,
-                                              jniDebugTimeout);
+                                              (char*)""); //Later may need to change to hblTdb.zkPort_);
       retcode = ehi_->initHBLC();
       if (retcode == 0) 
         retcode = ehi_->createCounterTable(hblTdb().getErrCountTable(), (char *)"ERRORS");
@@ -1983,13 +1981,9 @@ ExExeUtilHBaseBulkUnLoadTcb::ExExeUtilHBaseBulkUnLoadTcb(
        oneFile_(FALSE)
 {
   hdfsClient_ = NULL;
-  int jniDebugPort = 0;
-  int jniDebugTimeout = 0;
   ehi_ = ExpHbaseInterface::newInstance(getGlobals()->getDefaultHeap(),
                                    (char*)"", //Later may need to change to hblTdb.server_,
-                                   (char*)"", //Later may need to change to hblTdb.zkPort_,
-                                   jniDebugPort,
-                                   jniDebugTimeout);
+                                   (char*)""); //Later may need to change to hblTdb.zkPort_);
   qparent_.down->allocatePstate(this);
 
 }

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/ExFastTransport.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExFastTransport.cpp b/core/sql/executor/ExFastTransport.cpp
index bdde201..3a26467 100644
--- a/core/sql/executor/ExFastTransport.cpp
+++ b/core/sql/executor/ExFastTransport.cpp
@@ -1291,8 +1291,6 @@ void ExHdfsFastExtractTcb::createHdfsClientFileError(Int32 hdfsClientRetCode)
                   NULL, NULL, NULL, NULL,
                   errorMsg,
                 (char *)currContext->getJniErrorStr().data());
-  //ex_queue_entry *pentry_down = qParent_.down->getHeadEntry();
-  //pentry_down->setDiagsArea(diagsArea);
   updateWorkATPDiagsArea(diagsArea);
 }
 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/ExFastTransport.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExFastTransport.h b/core/sql/executor/ExFastTransport.h
index 94b091d..5bf1219 100644
--- a/core/sql/executor/ExFastTransport.h
+++ b/core/sql/executor/ExFastTransport.h
@@ -408,7 +408,7 @@ protected:
                           
   NABoolean isSequenceFile();
   void createSequenceFileError(Int32 sfwRetCode);
-  void createHdfsClientFileError(Int32 sfwRetCode);
+  void createHdfsClientFileError(Int32 hdfsClientRetCode);
   NABoolean isHdfsCompressed();
   NABoolean getEmptyNullString()
   {

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/ExHbaseAccess.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHbaseAccess.cpp b/core/sql/executor/ExHbaseAccess.cpp
index 42fd86e..2247b9a 100644
--- a/core/sql/executor/ExHbaseAccess.cpp
+++ b/core/sql/executor/ExHbaseAccess.cpp
@@ -358,15 +358,9 @@ ExHbaseAccessTcb::ExHbaseAccessTcb(
   registerSubtasks();
   registerResizeSubtasks();
 
-  int jniDebugPort = 0;
-  int jniDebugTimeout = 0;
   ehi_ = ExpHbaseInterface::newInstance(glob->getDefaultHeap(),
-					//					(char*)"localhost", 
 					(char*)hbaseAccessTdb.server_, 
-					//                                        (char*)"2181", 
-					(char*)hbaseAccessTdb.zkPort_,
-                                        jniDebugPort,
-                                        jniDebugTimeout);
+					(char*)hbaseAccessTdb.zkPort_);
 
   asciiRow_ = NULL;
   asciiRowMissingCols_ = NULL;
@@ -508,6 +502,8 @@ void ExHbaseAccessTcb::freeResources()
      NADELETEBASIC(colVal_.val, getHeap());
   if (hdfsClient_ != NULL) 
      NADELETE(hdfsClient_, HdfsClient, getHeap());
+  if (loggingFileName_ != NULL)
+     NADELETEBASIC(loggingFileName_, getHeap());
 }
 
 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/ExHbaseIUD.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHbaseIUD.cpp b/core/sql/executor/ExHbaseIUD.cpp
index 3bc1d93..e8896b2 100644
--- a/core/sql/executor/ExHbaseIUD.cpp
+++ b/core/sql/executor/ExHbaseIUD.cpp
@@ -1158,16 +1158,11 @@ ExHbaseAccessBulkLoadPrepSQTcb::ExHbaseAccessBulkLoadPrepSQTcb(
                       "traf_upsert_err",
                       fileNum,
                       loggingFileName_);
-   loggingFileCreated_ = FALSE;
    loggingRow_ =  new(glob->getDefaultHeap()) char[hbaseAccessTdb.updateRowLen_];
 }
 
 ExHbaseAccessBulkLoadPrepSQTcb::~ExHbaseAccessBulkLoadPrepSQTcb()
 {
-  if (loggingFileName_ != NULL) {
-     NADELETEBASIC(loggingFileName_, getHeap());
-     loggingFileName_ = NULL;
-  }
   // Flush and close sample file if used
   if (hdfs_)
     {

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/ExHdfsScan.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHdfsScan.cpp b/core/sql/executor/ExHdfsScan.cpp
index 90ac737..e29baf6 100644
--- a/core/sql/executor/ExHdfsScan.cpp
+++ b/core/sql/executor/ExHdfsScan.cpp
@@ -120,15 +120,35 @@ ExHdfsScanTcb::ExHdfsScanTcb(
   , loggingErrorDiags_(NULL)
   , loggingFileName_(NULL)
   , hdfsClient_(NULL)
+  , hdfsScan_(NULL)
   , hdfsFileInfoListAsArray_(glob->getDefaultHeap(), hdfsScanTdb.getHdfsFileInfoList()->numEntries())
 {
   Space * space = (glob ? glob->getSpace() : 0);
   CollHeap * heap = (glob ? glob->getDefaultHeap() : 0);
+  useLibhdfsScan_ = hdfsScanTdb.getUseLibhdfsScan();
   lobGlob_ = NULL;
-  const int readBufSize =  (Int32)hdfsScanTdb.hdfsBufSize_;
-  hdfsScanBuffer_ = new(space) char[ readBufSize + 1 ]; 
-  hdfsScanBuffer_[readBufSize] = '\0';
-
+  hdfsScanBufMaxSize_ = hdfsScanTdb.hdfsBufSize_;
+  headRoom_ = (Int32)hdfsScanTdb.rangeTailIOSize_;
+
+  if (useLibhdfsScan_) {
+     hdfsScanBuffer_ = new(heap) char[ hdfsScanBufMaxSize_ + 1 ]; 
+     hdfsScanBuffer_[hdfsScanBufMaxSize_] = '\0';
+  } else {
+     hdfsScanBufBacking_[0] = new (heap) BYTE[hdfsScanBufMaxSize_ + 2 * (headRoom_)];
+     hdfsScanBufBacking_[1] = new (heap) BYTE[hdfsScanBufMaxSize_ + 2 * (headRoom_)];
+     for (int i=0; i < 2; i++) {
+        BYTE *hdfsScanBufBacking = hdfsScanBufBacking_[i];
+        hdfsScanBuf_[i].headRoom_ = hdfsScanBufBacking;
+        hdfsScanBuf_[i].buf_ = hdfsScanBufBacking + headRoom_;
+     }
+     bufBegin_ = NULL;
+     bufEnd_ = NULL;
+     logicalBufEnd_ = NULL;
+     headRoomCopied_ = 0;
+     prevRangeNum_ = -1;
+     currRangeBytesRead_ = 0;
+     recordSkip_ = FALSE;
+  }
   moveExprColsBuffer_ = new(space) ExSimpleSQLBuffer( 1, // one row 
 						      (Int32)hdfsScanTdb.moveExprColsRowLength_,
 						      space);
@@ -202,9 +222,7 @@ ExHdfsScanTcb::ExHdfsScanTcb(
   int jniDebugTimeout = 0;
   ehi_ = ExpHbaseInterface::newInstance(glob->getDefaultHeap(),
                                         (char*)"",  //Later replace with server cqd
-                                        (char*)"", ////Later replace with port cqd
-                                        jniDebugPort,
-                                        jniDebugTimeout);
+                                        (char*)"");
 
   // Populate the hdfsInfo list into an array to gain o(1) lookup access
   Queue* hdfsInfoList = hdfsScanTdb.getHdfsFileInfoList();
@@ -238,9 +256,9 @@ void ExHdfsScanTcb::freeResources()
     deallocateAtp(workAtp_, getSpace());
     workAtp_ = NULL;
   }
-  if (hdfsScanBuffer_)
+  if (hdfsScanBuffer_ )
   {
-    NADELETEBASIC(hdfsScanBuffer_, getSpace());
+    NADELETEBASIC(hdfsScanBuffer_, getHeap());
     hdfsScanBuffer_ = NULL;
   }
   if (hdfsAsciiSourceBuffer_)
@@ -287,6 +305,8 @@ void ExHdfsScanTcb::freeResources()
   }
   if (hdfsClient_ != NULL) 
      NADELETE(hdfsClient_, HdfsClient, getHeap());
+  if (hdfsScan_ != NULL) 
+     NADELETE(hdfsScan_, HdfsScan, getHeap());
 }
 
 NABoolean ExHdfsScanTcb::needStatsEntry()
@@ -384,10 +404,12 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
   HdfsFileInfo *hdfo = NULL;
   Lng32 openType = 0;
   int changedLen = 0;
- ContextCli *currContext = getGlobals()->castToExExeStmtGlobals()->getCliGlobals()->currContext();
-   hdfsFS hdfs = currContext->getHdfsServerConnection(hdfsScanTdb().hostName_,hdfsScanTdb().port_);
-   hdfsFileInfo *dirInfo = NULL;
-   Int32 hdfsErrorDetail = 0;//this is errno returned form underlying hdfsOpenFile call.
+  ContextCli *currContext = getGlobals()->castToExExeStmtGlobals()->getCliGlobals()->currContext();
+  hdfsFS hdfs = currContext->getHdfsServerConnection(hdfsScanTdb().hostName_,hdfsScanTdb().port_);
+  hdfsFileInfo *dirInfo = NULL;
+  Int32 hdfsErrorDetail = 0;//this is errno returned form underlying hdfsOpenFile call.
+  HDFS_Scan_RetCode hdfsScanRetCode;
+
   while (!qparent_.down->isEmpty())
     {
       ex_queue_entry *pentry_down = qparent_.down->getHeadEntry();
@@ -442,8 +464,12 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
         case ASSIGN_RANGES_AT_RUNTIME:
           computeRangesAtRuntime();
           currRangeNum_ = beginRangeNum_;
-          if (numRanges_ > 0)
-            step_ = INIT_HDFS_CURSOR;
+          if (numRanges_ > 0) {
+            if (useLibhdfsScan_)
+               step_ = INIT_HDFS_CURSOR;
+            else
+               step_ = SETUP_HDFS_SCAN; 
+          }
           else
             step_ = DONE;
           break;
@@ -518,11 +544,93 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 
             if (step_ == CHECK_FOR_DATA_MOD_AND_DONE)
               step_ = DONE;
-            else
-              step_ = INIT_HDFS_CURSOR;
+            else {
+              if (useLibhdfsScan_)
+                 step_ = INIT_HDFS_CURSOR;
+              else
+                 step_ = SETUP_HDFS_SCAN;
+            }
           }        
           break;
-
+        case SETUP_HDFS_SCAN:
+          {   
+             if (hdfsScan_ != NULL)
+                NADELETE(hdfsScan_, HdfsScan, getHeap());
+             hdfsScan_ = HdfsScan::newInstance((NAHeap *)getHeap(), hdfsScanBuf_, hdfsScanBufMaxSize_, 
+                            &hdfsFileInfoListAsArray_, hdfsScanTdb().rangeTailIOSize_, hdfsScanRetCode);
+             if (hdfsScanRetCode != HDFS_SCAN_OK)
+             {
+                setupError(EXE_ERROR_HDFS_SCAN, hdfsScanRetCode, "SETUP_HDFS_SCAN", 
+                              currContext->getJniErrorStr(), NULL);              
+                step_ = HANDLE_ERROR_AND_DONE;
+                break;
+             } 
+             bufBegin_ = NULL;
+             bufEnd_ = NULL;
+             logicalBufEnd_ = NULL;
+             headRoomCopied_ = 0;
+             prevRangeNum_ = -1;                         
+             currRangeBytesRead_ = 0;                   
+             recordSkip_ = FALSE;
+             step_ = TRAF_HDFS_READ;
+          } 
+          break;
+        case TRAF_HDFS_READ:
+          {
+             hdfsScanRetCode = hdfsScan_->trafHdfsRead((NAHeap *)getHeap(), retArray_, sizeof(retArray_)/sizeof(int));
+             if (hdfsScanRetCode == HDFS_SCAN_EOR) {
+                step_ = DONE;
+                break;
+             }
+             else if (hdfsScanRetCode != HDFS_SCAN_OK) {
+                setupError(EXE_ERROR_HDFS_SCAN, hdfsScanRetCode, "SETUP_HDFS_SCAN", 
+                              currContext->getJniErrorStr(), NULL);              
+                step_ = HANDLE_ERROR_AND_DONE;
+                break;
+             } 
+             // Assign the starting address of the buffer
+             bufEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED];
+             if (retArray_[IS_EOF])
+                logicalBufEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED];
+             else if (retArray_[BYTES_COMPLETED] < hdfsScanBufMaxSize_) 
+                logicalBufEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED] - headRoom_;   
+             else
+                logicalBufEnd_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ + retArray_[BYTES_COMPLETED];
+             hdfo_ = getRange(retArray_[RANGE_NO]);
+             if (retArray_[RANGE_NO] != prevRangeNum_) {  
+                bufBegin_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_;
+                if (hdfo_->getStartOffset() == 0)
+                   recordSkip_ = FALSE;
+                else
+                   recordSkip_ = TRUE; 
+             } else {
+                bufBegin_ = hdfsScanBuf_[retArray_[BUF_NO]].buf_ - headRoomCopied_;
+                recordSkip_ = FALSE;
+             } 
+             prevRangeNum_ = retArray_[RANGE_NO];
+             if (recordSkip_) {
+		hdfsBufNextRow_ = hdfs_strchr((char *)bufBegin_,
+			      hdfsScanTdb().recordDelimiter_, 
+                              (char *)bufEnd_,
+			      checkRangeDelimiter_, 
+			      hdfsScanTdb().getHiveScanMode(), &changedLen);
+                if (hdfsBufNextRow_ == NULL) {
+                   setupError(8446, 0, "No record delimiter found in buffer from hdfsRead", 
+                              NULL, NULL);              
+                   step_ = HANDLE_ERROR_AND_DONE;
+                   break;
+                }
+             }
+             else
+                hdfsBufNextRow_ = (char *)bufBegin_; 
+             step_ = PROCESS_HDFS_ROW;
+          }
+          break;
+        case COPY_TAIL_TO_HEAD:
+          {
+             step_ = TRAF_HDFS_READ;  
+          }
+          break;
 	case INIT_HDFS_CURSOR:
 	  {
             hdfo_ = getRange(currRangeNum_);
@@ -949,7 +1057,10 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 
 	  if (startOfNextRow == NULL)
 	  {
-	    step_ = REPOS_HDFS_DATA;
+            if (useLibhdfsScan_)
+	       step_ = REPOS_HDFS_DATA;
+            else
+               step_ = COPY_TAIL_TO_HEAD;
 	    if (!exception_)
 	      break;
 	  }
@@ -1220,8 +1331,12 @@ ExWorkProcRetcode ExHdfsScanTcb::work()
 	  workAtp_->setDiagsArea(NULL);    // get rid of warnings.
           if (((pentry_down->downState.request == ex_queue::GET_N) &&
                (pentry_down->downState.requestValue == matches_)) ||
-              (pentry_down->downState.request == ex_queue::GET_NOMORE))
-              step_ = CLOSE_HDFS_CURSOR;
+              (pentry_down->downState.request == ex_queue::GET_NOMORE)) {
+              if (useLibhdfsScan_)
+                 step_ = CLOSE_HDFS_CURSOR;
+              else
+                 step_ = DONE;
+          }
           else
 	     step_ = PROCESS_HDFS_ROW;
 	  break;      
@@ -1568,18 +1683,26 @@ char * ExHdfsScanTcb::extractAndTransformAsciiSourceToSqlRow(int &err,
   
   const char cd = hdfsScanTdb().columnDelimiter_;
   const char rd = hdfsScanTdb().recordDelimiter_;
-  const char *sourceDataEnd = hdfsScanBuffer_+trailingPrevRead_+ bytesRead_;
-
+  const char *sourceDataEnd;
+  const char *endOfRequestedRange;
+  if (useLibhdfsScan_) {
+     sourceDataEnd  = hdfsScanBuffer_+trailingPrevRead_+ bytesRead_;
+     endOfRequestedRange = endOfRequestedRange_;
+  }
+  else {
+     sourceDataEnd = (const char *)bufEnd_;
+     endOfRequestedRange = (const char *)logicalBufEnd_;
+  }
   hdfsLoggingRow_ = hdfsBufNextRow_;
   if (asciiSourceTD->numAttrs() == 0)
   {
      sourceRowEnd = hdfs_strchr(sourceData, rd, sourceDataEnd, checkRangeDelimiter_, mode, &changedLen);
      hdfsLoggingRowEnd_  = sourceRowEnd + changedLen;
 
-     if (!sourceRowEnd)
-       return NULL; 
-     if ((endOfRequestedRange_) && 
-            (sourceRowEnd >= endOfRequestedRange_)) {
+     if (sourceRowEnd == NULL)
+        return NULL; 
+     if ((endOfRequestedRange) && 
+            (sourceRowEnd >= endOfRequestedRange)) {
         checkRangeDelimiter_ = TRUE;
         *(sourceRowEnd +1)= RANGE_DELIMITER;
      }
@@ -1623,8 +1746,8 @@ char * ExHdfsScanTcb::extractAndTransformAsciiSourceToSqlRow(int &err,
          if (rdSeen) {
             sourceRowEnd = sourceColEnd + changedLen; 
             hdfsLoggingRowEnd_  = sourceRowEnd;
-            if ((endOfRequestedRange_) && 
-                   (sourceRowEnd >= endOfRequestedRange_)) {
+            if ((endOfRequestedRange) && 
+                   (sourceRowEnd >= endOfRequestedRange)) {
                checkRangeDelimiter_ = TRUE;
                *(sourceRowEnd +1)= RANGE_DELIMITER;
             }
@@ -1697,8 +1820,8 @@ char * ExHdfsScanTcb::extractAndTransformAsciiSourceToSqlRow(int &err,
      sourceRowEnd = hdfs_strchr(sourceData, rd, sourceDataEnd, checkRangeDelimiter_,mode, &changedLen);
      if (sourceRowEnd) {
         hdfsLoggingRowEnd_  = sourceRowEnd + changedLen; //changedLen is when hdfs_strchr move the return pointer to remove the extra \r
-        if ((endOfRequestedRange_) &&
-              (sourceRowEnd >= endOfRequestedRange_ )) {
+        if ((endOfRequestedRange) &&
+              (sourceRowEnd >= endOfRequestedRange )) {
            checkRangeDelimiter_ = TRUE;
           *(sourceRowEnd +1)= RANGE_DELIMITER;
         }

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/ExHdfsScan.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHdfsScan.h b/core/sql/executor/ExHdfsScan.h
index 984fbb9..62bb11e 100644
--- a/core/sql/executor/ExHdfsScan.h
+++ b/core/sql/executor/ExHdfsScan.h
@@ -46,6 +46,8 @@
 // -----------------------------------------------------------------------
 class ExHdfsScanTdb;
 class ExHdfsScanTcb;
+class HdfsScan;
+class HdfsClient;
 
 // -----------------------------------------------------------------------
 // Classes referenced in this file
@@ -108,9 +110,46 @@ private:
   // ---------------------------------------------------------------------
 };
 
+/*
+   USE_LIBHDFS_SCAN - OFF enables hdfs access via java classes 
+      org.trafodion.sql.HdfsScan and org.trafodion.sql.HdfsClient
+   Steps involved:
+   1. Create a new HdfsScan object and set the scan ranges of the fragment instance in it
+      The scan range involves the following and it is determined either at runtime or compile time
+         a) filename
+         b) offset
+         c) len
+      Java layer always reads more than the len by rangeTailIOSize_ to accomdate the record split 
+   2. Two ByteBuffer objects are also passsed to HdfsScan object. These ByteBuffers are backed up by
+      2 native buffers where the data is fetched. The buffer has a head room of size rangeTailIOSize_ and the 
+      data is always read after the head room. 
+   3. HdfsScan returns an int array containing bytesRead, bufNo, rangeNo, isEOF and schedules either
+      the remaining bytes to be read or the next range using ByteBuffers alternatively.
+   4. HdfsScan returns null array when there is no more data to be read.
+   5. When the data is processed in one ByteBuffer in the native thread, the data is fetched into the other ByteBuffer by
+      another Java thread.
+   6. Native layer after processing all the rows in one ByteBuffer, moves the last incomplete row to head room of the
+      other ByteBuffer. Then it requests to check if the read is complete. The native layer processes the buffer starting
+      from the copied incomplete row.
+*/
+
 class ExHdfsScanTcb  : public ex_tcb
 {
+   
 public:
+  enum
+  {
+    BYTES_COMPLETED,
+    BUF_NO,
+    RANGE_NO,
+    IS_EOF
+  } retArrayIndices_;
+
+  struct HDFS_SCAN_BUF
+  {
+     BYTE *headRoom_;
+     BYTE *buf_;
+  };
   ExHdfsScanTcb( const ComTdbHdfsScan &tdb,
                          ex_globals *glob );
 
@@ -165,6 +204,9 @@ protected:
   , DONE
   , HANDLE_ERROR_WITH_CLOSE
   , HANDLE_ERROR_AND_DONE
+  , SETUP_HDFS_SCAN
+  , TRAF_HDFS_READ
+  , COPY_TAIL_TO_HEAD
   } step_,nextStep_;
 
   /////////////////////////////////////////////////////
@@ -296,7 +338,21 @@ protected:
 
   // this array is populated from the info list stored as Queue.
   HdfsFileInfoArray hdfsFileInfoListAsArray_;
+
   HdfsClient *hdfsClient_;
+  HdfsScan *hdfsScan_;
+  NABoolean useLibhdfsScan_;
+  BYTE *hdfsScanBufBacking_[2];
+  HDFS_SCAN_BUF hdfsScanBuf_[2];
+  int retArray_[4];
+  BYTE *bufBegin_;
+  BYTE *bufEnd_;
+  BYTE *logicalBufEnd_;
+  long currRangeBytesRead_;
+  int headRoomCopied_;
+  int headRoom_;
+  int prevRangeNum_;
+  NABoolean recordSkip_;
 };
 
 class ExOrcScanTcb  : public ExHdfsScanTcb

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/HBaseClient_JNI.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/HBaseClient_JNI.cpp b/core/sql/executor/HBaseClient_JNI.cpp
index fe56d94..6b400cd 100644
--- a/core/sql/executor/HBaseClient_JNI.cpp
+++ b/core/sql/executor/HBaseClient_JNI.cpp
@@ -114,8 +114,8 @@ static const char* const hbcErrorEnumStr[] =
 // 
 //////////////////////////////////////////////////////////////////////////////
 // private default constructor
-HBaseClient_JNI::HBaseClient_JNI(NAHeap *heap, int debugPort, int debugTimeout)
-                 :  JavaObjectInterface(heap, debugPort, debugTimeout)
+HBaseClient_JNI::HBaseClient_JNI(NAHeap *heap)
+                 :  JavaObjectInterface(heap)
                    ,isConnected_(FALSE)
 {
   for (int i=0; i<NUM_HBASE_WORKER_THREADS; i++) {
@@ -137,7 +137,7 @@ char* HBaseClient_JNI::getErrorText(HBC_RetCode errEnum)
 //////////////////////////////////////////////////////////////////////////////
 // 
 //////////////////////////////////////////////////////////////////////////////
-HBaseClient_JNI* HBaseClient_JNI::getInstance(int debugPort, int debugTimeout)
+HBaseClient_JNI* HBaseClient_JNI::getInstance()
 {
    ContextCli *currContext = GetCliGlobals()->currContext();
    HBaseClient_JNI *hbaseClient_JNI = currContext->getHBaseClient();
@@ -145,8 +145,7 @@ HBaseClient_JNI* HBaseClient_JNI::getInstance(int debugPort, int debugTimeout)
    {
      NAHeap *heap = currContext->exHeap();
     
-     hbaseClient_JNI  = new (heap) HBaseClient_JNI(heap,
-                   debugPort, debugTimeout);
+     hbaseClient_JNI  = new (heap) HBaseClient_JNI(heap);
      currContext->setHbaseClient(hbaseClient_JNI);
    }
    return hbaseClient_JNI;
@@ -301,7 +300,8 @@ HBC_RetCode HBaseClient_JNI::init()
     JavaMethods_[JM_TRUNCATE   ].jm_name      = "truncate";
     JavaMethods_[JM_TRUNCATE   ].jm_signature = "(Ljava/lang/String;ZJ)Z";
     rc = (HBC_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
-    javaMethodsInitialized_ = TRUE;
+    if (rc == HBC_OK)
+       javaMethodsInitialized_ = TRUE;
     pthread_mutex_unlock(&javaMethodsInitMutex_);
   }
   return rc;
@@ -1583,7 +1583,8 @@ HBLC_RetCode HBulkLoadClient_JNI::init()
     JavaMethods_[JM_ADD_TO_HFILE_DB  ].jm_signature = "(SLjava/lang/Object;Ljava/lang/Object;)Z";
 
     rc = (HBLC_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
-    javaMethodsInitialized_ = TRUE;
+    if (rc == HBLC_OK)
+       javaMethodsInitialized_ = TRUE;
     pthread_mutex_unlock(&javaMethodsInitMutex_);
   }
   return rc;
@@ -3208,7 +3209,8 @@ HTC_RetCode HTableClient_JNI::init()
     JavaMethods_[JM_COMPLETE_PUT ].jm_signature = "(I[Z)Z";
    
     rc = (HTC_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
-    javaMethodsInitialized_ = TRUE;
+    if (rc == HTC_OK)
+       javaMethodsInitialized_ = TRUE;
     pthread_mutex_unlock(&javaMethodsInitMutex_);
   }
   return rc;
@@ -3798,7 +3800,8 @@ HVC_RetCode HiveClient_JNI::init()
     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_);
-    javaMethodsInitialized_ = TRUE;
+    if (rc == HVC_OK)
+       javaMethodsInitialized_ = TRUE;
     pthread_mutex_unlock(&javaMethodsInitMutex_);
   }
   return rc;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/HBaseClient_JNI.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/HBaseClient_JNI.h b/core/sql/executor/HBaseClient_JNI.h
index f6667d9..3177a7a 100644
--- a/core/sql/executor/HBaseClient_JNI.h
+++ b/core/sql/executor/HBaseClient_JNI.h
@@ -246,7 +246,7 @@ public:
   std::string* getHTableName();
 
   // Get the error description.
-  virtual char* getErrorText(HTC_RetCode errEnum);
+  static char* getErrorText(HTC_RetCode errEnum);
 
   void setTableName(const char *tableName)
   {
@@ -429,7 +429,7 @@ typedef enum {
 class HBaseClient_JNI : public JavaObjectInterface
 {
 public:
-  static HBaseClient_JNI* getInstance(int debugPort, int debugTimeout);
+  static HBaseClient_JNI* getInstance();
   static void deleteInstance();
 
   // Destructor
@@ -488,7 +488,7 @@ public:
   HBaseClientRequest* getHBaseRequest();
   bool workerThreadsStarted() { return (threadID_[0] ? true : false); }
   // Get the error description.
-  virtual char* getErrorText(HBC_RetCode errEnum);
+  static char* getErrorText(HBC_RetCode errEnum);
   
   static void logIt(const char* str);
 
@@ -542,7 +542,7 @@ public:
 
 private:   
   // private default constructor
-  HBaseClient_JNI(NAHeap *heap, int debugPort, int debugTimeout);
+  HBaseClient_JNI(NAHeap *heap);
   NAArray<HbaseStr>* getKeys(Int32 funcIndex, NAHeap *heap, const char *tableName, bool useTRex);
 
 private:
@@ -665,7 +665,7 @@ public:
 
   HVC_RetCode executeHiveSQL(const char* hiveSQL);
   // Get the error description.
-  virtual char* getErrorText(HVC_RetCode errEnum);
+  static char* getErrorText(HVC_RetCode errEnum);
   
   static void logIt(const char* str);
 
@@ -757,7 +757,7 @@ public:
 
   HBLC_RetCode  bulkLoadCleanup(const HbaseStr &tblName, const Text& location);
   // Get the error description.
-  virtual char* getErrorText(HBLC_RetCode errEnum);
+  static char* getErrorText(HBLC_RetCode errEnum);
 
 
 private:

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/HdfsClient_JNI.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/HdfsClient_JNI.cpp b/core/sql/executor/HdfsClient_JNI.cpp
index a3aef5a..63c4ac1 100644
--- a/core/sql/executor/HdfsClient_JNI.cpp
+++ b/core/sql/executor/HdfsClient_JNI.cpp
@@ -37,6 +37,11 @@ pthread_mutex_t HdfsScan::javaMethodsInitMutex_ = PTHREAD_MUTEX_INITIALIZER;
 
 static const char* const hdfsScanErrorEnumStr[] = 
 {
+   "Error in HdfsScan::setScanRanges"
+  ,"Java Exception in HdfsScan::setScanRanges"
+  ,"Error in HdfsScan::trafHdfsRead"
+  ,"Java Exceptiokn in HdfsScan::trafHdfsRead"
+  , "Hdfs scan End of Ranges"
 };
 
  
@@ -62,13 +67,14 @@ HDFS_Scan_RetCode HdfsScan::init()
     
     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_SET_SCAN_RANGES].jm_name      = "setScanRanges";
+    JavaMethods_[JM_SET_SCAN_RANGES].jm_signature = "(Ljava/nio/ByteBuffer;Ljava/nio/ByteBuffer;[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;
+    if (rc == HDFS_SCAN_OK)
+       javaMethodsInitialized_ = TRUE;
     pthread_mutex_unlock(&javaMethodsInitMutex_);
   }
   return rc;
@@ -79,17 +85,131 @@ 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];
+    return (char*)hdfsScanErrorEnumStr[errEnum-HDFS_SCAN_FIRST];
 }
-//////////////////////////////////////////////////////////////////////////////
-HDFS_Scan_RetCode HdfsScan::initScanRanges()
+
+/////////////////////////////////////////////////////////////////////////////
+HDFS_Scan_RetCode HdfsScan::setScanRanges(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN_BUF *hdfsScanBuf,  int scanBufSize,
+      HdfsFileInfoArray *hdfsFileInfoArray, int rangeTailIOSize)
 {
+   QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsScan::setScanRanges() called.");
+
+  if (initJNIEnv() != JOI_OK)
+     return HDFS_SCAN_ERROR_SET_SCAN_RANGES_PARAM;
+
+   jobject j_buf1 = jenv_->NewDirectByteBuffer(hdfsScanBuf[0].buf_, scanBufSize);
+   if (j_buf1 == NULL) {
+      GetCliGlobals()->setJniErrorStr(getErrorText(HDFS_SCAN_ERROR_SET_SCAN_RANGES_PARAM));
+      jenv_->PopLocalFrame(NULL);
+      return HDFS_SCAN_ERROR_SET_SCAN_RANGES_PARAM;
+   }
+
+   jobject j_buf2 = jenv_->NewDirectByteBuffer(hdfsScanBuf[1].buf_, scanBufSize);
+   if (j_buf2 == NULL) {
+      GetCliGlobals()->setJniErrorStr(getErrorText(HDFS_SCAN_ERROR_SET_SCAN_RANGES_PARAM));
+      jenv_->PopLocalFrame(NULL);
+      return HDFS_SCAN_ERROR_SET_SCAN_RANGES_PARAM;
+   }
+   jobjectArray j_filenames = NULL;
+   jlongArray j_offsets = NULL;
+   jlongArray j_lens = NULL;  
+   HdfsFileInfo *hdfo;
+   jstring j_obj;
+
+   HDFS_Scan_RetCode hdfsScanRetCode =  HDFS_SCAN_ERROR_SET_SCAN_RANGES_PARAM;
+   int arrayLen = hdfsFileInfoArray->entries();
+   for (int i = 0; i < arrayLen; i++) {
+       hdfo = hdfsFileInfoArray->at(i);
+       j_obj = jenv_->NewStringUTF(hdfo->fileName());
+       if (jenv_->ExceptionCheck()) {
+          jenv_->PopLocalFrame(NULL);
+          return hdfsScanRetCode;
+       }
+       if (j_filenames == NULL) {
+          j_filenames = jenv_->NewObjectArray(arrayLen, jenv_->GetObjectClass(j_obj), NULL);
+          if (jenv_->ExceptionCheck()) {
+             jenv_->PopLocalFrame(NULL);
+             return hdfsScanRetCode;
+          }
+       }
+       jenv_->SetObjectArrayElement(j_filenames, i, (jobject)j_obj);
+       jenv_->DeleteLocalRef(j_obj);
+       if (j_offsets == NULL) {
+          j_offsets = jenv_->NewLongArray(arrayLen);
+          if (jenv_->ExceptionCheck()) {
+             jenv_->PopLocalFrame(NULL);
+             return hdfsScanRetCode;
+          }
+       }
+       long offset = hdfo->getStartOffset(); 
+       jenv_->SetLongArrayRegion(j_offsets, i, 1, &offset);
+       if (j_lens == NULL) {
+          j_lens = jenv_->NewLongArray(arrayLen);
+          if (jenv_->ExceptionCheck()) {
+             jenv_->PopLocalFrame(NULL);
+             return hdfsScanRetCode;
+          }
+       }
+       long len = hdfo->getBytesToRead()+rangeTailIOSize;
+       jenv_->SetLongArrayRegion(j_lens, i, 1, &len);
+   } 
+
+   jenv_->CallVoidMethod(javaObj_, JavaMethods_[JM_SET_SCAN_RANGES].methodID, j_buf1, j_buf2, j_filenames, j_offsets, j_lens);
+
+   if (jenv_->ExceptionCheck()) {
+      getExceptionDetails();
+      logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+      logError(CAT_SQL_HDFS, "HdfsScan::setScanRanges()", getLastError());
+      jenv_->PopLocalFrame(NULL);
+      return HDFS_SCAN_ERROR_SET_SCAN_RANGES_EXCEPTION;
+   }
    return HDFS_SCAN_OK; 
 }
 
-HDFS_Scan_RetCode HdfsScan::trafHdfsRead()
+HdfsScan *HdfsScan::newInstance(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN_BUF *hdfsScanBuf,  int scanBufSize,
+      HdfsFileInfoArray *hdfsFileInfoArray, int rangeTailIOSize, HDFS_Scan_RetCode &hdfsScanRetCode)
 {
-   return HDFS_SCAN_OK; 
+   QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsScan::newInstance() called.");
+
+   if (initJNIEnv() != JOI_OK)
+     return NULL;
+   hdfsScanRetCode = HDFS_SCAN_OK;
+   HdfsScan *hdfsScan = new (heap) HdfsScan(heap);
+   if (hdfsScan != NULL) {
+       hdfsScanRetCode = hdfsScan->init();
+       if (hdfsScanRetCode == HDFS_SCAN_OK) 
+          hdfsScanRetCode = hdfsScan->setScanRanges(heap, hdfsScanBuf, scanBufSize, 
+                    hdfsFileInfoArray, rangeTailIOSize); 
+       if (hdfsScanRetCode != HDFS_SCAN_OK) {
+          NADELETE(hdfsScan, HdfsScan, heap);
+          hdfsScan = NULL;
+       }
+   }
+   return hdfsScan;
+}
+
+
+HDFS_Scan_RetCode HdfsScan::trafHdfsRead(NAHeap *heap, int retArray[], short arrayLen)
+{
+   QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsScan::trafHdfsRead() called.");
+
+   if (initJNIEnv() != JOI_OK)
+     return HDFS_SCAN_ERROR_TRAF_HDFS_READ_PARAM;
+
+   jintArray j_retArray = (jintArray)jenv_->CallObjectMethod(javaObj_, JavaMethods_[JM_TRAF_HDFS_READ].methodID);
+   if (jenv_->ExceptionCheck()) {
+      getExceptionDetails();
+      logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+      logError(CAT_SQL_HDFS, "HdfsScan::setScanRanges()", getLastError());
+      jenv_->PopLocalFrame(NULL);
+      return HDFS_SCAN_ERROR_TRAF_HDFS_READ_EXCEPTION;
+   }
+   if (j_retArray == NULL)
+      return HDFS_SCAN_EOR;
+   short retArrayLen = jenv_->GetArrayLength(j_retArray);
+   ex_assert(retArrayLen == arrayLen, "HdfsScan::trafHdfsRead() InternalError: retArrayLen != arrayLen");
+   jenv_->GetIntArrayRegion(j_retArray, 0, 4, retArray);
+   return HDFS_SCAN_OK;
 }
 
 // ===========================================================================
@@ -123,12 +243,15 @@ static const char* const hdfsClientErrorEnumStr[] =
 //////////////////////////////////////////////////////////////////////////////
 HdfsClient *HdfsClient::newInstance(NAHeap *heap, HDFS_Client_RetCode &retCode)
 {
+   QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::newInstance() called.");
+
+   if (initJNIEnv() != JOI_OK)
+     return NULL;
    retCode = HDFS_CLIENT_OK;
    HdfsClient *hdfsClient = new (heap) HdfsClient(heap);
    if (hdfsClient != NULL) {
        retCode = hdfsClient->init();
-       if (retCode != HDFS_CLIENT_OK)
-       {
+       if (retCode != HDFS_CLIENT_OK) {
           NADELETE(hdfsClient, HdfsClient, heap);
           hdfsClient = NULL;
        }
@@ -170,7 +293,8 @@ HDFS_Client_RetCode HdfsClient::init()
     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;
+    if (rc == HDFS_CLIENT_OK)
+       javaMethodsInitialized_ = TRUE;
     pthread_mutex_unlock(&javaMethodsInitMutex_);
   }
   return rc;
@@ -189,7 +313,7 @@ char* HdfsClient::getErrorText(HDFS_Client_RetCode errEnum)
 
 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);
+  QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::hdfsCreate(%s) called.", path);
 
   if (initJNIEnv() != JOI_OK)
      return HDFS_CLIENT_ERROR_HDFS_CREATE_PARAM;
@@ -209,15 +333,15 @@ HDFS_Client_RetCode HdfsClient::hdfsCreate(const char* path, NABoolean compress)
   if (jenv_->ExceptionCheck())
   {
     getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsCreate()", getLastError());
+    logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsCreate()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_CREATE_EXCEPTION;
   }
 
   if (jresult == false)
   {
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsCreate()", getLastError());
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsCreate()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_CREATE_PARAM;
   }
@@ -231,7 +355,7 @@ HDFS_Client_RetCode HdfsClient::hdfsCreate(const char* path, NABoolean compress)
 //////////////////////////////////////////////////////////////////////////////
 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);
+  QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::hdfsWrite(%ld) called.", len);
 
   if (initJNIEnv() != JOI_OK)
      return HDFS_CLIENT_ERROR_HDFS_WRITE_EXCEPTION;
@@ -252,15 +376,15 @@ HDFS_Client_RetCode HdfsClient::hdfsWrite(const char* data, Int64 len)
   if (jenv_->ExceptionCheck())
   {
     getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsWrite()", getLastError());
+    logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsWrite()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_WRITE_EXCEPTION;
   }
 
   if (jresult == false)
   {
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsWrite()", getLastError());
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsWrite()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_WRITE_EXCEPTION;
   }
@@ -275,7 +399,7 @@ HDFS_Client_RetCode HdfsClient::hdfsWrite(const char* data, Int64 len)
 //////////////////////////////////////////////////////////////////////////////
 HDFS_Client_RetCode HdfsClient::hdfsClose()
 {
-  QRLogger::log(CAT_SQL_HDFS_SEQ_FILE_WRITER, LL_DEBUG, "HdfsClient::close() called.");
+  QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::close() called.");
 
   if (initJNIEnv() != JOI_OK)
      return HDFS_CLIENT_ERROR_HDFS_CLOSE_EXCEPTION;
@@ -287,15 +411,15 @@ HDFS_Client_RetCode HdfsClient::hdfsClose()
   if (jenv_->ExceptionCheck())
   {
     getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsClose()", getLastError());
+    logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsClose()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_CLOSE_EXCEPTION;
   }
 
   if (jresult == false)
   {
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsClose()", getLastError());
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsClose()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_CLOSE_EXCEPTION;
   }
@@ -306,7 +430,7 @@ HDFS_Client_RetCode HdfsClient::hdfsClose()
 
 HDFS_Client_RetCode HdfsClient::hdfsCleanUnloadPath( const NAString& uldPath)
 {
-  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HdfsClient::hdfsCleanUnloadPath(%s) called.",
+  QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::hdfsCleanUnloadPath(%s) called.",
                                                                              uldPath.data());
   if (initJNIEnv() != JOI_OK)
      return HDFS_CLIENT_ERROR_HDFS_CLEANUP_PARAM;
@@ -324,8 +448,8 @@ HDFS_Client_RetCode HdfsClient::hdfsCleanUnloadPath( const NAString& uldPath)
   if (jenv_->ExceptionCheck())
   {
     getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsCleanUnloadPath()", getLastError());
+    logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsCleanUnloadPath()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_CLEANUP_EXCEPTION;
   }
@@ -337,7 +461,7 @@ HDFS_Client_RetCode HdfsClient::hdfsCleanUnloadPath( const NAString& uldPath)
 HDFS_Client_RetCode HdfsClient::hdfsMergeFiles( const NAString& srcPath,
                                                 const NAString& dstPath)
 {
-  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HdfsClient::hdfsMergeFiles(%s, %s) called.",
+  QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::hdfsMergeFiles(%s, %s) called.",
                   srcPath.data(), dstPath.data());
 
   if (initJNIEnv() != JOI_OK)
@@ -364,15 +488,15 @@ HDFS_Client_RetCode HdfsClient::hdfsMergeFiles( const NAString& srcPath,
   if (jenv_->ExceptionCheck())
   {
     getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsMergeFiles()", getLastError());
+    logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsMergeFiles()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_EXCEPTION;
   }
 
   if (jresult == false)
   {
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsMergeFiles()", getLastError());
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsMergeFiles()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_MERGE_FILES_EXCEPTION;
   } 
@@ -383,7 +507,7 @@ HDFS_Client_RetCode HdfsClient::hdfsMergeFiles( const NAString& srcPath,
 
 HDFS_Client_RetCode HdfsClient::hdfsDeletePath( const NAString& delPath)
 {
-  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HdfsClient::hdfsDeletePath(%s called.",
+  QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::hdfsDeletePath(%s called.",
                   delPath.data());
   if (initJNIEnv() != JOI_OK)
      return HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_EXCEPTION;
@@ -402,15 +526,15 @@ HDFS_Client_RetCode HdfsClient::hdfsDeletePath( const NAString& delPath)
   if (jenv_->ExceptionCheck())
   {
     getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsDeletePath()", getLastError());
+    logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsDeletePath()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_EXCEPTION;
   }
 
   if (jresult == false)
   {
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsDeletePath()", getLastError());
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsDeletePath()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_DELETE_PATH_EXCEPTION;
   }
@@ -421,7 +545,7 @@ HDFS_Client_RetCode HdfsClient::hdfsDeletePath( const NAString& delPath)
 
 HDFS_Client_RetCode HdfsClient::hdfsExists( const NAString& uldPath, NABoolean & exist)
 {
-  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HdfsClient::hdfsExists(%s) called.",
+  QRLogger::log(CAT_SQL_HDFS, LL_DEBUG, "HdfsClient::hdfsExists(%s) called.",
                                                       uldPath.data());
 
   if (initJNIEnv() != JOI_OK)
@@ -441,8 +565,8 @@ HDFS_Client_RetCode HdfsClient::hdfsExists( const NAString& uldPath, NABoolean &
   if (jenv_->ExceptionCheck())
   {
     getExceptionDetails();
-    logError(CAT_SQL_HBASE, __FILE__, __LINE__);
-    logError(CAT_SQL_HBASE, "HdfsClient::hdfsExists()", getLastError());
+    logError(CAT_SQL_HDFS, __FILE__, __LINE__);
+    logError(CAT_SQL_HDFS, "HdfsClient::hdfsExists()", getLastError());
     jenv_->PopLocalFrame(NULL);
     return HDFS_CLIENT_ERROR_HDFS_EXISTS_EXCEPTION;
   } 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/HdfsClient_JNI.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/HdfsClient_JNI.h b/core/sql/executor/HdfsClient_JNI.h
index 8adf42f..0426ebc 100644
--- a/core/sql/executor/HdfsClient_JNI.h
+++ b/core/sql/executor/HdfsClient_JNI.h
@@ -24,6 +24,7 @@
 #define HDFS_CLIENT_H
 
 #include "JavaObjectInterface.h"
+#include "ExHdfsScan.h"
 
 // ===========================================================================
 // ===== The native HdfsScan class implements access to the Java methods 
@@ -33,6 +34,11 @@
 typedef enum {
    HDFS_SCAN_OK     = JOI_OK
   ,HDFS_SCAN_FIRST = JOI_LAST
+  ,HDFS_SCAN_ERROR_SET_SCAN_RANGES_PARAM = HDFS_SCAN_FIRST
+  ,HDFS_SCAN_ERROR_SET_SCAN_RANGES_EXCEPTION
+  ,HDFS_SCAN_ERROR_TRAF_HDFS_READ_PARAM
+  ,HDFS_SCAN_ERROR_TRAF_HDFS_READ_EXCEPTION
+  ,HDFS_SCAN_EOR
   ,HDFS_SCAN_LAST
 } HDFS_Scan_RetCode;
 
@@ -44,28 +50,28 @@ public:
   :  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();
+  // Get the error description.
+  static char* getErrorText(HDFS_Scan_RetCode errEnum);
+
+  static HdfsScan *newInstance(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN_BUF *hdfsScanBuf, int scanBufSize, 
+            HdfsFileInfoArray *hdfsFileInfoArray, int rangeTailIOSize, HDFS_Scan_RetCode &hdfsScanRetCode);
+
+  HDFS_Scan_RetCode setScanRanges(NAHeap *heap, ExHdfsScanTcb::HDFS_SCAN_BUF *hdfsScanBuf, int scanBufSize, 
+            HdfsFileInfoArray *hdfsFileInfoArray, int rangeTailIOSize);
 
-  HDFS_Scan_RetCode trafHdfsRead();
+  HDFS_Scan_RetCode trafHdfsRead(NAHeap *heap, int retArray[], short arrayLen);
 
 private:
   enum JAVA_METHODS {
     JM_CTOR = 0, 
-    JM_INIT_SCAN_RANGES,
+    JM_SET_SCAN_RANGES,
     JM_TRAF_HDFS_READ,
     JM_LAST
   };
- 
   static jclass javaClass_;
   static JavaMethodInit* JavaMethods_;
   static bool javaMethodsInitialized_;
@@ -109,7 +115,7 @@ public:
   static HdfsClient *newInstance(NAHeap *heap, HDFS_Client_RetCode &retCode);
 
   // Get the error description.
-  virtual char* getErrorText(HDFS_Client_RetCode errEnum);
+  static char* getErrorText(HDFS_Client_RetCode errEnum);
   
   // Initialize JVM and all the JNI configuration.
   // Must be called.

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/JavaObjectInterface.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/JavaObjectInterface.cpp b/core/sql/executor/JavaObjectInterface.cpp
index 2f5b2c7..ecd8c1e 100644
--- a/core/sql/executor/JavaObjectInterface.cpp
+++ b/core/sql/executor/JavaObjectInterface.cpp
@@ -40,6 +40,9 @@
 
 JavaVM* JavaObjectInterface::jvm_  = NULL;
 jint JavaObjectInterface::jniHandleCapacity_ = 0;
+int JavaObjectInterface::debugPort_ = 0;
+int JavaObjectInterface::debugTimeout_ = 0;
+
 __thread JNIEnv* jenv_ = NULL;
 __thread NAString *tsRecentJMFromJNI = NULL;
 jclass JavaObjectInterface::gThrowableClass = NULL;
@@ -285,7 +288,7 @@ int JavaObjectInterface::createJVM(LmJavaOptions *options)
       if (mySqRoot != NULL)
         {
           len = strlen(mySqRoot); 
-          oomDumpDir = new (heap_) char[len+50];
+          oomDumpDir = new char[len+50];
           strcpy(oomDumpDir, "-XX:HeapDumpPath="); 
           strcat(oomDumpDir, mySqRoot);
           strcat(oomDumpDir, "/logs");
@@ -317,7 +320,7 @@ int JavaObjectInterface::createJVM(LmJavaOptions *options)
   if (classPathArg)
     free(classPathArg);
   if (oomDumpDir)
-    NADELETEBASIC(oomDumpDir, heap_);
+    delete oomDumpDir;
   return ret;
 }
 
@@ -346,8 +349,6 @@ JOI_RetCode JavaObjectInterface::initJVM(LmJavaOptions *options)
          GetCliGlobals()->setJniErrorStr(getErrorText(JOI_ERROR_CHECK_JVM));
          return JOI_ERROR_CREATE_JVM;
       }
-        
-      needToDetach_ = false;
       QRLogger::log(CAT_SQL_HDFS_JNI_TOP, LL_DEBUG, "Created a new JVM.");
     }
     char *jniHandleCapacityStr =  getenv("TRAF_JNIHANDLE_CAPACITY");
@@ -371,7 +372,6 @@ JOI_RetCode JavaObjectInterface::initJVM(LmJavaOptions *options)
       if (result != JNI_OK)
         return JOI_ERROR_ATTACH_JVM;
       
-      needToDetach_ = true;
       QRLogger::log(CAT_SQL_HDFS_JNI_TOP, LL_DEBUG, "Attached to an existing JVM from another thread.");
       break;
        
@@ -537,11 +537,10 @@ void JavaObjectInterface::logError(std::string &cat, const char* file, int line)
 
 NABoolean  JavaObjectInterface::getExceptionDetails(JNIEnv *jenv)
 {
-   NAString error_msg(heap_);
-
    if (jenv == NULL)
        jenv = jenv_;
    CliGlobals *cliGlobals = GetCliGlobals();
+   NAString error_msg(heap_);
    if (jenv == NULL)
    {
       error_msg = "Internal Error - Unable to obtain jenv";
@@ -646,7 +645,6 @@ JOI_RetCode JavaObjectInterface::initJNIEnv()
          return retcode;
   }
   if (jenv_->PushLocalFrame(jniHandleCapacity_) != 0) {
-    getExceptionDetails();
     return JOI_ERROR_INIT_JNI;
   }
   return JOI_OK;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/JavaObjectInterface.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/JavaObjectInterface.h b/core/sql/executor/JavaObjectInterface.h
index 772148b..b167420 100644
--- a/core/sql/executor/JavaObjectInterface.h
+++ b/core/sql/executor/JavaObjectInterface.h
@@ -79,13 +79,10 @@ public:
 protected:
 
   // Default constructor - for creating a new JVM		
-  JavaObjectInterface(NAHeap *heap , int debugPort = 0, int debugTimeout = 0)
+  JavaObjectInterface(NAHeap *heap)
     :  heap_(heap)
       ,javaObj_(NULL)
-      ,needToDetach_(false)
       ,isInitialized_(false)
-      ,debugPort_(debugPort)
-      ,debugTimeout_(debugTimeout)
   {
      tid_ = syscall(SYS_gettid);
   }
@@ -94,10 +91,7 @@ protected:
   JavaObjectInterface(NAHeap *heap, jobject jObj)
     :  heap_(heap)
       ,javaObj_(NULL)
-      ,needToDetach_(false)
       ,isInitialized_(false)
-      ,debugPort_(0)
-      ,debugTimeout_(0)
   {
     tid_ = syscall(SYS_gettid);
     // When jObj is not null in the constructor
@@ -113,17 +107,17 @@ protected:
   virtual ~JavaObjectInterface();
   
   // Create a new JVM
-  int createJVM(LmJavaOptions *options);
+  static int createJVM(LmJavaOptions *options);
   
   // Initialize the JVM.
-  JOI_RetCode    initJVM(LmJavaOptions *options = NULL);
+  static JOI_RetCode    initJVM(LmJavaOptions *options = NULL);
   
   // Initialize JVM and all the JNI configuration.
   // Must be called.
   JOI_RetCode    init(char *className, jclass &javaclass, JavaMethodInit* JavaMethods, Int32 howManyMethods, bool methodsInitialized);
 
   // Get the error description.
-  virtual char* getErrorText(JOI_RetCode errEnum);
+  static char* getErrorText(JOI_RetCode errEnum);
  
   NAString getLastError();
 
@@ -132,8 +126,8 @@ protected:
   void logError(std::string &cat, const char* methodName, jstring jresult);
   void logError(std::string &cat, const char* file, int line);
 
-  JOI_RetCode initJNIEnv();
-  char* buildClassPath();  
+  static JOI_RetCode initJNIEnv();
+  static char* buildClassPath();  
   
 public:
   void setJavaObject(jobject jobj);
@@ -152,6 +146,7 @@ public:
   // Pass in jenv if the thread where the object is created is different than
   // the thread where exception occurred
   NABoolean getExceptionDetails(JNIEnv *jenv = NULL);  
+
   void appendExceptionMessages(JNIEnv *jenv, jthrowable a_exception, NAString &error_msg);
   
   NAHeap *getHeap() { return heap_; }
@@ -166,10 +161,9 @@ protected:
   static jint jniHandleCapacity_;
 
   jobject   javaObj_;
-  bool      needToDetach_;
   bool      isInitialized_;
-  int       debugPort_;
-  int       debugTimeout_;
+  static int debugPort_;
+  static int debugTimeout_;
   pid_t     tid_;
   NAHeap    *heap_;
 };

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/OrcFileReader.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/OrcFileReader.cpp b/core/sql/executor/OrcFileReader.cpp
index ddaa27a..988704d 100644
--- a/core/sql/executor/OrcFileReader.cpp
+++ b/core/sql/executor/OrcFileReader.cpp
@@ -138,7 +138,8 @@ OFR_RetCode OrcFileReader::init()
 							javaClass_,
 							JavaMethods_,
 							(Int32)JM_LAST, javaMethodsInitialized_);
-    javaMethodsInitialized_ = TRUE;
+    if (lv_retcode == OFR_OK)
+       javaMethodsInitialized_ = TRUE;
     pthread_mutex_unlock(&javaMethodsInitMutex_);
   }
   return lv_retcode;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/executor/SequenceFileReader.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/SequenceFileReader.cpp b/core/sql/executor/SequenceFileReader.cpp
index 5bc2f7f..8feb1d5 100644
--- a/core/sql/executor/SequenceFileReader.cpp
+++ b/core/sql/executor/SequenceFileReader.cpp
@@ -115,7 +115,8 @@ SFR_RetCode SequenceFileReader::init()
     JavaMethods_[JM_CLOSE     ].jm_signature = "()Ljava/lang/String;";
    
     rc = (SFR_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
-    javaMethodsInitialized_ = TRUE;
+    if (rc == SFR_OK)
+       javaMethodsInitialized_ = TRUE;
     pthread_mutex_unlock(&javaMethodsInitMutex_);
   }
   return rc;
@@ -440,7 +441,8 @@ SFW_RetCode SequenceFileWriter::init()
     JavaMethods_[JM_CLOSE     ].jm_signature = "()Ljava/lang/String;";
 
     rc = (SFW_RetCode)JavaObjectInterface::init(className, javaClass_, JavaMethods_, (Int32)JM_LAST, javaMethodsInitialized_);
-    javaMethodsInitialized_ = TRUE;
+    if (rc == SFW_OK)
+       javaMethodsInitialized_ = TRUE;
     pthread_mutex_unlock(&javaMethodsInitMutex_);
   }
   return rc;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/exp/ExpErrorEnums.h
----------------------------------------------------------------------
diff --git a/core/sql/exp/ExpErrorEnums.h b/core/sql/exp/ExpErrorEnums.h
index ba604bf..8227cb8 100644
--- a/core/sql/exp/ExpErrorEnums.h
+++ b/core/sql/exp/ExpErrorEnums.h
@@ -163,6 +163,7 @@ enum ExeErrorCode
   EXE_OLAP_OVERFLOW_NOT_SUPPORTED       = 8441,
   EXE_ERROR_FROM_LOB_INTERFACE          = 8442,
   EXE_INVALID_LOB_HANDLE                = 8443,
+  EXE_ERROR_HDFS_SCAN                   = 8447,
   EXE_LAST_EXPRESSIONS_ERROR		= 8499,
 
   // ---------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/exp/ExpHbaseInterface.cpp
----------------------------------------------------------------------
diff --git a/core/sql/exp/ExpHbaseInterface.cpp b/core/sql/exp/ExpHbaseInterface.cpp
index b7746e0..8a1d426 100644
--- a/core/sql/exp/ExpHbaseInterface.cpp
+++ b/core/sql/exp/ExpHbaseInterface.cpp
@@ -48,9 +48,7 @@
 
 ExpHbaseInterface::ExpHbaseInterface(CollHeap * heap,
                                      const char * server,
-                                     const char * zkPort,
-                                     int debugPort,
-                                     int debugTimeout)
+                                     const char * zkPort)
 {
   heap_ = heap;
   hbs_ = NULL;
@@ -66,19 +64,13 @@ ExpHbaseInterface::ExpHbaseInterface(CollHeap * heap,
     strcpy(zkPort_, zkPort);
   else
     zkPort_[0] = 0;
-
-  debugPort_ = debugPort;
-  debugTimeout_ = debugTimeout;
 }
 
 ExpHbaseInterface* ExpHbaseInterface::newInstance(CollHeap* heap,
                                                   const char* server,
-                                                  const char *zkPort,
-                                                  int debugPort,
-                                                  int debugTimeout)
+                                                  const char *zkPort)
 {
-  return new (heap) ExpHbaseInterface_JNI(heap, server, TRUE, zkPort,
-                                          debugPort, debugTimeout); // This is the transactional interface
+  return new (heap) ExpHbaseInterface_JNI(heap, server, TRUE,zkPort);
 }
 
 NABoolean isParentQueryCanceled()
@@ -283,8 +275,8 @@ char * getHbaseErrStr(Lng32 errEnum)
 // ===========================================================================
 
 ExpHbaseInterface_JNI::ExpHbaseInterface_JNI(CollHeap* heap, const char* server, bool useTRex,
-                                             const char *zkPort, int debugPort, int debugTimeout)
-     : ExpHbaseInterface(heap, server, zkPort, debugPort, debugTimeout)
+                                             const char *zkPort)
+     : ExpHbaseInterface(heap, server, zkPort)
    ,useTRex_(useTRex)
    ,client_(NULL)
    ,htc_(NULL)
@@ -324,7 +316,7 @@ Lng32 ExpHbaseInterface_JNI::init(ExHbaseAccessStats *hbs)
   if (client_ == NULL)
   {
     HBaseClient_JNI::logIt("ExpHbaseInterface_JNI::init() creating new client.");
-    client_ = HBaseClient_JNI::getInstance(debugPort_, debugTimeout_);
+    client_ = HBaseClient_JNI::getInstance();
     
     if (client_->isInitialized() == FALSE)
     {

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/exp/ExpHbaseInterface.h
----------------------------------------------------------------------
diff --git a/core/sql/exp/ExpHbaseInterface.h b/core/sql/exp/ExpHbaseInterface.h
index f7b23cb..f68de05 100644
--- a/core/sql/exp/ExpHbaseInterface.h
+++ b/core/sql/exp/ExpHbaseInterface.h
@@ -71,9 +71,7 @@ class ExpHbaseInterface : public NABasicObject
 
   static ExpHbaseInterface* newInstance(CollHeap* heap, 
                                         const char* server = NULL, 
-                                        const char *zkPort = NULL, 
-                                        int debugPort = 0, 
-                                        int DebugTimeout = 0);
+                                        const char *zkPort = NULL);
 
   virtual ~ExpHbaseInterface()
   {}
@@ -389,16 +387,12 @@ protected:
 
   ExpHbaseInterface(CollHeap * heap,
                     const char * server = NULL,
-                    const char * zkPort = NULL,
-                    int debugPort = 0,
-                    int debugTimeout = 0);
+                    const char * zkPort = NULL);
   
   CollHeap * heap_;
   ExHbaseAccessStats * hbs_;
   char server_[MAX_SERVER_SIZE+1];
   char zkPort_[MAX_PORT_SIZE+1];
-  int  debugPort_;
-  int  debugTimeout_;
 };
 
 char * getHbaseErrStr(Lng32 errEnum);
@@ -410,7 +404,7 @@ class ExpHbaseInterface_JNI : public ExpHbaseInterface
 
   ExpHbaseInterface_JNI(CollHeap* heap,
                         const char* server, bool useTRex,
-                        const char *zkPort, int debugPort, int debugTimeout);
+                        const char *zkPort);
   
   virtual ~ExpHbaseInterface_JNI();
   

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/exp/ExpLOBinterface.h
----------------------------------------------------------------------
diff --git a/core/sql/exp/ExpLOBinterface.h b/core/sql/exp/ExpLOBinterface.h
index df6c142..b98d2b4 100644
--- a/core/sql/exp/ExpLOBinterface.h
+++ b/core/sql/exp/ExpLOBinterface.h
@@ -344,19 +344,6 @@ Lng32 ExpLOBInterfaceGetLobLength(ExLobGlobals * exLobGlob,
 				  );
 
 
-/*
-class HdfsFileInfo
-{
- public:
-  char * fileName() { return fileName_; }
-  Int64 getStartOffset() { return startOffset_; }
-  Int64 getBytesToRead() { return bytesToRead_; }
-  Lng32 entryNum_; // 0 based, first entry is entry num 0.
-  NABasicPtr  fileName_;
-  Int64 startOffset_;
-  Int64 bytesToRead_;
-};
-*/
 #endif
 
 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/generator/GenRelScan.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenRelScan.cpp b/core/sql/generator/GenRelScan.cpp
index 013651b..3d01223 100644
--- a/core/sql/generator/GenRelScan.cpp
+++ b/core/sql/generator/GenRelScan.cpp
@@ -1391,6 +1391,9 @@ if (hTabStats->isOrcFile())
   hdfsscan_tdb->setUseCif(useCIF);
   hdfsscan_tdb->setUseCifDefrag(useCIFDegrag);
 
+  if (CmpCommon::getDefault(USE_LIBHDFS_SCAN) == DF_ON)
+     hdfsscan_tdb->setUseLibhdfsScan(TRUE);
+
   if(!generator->explainDisabled()) {
     generator->setExplainTuple(
        addExplainInfo(hdfsscan_tdb, 0, 0, generator));

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/qmscommon/QRLogger.cpp
----------------------------------------------------------------------
diff --git a/core/sql/qmscommon/QRLogger.cpp b/core/sql/qmscommon/QRLogger.cpp
index f4fb75d..0b8c398 100644
--- a/core/sql/qmscommon/QRLogger.cpp
+++ b/core/sql/qmscommon/QRLogger.cpp
@@ -60,6 +60,7 @@ std::string CAT_SQL_HDFS_SEQ_FILE_READER      =  "SQL.HDFS.SeqFileReader";
 std::string CAT_SQL_HDFS_SEQ_FILE_WRITER      =  "SQL.HDFS.SeqFileWriter";
 std::string CAT_SQL_HDFS_ORC_FILE_READER      =  "SQL.HDFS.OrcFileReader";
 std::string CAT_SQL_HBASE                     =  "SQL.HBase";
+std::string CAT_SQL_HDFS                      =  "SQL.HDFS";
 
 // these categories are currently not used 
 std::string CAT_SQL_QMP                       = "SQL.Qmp";

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/qmscommon/QRLogger.h
----------------------------------------------------------------------
diff --git a/core/sql/qmscommon/QRLogger.h b/core/sql/qmscommon/QRLogger.h
index 5cabac4..3be016e 100644
--- a/core/sql/qmscommon/QRLogger.h
+++ b/core/sql/qmscommon/QRLogger.h
@@ -67,6 +67,7 @@ extern std::string CAT_SQL_HDFS_SEQ_FILE_READER;
 extern std::string CAT_SQL_HDFS_SEQ_FILE_WRITER;
 extern std::string CAT_SQL_HDFS_ORC_FILE_READER;
 extern std::string CAT_SQL_HBASE;
+extern std::string CAT_SQL_HDFS;
 
 class ComDiagsArea;
 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/sqlcomp/DefaultConstants.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/DefaultConstants.h b/core/sql/sqlcomp/DefaultConstants.h
index b7e4d0d..339d55f 100644
--- a/core/sql/sqlcomp/DefaultConstants.h
+++ b/core/sql/sqlcomp/DefaultConstants.h
@@ -3306,6 +3306,9 @@ enum DefaultConstants
   SUPPRESS_CHAR_LIMIT_CHECK,
  
   BMO_MEMORY_ESTIMATE_OUTLIER_FACTOR,
+
+  // Use the earlier implementation of HdfsScan via libhdfs
+  USE_LIBHDFS_SCAN,
   // This enum constant must be the LAST one in the list; it's a count,
   // not an Attribute (it's not IN DefaultDefaults; it's the SIZE of it)!
   __NUM_DEFAULT_ATTRIBUTES

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/sqlcomp/nadefaults.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/nadefaults.cpp b/core/sql/sqlcomp/nadefaults.cpp
index a75c20b..28148cd 100644
--- a/core/sql/sqlcomp/nadefaults.cpp
+++ b/core/sql/sqlcomp/nadefaults.cpp
@@ -3033,6 +3033,8 @@ XDDkwd__(SUBQUERY_UNNESTING,			"ON"),
   // Use large queues on RHS of Flow/Nested Join when appropriate
   DDkwd__(USE_LARGE_QUEUES,                     "ON"),
 
+  DDkwd__(USE_LIBHDFS_SCAN,                     "ON"),
+
   DDkwd__(USE_MAINTAIN_CONTROL_TABLE,          "OFF"),
 
   DDkwd__(USE_OLD_DT_CONSTRUCTOR,      "OFF"),

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
index 8d2052f..1af2c49 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
@@ -54,6 +54,7 @@ public class HDFSClient
    private static FileSystem defaultFs_ = null;
    private FileSystem fs_ = null;
    private int bufNo_;
+   private int rangeNo_;
    private FSDataInputStream fsdis_; 
    private OutputStream outStream_;
    private String filename_;
@@ -66,7 +67,7 @@ public class HDFSClient
    private int blockSize_; 
    private int bytesRead_;
    private Future future_ = null;
-    
+   private int isEOF_ = 0; 
    static {
       String confFile = System.getProperty("trafodion.log4j.configFile");
       System.setProperty("trafodion.root", System.getenv("TRAF_HOME"));
@@ -111,9 +112,10 @@ public class HDFSClient
    {
    }
  
-   public HDFSClient(int bufNo, String filename, ByteBuffer buffer, long position, int length) throws IOException
+   public HDFSClient(int bufNo, int rangeNo, String filename, ByteBuffer buffer, long position, int length) throws IOException
    {
       bufNo_ = bufNo; 
+      rangeNo_ = rangeNo;
       filename_ = filename;
       Path filepath = new Path(filename_);
       fs_ = FileSystem.get(filepath.toUri(),config_);
@@ -164,13 +166,27 @@ public class HDFSClient
       return bytesRead;
    } 
 
+   public int getRangeNo()
+   {
+      return rangeNo_;
+   }
+  
+   public int isEOF()
+   {
+      return isEOF_;
+   }
+
    public int trafHdfsReadBuffer() throws IOException, InterruptedException, ExecutionException
    {
       int bytesRead;
       int totalBytesRead = 0;
       while (true) {
          bytesRead = trafHdfsRead();
-         if (bytesRead == -1 || bytesRead == 0)
+         if (bytesRead == -1) {
+            isEOF_ = 1;
+            return totalBytesRead;
+         }
+         if (bytesRead == 0)
             return totalBytesRead;
          totalBytesRead += bytesRead;
          if (totalBytesRead == bufLen_)

http://git-wip-us.apache.org/repos/asf/trafodion/blob/f17e15ee/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
index bf81ab0..9fb145e 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
@@ -88,7 +88,11 @@ public class HdfsScan
       System.setProperty("trafodion.root", System.getenv("TRAF_HOME"));
    }
 
-   HdfsScan(ByteBuffer buf1, ByteBuffer buf2, String filename[], long pos[], long len[]) throws IOException
+   public HdfsScan() 
+   {
+   }
+
+   public void setScanRanges(ByteBuffer buf1, ByteBuffer buf2, String filename[], long pos[], long len[]) throws IOException
    {
       buf_ = new ByteBuffer[2];
       bufLen_ = new int[2];
@@ -119,13 +123,14 @@ public class HdfsScan
 
    public void hdfsScanRange(int bufNo) throws IOException
    {
-      System.out.println (" CurrentRange " + currRange_ + " LenRemain " + lenRemain_ + " BufNo " + bufNo); 
+      if (logger_.isDebugEnabled())
+         logger_.debug(" CurrentRange " + currRange_ + " LenRemain " + lenRemain_ + " BufNo " + bufNo); 
       int readLength;
       if (lenRemain_ > bufLen_[bufNo])
          readLength = bufLen_[bufNo];
       else
          readLength = (int)lenRemain_;
-      hdfsClient_[bufNo] = new HDFSClient(bufNo, hdfsScanRanges_[currRange_].filename_, buf_[bufNo], currPos_, readLength);
+      hdfsClient_[bufNo] = new HDFSClient(bufNo, currRange_, hdfsScanRanges_[currRange_].filename_, buf_[bufNo], currPos_, readLength);
       lenRemain_ -= readLength;
       currPos_ += readLength; 
       if (lenRemain_ == 0) {
@@ -144,29 +149,44 @@ public class HdfsScan
       int[] retArray;
       int byteCompleted;
       int bufNo;
- 
+      int rangeNo;
+      int isEOF;
+  
+      if (hdfsScanRanges_ == null)
+         throw new IOException("Scan ranges are not yet set"); 
       if (scanCompleted_)
          return null; 
-      retArray = new int[2];
+      retArray = new int[4];
       switch (lastBufCompleted_) {
          case -1:
          case 1:
             byteCompleted = hdfsClient_[0].trafHdfsReadBuffer(); 
             bufNo = 0;
+            rangeNo = hdfsClient_[0].getRangeNo();
+            isEOF = hdfsClient_[0].isEOF();
             break;
          case 0:
             byteCompleted = hdfsClient_[1].trafHdfsReadBuffer(); 
             bufNo = 1;
+            rangeNo = hdfsClient_[1].getRangeNo();
+            isEOF = hdfsClient_[1].isEOF();
             break;
          default:
             bufNo = -1;
             byteCompleted = -1;
+            rangeNo = -1;
+            isEOF = 0;
       }    
       lastBufCompleted_ = bufNo;
       retArray[0] = byteCompleted;
       retArray[1] = bufNo;
-      System.out.println (" Buffer No " + retArray[1] + " Bytes Read " + retArray[0]); 
+      retArray[2] = rangeNo; 
+      retArray[3] = isEOF;
+      if (logger_.isDebugEnabled())
+         logger_.debug(" Range No " + retArray[2] + " Buffer No " + retArray[1] + " Bytes Read " + retArray[0] + " isEOF " + retArray[3]); 
       lastBufCompleted_ = bufNo;
+      if ((isEOF == 1) && (currRange_ == (hdfsScanRanges_.length-1))) 
+         lastScanRangeScheduled_ = true;
       if (lastScanRangeScheduled_) {
          scanCompleted_ = true;
          return retArray; 
@@ -233,7 +253,8 @@ public class HdfsScan
          }
       }
       long time1 = System.currentTimeMillis();
-      HdfsScan hdfsScan = new HdfsScan(buf1, buf2, fileName, pos, len);
+      HdfsScan hdfsScan = new HdfsScan();
+      hdfsScan.setScanRanges(buf1, buf2, fileName, pos, len);
       int[] retArray;
       int bytesCompleted;
       while (true) {


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

Posted by se...@apache.org.
Merge PR 1417 [TRAFODION-2917] Refactor Trafodion implementation of hdfs scan for text formatted hive tables


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

Branch: refs/heads/master
Commit: de8357677276e3987f24b550da5b7d0fc2b8200c
Parents: 087af70 cc43432
Author: selvaganesang <se...@apache.org>
Authored: Fri Feb 16 19:59:39 2018 +0000
Committer: selvaganesang <se...@apache.org>
Committed: Fri Feb 16 19:59:39 2018 +0000

----------------------------------------------------------------------
 core/sql/comexe/ComTdbHdfsScan.h                |  10 +-
 core/sql/executor/ExExeUtil.h                   |   3 +-
 core/sql/executor/ExExeUtilCli.cpp              |   1 -
 core/sql/executor/ExExeUtilGet.cpp              |  16 +-
 core/sql/executor/ExExeUtilLoad.cpp             |  53 +-
 core/sql/executor/ExExeUtilMisc.cpp             |   1 -
 core/sql/executor/ExFastTransport.cpp           | 100 ++-
 core/sql/executor/ExFastTransport.h             |   5 +-
 core/sql/executor/ExHbaseAccess.cpp             |  53 +-
 core/sql/executor/ExHbaseAccess.h               |  32 +-
 core/sql/executor/ExHbaseIUD.cpp                |  58 +-
 core/sql/executor/ExHdfsScan.cpp                | 378 ++++++---
 core/sql/executor/ExHdfsScan.h                  |  69 +-
 core/sql/executor/HBaseClient_JNI.cpp           | 147 +---
 core/sql/executor/HBaseClient_JNI.h             |  23 +-
 core/sql/executor/HdfsClient_JNI.cpp            | 763 +++++++++++++++++++
 core/sql/executor/HdfsClient_JNI.h              | 192 +++++
 core/sql/executor/JavaObjectInterface.cpp       |  24 +-
 core/sql/executor/JavaObjectInterface.h         |  24 +-
 core/sql/executor/OrcFileReader.cpp             |   3 +-
 core/sql/executor/SequenceFileReader.cpp        | 304 +-------
 core/sql/executor/SequenceFileReader.h          |  30 -
 core/sql/exp/ExpErrorEnums.h                    |   1 +
 core/sql/exp/ExpHbaseInterface.cpp              |  93 +--
 core/sql/exp/ExpHbaseInterface.h                |  18 +-
 core/sql/exp/ExpLOBinterface.h                  |  13 -
 core/sql/generator/GenRelScan.cpp               |   3 +
 core/sql/nskgmake/executor/Makefile             |   1 +
 core/sql/qmscommon/QRLogger.cpp                 |   1 +
 core/sql/qmscommon/QRLogger.h                   |   1 +
 core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp        |   1 -
 core/sql/sqlcomp/DefaultConstants.h             |   3 +
 core/sql/sqlcomp/nadefaults.cpp                 |   2 +
 .../main/java/org/trafodion/sql/HDFSClient.java | 394 ++++++++++
 .../main/java/org/trafodion/sql/HdfsScan.java   | 289 +++++++
 .../org/trafodion/sql/SequenceFileWriter.java   | 154 ----
 .../org/trafodion/sql/TrafConfiguration.java    |   1 +
 37 files changed, 2244 insertions(+), 1020 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/de835767/core/sql/executor/ExExeUtil.h
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/de835767/core/sql/executor/ExExeUtilLoad.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/de835767/core/sql/executor/ExExeUtilMisc.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/de835767/core/sql/executor/ExFastTransport.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/de835767/core/sql/executor/ExHdfsScan.cpp
----------------------------------------------------------------------
diff --cc core/sql/executor/ExHdfsScan.cpp
index 7dd933b,c36270e..2b73feb
--- a/core/sql/executor/ExHdfsScan.cpp
+++ b/core/sql/executor/ExHdfsScan.cpp
@@@ -280,9 -307,15 +307,15 @@@ void ExHdfsScanTcb::freeResources(
    }
    deallocateRuntimeRanges();
    if (lobGlob_) { 
 -     ExpLOBinterfaceCleanup(lobGlob_, (NAHeap *)getGlobals()->getDefaultHeap());
 +     ExpLOBinterfaceCleanup(lobGlob_);
       lobGlob_ = NULL;
    }
+   if (hdfsClient_ != NULL) 
+      NADELETE(hdfsClient_, HdfsClient, getHeap());
+   if (logFileHdfsClient_ != NULL) 
+      NADELETE(logFileHdfsClient_, HdfsClient, getHeap());
+   if (hdfsScan_ != NULL) 
+      NADELETE(hdfsScan_, HdfsScan, getHeap());
  }
  
  NABoolean ExHdfsScanTcb::needStatsEntry()

http://git-wip-us.apache.org/repos/asf/trafodion/blob/de835767/core/sql/exp/ExpLOBinterface.h
----------------------------------------------------------------------
diff --cc core/sql/exp/ExpLOBinterface.h
index c8a8591,b98d2b4..8194859
--- a/core/sql/exp/ExpLOBinterface.h
+++ b/core/sql/exp/ExpLOBinterface.h
@@@ -342,42 -342,8 +342,29 @@@ Lng32 ExpLOBInterfaceGetLobLength(ExLob
  			          Int64 &outLobLen 
                                    
  				  );
 +Lng32 ExpLOBInterfaceGetFileName(ExLobGlobals * exLobGlob, 
 +				  char * lobName, 
 +				  char * lobLoc,
 +				  Lng32 lobType,
 +				  char * lobHdfsServer,
 +				  Lng32 lobHdfsPort,
 +				  Int32 handleLen, 
 +                                  char * lobHandle,  
 +                                 char * outFileName,
 +                                  Int32 &outFileLen);
  
 +Lng32 ExpLOBInterfaceGetOffset(ExLobGlobals * exLobGlob, 
 +				  char * lobName, 
 +				  char * lobLoc,
 +				  Lng32 lobType,
 +				  char * lobHdfsServer,
 +				  Lng32 lobHdfsPort,
 +				  Int32 handleLen, 
 +				  char * lobHandle,
 +			          Int64 &outLobOffset 
 +                                  
 +				  );
  
- /*
- class HdfsFileInfo
- {
-  public:
-   char * fileName() { return fileName_; }
-   Int64 getStartOffset() { return startOffset_; }
-   Int64 getBytesToRead() { return bytesToRead_; }
-   Lng32 entryNum_; // 0 based, first entry is entry num 0.
-   NABasicPtr  fileName_;
-   Int64 startOffset_;
-   Int64 bytesToRead_;
- };
- */
  #endif
  
  

http://git-wip-us.apache.org/repos/asf/trafodion/blob/de835767/core/sql/qmscommon/QRLogger.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/de835767/core/sql/qmscommon/QRLogger.h
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/de835767/core/sql/sqlcomp/DefaultConstants.h
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/de835767/core/sql/sqlcomp/nadefaults.cpp
----------------------------------------------------------------------


[6/9] trafodion git commit: [TRAFODION-2917] Refactor Trafodion implementation of hdfs scan for trafodion

Posted by se...@apache.org.
[TRAFODION-2917] Refactor Trafodion implementation of hdfs scan for trafodion

With the change hive regressions pass except hive/TEST003 when
cqd use_libhdfs_scan 'off'


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

Branch: refs/heads/master
Commit: a187b03bc37d62d00278d21eef519496ea6ce1aa
Parents: 7066e3e
Author: selvaganesang <se...@esgyn.com>
Authored: Fri Feb 9 00:56:48 2018 +0000
Committer: selvaganesang <se...@esgyn.com>
Committed: Fri Feb 9 00:56:48 2018 +0000

----------------------------------------------------------------------
 core/sql/executor/ExHdfsScan.cpp                       |  2 ++
 .../src/main/java/org/trafodion/sql/HDFSClient.java    | 12 ++++++++++--
 core/sql/src/main/java/org/trafodion/sql/HdfsScan.java | 13 ++++++++++---
 3 files changed, 22 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/a187b03b/core/sql/executor/ExHdfsScan.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExHdfsScan.cpp b/core/sql/executor/ExHdfsScan.cpp
index f8ec9a1..cd95899 100644
--- a/core/sql/executor/ExHdfsScan.cpp
+++ b/core/sql/executor/ExHdfsScan.cpp
@@ -127,6 +127,8 @@ ExHdfsScanTcb::ExHdfsScanTcb(
   Space * space = (glob ? glob->getSpace() : 0);
   CollHeap * heap = (glob ? glob->getDefaultHeap() : 0);
   useLibhdfsScan_ = hdfsScanTdb.getUseLibhdfsScan();
+  if (isSequenceFile())
+     useLibhdfsScan_ = TRUE;
   lobGlob_ = NULL;
   hdfsScanBufMaxSize_ = hdfsScanTdb.hdfsBufSize_;
   headRoom_ = (Int32)hdfsScanTdb.rangeTailIOSize_;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a187b03b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
index 3b83c8f..5c8c487 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HDFSClient.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.conf.Configuration;
 import java.nio.ByteBuffer;
 import java.io.IOException;
+import java.io.EOFException;
 import java.io.OutputStream;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Future;
@@ -95,8 +96,14 @@ public class HDFSClient
       {
          int bytesRead;
          int totalBytesRead = 0;
-         if (! buf_.hasArray())
-            fsdis_.seek(pos_);
+         if (! buf_.hasArray()) {
+            try {
+              fsdis_.seek(pos_);
+            } catch (EOFException e) {
+              isEOF_ = 1;
+              return new Integer(totalBytesRead);
+            } 
+         }
          do
          {
             if (buf_.hasArray())
@@ -157,6 +164,7 @@ public class HDFSClient
       int bytesRead;
       retObject = (Integer)future_.get();
       bytesRead = retObject.intValue();
+      fsdis_.close();
       return bytesRead;
    }
 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/a187b03b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
index f3d505d..e216555 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HdfsScan.java
@@ -188,9 +188,16 @@ public class HdfsScan
       if (logger_.isDebugEnabled())
          logger_.debug(" Range No " + retArray[2] + " Buffer No " + retArray[1] + " Bytes Read " + retArray[0] + " isEOF " + retArray[3]); 
       lastBufCompleted_ = bufNo;
-      if ((isEOF == 1) && (currRange_ == (hdfsScanRanges_.length-1))) {
-         scanCompleted_ = true;
-         return retArray;
+      if (isEOF == 1) {
+         if (currRange_ == (hdfsScanRanges_.length-1)) {
+            scanCompleted_ = true;
+            return retArray;
+         } else {
+            currRange_++;
+            currPos_ = hdfsScanRanges_[currRange_].pos_;
+            lenRemain_ = hdfsScanRanges_[currRange_].len_;
+            bytesRead = 0;
+         }
       }
       switch (lastBufCompleted_)
       {