[
https://issues.apache.org/jira/browse/TRAFODION-2917?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16360300#comment-16360300
]
ASF GitHub Bot commented on TRAFODION-2917:
-------------------------------------------
Github user sureshsubbiah commented on a diff in the pull request:
https://github.com/apache/trafodion/pull/1417#discussion_r167466431
--- Diff: 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.EOFException;
+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 int rangeNo_;
+ 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;
+ private int isEOF_ = 0;
+ 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
+ {
+ HDFSRead()
+ {
+ }
+
+ public Object call() throws IOException
+ {
+ int bytesRead;
+ int totalBytesRead = 0;
+ if (! buf_.hasArray()) {
--- End diff --
Does this condition mean that somehow the buffer we got from the C side is
empty or not accessible?
> Refactor Trafodion implementation of hdfs scan for text formatted hive tables
> -----------------------------------------------------------------------------
>
> Key: TRAFODION-2917
> URL: https://issues.apache.org/jira/browse/TRAFODION-2917
> Project: Apache Trafodion
> Issue Type: New Feature
> Components: sql-general
> Reporter: Selvaganesan Govindarajan
> Priority: Major
> Fix For: 2.3
>
>
> Find below the general outline of hdfs scan for text formatted hive tables.
> Compiler returns a list of scan ranges and the begin range and number of
> ranges to be done by each instance of TCB in TDB. This list of scan ranges is
> also re-computed at run time possibly based on a CQD
> The scan range for a TCB can come from the same or different hdfs files. TCB
> creates two threads to read these ranges.Two ranges (for the TCB) are
> initially assigned to these threads. As and when a range is completed, the
> next range (assigned for the TCB) is picked up by the thread. Ranges are read
> in multiples of hdfs scan buffer size at the TCB level. Default hdfs scan
> buffer size is 64 MB. Rows from hdfs scan buffer is processed and moved into
> up queue. If the range contains a record split, then the range is extended to
> read up to range tail IO size to get the full row. The range that had the
> latter part of the row ignores it because the former range processes it.
> Record split at the file level is not possible and/or not supported.
> For compression, the compiler returns the range info such that the hdfs scan
> buffer can hold the full uncompressed buffer.
> Cons:
> Reader threads feature too complex to maintain in C++
> Error handling at the layer below the TCB is missing or errors are not
> propagated to work method causing incorrect results
> Possible multiple copying of data
> Libhdfs calls are not optimized. It was observed that the method Ids are
> being obtained many times. Need to check if this problem still exists.
> Now that we clearly know what is expected, it could be optimized better
> - Reduced scan buffer size for smoother data flow
> - Better thread utilization
> - Avoid multiple copying of data.
> Unable to comprehend the need for two threads for pre-fetch especially when
> one range is completed fully before the data from next range is processed.
> Following are the hdfsCalls used by programs at exp and executor directory.
> U hdfsCloseFile
> U hdfsConnect
> U hdfsDelete
> U hdfsExists
> U hdfsFlush
> U hdfsFreeFileInfo
> U hdfsGetPathInfo
> U hdfsListDirectory
> U hdfsOpenFile
> U hdfsPread
> U hdfsRename
> U hdfsWrite
> U hdfsCreateDirectory
> New implementation
> Make changes to use direct Java APIs for these calls. However, come up with
> better mechanism to move the data from Java and JNI, avoid unnecessary
> copying of data, better thread management via Executor concepts in Java.
> Hence it won’t be direct mapping of these calls to hdfs Java API. Instead,
> use the abstraction like what is being done for HBase access.
> I believe newer implementation will be optimized better and hence improved
> performance. (but not many folds)
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)