[2/2] hive git commit: HIVE-14136 : LLAP ZK SecretManager should resolve _HOST in principal (Sergey Shelukhin, reviewed by Siddharth Seth)

2016-06-30 Thread sershe
HIVE-14136 : LLAP ZK SecretManager should resolve _HOST in principal (Sergey 
Shelukhin, reviewed by Siddharth Seth)

Conflicts:

llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java


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

Branch: refs/heads/branch-2.1
Commit: e2da0e163dba1ecb58fe66d319df22cf1efd970a
Parents: 19192a6
Author: Sergey Shelukhin 
Authored: Thu Jun 30 16:00:04 2016 -0700
Committer: Sergey Shelukhin 
Committed: Thu Jun 30 16:02:33 2016 -0700

--
 .../hadoop/hive/llap/security/SecretManager.java   | 13 ++---
 1 file changed, 10 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/e2da0e16/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
--
diff --git 
a/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java 
b/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
index 8eca946..dc06cc9 100644
--- 
a/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
+++ 
b/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.LlapUtil;
 import org.apache.hadoop.hive.llap.security.LlapTokenIdentifier;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
@@ -61,6 +62,7 @@ public class SecretManager extends 
ZKDelegationTokenSecretManager() {
   @Override



[1/2] hive git commit: HIVE-14136 : LLAP ZK SecretManager should resolve _HOST in principal (Sergey Shelukhin, reviewed by Siddharth Seth)

2016-06-30 Thread sershe
Repository: hive
Updated Branches:
  refs/heads/branch-2.1 19192a631 -> e2da0e163
  refs/heads/master 2de64b0b0 -> 4d349dadb


HIVE-14136 : LLAP ZK SecretManager should resolve _HOST in principal (Sergey 
Shelukhin, reviewed by Siddharth Seth)


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

Branch: refs/heads/master
Commit: 4d349dadb2c6103b8b1f50226f2384b519dcc59b
Parents: 2de64b0
Author: Sergey Shelukhin 
Authored: Thu Jun 30 16:00:04 2016 -0700
Committer: Sergey Shelukhin 
Committed: Thu Jun 30 16:00:04 2016 -0700

--
 .../hadoop/hive/llap/security/SecretManager.java | 15 ++-
 1 file changed, 10 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/4d349dad/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
--
diff --git 
a/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java 
b/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
index 9431e4f..f43e74d 100644
--- 
a/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
+++ 
b/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.LlapUtil;
 import org.apache.hadoop.hive.llap.security.LlapTokenIdentifier;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
@@ -44,8 +45,6 @@ import org.apache.zookeeper.data.Id;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.protobuf.ByteString;
-
 public class SecretManager extends 
ZKDelegationTokenSecretManager
   implements SigningSecretManager {
   private static final Logger LOG = 
LoggerFactory.getLogger(SecretManager.class);
@@ -63,6 +62,7 @@ public class SecretManager extends 
ZKDelegationTokenSecretManager() {



hive git commit: HIVE-14078: LLAP input split should get task attempt number from conf if available (Jason Dere, reviewed by Prasanth Jayachandran/Siddharth Seth)

2016-06-30 Thread jdere
Repository: hive
Updated Branches:
  refs/heads/master df51738ad -> 2de64b0b0


HIVE-14078: LLAP input split should get task attempt number from conf if 
available (Jason Dere, reviewed by Prasanth Jayachandran/Siddharth Seth)


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

Branch: refs/heads/master
Commit: 2de64b0b0575264aa9716bc6fa824cd076884257
Parents: df51738
Author: Jason Dere 
Authored: Thu Jun 30 13:14:03 2016 -0700
Committer: Jason Dere 
Committed: Thu Jun 30 13:14:03 2016 -0700

--
 .../hadoop/hive/llap/LlapBaseInputFormat.java   | 22 +++-
 1 file changed, 17 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/2de64b0b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
--
diff --git 
a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java 
b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
index aef5762..c2fca54 100644
--- 
a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
+++ 
b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
@@ -59,6 +59,8 @@ import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
@@ -137,15 +139,25 @@ public class LlapBaseInputFormat>
 
 LlapRecordReaderTaskUmbilicalExternalResponder umbilicalResponder =
 new LlapRecordReaderTaskUmbilicalExternalResponder();
-// TODO: close this
 LlapTaskUmbilicalExternalClient llapClient =
   new LlapTaskUmbilicalExternalClient(job, 
submitWorkInfo.getTokenIdentifier(),
   submitWorkInfo.getToken(), umbilicalResponder, llapToken);
 llapClient.init(job);
 llapClient.start();
 
+int attemptNum = 0;
+// Use task attempt number from conf if provided
+TaskAttemptID taskAttemptId = 
TaskAttemptID.forName(job.get(MRJobConfig.TASK_ATTEMPT_ID));
+if (taskAttemptId != null) {
+  attemptNum = taskAttemptId.getId();
+  if (LOG.isDebugEnabled()) {
+LOG.debug("Setting attempt number to " + attemptNum + " from task 
attempt ID in conf: " +
+job.get(MRJobConfig.TASK_ATTEMPT_ID));
+  }
+}
+
 SubmitWorkRequestProto request = constructSubmitWorkRequestProto(
-submitWorkInfo, llapSplit.getSplitNum(), llapClient.getAddress(),
+submitWorkInfo, llapSplit.getSplitNum(), attemptNum, 
llapClient.getAddress(),
 submitWorkInfo.getToken(), llapSplit.getFragmentBytes(),
 llapSplit.getFragmentBytesSignature());
 llapClient.submitWork(request, host, llapSubmitPort);
@@ -275,7 +287,7 @@ public class LlapBaseInputFormat>
   }
 
   private SubmitWorkRequestProto 
constructSubmitWorkRequestProto(SubmitWorkInfo submitWorkInfo,
-  int taskNum, InetSocketAddress address, Token token,
+  int taskNum, int attemptNum, InetSocketAddress address, 
Token token,
   byte[] fragmentBytes, byte[] fragmentBytesSignature) throws IOException {
 ApplicationId appId = submitWorkInfo.getFakeAppId();
 
@@ -284,7 +296,7 @@ public class LlapBaseInputFormat>
 LOG.info("Setting user in submitWorkRequest to: " + user);
 
 ContainerId containerId =
-ContainerId.newInstance(ApplicationAttemptId.newInstance(appId, 0), 
taskNum);
+ContainerId.newInstance(ApplicationAttemptId.newInstance(appId, 
attemptNum), taskNum);
 
 // Credentials can change across DAGs. Ideally construct only once per DAG.
 Credentials credentials = new Credentials();
@@ -309,7 +321,7 @@ public class LlapBaseInputFormat>
 }
 builder.setWorkSpec(vertexBuilder.build());
 builder.setFragmentNumber(taskNum);
-builder.setAttemptNumber(0); // TODO: hmm
+builder.setAttemptNumber(attemptNum);
 builder.setContainerIdString(containerId.toString());
 builder.setAmHost(address.getHostName());
 builder.setAmPort(address.getPort());



hive git commit: HIVE-14091 : some errors are not propagated to LLAP external clients (Sergey Shelukhin/Jason Dere, reviewed by Jason Dere/Sergey Shelukhin)

2016-06-30 Thread sershe
Repository: hive
Updated Branches:
  refs/heads/master 2c7f2e9d6 -> df51738ad


HIVE-14091 : some errors are not propagated to LLAP external clients (Sergey 
Shelukhin/Jason Dere, reviewed by Jason Dere/Sergey Shelukhin)


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

Branch: refs/heads/master
Commit: df51738ad8c0de6e57b5b08bc20a06f0b3c734b7
Parents: 2c7f2e9
Author: Sergey Shelukhin 
Authored: Thu Jun 30 12:21:22 2016 -0700
Committer: Sergey Shelukhin 
Committed: Thu Jun 30 12:23:14 2016 -0700

--
 .../hadoop/hive/llap/LlapBaseRecordReader.java   | 19 +++
 .../hadoop/hive/llap/LlapBaseInputFormat.java|  4 ++--
 .../hadoop/hive/llap/TestLlapOutputFormat.java   |  3 ++-
 3 files changed, 19 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/df51738a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
--
diff --git 
a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java 
b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
index f2700c8..59dec1b 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
@@ -54,8 +54,10 @@ public class LlapBaseRecordReader implements Recor
   protected final LinkedBlockingQueue readerEvents = new 
LinkedBlockingQueue();
   protected final long timeout;
   protected final Closeable client;
+  private final Closeable socket;
 
-  public LlapBaseRecordReader(InputStream in, Schema schema, Class clazz, 
JobConf job, Closeable client) {
+  public LlapBaseRecordReader(InputStream in, Schema schema,
+  Class clazz, JobConf job, Closeable client, Closeable socket) {
 din = new DataInputStream(in);
 this.schema = schema;
 this.clazz = clazz;
@@ -63,6 +65,7 @@ public class LlapBaseRecordReader implements Recor
 this.timeout = 3 * HiveConf.getTimeVar(job,
 HiveConf.ConfVars.LLAP_DAEMON_AM_LIVENESS_CONNECTION_TIMEOUT_MS, 
TimeUnit.MILLISECONDS);
 this.client = client;
+this.socket = socket;
   }
 
   public Schema getSchema() {
@@ -78,6 +81,7 @@ public class LlapBaseRecordReader implements Recor
   LOG.error("Error closing input stream:" + err.getMessage(), err);
   caughtException = err;
 }
+// Don't close the socket - the stream already does that if needed.
 
 if (client != null) {
   try {
@@ -152,9 +156,10 @@ public class LlapBaseRecordReader implements Recor
   ReaderEvent event = getReaderEvent();
   switch (event.getEventType()) {
 case ERROR:
-  throw new IOException("Received reader event error: " + 
event.getMessage());
+  throw new IOException("Received reader event error: " + 
event.getMessage(), io);
 default:
-  throw new IOException("Got reader event type " + 
event.getEventType() + ", expected error event");
+  throw new IOException("Got reader event type " + 
event.getEventType()
+  + ", expected error event", io);
   }
 }
   } else {
@@ -214,7 +219,13 @@ public class LlapBaseRecordReader implements Recor
 if (LOG.isDebugEnabled()) {
   LOG.debug("Interrupting reader thread due to reader event with error 
" + event.getMessage());
 }
-getReaderThread().interrupt();
+readerThread.interrupt();
+try {
+  socket.close();
+} catch (IOException e) {
+  // Leave the client to time out.
+  LOG.error("Cannot close the socket on error", e);
+}
 break;
   default:
 throw new RuntimeException("Unhandled ReaderEvent type " + 
event.getEventType() + " with message " + event.getMessage());

http://git-wip-us.apache.org/repos/asf/hive/blob/df51738a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
--
diff --git 
a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java 
b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
index 6d63797..aef5762 100644
--- 
a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
+++ 
b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
@@ -170,8 +170,8 @@ public class LlapBaseInputFormat>
 LOG.info("Registered id: " + fragmentId);
 
 @SuppressWarnings("rawtypes")
-LlapBaseRecordReader recordReader = new LlapBaseRecordReader(
-   

hive git commit: HIVE-14093: LLAP output format connection should wait for all writes to finish before closing channel (Jason Dere, reviewed by Prasanth Jayachandran)

2016-06-30 Thread jdere
Repository: hive
Updated Branches:
  refs/heads/master 8ce0118ff -> 2c7f2e9d6


HIVE-14093: LLAP output format connection should wait for all writes to finish 
before closing channel (Jason Dere, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/master
Commit: 2c7f2e9d609e42154d0f699151f4e854051ba167
Parents: 8ce0118
Author: Jason Dere 
Authored: Thu Jun 30 11:15:58 2016 -0700
Committer: Jason Dere 
Committed: Thu Jun 30 11:17:49 2016 -0700

--
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  3 +
 .../hadoop/hive/llap/ChannelOutputStream.java   | 73 ++--
 .../hive/llap/LlapOutputFormatService.java  | 14 ++--
 3 files changed, 57 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/2c7f2e9d/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
--
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index ad467c5..680b623 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2955,6 +2955,9 @@ public class HiveConf extends Configuration {
 "output after sending the fragment. The fragment will fail if its 
output is not claimed."),
 
LLAP_DAEMON_OUTPUT_SERVICE_SEND_BUFFER_SIZE("hive.llap.daemon.output.service.send.buffer.size",
 128 * 1024, "Send buffer size to be used by LLAP daemon output 
service"),
+
LLAP_DAEMON_OUTPUT_SERVICE_MAX_PENDING_WRITES("hive.llap.daemon.output.service.max.pending.writes",
+8, "Maximum number of queued writes allowed per connection when 
sending data\n" +
+" via the LLAP output service to external clients."),
 LLAP_ENABLE_GRACE_JOIN_IN_LLAP("hive.llap.enable.grace.join.in.llap", 
false,
 "Override if grace join should be allowed to run in llap."),
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2c7f2e9d/ql/src/java/org/apache/hadoop/hive/llap/ChannelOutputStream.java
--
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/ChannelOutputStream.java 
b/ql/src/java/org/apache/hadoop/hive/llap/ChannelOutputStream.java
index 239e061..dbe90d6 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/ChannelOutputStream.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/ChannelOutputStream.java
@@ -40,25 +40,45 @@ public class ChannelOutputStream extends OutputStream {
   private ByteBuf buf;
   private byte[] singleByte = new byte[1];
   private boolean closed = false;
-  private final Object channelWritabilityMonitor;
+  private final Object writeMonitor = new Object();
+  private final int maxPendingWrites;
+  private volatile int pendingWrites = 0;
 
-  private ChannelFutureListener listener = new ChannelFutureListener() {
+  private ChannelFutureListener writeListener = new ChannelFutureListener() {
+@Override
+public void operationComplete(ChannelFuture future) {
+
+  pendingWrites--;
+
+  if (future.isCancelled()) {
+LOG.error("Write cancelled on ID " + id);
+  } else if (!future.isSuccess()) {
+LOG.error("Write error on ID " + id, future.cause());
+  }
+
+  synchronized (writeMonitor) {
+writeMonitor.notifyAll();
+  }
+}
+  };
+
+  private ChannelFutureListener closeListener = new ChannelFutureListener() {
 @Override
 public void operationComplete(ChannelFuture future) {
   if (future.isCancelled()) {
-LOG.error(id + " was cancelled");
+LOG.error("Close cancelled on ID " + id);
   } else if (!future.isSuccess()) {
-LOG.error("Error on ID " + id, future.cause());
+LOG.error("Close failed on ID " + id, future.cause());
   }
 }
   };
 
-  public ChannelOutputStream(ChannelHandlerContext chc, String id, int 
bufSize, final Object monitor) {
+  public ChannelOutputStream(ChannelHandlerContext chc, String id, int 
bufSize, int maxOutstandingWrites) {
 this.chc = chc;
 this.id = id;
 this.bufSize = bufSize;
 this.buf = chc.alloc().buffer(bufSize);
-this.channelWritabilityMonitor = monitor;
+this.maxPendingWrites = maxOutstandingWrites;
   }
 
   @Override
@@ -109,10 +129,13 @@ public class ChannelOutputStream extends OutputStream {
   LOG.error("Error flushing stream before close", err);
 }
 
+closed = true;
+
+// Wait for all writes to finish before we actually close.
+waitForWritesToFinish(0);
+
 try {
-  chc.close().addListener(

hive git commit: HIVE-14037: java.lang.ClassNotFoundException for the jar in hive.reloadable.aux.jars.path in mapreduce (Reviewed by Ferdinand Xu)

2016-06-30 Thread aihuaxu
Repository: hive
Updated Branches:
  refs/heads/master 3cba487f5 -> 8ce0118ff


HIVE-14037: java.lang.ClassNotFoundException for the jar in 
hive.reloadable.aux.jars.path in mapreduce (Reviewed by Ferdinand Xu)


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

Branch: refs/heads/master
Commit: 8ce0118ffe517f0c622571778251cbd9f760c4f5
Parents: 3cba487
Author: Aihua Xu 
Authored: Fri Jun 24 15:16:49 2016 -0400
Committer: Aihua Xu 
Committed: Thu Jun 30 14:09:41 2016 -0400

--
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  3 +-
 .../hive/common/util/HiveStringUtils.java   | 21 -
 .../apache/hadoop/hive/ql/exec/Utilities.java   | 69 ++-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java  | 88 
 .../hadoop/hive/ql/exec/mr/MapRedTask.java  | 26 ++
 .../hadoop/hive/ql/session/SessionState.java| 10 ++-
 .../hadoop/hive/ql/exec/TestUtilities.java  | 17 ++--
 ql/src/test/queries/clientpositive/reloadJar.q  | 17 
 .../test/results/clientpositive/reloadJar.q.out | 64 ++
 9 files changed, 214 insertions(+), 101 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/8ce0118f/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
--
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index d75ab40..ad467c5 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -936,7 +936,8 @@ public class HiveConf extends Configuration {
 
 // reloadable jars
 HIVERELOADABLEJARS("hive.reloadable.aux.jars.path", "",
-"Jars can be renewed by executing reload command. And these jars can 
be "
+"The locations of the plugin jars, which can be a comma-separated 
folders or jars. Jars can be renewed\n"
++ "by executing reload command. And these jars can be "
 + "used as the auxiliary classes like creating a UDF or SerDe."),
 
 // hive added files and jars

http://git-wip-us.apache.org/repos/asf/hive/blob/8ce0118f/common/src/java/org/apache/hive/common/util/HiveStringUtils.java
--
diff --git a/common/src/java/org/apache/hive/common/util/HiveStringUtils.java 
b/common/src/java/org/apache/hive/common/util/HiveStringUtils.java
index c2ff635..72c3fa9 100644
--- a/common/src/java/org/apache/hive/common/util/HiveStringUtils.java
+++ b/common/src/java/org/apache/hive/common/util/HiveStringUtils.java
@@ -43,6 +43,7 @@ import java.util.regex.Pattern;
 import com.google.common.collect.Interner;
 import com.google.common.collect.Interners;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang3.text.translate.CharSequenceTranslator;
 import org.apache.commons.lang3.text.translate.EntityArrays;
 import org.apache.commons.lang3.text.translate.LookupTranslator;
@@ -901,6 +902,24 @@ public class HiveStringUtils {
   }
 
   /**
+   * Concatenates strings, using a separator. Empty/blank string or null will 
be
+   * ignored.
+   *
+   * @param strings Strings to join.
+   * @param separator Separator to join with.
+   */
+  public static String joinIgnoringEmpty(String[] strings, char separator) {
+ArrayList list = new ArrayList();
+for(String str : strings) {
+  if (StringUtils.isNotBlank(str)) {
+list.add(str);
+  }
+}
+
+return StringUtils.join(list, separator);
+  }
+
+  /**
* Convert SOME_STUFF to SomeStuff
*
* @param s input string
@@ -911,7 +930,7 @@ public class HiveStringUtils {
 String[] words = split(s.toLowerCase(Locale.US), ESCAPE_CHAR, '_');
 
 for (String word : words) {
-  sb.append(org.apache.commons.lang.StringUtils.capitalize(word));
+  sb.append(StringUtils.capitalize(word));
 }
 
 return sb.toString();

http://git-wip-us.apache.org/repos/asf/hive/blob/8ce0118f/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
--
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index 3fab298..12a929a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -36,6 +36,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
 import java.net.URI;
+import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URLClassLoader;