You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/10/01 01:08:43 UTC

[01/44] hive git commit: HIVE-14680 : retain consistent splits /during/ (as opposed to across) LLAP failures on top of HIVE-14589 (Sergey Shelukhin, reviewed by Siddharth Seth)

Repository: hive
Updated Branches:
  refs/heads/hive-14535 70299dc48 -> 6d9144835


HIVE-14680 : retain consistent splits /during/ (as opposed to across) LLAP failures on top of HIVE-14589 (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/83ef6f92
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/83ef6f92
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/83ef6f92

Branch: refs/heads/hive-14535
Commit: 83ef6f9272d71e1918ffc89635709b4f81e8aba9
Parents: 4340d46
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Sep 19 16:11:16 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Sep 19 16:11:16 2016 -0700

----------------------------------------------------------------------
 .../hive/llap/registry/ServiceInstanceSet.java  |   7 +-
 .../registry/impl/InactiveServiceInstance.java  |  77 ++++++++++
 .../registry/impl/LlapFixedRegistryImpl.java    |   2 +-
 .../impl/LlapZookeeperRegistryImpl.java         |  34 ++++-
 .../daemon/services/impl/LlapWebServices.java   |   2 +-
 .../tez/HostAffinitySplitLocationProvider.java  |  80 +++++++---
 .../apache/hadoop/hive/ql/exec/tez/Utils.java   |   8 +-
 .../TestHostAffinitySplitLocationProvider.java  | 150 +++++++++++++++----
 .../apache/hadoop/hive/serde2/SerDeUtils.java   |  11 ++
 9 files changed, 306 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/83ef6f92/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstanceSet.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstanceSet.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstanceSet.java
index 13b668d..1e8c895 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstanceSet.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstanceSet.java
@@ -14,7 +14,6 @@
 package org.apache.hadoop.hive.llap.registry;
 
 import java.util.Collection;
-import java.util.List;
 import java.util.Set;
 
 public interface ServiceInstanceSet {
@@ -32,9 +31,11 @@ public interface ServiceInstanceSet {
   /**
    * Gets a list containing all the instances. This list has the same iteration order across
    * different processes, assuming the list of registry entries is the same.
-   * @return
+   * @param consistentIndexes if true, also try to maintain the same exact index for each node
+   *                          across calls, by inserting inactive instances to replace the
+   *                          removed ones.
    */
-  public Collection<ServiceInstance> getAllInstancesOrdered();
+  public Collection<ServiceInstance> getAllInstancesOrdered(boolean consistentIndexes);
 
   /**
    * Get an instance by worker identity.

http://git-wip-us.apache.org/repos/asf/hive/blob/83ef6f92/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/InactiveServiceInstance.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/InactiveServiceInstance.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/InactiveServiceInstance.java
new file mode 100644
index 0000000..79b7d51
--- /dev/null
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/InactiveServiceInstance.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed 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.
+ */
+
+package org.apache.hadoop.hive.llap.registry.impl;
+
+import java.util.Map;
+
+import org.apache.hadoop.hive.llap.registry.ServiceInstance;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class InactiveServiceInstance implements ServiceInstance {
+  private final String name;
+  public InactiveServiceInstance(String name) {
+    this.name = name;
+  }
+
+  @Override
+  public String getWorkerIdentity() {
+    return name;
+  }
+
+  @Override
+  public boolean isAlive() {
+    return false;
+  }
+
+  @Override
+  public String getHost() {
+    return null;
+  }
+
+  @Override
+  public int getRpcPort() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getManagementPort() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getShufflePort() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getServicesAddress() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getOutputFormatPort() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Map<String, String> getProperties() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Resource getResource() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/83ef6f92/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
index de4d7f2..bbfcbf6 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
@@ -228,7 +228,7 @@ public class LlapFixedRegistryImpl implements ServiceRegistry {
     }
 
     @Override
-    public List<ServiceInstance> getAllInstancesOrdered() {
+    public List<ServiceInstance> getAllInstancesOrdered(boolean consistentIndexes) {
       List<ServiceInstance> list = new LinkedList<>();
       list.addAll(instances.values());
       Collections.sort(list, new Comparator<ServiceInstance>() {

http://git-wip-us.apache.org/repos/asf/hive/blob/83ef6f92/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
index 5e17ebf..59f7c9e 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
@@ -542,8 +542,8 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
     }
 
     @Override
-    public Collection<ServiceInstance> getAllInstancesOrdered() {
-      Map<String, String> slotByWorker = new HashMap<String, String>();
+    public Collection<ServiceInstance> getAllInstancesOrdered(boolean consistentIndexes) {
+      Map<String, Long> slotByWorker = new HashMap<String, Long>();
       List<ServiceInstance> unsorted = new LinkedList<ServiceInstance>();
       for (ChildData childData : instancesCache.getCurrentData()) {
         if (childData == null) continue;
@@ -560,21 +560,45 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
                 " Ignoring from current instances list..", childData.getPath());
           }
         } else if (nodeName.startsWith(SLOT_PREFIX)) {
-          slotByWorker.put(extractWorkerIdFromSlot(childData), nodeName);
+          slotByWorker.put(extractWorkerIdFromSlot(childData),
+              Long.parseLong(nodeName.substring(SLOT_PREFIX.length())));
         } else {
           LOG.info("Ignoring unknown node {}", childData.getPath());
         }
       }
 
-      TreeMap<String, ServiceInstance> sorted = new TreeMap<>();
+      TreeMap<Long, ServiceInstance> sorted = new TreeMap<>();
+      long maxSlot = Long.MIN_VALUE;
       for (ServiceInstance worker : unsorted) {
-        String slot = slotByWorker.get(worker.getWorkerIdentity());
+        Long slot = slotByWorker.get(worker.getWorkerIdentity());
         if (slot == null) {
           LOG.info("Unknown slot for {}", worker.getWorkerIdentity());
           continue;
         }
+        maxSlot = Math.max(maxSlot, slot);
         sorted.put(slot, worker);
       }
+
+      if (consistentIndexes) {
+        // Add dummy instances to all slots where LLAPs are MIA... I can haz insert_iterator? 
+        TreeMap<Long, ServiceInstance> dummies = new TreeMap<>();
+        Iterator<Long> keyIter = sorted.keySet().iterator();
+        long expected = 0;
+        Long ts = null;
+        while (keyIter.hasNext()) {
+          Long slot = keyIter.next();
+          assert slot >= expected;
+          while (slot > expected) {
+            if (ts == null) {
+              ts = System.nanoTime(); // Inactive nodes restart every call!
+            }
+            dummies.put(expected, new InactiveServiceInstance("inactive-" + expected + "-" + ts));
+            ++expected;
+          }
+          ++expected;
+        }
+        sorted.putAll(dummies);
+      }
       return sorted.values();
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/83ef6f92/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
index f85bbf2..0614355 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
@@ -226,7 +226,7 @@ public class LlapWebServices extends AbstractService {
           }
           jg.writeStringField("identity", registry.getWorkerIdentity());
           jg.writeArrayFieldStart("peers");
-          for (ServiceInstance s : registry.getInstances().getAllInstancesOrdered()) {
+          for (ServiceInstance s : registry.getInstances().getAllInstancesOrdered(false)) {
             jg.writeStartObject();
             jg.writeStringField("identity", s.getWorkerIdentity());
             jg.writeStringField("host", s.getHost());

http://git-wip-us.apache.org/repos/asf/hive/blob/83ef6f92/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HostAffinitySplitLocationProvider.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HostAffinitySplitLocationProvider.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HostAffinitySplitLocationProvider.java
index aafc27e..dcb985f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HostAffinitySplitLocationProvider.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HostAffinitySplitLocationProvider.java
@@ -15,10 +15,13 @@
 package org.apache.hadoop.hive.ql.exec.tez;
 
 import java.io.IOException;
+import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.hash.Hashing;
 
+import org.apache.hadoop.hive.serde2.SerDeUtils;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.split.SplitLocationProvider;
@@ -39,46 +42,83 @@ import org.slf4j.LoggerFactory;
  */
 public class HostAffinitySplitLocationProvider implements SplitLocationProvider {
 
-  private final Logger LOG = LoggerFactory.getLogger(HostAffinitySplitLocationProvider.class);
+  private final static Logger LOG = LoggerFactory.getLogger(
+      HostAffinitySplitLocationProvider.class);
   private final boolean isDebugEnabled = LOG.isDebugEnabled();
 
-  private final String[] knownLocations;
+  private final List<String> locations;
 
-  public HostAffinitySplitLocationProvider(String[] knownLocations) {
-    Preconditions.checkState(knownLocations != null && knownLocations.length != 0,
+  public HostAffinitySplitLocationProvider(List<String> knownLocations) {
+    Preconditions.checkState(knownLocations != null && !knownLocations.isEmpty(),
         HostAffinitySplitLocationProvider.class.getName() +
-            "needs at least 1 location to function");
-    this.knownLocations = knownLocations;
+            " needs at least 1 location to function");
+    this.locations = knownLocations;
   }
 
   @Override
   public String[] getLocations(InputSplit split) throws IOException {
-    if (split instanceof FileSplit) {
-      FileSplit fsplit = (FileSplit) split;
-      int index = chooseBucket(fsplit.getPath().toString(), fsplit.getStart());
-      if (isDebugEnabled) {
-        LOG.debug(
-            "Split at " + fsplit.getPath() + " with offset= " + fsplit.getStart() + ", length=" +
-                fsplit.getLength() + " mapped to index=" + index + ", location=" +
-                knownLocations[index]);
-      }
-      return new String[]{knownLocations[index]};
-    } else {
+    if (!(split instanceof FileSplit)) {
       if (isDebugEnabled) {
         LOG.debug("Split: " + split + " is not a FileSplit. Using default locations");
       }
       return split.getLocations();
     }
+    FileSplit fsplit = (FileSplit) split;
+    String splitDesc = "Split at " + fsplit.getPath() + " with offset= " + fsplit.getStart()
+        + ", length=" + fsplit.getLength();
+    String location = locations.get(determineLocation(
+        locations, fsplit.getPath().toString(), fsplit.getStart(), splitDesc));
+    return (location != null) ? new String[] { location } : null;
   }
 
+  @VisibleForTesting
+  public static int determineLocation(
+      List<String> locations, String path, long start, String desc) {
+    byte[] bytes = getHashInputForSplit(path, start);
+    long hash1 = hash1(bytes);
+    int index = Hashing.consistentHash(hash1, locations.size());
+    String location = locations.get(index);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(desc + " mapped to index=" + index + ", location=" + location);
+    }
+    int iter = 1;
+    long hash2 = 0;
+    // Since our probing method is totally bogus, give up after some time.
+    while (location == null && iter < locations.size() * 2) {
+      if (iter == 1) {
+        hash2 = hash2(bytes);
+      }
+      // Note that this is not real double hashing since we have consistent hash on top.
+      index = Hashing.consistentHash(hash1 + iter * hash2, locations.size());
+      location = locations.get(index);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(desc + " remapped to index=" + index + ", location=" + location);
+      }
+      ++iter;
+    }
+    return index;
+  }
 
-  private int chooseBucket(String path, long startOffset) throws IOException {
+  private static byte[] getHashInputForSplit(String path, long start) {
     // Explicitly using only the start offset of a split, and not the length. Splits generated on
     // block boundaries and stripe boundaries can vary slightly. Try hashing both to the same node.
     // There is the drawback of potentially hashing the same data on multiple nodes though, when a
     // large split is sent to 1 node, and a second invocation uses smaller chunks of the previous
     // large split and send them to different nodes.
-    long hashCode = ((startOffset >> 2) * 37) ^ Murmur3.hash64(path.getBytes());
-    return Hashing.consistentHash(hashCode, knownLocations.length);
+    byte[] pathBytes = path.getBytes();
+    byte[] allBytes = new byte[pathBytes.length + 8];
+    System.arraycopy(pathBytes, 0, allBytes, 0, pathBytes.length);
+    SerDeUtils.writeLong(allBytes, pathBytes.length, start >> 3);
+    return allBytes;
+  }
+
+  private static long hash1(byte[] bytes) {
+    final int PRIME = 104729; // Same as hash64's default seed.
+    return Murmur3.hash64(bytes, 0, bytes.length, PRIME);
+  }
+
+  private static long hash2(byte[] bytes) {
+    final int PRIME = 1366661;
+    return Murmur3.hash64(bytes, 0, bytes.length, PRIME);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/83ef6f92/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
index 2e9918e..113aa49 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.ql.exec.tez;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 
 import org.apache.commons.lang.ArrayUtils;
@@ -41,15 +42,14 @@ public class Utils {
       serviceRegistry = LlapRegistryService.getClient(conf);
 
       Collection<ServiceInstance> serviceInstances =
-          serviceRegistry.getInstances().getAllInstancesOrdered();
-      String[] locations = new String[serviceInstances.size()];
-      int i = 0;
+          serviceRegistry.getInstances().getAllInstancesOrdered(true);
+      ArrayList<String> locations = new ArrayList<>(serviceInstances.size());
       for (ServiceInstance serviceInstance : serviceInstances) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Adding " + serviceInstance.getWorkerIdentity() + " with hostname=" +
               serviceInstance.getHost() + " to list for split locations");
         }
-        locations[i++] = serviceInstance.getHost();
+        locations.add(serviceInstance.getHost());
       }
       splitLocationProvider = new HostAffinitySplitLocationProvider(locations);
     } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/83ef6f92/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestHostAffinitySplitLocationProvider.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestHostAffinitySplitLocationProvider.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestHostAffinitySplitLocationProvider.java
index 7ed3df1..f5ca623 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestHostAffinitySplitLocationProvider.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestHostAffinitySplitLocationProvider.java
@@ -19,15 +19,22 @@ import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.math.stat.descriptive.SummaryStatistics;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.io.orc.OrcSplit;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputSplit;
 import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,20 +42,20 @@ public class TestHostAffinitySplitLocationProvider {
   private final Logger LOG = LoggerFactory.getLogger(TestHostAffinitySplitLocationProvider.class);
 
 
-  private static final String[] locations = new String[5];
+  private static final List<String> locations = new ArrayList<>();
   private static final Set<String> locationsSet = new HashSet<>();
-  private static final String[] executorLocations = new String[9];
+  private static final List<String> executorLocations = new ArrayList<>();
   private static final Set<String> executorLocationsSet = new HashSet<>();
 
   static {
     for (int i = 0 ; i < 5 ; i++) {
-      locations[i] = "location" + i;
-      locationsSet.add(locations[i]);
+      locations.add("location" + i);
+      locationsSet.add(locations.get(i));
     }
 
     for (int i = 0 ; i < 9 ; i++) {
-      executorLocations[i] = "execLocation" + i;
-      executorLocationsSet.add(executorLocations[i]);
+      executorLocations.add("execLocation" + i);
+      executorLocationsSet.add(executorLocations.get(i));
     }
 
   }
@@ -58,20 +65,20 @@ public class TestHostAffinitySplitLocationProvider {
 
     HostAffinitySplitLocationProvider locationProvider = new HostAffinitySplitLocationProvider(executorLocations);
 
-    InputSplit inputSplit1 = createMockInputSplit(new String[] {locations[0], locations[1]});
-    InputSplit inputSplit2 = createMockInputSplit(new String[] {locations[2], locations[3]});
+    InputSplit inputSplit1 = createMockInputSplit(new String[] {locations.get(0), locations.get(1)});
+    InputSplit inputSplit2 = createMockInputSplit(new String[] {locations.get(2), locations.get(3)});
 
-    assertArrayEquals(new String[] {locations[0], locations[1]}, locationProvider.getLocations(inputSplit1));
-    assertArrayEquals(new String[] {locations[2], locations[3]}, locationProvider.getLocations(inputSplit2));
+    assertArrayEquals(new String[] {locations.get(0), locations.get(1)}, locationProvider.getLocations(inputSplit1));
+    assertArrayEquals(new String[] {locations.get(2), locations.get(3)}, locationProvider.getLocations(inputSplit2));
   }
 
   @Test (timeout = 5000)
   public void testOrcSplitsBasic() throws IOException {
     HostAffinitySplitLocationProvider locationProvider = new HostAffinitySplitLocationProvider(executorLocations);
 
-    InputSplit os1 = createMockFileSplit(true, "path1", 0, 1000, new String[] {locations[0], locations[1]});
-    InputSplit os2 = createMockFileSplit(true, "path2", 0, 2000, new String[] {locations[2], locations[3]});
-    InputSplit os3 = createMockFileSplit(true, "path3", 1000, 2000, new String[] {locations[0], locations[3]});
+    InputSplit os1 = createMockFileSplit(true, "path1", 0, 1000, new String[] {locations.get(0), locations.get(1)});
+    InputSplit os2 = createMockFileSplit(true, "path2", 0, 2000, new String[] {locations.get(2), locations.get(3)});
+    InputSplit os3 = createMockFileSplit(true, "path3", 1000, 2000, new String[] {locations.get(0), locations.get(3)});
 
     String[] retLoc1 = locationProvider.getLocations(os1);
     String[] retLoc2 = locationProvider.getLocations(os2);
@@ -94,25 +101,18 @@ public class TestHostAffinitySplitLocationProvider {
   @Test (timeout = 10000)
   public void testConsistentHashing() throws IOException {
     final int LOC_COUNT = 20, MIN_LOC_COUNT = 4, SPLIT_COUNT = 100;
-
-    String[] locations = new String[LOC_COUNT];
-    for (int i = 0; i < locations.length; ++i) {
-      locations[i] = String.valueOf(i);
-    }
-    InputSplit[] splits = new InputSplit[SPLIT_COUNT];
-    for (int i = 0; i < splits.length; ++i) {
-      splits[i] = createMockFileSplit(true, "path" + i, 0, 1000, new String[] {});
-    }
+    List<String> locations = createLocations(LOC_COUNT);
+    InputSplit[] splits = createSplits(SPLIT_COUNT);
 
     StringBuilder failBuilder = new StringBuilder("\n");
     String[] lastLocations = new String[splits.length];
     double movedRatioSum = 0, newRatioSum = 0,
         movedRatioWorst = 0, newRatioWorst = Double.MAX_VALUE;
-    for (int locs = MIN_LOC_COUNT; locs <= locations.length; ++locs) {
-      String[] partLoc = Arrays.copyOf(locations, locs);
+    for (int locs = MIN_LOC_COUNT; locs <= locations.size(); ++locs) {
+      List<String> partLoc = locations.subList(0, locs);
       HostAffinitySplitLocationProvider lp = new HostAffinitySplitLocationProvider(partLoc);
       int moved = 0, newLoc = 0;
-      String newNode = partLoc[locs - 1];
+      String newNode = partLoc.get(locs - 1);
       for (int splitIx = 0; splitIx < splits.length; ++splitIx) {
         String[] splitLocations = lp.getLocations(splits[splitIx]);
         assertEquals(1, splitLocations.length);
@@ -137,18 +137,106 @@ public class TestHostAffinitySplitLocationProvider {
       newRatioWorst = Math.min(newLoc / minNew, newRatioWorst);
       logBadRatios(failBuilder, moved, newLoc, msgTail, movedMsg, newMsg, maxMoved, minNew);
     }
-    int count = locations.length - MIN_LOC_COUNT;
+    int count = locations.size() - MIN_LOC_COUNT;
     double moveRatioAvg = movedRatioSum / count, newRatioAvg = newRatioSum / count;
     String errorMsg = "Move counts: average " + moveRatioAvg + ", worst " + movedRatioWorst
         + "; assigned to new node: average " + newRatioAvg + ", worst " + newRatioWorst;
     LOG.info(errorMsg);
     // Give it a LOT of slack, since on low numbers consistent hashing is very imprecise.
     if (moveRatioAvg > 1.2f || newRatioAvg < 0.8f
-        || movedRatioWorst > 1.5f || newRatioWorst < 0.5f) {
+        || movedRatioWorst > 1.67f || newRatioWorst < 0.5f) {
       fail(errorMsg + "; example failures: " + failBuilder.toString());
     }
   }
 
+  public FileSplit[] createSplits(final int splitCount) throws IOException {
+    FileSplit[] splits = new FileSplit[splitCount];
+    for (int i = 0; i < splits.length; ++i) {
+      splits[i] = createMockFileSplit(true, "path" + i, 0, 1000, new String[] {});
+    }
+    return splits;
+  }
+
+  public List<String> createLocations(final int locCount) {
+    List<String> locations = new ArrayList<>(locCount);
+    for (int i = 0; i < locCount; ++i) {
+      locations.add(String.valueOf(i));
+    }
+    return locations;
+  }
+
+
+  @Test (timeout = 10000)
+  public void testConsistentHashingFallback() throws IOException {
+    final int LOC_COUNT_TO = 20, SPLIT_COUNT = 500, MAX_MISS_COUNT = 4,
+        LOC_COUNT_FROM = MAX_MISS_COUNT + 1;
+    FileSplit[] splits = createSplits(SPLIT_COUNT);
+    AtomicInteger errorCount = new AtomicInteger(0);
+    int cvErrorCount = 0;
+    for (int locs = LOC_COUNT_FROM; locs <= LOC_COUNT_TO; ++locs) {
+      int aboveAvgCount = 0;
+      double sum = 0;
+      double[] cvs = new double[MAX_MISS_COUNT + 1];
+      for (int missCount = 0; missCount <= MAX_MISS_COUNT; ++missCount) {
+        double cv = cvs[missCount] = testHashDistribution(locs, missCount, splits, errorCount);
+        sum += cv;
+        if (missCount > 0 && cv > sum / (missCount + 1)) {
+          ++aboveAvgCount;
+        }
+      }
+      if (aboveAvgCount > 2) {
+        LOG.info("CVs for " + locs + " locations aren't to our liking: " + Arrays.toString(cvs));
+        ++cvErrorCount;
+      }
+    }
+    assertTrue("Found " + errorCount.get() + " abnormalities", errorCount.get() < 3);
+    // TODO: the way we add hash fns does exhibit some irregularities.
+    //       Seems like the 3rd iter has a better distribution in many cases, even better
+    //       that the original hash. That trips the "above MA" criteria, even if the rest is flat.
+    assertTrue("Found " + cvErrorCount + " abnormalities", cvErrorCount< 7);
+  }
+
+  private double testHashDistribution(int locs, final int missCount, FileSplit[] splits,
+      AtomicInteger errorCount) {
+    // This relies heavily on what method determineSplits ... calls and doesn't.
+    // We could do a wrapper with only size() and get() methods instead of List, to be sure.
+    @SuppressWarnings("unchecked")
+    List<String> partLocs = (List<String>)Mockito.mock(List.class);
+    Mockito.when(partLocs.size()).thenReturn(locs);
+    final AtomicInteger state = new AtomicInteger(0);
+    Mockito.when(partLocs.get(Mockito.anyInt())).thenAnswer(new Answer<String>() {
+      @Override
+      public String answer(InvocationOnMock invocation) throws Throwable {
+        return (state.getAndIncrement() == missCount) ? "not-null" : null;
+      }
+    });
+    int[] hitCounts = new int[locs];
+    for (int splitIx = 0; splitIx < splits.length; ++splitIx) {
+      state.set(0);
+      int index = HostAffinitySplitLocationProvider.determineLocation(partLocs,
+          splits[splitIx].getPath().toString(), splits[splitIx].getStart(), null);
+      ++hitCounts[index];
+    }
+    SummaryStatistics ss = new SummaryStatistics();
+    for (int hitCount : hitCounts) {
+      ss.addValue(hitCount);
+    }
+    // All of this is completely bogus and mostly captures the following function:
+    // f(output) = I-eyeballed-the(output) == they-look-ok.
+    // It's pretty much a golden file... 
+    // The fact that stdev doesn't increase with increasing missCount is captured outside.
+    double avg = ss.getSum()/ss.getN(), stdev = ss.getStandardDeviation(), cv = stdev/avg;
+    double allowedMin = avg - 2.5 * stdev, allowedMax = avg + 2.5 * stdev;
+    if (allowedMin > ss.getMin() || allowedMax < ss.getMax() || cv > 0.22) {
+      LOG.info("The distribution for " + locs + " locations, " + missCount + " misses isn't to "
+          + "our liking: avg " + avg + ", stdev " + stdev  + ", cv " + cv + ", min " + ss.getMin()
+          + ", max " + ss.getMax());
+      errorCount.incrementAndGet();
+    }
+    return cv;
+  }
+
+
   private void logBadRatios(StringBuilder failBuilder, int moved, int newLoc, String msgTail,
       String movedMsg, String newMsg, double maxMoved, double minNew) {
     boolean logged = false;
@@ -170,10 +258,10 @@ public class TestHostAffinitySplitLocationProvider {
     HostAffinitySplitLocationProvider locationProvider = new HostAffinitySplitLocationProvider(executorLocations);
 
     // Same file, offset, different lengths
-    InputSplit os11 = createMockFileSplit(true, "path1", 0, 15000, new String[] {locations[0], locations[1]});
-    InputSplit os12 = createMockFileSplit(true, "path1", 0, 30000, new String[] {locations[0], locations[1]});
+    InputSplit os11 = createMockFileSplit(true, "path1", 0, 15000, new String[] {locations.get(0), locations.get(1)});
+    InputSplit os12 = createMockFileSplit(true, "path1", 0, 30000, new String[] {locations.get(0), locations.get(1)});
     // Same file, different offset
-    InputSplit os13 = createMockFileSplit(true, "path1", 15000, 30000, new String[] {locations[0], locations[1]});
+    InputSplit os13 = createMockFileSplit(true, "path1", 15000, 30000, new String[] {locations.get(0), locations.get(1)});
 
     String[] retLoc11 = locationProvider.getLocations(os11);
     String[] retLoc12 = locationProvider.getLocations(os12);
@@ -216,7 +304,7 @@ public class TestHostAffinitySplitLocationProvider {
     return inputSplit;
   }
 
-  private InputSplit createMockFileSplit(boolean createOrcSplit, String fakePathString, long start,
+  private FileSplit createMockFileSplit(boolean createOrcSplit, String fakePathString, long start,
                                          long length, String[] locations) throws IOException {
     FileSplit fileSplit;
     if (createOrcSplit) {

http://git-wip-us.apache.org/repos/asf/hive/blob/83ef6f92/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
index 6e08dfd..7ffc964 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
@@ -567,4 +567,15 @@ public final class SerDeUtils {
   public static Text transformTextFromUTF8(Text text, Charset targetCharset) {
     return new Text(new String(text.getBytes(), 0, text.getLength()).getBytes(targetCharset));
   }
+
+  public static void writeLong(byte[] writeBuffer, int offset, long value) {
+    writeBuffer[offset] = (byte) ((value >> 0)  & 0xff);
+    writeBuffer[offset + 1] = (byte) ((value >> 8)  & 0xff);
+    writeBuffer[offset + 2] = (byte) ((value >> 16) & 0xff);
+    writeBuffer[offset + 3] = (byte) ((value >> 24) & 0xff);
+    writeBuffer[offset + 4] = (byte) ((value >> 32) & 0xff);
+    writeBuffer[offset + 5] = (byte) ((value >> 40) & 0xff);
+    writeBuffer[offset + 6] = (byte) ((value >> 48) & 0xff);
+    writeBuffer[offset + 7] = (byte) ((value >> 56) & 0xff);
+  }
 }


[30/44] hive git commit: Revert "HIVE-14835: Improve ptest2 build time (Prasanth Jayachandran reviewed by Sergio Pena)"

Posted by se...@apache.org.
Revert "HIVE-14835: Improve ptest2 build time (Prasanth Jayachandran reviewed by Sergio Pena)"

This reverts commit d3b88f664415ff114de74aa2a0da2f1e1acbf60d.


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

Branch: refs/heads/hive-14535
Commit: 871b55fa17e3d63a91a34f48df0bc2156f3473ce
Parents: ac977cc
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Tue Sep 27 21:42:09 2016 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Tue Sep 27 21:42:09 2016 -0700

----------------------------------------------------------------------
 dev-support/jenkins-execute-build.sh               | 4 +++-
 testutils/ptest2/src/main/resources/source-prep.vm | 4 ++--
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/871b55fa/dev-support/jenkins-execute-build.sh
----------------------------------------------------------------------
diff --git a/dev-support/jenkins-execute-build.sh b/dev-support/jenkins-execute-build.sh
index 972abae..2142942 100644
--- a/dev-support/jenkins-execute-build.sh
+++ b/dev-support/jenkins-execute-build.sh
@@ -70,7 +70,9 @@ test -n "$TEST_HANDLE" || fail "TEST_HANDLE must be specified and cannot be empt
 test -n "$PTEST_API_ENDPOINT" || fail "PTEST_API_ENDPOINT must be specified and cannot be empty."
 test -n "$PTEST_LOG_ENDPOINT" || fail "PTEST_LOG_ENDPOINT must be specified and cannot be empty."
 
-MVN_REPO_LOCAL=${HOME}/.m2/repository
+# WORKSPACE is an environment variable created by Jenkins, and it is the directory where the build is executed.
+# If not set, then default to $HOME
+MVN_REPO_LOCAL=${WORKSPACE:-$HOME}/.m2/repository
 
 # Directory where to build the ptest framework
 PTEST_BUILD_DIR="$PWD/hive/build"

http://git-wip-us.apache.org/repos/asf/hive/blob/871b55fa/testutils/ptest2/src/main/resources/source-prep.vm
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/resources/source-prep.vm b/testutils/ptest2/src/main/resources/source-prep.vm
index 0fc22be..67e6a95 100644
--- a/testutils/ptest2/src/main/resources/source-prep.vm
+++ b/testutils/ptest2/src/main/resources/source-prep.vm
@@ -102,11 +102,11 @@ cd $workingDir/
         fi
       done
     #end
-    mvn -B clean install -DskipTests -T 4 -q -Dmaven.repo.local=$workingDir/maven $mavenArgs $mavenBuildArgs
+    mvn -B clean install -DskipTests -Dmaven.repo.local=$workingDir/maven $mavenArgs $mavenBuildArgs
     if [[ -d "itests" ]]
     then
       cd itests
-      mvn -B clean install -DskipTests -T 4 -q -Dmaven.repo.local=$workingDir/maven $mavenArgs $mavenBuildArgs
+      mvn -B clean install -DskipTests -Dmaven.repo.local=$workingDir/maven $mavenArgs $mavenBuildArgs
     fi
   elif [[ "${buildTool}" == "ant" ]]
   then


[32/44] hive git commit: HIVE-12222: Define port range in property for RPCServer (Aihua Xu, reviewed by Xuefu Zhang)

Posted by se...@apache.org.
HIVE-12222: Define port range in property for RPCServer (Aihua Xu, reviewed by Xuefu Zhang)


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

Branch: refs/heads/hive-14535
Commit: e2bd513a3970b141576f7ead25fc6cfcc5fcda17
Parents: 667e9dd
Author: Aihua Xu <ai...@apache.org>
Authored: Thu Sep 22 14:20:51 2016 -0400
Committer: Aihua Xu <ai...@apache.org>
Committed: Wed Sep 28 12:07:40 2016 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  3 ++
 .../hive/spark/client/rpc/RpcConfiguration.java | 38 +++++++++++++++++
 .../apache/hive/spark/client/rpc/RpcServer.java | 44 +++++++++++++++++---
 .../apache/hive/spark/client/rpc/TestRpc.java   | 37 +++++++++++++++-
 4 files changed, 115 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e2bd513a/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 43a16d7..4c3ef3e 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3080,6 +3080,9 @@ public class HiveConf extends Configuration {
       "Default is empty, which means the address will be determined in the same way as for hive.server2.thrift.bind.host." +
       "This is only necessary if the host has mutiple network addresses and if a different network address other than " +
       "hive.server2.thrift.bind.host is to be used."),
+    SPARK_RPC_SERVER_PORT("hive.spark.client.rpc.server.port", "", "A list of port ranges which can be used by RPC server " +
+        "with the format of 49152-49222,49228 and a random one is selected from the list. Default is empty, which randomly " +
+        "selects one port from all available ones."),
     SPARK_DYNAMIC_PARTITION_PRUNING(
         "hive.spark.dynamic.partition.pruning", false,
         "When dynamic pruning is enabled, joins on partition keys will be processed by writing\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/e2bd513a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java
index 210f8a4..8c59015 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java
@@ -18,7 +18,9 @@
 package org.apache.hive.spark.client.rpc;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
@@ -107,6 +109,42 @@ public final class RpcConfiguration {
     return ServerUtils.getHostAddress(hiveHost).getHostName();
   }
 
+  /**
+   * Parses the port string like 49152-49222,49228 into the port list. A default 0
+   * is added for the empty port string.
+   * @return a list of configured ports.
+   * @exception IOException is thrown if the property is not configured properly
+   */
+  List<Integer> getServerPorts() throws IOException {
+    String errMsg = "Incorrect RPC server port configuration for HiveServer2";
+    String portString = config.get(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname);
+    ArrayList<Integer> ports = new ArrayList<Integer>();
+    try {
+      if(!StringUtils.isEmpty(portString)) {
+        for (String portRange : portString.split(",")) {
+          String[] range = portRange.split("-");
+          if (range.length == 0 || range.length > 2
+              || (range.length == 2 && Integer.valueOf(range[0]) > Integer.valueOf(range[1]))) {
+            throw new IOException(errMsg);
+          }
+          if (range.length == 1) {
+            ports.add(Integer.valueOf(range[0]));
+          } else {
+            for (int i = Integer.valueOf(range[0]); i <= Integer.valueOf(range[1]); i++) {
+              ports.add(i);
+            }
+          }
+        }
+      } else {
+        ports.add(0);
+      }
+
+      return ports;
+    } catch(NumberFormatException e) {
+      throw new IOException(errMsg);
+    }
+  }
+
   String getRpcChannelLogLevel() {
     return config.get(HiveConf.ConfVars.SPARK_RPC_CHANNEL_LOG_LEVEL.varname);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/e2bd513a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java
index 68ee627..657494a 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java
@@ -21,10 +21,13 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.SecureRandom;
+import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeUnit;
+
 import javax.security.auth.callback.Callback;
 import javax.security.auth.callback.CallbackHandler;
 import javax.security.auth.callback.NameCallback;
@@ -39,8 +42,10 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.EventLoopGroup;
@@ -51,9 +56,9 @@ import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.GenericFutureListener;
 import io.netty.util.concurrent.Promise;
 import io.netty.util.concurrent.ScheduledFuture;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 
 /**
@@ -82,7 +87,7 @@ public class RpcServer implements Closeable {
             .setNameFormat("RPC-Handler-%d")
             .setDaemon(true)
             .build());
-    this.channel = new ServerBootstrap()
+     ServerBootstrap serverBootstrap = new ServerBootstrap()
       .group(group)
       .channel(NioServerSocketChannel.class)
       .childHandler(new ChannelInitializer<SocketChannel>() {
@@ -107,16 +112,43 @@ public class RpcServer implements Closeable {
       })
       .option(ChannelOption.SO_BACKLOG, 1)
       .option(ChannelOption.SO_REUSEADDR, true)
-      .childOption(ChannelOption.SO_KEEPALIVE, true)
-      .bind(0)
-      .sync()
-      .channel();
+      .childOption(ChannelOption.SO_KEEPALIVE, true);
+
+    this.channel = bindServerPort(serverBootstrap).channel();
     this.port = ((InetSocketAddress) channel.localAddress()).getPort();
     this.pendingClients = Maps.newConcurrentMap();
     this.address = this.config.getServerAddress();
   }
 
   /**
+   * Retry the list of configured ports until one is found
+   * @param serverBootstrap
+   * @return
+   * @throws InterruptedException
+   * @throws IOException
+   */
+  private ChannelFuture bindServerPort(ServerBootstrap serverBootstrap)
+      throws InterruptedException, IOException {
+    List<Integer> ports = config.getServerPorts();
+    if (ports.contains(0)) {
+      return serverBootstrap.bind(0).sync();
+    } else {
+      Random rand = new Random();
+      while(!ports.isEmpty()) {
+        int index = rand.nextInt(ports.size());
+        int port = ports.get(index);
+        ports.remove(index);
+        try {
+          return serverBootstrap.bind(port).sync();
+        } catch(Exception e) {
+          // Retry the next port
+        }
+      }
+      throw new IOException("No available ports from configured RPC Server ports for HiveServer2");
+    }
+  }
+
+  /**
    * Tells the RPC server to expect a connection from a new client.
    *
    * @param clientId An identifier for the client. Must be unique.

http://git-wip-us.apache.org/repos/asf/hive/blob/e2bd513a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
----------------------------------------------------------------------
diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
index 7bcf1df..77c3d02 100644
--- a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
+++ b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
@@ -19,10 +19,10 @@ package org.apache.hive.spark.client.rpc;
 
 import java.io.Closeable;
 import java.net.InetAddress;
+import java.io.IOException;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.concurrent.CancellationException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -168,6 +168,41 @@ public class TestRpc {
   }
 
   @Test
+  public void testServerPort() throws Exception {
+    Map<String, String> config = new HashMap<String, String>();
+
+    RpcServer server0 = new RpcServer(config);
+    assertTrue("Empty port range should return a random valid port: " + server0.getPort(), server0.getPort() >= 0);
+    IOUtils.closeQuietly(server0);
+
+    config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, "49152-49222,49223,49224-49333");
+    RpcServer server1 = new RpcServer(config);
+    assertTrue("Port should be within configured port range:" + server1.getPort(), server1.getPort() >= 49152 && server1.getPort() <= 49333);
+    IOUtils.closeQuietly(server1);
+
+    int expectedPort = 65535;
+    config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort));
+    RpcServer server2 = new RpcServer(config);
+    assertTrue("Port should match configured one: " + server2.getPort(), server2.getPort() == expectedPort);
+    IOUtils.closeQuietly(server2);
+
+    config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, "49552-49222,49223,49224-49333");
+    try {
+      autoClose(new RpcServer(config));
+      assertTrue("Invalid port range should throw an exception", false); // Should not reach here
+    } catch(IOException e) {
+      assertEquals("Incorrect RPC server port configuration for HiveServer2", e.getMessage());
+    }
+
+    // Retry logic
+    expectedPort = 65535;
+    config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort) + ",21-23");
+    RpcServer server3 = new RpcServer(config);
+    assertTrue("Port should match configured one:" + server3.getPort(), server3.getPort() == expectedPort);
+    IOUtils.closeQuietly(server3);
+  }
+
+  @Test
   public void testCloseListener() throws Exception {
     RpcServer server = autoClose(new RpcServer(emptyConfig));
     Rpc[] rpcs = createRpcConnection(server);


[23/44] hive git commit: HIVE-14831: Missing Druid dependencies at runtime (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-14831: Missing Druid dependencies at runtime (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: 4ce5fe131a5861b07c024c0529f6b2ebe63a4456
Parents: a213115
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri Sep 23 19:45:56 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Sat Sep 24 07:51:25 2016 +0100

----------------------------------------------------------------------
 druid-handler/pom.xml | 11 +++--------
 1 file changed, 3 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4ce5fe13/druid-handler/pom.xml
----------------------------------------------------------------------
diff --git a/druid-handler/pom.xml b/druid-handler/pom.xml
index 2173cdc..0db542e 100644
--- a/druid-handler/pom.xml
+++ b/druid-handler/pom.xml
@@ -150,12 +150,8 @@
                   <shadedPattern>org.apache.hive.druid.io.druid</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>com.metamx.emitter</pattern>
-                  <shadedPattern>org.apache.hive.druid.com.metamx.emitter</shadedPattern>
-                </relocation>
-                <relocation>
-                  <pattern>com.metamx.http.client</pattern>
-                  <shadedPattern>org.apache.hive.druid.com.metamx.http.client</shadedPattern>
+                  <pattern>com.metamx</pattern>
+                  <shadedPattern>org.apache.hive.druid.com.metamx</shadedPattern>
                 </relocation>
                 <relocation>
                   <pattern>io.netty</pattern>
@@ -173,8 +169,7 @@
               <artifactSet>
                 <includes>
                   <include>io.druid:*</include>
-                  <include>com.metamx:emitter:*</include>
-                  <include>com.metamx:http-client:*</include>
+                  <include>com.metamx:*</include>
                   <include>io.netty:*</include>
                   <include>com.fasterxml.jackson.core:*</include>
                   <include>com.fasterxml.jackson.datatype:*</include>


[38/44] hive git commit: HIVE-14854. Add a core cluster type to QTestUtil. (Siddharth Seth, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-14854. Add a core cluster type to QTestUtil. (Siddharth Seth, 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/474425aa
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/474425aa
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/474425aa

Branch: refs/heads/hive-14535
Commit: 474425aa62e3f25b119419439373aa684c6c2121
Parents: a6c6080
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Sep 29 13:10:44 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Sep 29 13:10:44 2016 -0700

----------------------------------------------------------------------
 .../hive/cli/control/AbstractCliConfig.java     | 13 +++-
 .../hadoop/hive/cli/control/CoreCliDriver.java  |  2 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 75 +++++++++++---------
 .../hive/llap/daemon/impl/LlapDaemon.java       |  5 +-
 4 files changed, 56 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/474425aa/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
index 03d4075..c12f51e 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
@@ -409,7 +409,18 @@ public abstract class AbstractCliConfig {
   }
 
   protected void setMetastoreType(MetastoreType mt) {
-    metastoreType=mt;
+    String metaStoreTypeProperty = getSysPropValue("metaStoreType");
+    if (metaStoreTypeProperty != null) {
+      if (metaStoreTypeProperty.equalsIgnoreCase("sql")) {
+        metastoreType = MetastoreType.sql;
+      } else if (metaStoreTypeProperty.equalsIgnoreCase("hbase")) {
+        metastoreType = MetastoreType.hbase;
+      } else {
+        throw new IllegalArgumentException("Unknown metastore type: " + metaStoreTypeProperty);
+      }
+    } else {
+      metastoreType = mt;
+    }
   }
 
   public MetastoreType getMetastoreType() {

http://git-wip-us.apache.org/repos/asf/hive/blob/474425aa/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
index d83ff45..a735346 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
@@ -61,7 +61,7 @@ public class CoreCliDriver extends CliAdapter {
         @Override
         public QTestUtil invokeInternal() throws Exception {
           return new QTestUtil((cliConfig.getResultsDir()), (cliConfig.getLogDir()), miniMR,
-              hiveConfDir, hadoopVer, initScript, cleanupScript, useHBaseMetastore, true, false,
+              hiveConfDir, hadoopVer, initScript, cleanupScript, useHBaseMetastore, true,
               cliConfig.getFsType());
         }
       }.invoke("QtestUtil instance created", LOG, true);

http://git-wip-us.apache.org/repos/asf/hive/blob/474425aa/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 909d7f6..e49ecd9 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -50,6 +50,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Deque;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -152,7 +153,6 @@ public class QTestUtil {
 
   private String testWarehouse;
   private final String testFiles;
-  private final boolean localMode;
   protected final String outDir;
   protected final String logDir;
   private final TreeMap<String, String> qMap;
@@ -411,6 +411,11 @@ public class QTestUtil {
     }
   }
 
+  private enum CoreClusterType {
+    MR,
+    TEZ,
+    SPARK
+  }
 
   public enum FsType {
     local,
@@ -420,35 +425,48 @@ public class QTestUtil {
 
   public enum MiniClusterType {
 
-    mr(FsType.hdfs),
-    tez(FsType.hdfs),
-    spark(FsType.local),
-    miniSparkOnYarn(FsType.hdfs),
-    llap(FsType.hdfs),
-    none(FsType.local);
+    mr(CoreClusterType.MR, FsType.hdfs),
+    tez(CoreClusterType.TEZ, FsType.hdfs),
+    tez_local(CoreClusterType.TEZ, FsType.local),
+    spark(CoreClusterType.SPARK, FsType.local),
+    miniSparkOnYarn(CoreClusterType.SPARK, FsType.hdfs),
+    llap(CoreClusterType.TEZ, FsType.hdfs),
+    llap_local(CoreClusterType.TEZ, FsType.local),
+    none(CoreClusterType.MR, FsType.local);
 
 
+    private final CoreClusterType coreClusterType;
     private final FsType defaultFsType;
 
-    MiniClusterType(FsType defaultFsType) {
+    MiniClusterType(CoreClusterType coreClusterType, FsType defaultFsType) {
+      this.coreClusterType = coreClusterType;
       this.defaultFsType = defaultFsType;
     }
 
+    public CoreClusterType getCoreClusterType() {
+      return coreClusterType;
+    }
+
     public FsType getDefaultFsType() {
       return defaultFsType;
     }
 
     public static MiniClusterType valueForString(String type) {
+      // Replace this with valueOf.
       if (type.equals("miniMR")) {
         return mr;
       } else if (type.equals("tez")) {
         return tez;
+      } else if (type.equals("tez_local")) {
+        return tez_local;
       } else if (type.equals("spark")) {
         return spark;
       } else if (type.equals("miniSparkOnYarn")) {
         return miniSparkOnYarn;
       } else if (type.equals("llap")) {
         return llap;
+      } else if (type.equals("llap_local")) {
+        return llap_local;
       } else {
         return none;
       }
@@ -487,25 +505,24 @@ public class QTestUtil {
                    String confDir, String hadoopVer, String initScript, String cleanupScript,
                    boolean useHBaseMetastore, boolean withLlapIo) throws Exception {
     this(outDir, logDir, clusterType, confDir, hadoopVer, initScript, cleanupScript,
-        useHBaseMetastore, withLlapIo, false, null);
+        useHBaseMetastore, withLlapIo, null);
   }
 
   public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
       String confDir, String hadoopVer, String initScript, String cleanupScript,
-      boolean useHBaseMetastore, boolean withLlapIo, boolean localMode, FsType fsType)
+      boolean useHBaseMetastore, boolean withLlapIo, FsType fsType)
     throws Exception {
     LOG.info("Setting up QTestUtil with outDir={}, logDir={}, clusterType={}, confDir={}," +
         " hadoopVer={}, initScript={}, cleanupScript={}, useHbaseMetaStore={}, withLlapIo={}," +
-            " localMode={}, fsType={}"
+            " fsType={}"
         , outDir, logDir, clusterType, confDir, hadoopVer, initScript, cleanupScript,
-        useHBaseMetastore, withLlapIo, localMode, fsType);
+        useHBaseMetastore, withLlapIo, fsType);
     Preconditions.checkNotNull(clusterType, "ClusterType cannot be null");
     if (fsType != null) {
       this.fsType = fsType;
     } else {
       this.fsType = clusterType.getDefaultFsType();
     }
-    this.localMode = localMode;
     this.outDir = outDir;
     this.logDir = logDir;
     this.useHBaseMetastore = useHBaseMetastore;
@@ -604,28 +621,22 @@ public class QTestUtil {
   private void setupMiniCluster(HadoopShims shims, String confDir) throws
       IOException {
 
-    if (localMode) {
-      Preconditions
-          .checkState(clusterType == MiniClusterType.tez || clusterType == MiniClusterType.llap,
-              "localMode can currently only be set for tez or llap");
-    }
-
     String uriString = WindowsPathUtil.getHdfsUriString(fs.getUri().toString());
 
-    if (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.llap) {
+    if (clusterType.getCoreClusterType() == CoreClusterType.TEZ) {
       if (confDir != null && !confDir.isEmpty()) {
         conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
             + "/tez-site.xml"));
       }
       int numTrackers;
-      if (clusterType == MiniClusterType.tez) {
-        numTrackers = 4;
-      } else {
+      if (EnumSet.of(MiniClusterType.llap, MiniClusterType.llap_local).contains(clusterType)) {
         llapCluster = LlapItUtils.startAndGetMiniLlapCluster(conf, setup.zooKeeperCluster, confDir);
         numTrackers = 2;
+      } else {
+        numTrackers = 4;
       }
-      if (localMode) {
-        mr = shims.getLocalMiniTezCluster(conf, clusterType == MiniClusterType.llap);
+      if (EnumSet.of(MiniClusterType.llap_local, MiniClusterType.tez_local).contains(clusterType)) {
+        mr = shims.getLocalMiniTezCluster(conf, clusterType == MiniClusterType.llap_local);
       } else {
         mr = shims.getMiniTezCluster(conf, numTrackers, uriString);
       }
@@ -642,7 +653,7 @@ public class QTestUtil {
       cleanUp();
     }
 
-    if (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.llap) {
+    if (clusterType.getCoreClusterType() == CoreClusterType.TEZ) {
       SessionState.get().getTezSession().close(false);
     }
     setup.tearDown();
@@ -1152,8 +1163,7 @@ public class QTestUtil {
     SessionState oldSs = SessionState.get();
 
     boolean canReuseSession = !qNoSessionReuseQuerySet.contains(tname);
-    if (oldSs != null && canReuseSession
-        && (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.llap)) {
+    if (oldSs != null && canReuseSession && clusterType.getCoreClusterType() == CoreClusterType.TEZ) {
       // Copy the tezSessionState from the old CliSessionState.
       tezSessionState = oldSs.getTezSession();
       oldSs.setTezSession(null);
@@ -1161,8 +1171,7 @@ public class QTestUtil {
       oldSs.close();
     }
 
-    if (oldSs != null && (clusterType == MiniClusterType.spark
-        || clusterType == MiniClusterType.miniSparkOnYarn)) {
+    if (oldSs != null && clusterType.getCoreClusterType() == CoreClusterType.SPARK) {
       sparkSession = oldSs.getSparkSession();
       ss.setSparkSession(sparkSession);
       oldSs.setSparkSession(null);
@@ -1225,8 +1234,7 @@ public class QTestUtil {
     ss.err = System.out;
 
     SessionState oldSs = SessionState.get();
-    if (oldSs != null && canReuseSession
-        && (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.llap)) {
+    if (oldSs != null && canReuseSession && clusterType.getCoreClusterType() == CoreClusterType.TEZ) {
       // Copy the tezSessionState from the old CliSessionState.
       tezSessionState = oldSs.getTezSession();
       ss.setTezSession(tezSessionState);
@@ -1234,8 +1242,7 @@ public class QTestUtil {
       oldSs.close();
     }
 
-    if (oldSs != null && (clusterType == MiniClusterType.spark
-        || clusterType == MiniClusterType.miniSparkOnYarn)) {
+    if (oldSs != null && clusterType.getCoreClusterType() == CoreClusterType.SPARK) {
       sparkSession = oldSs.getSparkSession();
       ss.setSparkSession(sparkSession);
       oldSs.setSparkSession(null);

http://git-wip-us.apache.org/repos/asf/hive/blob/474425aa/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index 53a2295..752e6ee 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -116,8 +116,6 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       int mngPort, int shufflePort, int webPort, String appName) {
     super("LlapDaemon");
 
-    initializeLogging(daemonConf);
-
     printAsciiArt();
 
     Preconditions.checkArgument(numExecutors > 0);
@@ -278,7 +276,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     addIfService(amReporter);
   }
 
-  private void initializeLogging(final Configuration conf) {
+  private static void initializeLogging(final Configuration conf) {
     long start = System.currentTimeMillis();
     URL llap_l4j2 = LlapDaemon.class.getClassLoader().getResource(
         LlapConstants.LOG4j2_PROPERTIES_FILE);
@@ -450,6 +448,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       long ioMemoryBytes = HiveConf.getSizeVar(daemonConf, ConfVars.LLAP_IO_MEMORY_MAX_SIZE);
       boolean isDirectCache = HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_ALLOCATOR_DIRECT);
       boolean isLlapIo = HiveConf.getBoolVar(daemonConf, HiveConf.ConfVars.LLAP_IO_ENABLED, true);
+      LlapDaemon.initializeLogging(daemonConf);
       llapDaemon = new LlapDaemon(daemonConf, numExecutors, executorMemoryBytes, isLlapIo,
           isDirectCache, ioMemoryBytes, localDirs, rpcPort, mngPort, shufflePort, webPort,
           appName);


[27/44] hive git commit: HIVE-14029: Update Spark version to 2.0.0 (Ferdinand Xu, via Li Rui, Szehon Ho and Sergio Pena)

Posted by se...@apache.org.
HIVE-14029: Update Spark version to 2.0.0 (Ferdinand Xu, via Li Rui, Szehon Ho and Sergio Pena)


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

Branch: refs/heads/hive-14535
Commit: ac977cc88757b49fbbd5c3bb236adcedcaae396c
Parents: 7d3da17
Author: Ferdinand Xu <ch...@intel.com>
Authored: Wed Sep 28 01:44:32 2016 +0800
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Wed Sep 28 01:44:32 2016 +0800

----------------------------------------------------------------------
 pom.xml                                         | 12 ++-
 ql/pom.xml                                      | 26 +++++-
 .../exec/spark/HiveBaseFunctionResultList.java  | 96 +++++++++-----------
 .../hive/ql/exec/spark/HiveMapFunction.java     |  2 +-
 .../hive/ql/exec/spark/HiveReduceFunction.java  |  2 +-
 .../hive/ql/exec/spark/SortByShuffler.java      | 84 ++++++++---------
 .../spark/status/impl/JobMetricsListener.java   |  4 +-
 .../ql/exec/spark/TestHiveKVResultCache.java    |  5 +-
 spark-client/pom.xml                            | 15 ++-
 .../hive/spark/client/MetricsCollection.java    |  8 +-
 .../apache/hive/spark/client/RemoteDriver.java  |  4 +-
 .../hive/spark/client/metrics/InputMetrics.java |  9 +-
 .../hive/spark/client/metrics/Metrics.java      |  6 +-
 .../client/metrics/ShuffleReadMetrics.java      | 18 ++--
 .../client/metrics/ShuffleWriteMetrics.java     |  4 +-
 .../spark/client/TestMetricsCollection.java     |  8 +-
 16 files changed, 153 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2fb78cd..756cc34 100644
--- a/pom.xml
+++ b/pom.xml
@@ -145,7 +145,7 @@
     <ivy.version>2.4.0</ivy.version>
     <jackson.version>1.9.13</jackson.version>
     <!-- jackson 1 and 2 lines can coexist without issue, as they have different artifactIds -->
-    <jackson.new.version>2.4.2</jackson.new.version>
+    <jackson.new.version>2.6.5</jackson.new.version>
     <jasper.version>5.5.23</jasper.version>
     <jamon.plugin.version>2.3.4</jamon.plugin.version>
     <jamon-runtime.version>2.3.1</jamon-runtime.version>
@@ -155,6 +155,8 @@
     <jdo-api.version>3.0.1</jdo-api.version>
     <jetty.version>7.6.0.v20120127</jetty.version>
     <jersey.version>1.14</jersey.version>
+    <!-- Glassfish jersey is included for Spark client test only -->
+    <glassfish.jersey.version>2.22.2</glassfish.jersey.version>
     <jline.version>2.12</jline.version>
     <jms.version>1.1</jms.version>
     <joda.version>2.8.1</joda.version>
@@ -168,7 +170,7 @@
     <opencsv.version>2.3</opencsv.version>
     <mockito-all.version>1.9.5</mockito-all.version>
     <mina.version>2.0.0-M5</mina.version>
-    <netty.version>4.0.23.Final</netty.version>
+    <netty.version>4.0.29.Final</netty.version>
     <parquet.version>1.8.1</parquet.version>
     <pig.version>0.16.0</pig.version>
     <protobuf.version>2.5.0</protobuf.version>
@@ -178,9 +180,9 @@
     <tez.version>0.8.4</tez.version>
     <slider.version>0.90.2-incubating</slider.version>
     <super-csv.version>2.2.0</super-csv.version>
-    <spark.version>1.6.0</spark.version>
-    <scala.binary.version>2.10</scala.binary.version>
-    <scala.version>2.10.4</scala.version>
+    <spark.version>2.0.0</spark.version>
+    <scala.binary.version>2.11</scala.binary.version>
+    <scala.version>2.11.8</scala.version>
     <tempus-fugit.version>1.1</tempus-fugit.version>
     <snappy.version>0.2</snappy.version>
     <wadl-resourcedoc-doclet.version>1.4</wadl-resourcedoc-doclet.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index 02ddb80..2a93bb7 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -361,7 +361,7 @@
       <version>${calcite.version}</version>
       <exclusions>
         <!-- hsqldb interferes with the use of derby as the default db
-          in hive's use of datanucleus. 
+          in hive's use of datanucleus.
         -->
         <exclusion>
           <groupId>org.hsqldb</groupId>
@@ -380,14 +380,14 @@
           <artifactId>jackson-core</artifactId>
         </exclusion>
       </exclusions>
-    </dependency>   
+    </dependency>
     <dependency>
       <groupId>org.apache.calcite</groupId>
       <artifactId>calcite-avatica</artifactId>
       <version>${calcite.version}</version>
       <exclusions>
         <!-- hsqldb interferes with the use of derby as the default db
-          in hive's use of datanucleus. 
+          in hive's use of datanucleus.
         -->
         <exclusion>
           <groupId>org.hsqldb</groupId>
@@ -685,6 +685,14 @@
          <groupId>commmons-logging</groupId>
          <artifactId>commons-logging</artifactId>
        </exclusion>
+       <exclusion>
+         <groupId>org.glassfish.jersey.containers</groupId>
+         <artifactId>*</artifactId>
+       </exclusion>
+       <exclusion>
+         <groupId>org.glassfish.jersey.core</groupId>
+         <artifactId>*</artifactId>
+       </exclusion>
      </exclusions>
    </dependency>
     <dependency>
@@ -692,6 +700,18 @@
       <artifactId>jersey-servlet</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.core</groupId>
+      <artifactId>jersey-server</artifactId>
+      <version>${glassfish.jersey.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.containers</groupId>
+      <artifactId>jersey-container-servlet-core</artifactId>
+      <version>${glassfish.jersey.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java
index 5b65036..0fc79f4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java
@@ -38,15 +38,14 @@ import com.google.common.base.Preconditions;
  *     through Iterator interface.
  */
 @SuppressWarnings("rawtypes")
-public abstract class HiveBaseFunctionResultList<T> implements
-    Iterable, OutputCollector<HiveKey, BytesWritable>, Serializable {
+public abstract class HiveBaseFunctionResultList<T>
+  implements Iterator, OutputCollector<HiveKey, BytesWritable>, Serializable {
   private static final long serialVersionUID = -1L;
   private final Iterator<T> inputIterator;
   private boolean isClosed = false;
 
   // Contains results from last processed input record.
   private final HiveKVResultCache lastRecordOutput;
-  private boolean iteratorAlreadyCreated = false;
 
   public HiveBaseFunctionResultList(Iterator<T> inputIterator) {
     this.inputIterator = inputIterator;
@@ -54,13 +53,6 @@ public abstract class HiveBaseFunctionResultList<T> implements
   }
 
   @Override
-  public Iterator iterator() {
-    Preconditions.checkState(!iteratorAlreadyCreated, "Iterator can only be created once.");
-    iteratorAlreadyCreated = true;
-    return new ResultIterator();
-  }
-
-  @Override
   public void collect(HiveKey key, BytesWritable value) throws IOException {
     lastRecordOutput.add(SparkUtilities.copyHiveKey(key),
         SparkUtilities.copyBytesWritable(value));
@@ -77,57 +69,55 @@ public abstract class HiveBaseFunctionResultList<T> implements
   /** Close the record processor. */
   protected abstract void closeRecordProcessor();
 
-  /** Implement Iterator interface. */
-  public class ResultIterator implements Iterator {
-    @Override
-    public boolean hasNext(){
-      // Return remaining records (if any) from last processed input record.
-      if (lastRecordOutput.hasNext()) {
-        return true;
-      }
+  @Override
+  public boolean hasNext() {
+    // Return remaining records (if any) from last processed input record.
+    if (lastRecordOutput.hasNext()) {
+      return true;
+    }
 
-      // Process the records in the input iterator until
-      //  - new output records are available for serving downstream operator,
-      //  - input records are exhausted or
-      //  - processing is completed.
-      while (inputIterator.hasNext() && !processingDone()) {
-        try {
-          processNextRecord(inputIterator.next());
-          if (lastRecordOutput.hasNext()) {
-            return true;
-          }
-        } catch (IOException ex) {
-          throw new IllegalStateException("Error while processing input.", ex);
+    // Process the records in the input iterator until
+    //  - new output records are available for serving downstream operator,
+    //  - input records are exhausted or
+    //  - processing is completed.
+    while (inputIterator.hasNext() && !processingDone()) {
+      try {
+        processNextRecord(inputIterator.next());
+        if (lastRecordOutput.hasNext()) {
+          return true;
         }
+      } catch (IOException ex) {
+        throw new IllegalStateException("Error while processing input.", ex);
       }
+    }
 
-      // At this point we are done processing the input. Close the record processor
-      if (!isClosed) {
-        closeRecordProcessor();
-        isClosed = true;
-      }
-
-      // It is possible that some operators add records after closing the processor, so make sure
-      // to check the lastRecordOutput
-      if (lastRecordOutput.hasNext()) {
-        return true;
-      }
-
-      lastRecordOutput.clear();
-      return false;
+    // At this point we are done processing the input. Close the record processor
+    if (!isClosed) {
+      closeRecordProcessor();
+      isClosed = true;
     }
 
-    @Override
-    public Tuple2<HiveKey, BytesWritable> next() {
-      if (hasNext()) {
-        return lastRecordOutput.next();
-      }
-      throw new NoSuchElementException("There are no more elements");
+    // It is possible that some operators add records after closing the processor, so make sure
+    // to check the lastRecordOutput
+    if (lastRecordOutput.hasNext()) {
+      return true;
     }
 
-    @Override
-    public void remove() {
-      throw new UnsupportedOperationException("Iterator.remove() is not supported");
+    lastRecordOutput.clear();
+    return false;
+  }
+
+  @Override
+  public Tuple2<HiveKey, BytesWritable> next() {
+    if (hasNext()) {
+      return lastRecordOutput.next();
     }
+    throw new NoSuchElementException("There are no more elements");
   }
+
+  @Override
+  public void remove() {
+    throw new UnsupportedOperationException("Iterator.remove() is not supported");
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java
index 53c5c0e..ff21a52 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java
@@ -38,7 +38,7 @@ public class HiveMapFunction extends HivePairFlatMapFunction<
 
   @SuppressWarnings("unchecked")
   @Override
-  public Iterable<Tuple2<HiveKey, BytesWritable>>
+  public Iterator<Tuple2<HiveKey, BytesWritable>>
   call(Iterator<Tuple2<BytesWritable, BytesWritable>> it) throws Exception {
     initJobConf();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java
index f6595f1..eeb4443 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java
@@ -36,7 +36,7 @@ public class HiveReduceFunction extends HivePairFlatMapFunction<
 
   @SuppressWarnings("unchecked")
   @Override
-  public Iterable<Tuple2<HiveKey, BytesWritable>>
+  public Iterator<Tuple2<HiveKey, BytesWritable>>
   call(Iterator<Tuple2<HiveKey, Iterable<BytesWritable>>> it) throws Exception {
     initJobConf();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SortByShuffler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SortByShuffler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SortByShuffler.java
index a6350d3..997ab7e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SortByShuffler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SortByShuffler.java
@@ -75,60 +75,52 @@ public class SortByShuffler implements SparkShuffler {
     private static final long serialVersionUID = 1L;
 
     @Override
-    public Iterable<Tuple2<HiveKey, Iterable<BytesWritable>>> call(
-        final Iterator<Tuple2<HiveKey, BytesWritable>> it) throws Exception {
+    public Iterator<Tuple2<HiveKey, Iterable<BytesWritable>>> call(
+      final Iterator<Tuple2<HiveKey, BytesWritable>> it) throws Exception {
       // Use input iterator to back returned iterable object.
-      final Iterator<Tuple2<HiveKey, Iterable<BytesWritable>>> resultIt =
-          new Iterator<Tuple2<HiveKey, Iterable<BytesWritable>>>() {
-            HiveKey curKey = null;
-            List<BytesWritable> curValues = new ArrayList<BytesWritable>();
+      return new Iterator<Tuple2<HiveKey, Iterable<BytesWritable>>>() {
+        HiveKey curKey = null;
+        List<BytesWritable> curValues = new ArrayList<BytesWritable>();
 
-            @Override
-            public boolean hasNext() {
-              return it.hasNext() || curKey != null;
-            }
+        @Override
+        public boolean hasNext() {
+          return it.hasNext() || curKey != null;
+        }
 
-            @Override
-            public Tuple2<HiveKey, Iterable<BytesWritable>> next() {
-              // TODO: implement this by accumulating rows with the same key into a list.
-              // Note that this list needs to improved to prevent excessive memory usage, but this
-              // can be done in later phase.
-              while (it.hasNext()) {
-                Tuple2<HiveKey, BytesWritable> pair = it.next();
-                if (curKey != null && !curKey.equals(pair._1())) {
-                  HiveKey key = curKey;
-                  List<BytesWritable> values = curValues;
-                  curKey = pair._1();
-                  curValues = new ArrayList<BytesWritable>();
-                  curValues.add(pair._2());
-                  return new Tuple2<HiveKey, Iterable<BytesWritable>>(key, values);
-                }
-                curKey = pair._1();
-                curValues.add(pair._2());
-              }
-              if (curKey == null) {
-                throw new NoSuchElementException();
-              }
-              // if we get here, this should be the last element we have
+        @Override
+        public Tuple2<HiveKey, Iterable<BytesWritable>> next() {
+          // TODO: implement this by accumulating rows with the same key into a list.
+          // Note that this list needs to improved to prevent excessive memory usage, but this
+          // can be done in later phase.
+          while (it.hasNext()) {
+            Tuple2<HiveKey, BytesWritable> pair = it.next();
+            if (curKey != null && !curKey.equals(pair._1())) {
               HiveKey key = curKey;
-              curKey = null;
-              return new Tuple2<HiveKey, Iterable<BytesWritable>>(key, curValues);
+              List<BytesWritable> values = curValues;
+              curKey = pair._1();
+              curValues = new ArrayList<BytesWritable>();
+              curValues.add(pair._2());
+              return new Tuple2<HiveKey, Iterable<BytesWritable>>(key, values);
             }
+            curKey = pair._1();
+            curValues.add(pair._2());
+          }
+          if (curKey == null) {
+            throw new NoSuchElementException();
+          }
+          // if we get here, this should be the last element we have
+          HiveKey key = curKey;
+          curKey = null;
+          return new Tuple2<HiveKey, Iterable<BytesWritable>>(key, curValues);
+        }
 
-            @Override
-            public void remove() {
-              // Not implemented.
-              // throw Unsupported Method Invocation Exception.
-              throw new UnsupportedOperationException();
-            }
-
-          };
-
-      return new Iterable<Tuple2<HiveKey, Iterable<BytesWritable>>>() {
         @Override
-        public Iterator<Tuple2<HiveKey, Iterable<BytesWritable>>> iterator() {
-          return resultIt;
+        public void remove() {
+          // Not implemented.
+          // throw Unsupported Method Invocation Exception.
+          throw new UnsupportedOperationException();
         }
+
       };
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java
index 09c54c1..b48de3e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java
@@ -24,15 +24,15 @@ import java.util.Map;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.spark.JavaSparkListener;
 import org.apache.spark.executor.TaskMetrics;
+import org.apache.spark.scheduler.SparkListener;
 import org.apache.spark.scheduler.SparkListenerJobStart;
 import org.apache.spark.scheduler.SparkListenerTaskEnd;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-public class JobMetricsListener extends JavaSparkListener {
+public class JobMetricsListener extends SparkListener {
 
   private static final Logger LOG = LoggerFactory.getLogger(JobMetricsListener.class);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveKVResultCache.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveKVResultCache.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveKVResultCache.java
index ee9f9b7..7bb9c62 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveKVResultCache.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveKVResultCache.java
@@ -282,9 +282,8 @@ public class TestHiveKVResultCache {
 
     resultList.init(rows, threshold, separate, prefix1, prefix2);
     long startTime = System.currentTimeMillis();
-    Iterator it = resultList.iterator();
-    while (it.hasNext()) {
-      Object item = it.next();
+    while (resultList.hasNext()) {
+      Object item = resultList.next();
       if (output != null) {
         output.add((Tuple2<HiveKey, BytesWritable>)item);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/spark-client/pom.xml
----------------------------------------------------------------------
diff --git a/spark-client/pom.xml b/spark-client/pom.xml
index 6cf3b17..effc13b 100644
--- a/spark-client/pom.xml
+++ b/spark-client/pom.xml
@@ -33,7 +33,6 @@
 
   <properties>
     <hive.path.to.root>..</hive.path.to.root>
-    <scala.binary.version>2.10</scala.binary.version>
     <test.redirectToFile>true</test.redirectToFile>
   </properties>
 
@@ -70,6 +69,14 @@
          <groupId>com.esotericsoftware.kryo</groupId>
          <artifactId>kryo</artifactId>
        </exclusion>
+        <exclusion>
+          <groupId>org.glassfish.jersey.containers</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.glassfish.jersey.core</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
        <exclusion>
          <groupId>org.slf4j</groupId>
          <artifactId>slf4j-log4j12</artifactId>
@@ -96,6 +103,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.glassfish.jersey.containers</groupId>
+      <artifactId>jersey-container-servlet</artifactId>
+      <version>${glassfish.jersey.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hive</groupId>
       <artifactId>hive-service-rpc</artifactId>
       <version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java b/spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java
index e77aa78..0f03a64 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java
@@ -151,7 +151,6 @@ public class MetricsCollection {
 
       // Input metrics.
       boolean hasInputMetrics = false;
-      DataReadMethod readMethod = null;
       long bytesRead = 0L;
 
       // Shuffle read metrics.
@@ -177,11 +176,6 @@ public class MetricsCollection {
 
         if (m.inputMetrics != null) {
           hasInputMetrics = true;
-          if (readMethod == null) {
-            readMethod = m.inputMetrics.readMethod;
-          } else if (readMethod != m.inputMetrics.readMethod) {
-            readMethod = DataReadMethod.Multiple;
-          }
           bytesRead += m.inputMetrics.bytesRead;
         }
 
@@ -201,7 +195,7 @@ public class MetricsCollection {
 
       InputMetrics inputMetrics = null;
       if (hasInputMetrics) {
-        inputMetrics = new InputMetrics(readMethod, bytesRead);
+        inputMetrics = new InputMetrics(bytesRead);
       }
 
       ShuffleReadMetrics shuffleReadMetrics = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java b/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java
index e3b88d1..ede8ce9 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java
@@ -43,11 +43,11 @@ import org.apache.hive.spark.client.metrics.Metrics;
 import org.apache.hive.spark.client.rpc.Rpc;
 import org.apache.hive.spark.client.rpc.RpcConfiguration;
 import org.apache.hive.spark.counter.SparkCounters;
-import org.apache.spark.JavaSparkListener;
 import org.apache.spark.SparkConf;
 import org.apache.spark.SparkJobInfo;
 import org.apache.spark.api.java.JavaFutureAction;
 import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.scheduler.SparkListener;
 import org.apache.spark.scheduler.SparkListenerJobEnd;
 import org.apache.spark.scheduler.SparkListenerJobStart;
 import org.apache.spark.scheduler.SparkListenerTaskEnd;
@@ -441,7 +441,7 @@ public class RemoteDriver {
 
   }
 
-  private class ClientListener extends JavaSparkListener {
+  private class ClientListener extends SparkListener {
 
     private final Map<Integer, Integer> stageToJobId = Maps.newHashMap();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java
index e46b67d..f137007 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java
@@ -28,25 +28,20 @@ import org.apache.hadoop.hive.common.classification.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public class InputMetrics implements Serializable {
-
-  public final DataReadMethod readMethod;
   public final long bytesRead;
 
   private InputMetrics() {
     // For Serialization only.
-    this(null, 0L);
+    this(0L);
   }
 
   public InputMetrics(
-      DataReadMethod readMethod,
       long bytesRead) {
-    this.readMethod = readMethod;
     this.bytesRead = bytesRead;
   }
 
   public InputMetrics(TaskMetrics metrics) {
-    this(DataReadMethod.valueOf(metrics.inputMetrics().get().readMethod().toString()),
-      metrics.inputMetrics().get().bytesRead());
+    this(metrics.inputMetrics().bytesRead());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java
index a7305cf..418d534 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java
@@ -99,15 +99,15 @@ public class Metrics implements Serializable {
   }
 
   private static InputMetrics optionalInputMetric(TaskMetrics metrics) {
-    return metrics.inputMetrics().isDefined() ? new InputMetrics(metrics) : null;
+    return (metrics.inputMetrics() != null) ? new InputMetrics(metrics) : null;
   }
 
   private static ShuffleReadMetrics optionalShuffleReadMetric(TaskMetrics metrics) {
-    return metrics.shuffleReadMetrics().isDefined() ? new ShuffleReadMetrics(metrics) : null;
+    return (metrics.shuffleReadMetrics() != null) ? new ShuffleReadMetrics(metrics) : null;
   }
 
   private static ShuffleWriteMetrics optionalShuffleWriteMetrics(TaskMetrics metrics) {
-    return metrics.shuffleWriteMetrics().isDefined() ? new ShuffleWriteMetrics(metrics) : null;
+    return (metrics.shuffleWriteMetrics() != null) ? new ShuffleWriteMetrics(metrics) : null;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleReadMetrics.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleReadMetrics.java b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleReadMetrics.java
index be14c06..9ff4d0f 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleReadMetrics.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleReadMetrics.java
@@ -30,9 +30,9 @@ import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 public class ShuffleReadMetrics implements Serializable {
 
   /** Number of remote blocks fetched in shuffles by tasks. */
-  public final int remoteBlocksFetched;
+  public final long remoteBlocksFetched;
   /** Number of local blocks fetched in shuffles by tasks. */
-  public final int localBlocksFetched;
+  public final long localBlocksFetched;
   /**
    * Time tasks spent waiting for remote shuffle blocks. This only includes the
    * time blocking on shuffle input data. For instance if block B is being
@@ -49,8 +49,8 @@ public class ShuffleReadMetrics implements Serializable {
   }
 
   public ShuffleReadMetrics(
-      int remoteBlocksFetched,
-      int localBlocksFetched,
+      long remoteBlocksFetched,
+      long localBlocksFetched,
       long fetchWaitTime,
       long remoteBytesRead) {
     this.remoteBlocksFetched = remoteBlocksFetched;
@@ -60,16 +60,16 @@ public class ShuffleReadMetrics implements Serializable {
   }
 
   public ShuffleReadMetrics(TaskMetrics metrics) {
-    this(metrics.shuffleReadMetrics().get().remoteBlocksFetched(),
-      metrics.shuffleReadMetrics().get().localBlocksFetched(),
-      metrics.shuffleReadMetrics().get().fetchWaitTime(),
-      metrics.shuffleReadMetrics().get().remoteBytesRead());
+    this(metrics.shuffleReadMetrics().remoteBlocksFetched(),
+      metrics.shuffleReadMetrics().localBlocksFetched(),
+      metrics.shuffleReadMetrics().fetchWaitTime(),
+      metrics.shuffleReadMetrics().remoteBytesRead());
   }
 
   /**
    * Number of blocks fetched in shuffle by tasks (remote or local).
    */
-  public int getTotalBlocksFetched() {
+  public long getTotalBlocksFetched() {
     return remoteBlocksFetched + localBlocksFetched;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleWriteMetrics.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleWriteMetrics.java b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleWriteMetrics.java
index 4420e4d..64a4b86 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleWriteMetrics.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleWriteMetrics.java
@@ -47,8 +47,8 @@ public class ShuffleWriteMetrics implements Serializable {
   }
 
   public ShuffleWriteMetrics(TaskMetrics metrics) {
-    this(metrics.shuffleWriteMetrics().get().shuffleBytesWritten(),
-      metrics.shuffleWriteMetrics().get().shuffleWriteTime());
+    this(metrics.shuffleWriteMetrics().shuffleBytesWritten(),
+      metrics.shuffleWriteMetrics().shuffleWriteTime());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ac977cc8/spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java
----------------------------------------------------------------------
diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java b/spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java
index 5146e91..8fef66b 100644
--- a/spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java
+++ b/spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java
@@ -95,22 +95,21 @@ public class TestMetricsCollection {
 
     long value = taskValue(1, 1, 1);
     Metrics metrics1 = new Metrics(value, value, value, value, value, value, value,
-      new InputMetrics(DataReadMethod.Memory, value), null, null);
+      new InputMetrics(value), null, null);
     Metrics metrics2 = new Metrics(value, value, value, value, value, value, value,
-      new InputMetrics(DataReadMethod.Disk, value), null, null);
+      new InputMetrics(value), null, null);
 
     collection.addMetrics(1, 1, 1, metrics1);
     collection.addMetrics(1, 1, 2, metrics2);
 
     Metrics global = collection.getAllMetrics();
     assertNotNull(global.inputMetrics);
-    assertEquals(DataReadMethod.Multiple, global.inputMetrics.readMethod);
   }
 
   private Metrics makeMetrics(int jobId, int stageId, long taskId) {
     long value = 1000000 * jobId + 1000 * stageId + taskId;
     return new Metrics(value, value, value, value, value, value, value,
-      new InputMetrics(DataReadMethod.Memory, value),
+      new InputMetrics(value),
       new ShuffleReadMetrics((int) value, (int) value, value, value),
       new ShuffleWriteMetrics(value, value));
   }
@@ -156,7 +155,6 @@ public class TestMetricsCollection {
     assertEquals(expected, metrics.memoryBytesSpilled);
     assertEquals(expected, metrics.diskBytesSpilled);
 
-    assertEquals(DataReadMethod.Memory, metrics.inputMetrics.readMethod);
     assertEquals(expected, metrics.inputMetrics.bytesRead);
 
     assertEquals(expected, metrics.shuffleReadMetrics.remoteBlocksFetched);


[31/44] hive git commit: HIVE-14843: HIVE-14751 introduced ambiguity in grammar (Jesus Camacho Rodriguez, reviewed by Pengcheng Xiong)

Posted by se...@apache.org.
HIVE-14843: HIVE-14751 introduced ambiguity in grammar (Jesus Camacho Rodriguez, reviewed by Pengcheng Xiong)


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

Branch: refs/heads/hive-14535
Commit: 667e9dd50ef4cfc9f743f8716da34339ec012f91
Parents: 871b55f
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Sep 27 17:00:45 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Sep 28 08:13:50 2016 +0100

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/667e9dd5/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index e6b70a0..04f87b8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -347,8 +347,8 @@ intervalLiteral
 
 intervalQualifiers
     :
-    KW_YEAR KW_TO KW_MONTH -> TOK_INTERVAL_YEAR_MONTH_LITERAL
-    | KW_DAY KW_TO KW_SECOND -> TOK_INTERVAL_DAY_TIME_LITERAL
+    (KW_YEAR KW_TO) => KW_YEAR KW_TO KW_MONTH -> TOK_INTERVAL_YEAR_MONTH_LITERAL
+    | (KW_DAY KW_TO) => KW_DAY KW_TO KW_SECOND -> TOK_INTERVAL_DAY_TIME_LITERAL
     | KW_YEAR -> TOK_INTERVAL_YEAR_LITERAL
     | KW_MONTH -> TOK_INTERVAL_MONTH_LITERAL
     | KW_DAY -> TOK_INTERVAL_DAY_LITERAL


[29/44] hive git commit: HIVE-7224: Set incremental printing to true by default in Beeline (Sahil Takiar, reviewed by Thejas M Nair>

Posted by se...@apache.org.
HIVE-7224: Set incremental printing to true by default in Beeline (Sahil Takiar, reviewed by Thejas M Nair>


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

Branch: refs/heads/hive-14535
Commit: 7d3da1778fdbfdb9b3eb0a19a10260b0258e1f87
Parents: d3b88f6
Author: Sahil Takiar <ta...@cloudera.com>
Authored: Tue Sep 27 17:41:59 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Tue Sep 27 17:41:59 2016 -0500

----------------------------------------------------------------------
 beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7d3da177/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
index 59fbca3..57b9c46 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
@@ -74,7 +74,7 @@ class BeeLineOpts implements Completer {
   private boolean autoCommit = false;
   private boolean verbose = false;
   private boolean force = false;
-  private boolean incremental = false;
+  private boolean incremental = true;
   private int incrementalBufferRows = DEFAULT_INCREMENTAL_BUFFER_ROWS;
   private boolean showWarnings = false;
   private boolean showNestedErrs = false;


[03/44] hive git commit: HIVE-14779 make DbTxnManager.HeartbeaterThread a daemon (Eugene Koifman, reviewed by Alan Gates)

Posted by se...@apache.org.
HIVE-14779 make DbTxnManager.HeartbeaterThread a daemon (Eugene Koifman, reviewed by Alan Gates)


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

Branch: refs/heads/hive-14535
Commit: 3c55115b6eaaa02442c9c487d5d328275e211567
Parents: eab7b40
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Mon Sep 19 17:14:03 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Mon Sep 19 17:14:03 2016 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3c55115b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index a446999..da7505b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -105,7 +105,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
       };
 
   private static AtomicInteger heartbeaterMSClientCount = new AtomicInteger(0);
-  private int heartbeaterThreadPoolSize = 0;
+  private static int heartbeaterThreadPoolSize = 0;
 
   private static SynchronizedMetaStoreClient getThreadLocalMSClient() {
     return threadLocalMSClient.get();
@@ -625,6 +625,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
   public static class HeartbeaterThread extends Thread {
     public HeartbeaterThread(Runnable target, String name) {
       super(target, name);
+      setDaemon(true);
     }
 
     @Override


[17/44] hive git commit: HIVE-14805: Subquery inside a view will have the object in the subquery as the direct input (Aihua Xu, reviewed by Yongzhi Chen)

Posted by se...@apache.org.
HIVE-14805: Subquery inside a view will have the object in the subquery as the direct input (Aihua Xu, reviewed by Yongzhi Chen)


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

Branch: refs/heads/hive-14535
Commit: f284b6d04aea51bce4e438f31a7b5ed8597df8fd
Parents: da376eb
Author: Aihua Xu <ai...@apache.org>
Authored: Wed Sep 21 13:59:14 2016 -0400
Committer: Aihua Xu <ai...@apache.org>
Committed: Fri Sep 23 09:07:15 2016 -0400

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  2 +-
 .../hadoop/hive/ql/plan/TestViewEntity.java     | 31 +++++++++++
 .../results/clientpositive/cbo_union_view.q.out | 12 +++++
 .../results/clientpositive/ppd_union_view.q.out | 12 +++++
 .../results/clientpositive/union_view.q.out     | 54 ++++++++++++++++++++
 5 files changed, 110 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f284b6d0/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 577d006..747f387 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -997,7 +997,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     // Recursively do the first phase of semantic analysis for the subquery
     QBExpr qbexpr = new QBExpr(alias);
 
-    doPhase1QBExpr(subqref, qbexpr, qb.getId(), alias);
+    doPhase1QBExpr(subqref, qbexpr, qb.getId(), alias, qb.isInsideView());
 
     // If the alias is already there then we have a conflict
     if (qb.exists(alias)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f284b6d0/ql/src/test/org/apache/hadoop/hive/ql/plan/TestViewEntity.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestViewEntity.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestViewEntity.java
index 2d70a1b..fa01416 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestViewEntity.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestViewEntity.java
@@ -141,4 +141,35 @@ public class TestViewEntity {
 
   }
 
+  /**
+   * Verify that the the query with the subquery inside a view will have the correct
+   * direct and indirect inputs.
+   * @throws Exception
+   */
+  @Test
+  public void testSubQueryInSubView() throws Exception {
+    String prefix = "tvsubqueryinsubview" + NAME_PREFIX;
+    final String tab1 = prefix + "t";
+    final String view1 = prefix + "v";
+    final String view2 = prefix + "v2";
+
+    int ret = driver.run("create table " + tab1 + "(id int)").getResponseCode();
+    assertEquals("Checking command success", 0, ret);
+    ret = driver.run("create view " + view1 + " as select * from " + tab1).getResponseCode();
+    assertEquals("Checking command success", 0, ret);
+
+    ret = driver.run("create view " + view2 + " as select * from (select * from " + view1 + ") x").getResponseCode();
+    assertEquals("Checking command success", 0, ret);
+
+    driver.compile("select * from " + view2);
+    // view entity
+    assertEquals("default@" + view2, CheckInputReadEntity.readEntities[0].getName());
+
+    // table1 and view1 as second read entity
+    assertEquals("default@" + view1, CheckInputReadEntity.readEntities[1].getName());
+    assertFalse("Table is not direct input", CheckInputReadEntity.readEntities[1].isDirect());
+    assertEquals("default@" + tab1, CheckInputReadEntity.readEntities[2].getName());
+    assertFalse("Table is not direct input", CheckInputReadEntity.readEntities[2].isDirect());
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f284b6d0/ql/src/test/results/clientpositive/cbo_union_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_union_view.q.out b/ql/src/test/results/clientpositive/cbo_union_view.q.out
index ed6bba9..d179c28 100644
--- a/ql/src/test/results/clientpositive/cbo_union_view.q.out
+++ b/ql/src/test/results/clientpositive/cbo_union_view.q.out
@@ -64,6 +64,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src_union_1
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -87,6 +89,8 @@ STAGE PLANS:
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             alias: src_union_2
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -110,6 +114,8 @@ STAGE PLANS:
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             alias: src_union_3
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -152,6 +158,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src_union_1
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '1')) (type: boolean)
@@ -175,6 +183,8 @@ STAGE PLANS:
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             alias: src_union_2
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '1')) (type: boolean)
@@ -198,6 +208,8 @@ STAGE PLANS:
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             alias: src_union_3
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '1')) (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/f284b6d0/ql/src/test/results/clientpositive/ppd_union_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ppd_union_view.q.out b/ql/src/test/results/clientpositive/ppd_union_view.q.out
index aa123d3..69234a1 100644
--- a/ql/src/test/results/clientpositive/ppd_union_view.q.out
+++ b/ql/src/test/results/clientpositive/ppd_union_view.q.out
@@ -149,6 +149,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: t1_old
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 14 Basic stats: COMPLETE Column stats: NONE
             GatherStats: false
             Filter Operator
@@ -170,6 +172,8 @@ STAGE PLANS:
                   auto parallelism: false
           TableScan
             alias: t1_mapping
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE
             GatherStats: false
             Filter Operator
@@ -322,6 +326,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: t1_new
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             GatherStats: false
             Filter Operator
@@ -477,6 +483,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: t1_old
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             GatherStats: false
             Filter Operator
@@ -498,6 +506,8 @@ STAGE PLANS:
                   auto parallelism: false
           TableScan
             alias: t1_mapping
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             GatherStats: false
             Filter Operator
@@ -554,6 +564,8 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: t1_new
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
             GatherStats: false
             Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/f284b6d0/ql/src/test/results/clientpositive/union_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_view.q.out b/ql/src/test/results/clientpositive/union_view.q.out
index 985ef9d..29f6758 100644
--- a/ql/src/test/results/clientpositive/union_view.q.out
+++ b/ql/src/test/results/clientpositive/union_view.q.out
@@ -181,6 +181,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_1
             filterExpr: ((key = 86) and (ds = '1')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -205,6 +207,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_2
             filterExpr: ((key = 86) and (ds = '1')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '1')) (type: boolean)
@@ -229,6 +233,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_3
             filterExpr: ((key = 86) and (ds = '1')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '1')) (type: boolean)
@@ -294,6 +300,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_1
             filterExpr: ((key = 86) and (ds = '2')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '2')) (type: boolean)
@@ -318,6 +326,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_2
             filterExpr: ((key = 86) and (ds = '2')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -342,6 +352,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_3
             filterExpr: ((key = 86) and (ds = '2')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '2')) (type: boolean)
@@ -407,6 +419,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_1
             filterExpr: ((key = 86) and (ds = '3')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '3')) (type: boolean)
@@ -431,6 +445,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_2
             filterExpr: ((key = 86) and (ds = '3')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '3')) (type: boolean)
@@ -455,6 +471,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_3
             filterExpr: ((key = 86) and (ds = '3')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -524,6 +542,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_1
             filterExpr: ((key = 86) and ds is not null) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -548,6 +568,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_2
             filterExpr: ((key = 86) and ds is not null) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -572,6 +594,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_3
             filterExpr: ((key = 86) and ds is not null) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -693,6 +717,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_1
             filterExpr: (ds = '1') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -712,6 +738,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_2
             filterExpr: (ds = '1') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (ds = '1') (type: boolean)
@@ -734,6 +762,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_3
             filterExpr: (ds = '1') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (ds = '1') (type: boolean)
@@ -808,6 +838,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_1
             filterExpr: (ds = '2') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (ds = '2') (type: boolean)
@@ -830,6 +862,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_2
             filterExpr: (ds = '2') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -849,6 +883,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_3
             filterExpr: (ds = '2') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (ds = '2') (type: boolean)
@@ -923,6 +959,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_1
             filterExpr: (ds = '3') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (ds = '3') (type: boolean)
@@ -945,6 +983,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_2
             filterExpr: (ds = '3') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (ds = '3') (type: boolean)
@@ -967,6 +1007,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_3
             filterExpr: (ds = '3') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -1043,6 +1085,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_1
             filterExpr: ((key = 86) and (ds = '4')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '4')) (type: boolean)
@@ -1067,6 +1111,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_2
             filterExpr: ((key = 86) and (ds = '4')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((key = 86) and (ds = '4')) (type: boolean)
@@ -1091,6 +1137,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_3
             filterExpr: ((key = 86) and (ds = '4')) (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -1155,6 +1203,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_1
             filterExpr: (ds = '4') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (ds = '4') (type: boolean)
@@ -1177,6 +1227,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_2
             filterExpr: (ds = '4') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (ds = '4') (type: boolean)
@@ -1199,6 +1251,8 @@ STAGE PLANS:
           TableScan
             alias: src_union_3
             filterExpr: (ds = '4') (type: boolean)
+            properties:
+              insideView TRUE
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE


[28/44] hive git commit: HIVE-14835: Improve ptest2 build time (Prasanth Jayachandran reviewed by Sergio Pena)

Posted by se...@apache.org.
HIVE-14835: Improve ptest2 build time (Prasanth Jayachandran reviewed by Sergio Pena)


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

Branch: refs/heads/hive-14535
Commit: d3b88f664415ff114de74aa2a0da2f1e1acbf60d
Parents: 0c55d46
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Tue Sep 27 10:59:33 2016 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Tue Sep 27 10:59:33 2016 -0700

----------------------------------------------------------------------
 dev-support/jenkins-execute-build.sh               | 4 +---
 testutils/ptest2/src/main/resources/source-prep.vm | 4 ++--
 2 files changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d3b88f66/dev-support/jenkins-execute-build.sh
----------------------------------------------------------------------
diff --git a/dev-support/jenkins-execute-build.sh b/dev-support/jenkins-execute-build.sh
index 2142942..972abae 100644
--- a/dev-support/jenkins-execute-build.sh
+++ b/dev-support/jenkins-execute-build.sh
@@ -70,9 +70,7 @@ test -n "$TEST_HANDLE" || fail "TEST_HANDLE must be specified and cannot be empt
 test -n "$PTEST_API_ENDPOINT" || fail "PTEST_API_ENDPOINT must be specified and cannot be empty."
 test -n "$PTEST_LOG_ENDPOINT" || fail "PTEST_LOG_ENDPOINT must be specified and cannot be empty."
 
-# WORKSPACE is an environment variable created by Jenkins, and it is the directory where the build is executed.
-# If not set, then default to $HOME
-MVN_REPO_LOCAL=${WORKSPACE:-$HOME}/.m2/repository
+MVN_REPO_LOCAL=${HOME}/.m2/repository
 
 # Directory where to build the ptest framework
 PTEST_BUILD_DIR="$PWD/hive/build"

http://git-wip-us.apache.org/repos/asf/hive/blob/d3b88f66/testutils/ptest2/src/main/resources/source-prep.vm
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/resources/source-prep.vm b/testutils/ptest2/src/main/resources/source-prep.vm
index 67e6a95..0fc22be 100644
--- a/testutils/ptest2/src/main/resources/source-prep.vm
+++ b/testutils/ptest2/src/main/resources/source-prep.vm
@@ -102,11 +102,11 @@ cd $workingDir/
         fi
       done
     #end
-    mvn -B clean install -DskipTests -Dmaven.repo.local=$workingDir/maven $mavenArgs $mavenBuildArgs
+    mvn -B clean install -DskipTests -T 4 -q -Dmaven.repo.local=$workingDir/maven $mavenArgs $mavenBuildArgs
     if [[ -d "itests" ]]
     then
       cd itests
-      mvn -B clean install -DskipTests -Dmaven.repo.local=$workingDir/maven $mavenArgs $mavenBuildArgs
+      mvn -B clean install -DskipTests -T 4 -q -Dmaven.repo.local=$workingDir/maven $mavenArgs $mavenBuildArgs
     fi
   elif [[ "${buildTool}" == "ant" ]]
   then


[36/44] hive git commit: HIVE-14778 document threading model of Streaming API (Eugene Koifman, reviewed by Alan Gates)

Posted by se...@apache.org.
HIVE-14778 document threading model of Streaming API (Eugene Koifman, reviewed by Alan Gates)


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

Branch: refs/heads/hive-14535
Commit: 20304c0705c4ad861b5915dacceaa6d6bdfe91fc
Parents: d16d4f1
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Thu Sep 29 10:41:42 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Thu Sep 29 10:41:42 2016 -0700

----------------------------------------------------------------------
 .../apache/hive/hcatalog/streaming/StreamingConnection.java  | 2 ++
 .../org/apache/hive/hcatalog/streaming/TransactionBatch.java | 8 ++++++--
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/20304c07/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java
index 8785a21..a8f4089 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java
@@ -22,6 +22,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 
 /**
  * Represents a connection to a HiveEndPoint. Used to acquire transaction batches.
+ * Note: the expectation is that there is at most 1 TransactionBatch outstanding for any given
+ * StreamingConnection.  Violating this may result in "out of sequence response".
  */
 public interface StreamingConnection {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/20304c07/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java
index 3c8670d..3bcc510 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java
@@ -24,8 +24,12 @@ import java.util.Collection;
 /**
  * Represents a set of Transactions returned by Hive. Supports opening, writing to
  * and commiting/aborting each transaction. The interface is designed to ensure
- * transactions in a batch are used up sequentially. Multiple transaction batches can be
- * used (initialized with separate RecordWriters) for concurrent streaming
+ * transactions in a batch are used up sequentially. To stream to the same HiveEndPoint
+ * concurrently, create separate StreamingConnections.
+ *
+ * Note on thread safety: At most 2 threads can run through a given TransactionBatch at the same
+ * time.  One thread may call {@link #heartbeat()} and the other all other methods.
+ * Violating this may result in "out of sequence response".
  *
  */
 public interface TransactionBatch  {


[42/44] hive git commit: HIVE-14775: Cleanup IOException usage in Metrics APIs (Barna Zsombor Klara reviewed by Peter Vary, Gabor Szadovszky, Szehon Ho, Mohit Sabharwal)

Posted by se...@apache.org.
HIVE-14775: Cleanup IOException usage in Metrics APIs (Barna Zsombor Klara reviewed by Peter Vary, Gabor Szadovszky, Szehon Ho, Mohit Sabharwal)


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

Branch: refs/heads/hive-14535
Commit: f903c4afad360ea66ec266abe8a3f414935c82ff
Parents: 45c1a09
Author: Mohit Sabharwal <mo...@cloudera.com>
Authored: Fri Sep 30 15:13:14 2016 -0400
Committer: Mohit Sabharwal <mo...@cloudera.com>
Committed: Fri Sep 30 15:13:14 2016 -0400

----------------------------------------------------------------------
 .../hive/common/metrics/LegacyMetrics.java      |  96 ++++++++++-------
 .../hive/common/metrics/MetricsMBean.java       |  13 +--
 .../hive/common/metrics/MetricsMBeanImpl.java   |  16 +--
 .../hive/common/metrics/common/Metrics.java     |  31 ++----
 .../metrics/metrics2/CodahaleMetrics.java       |  70 ++++++-------
 .../apache/hadoop/hive/ql/log/PerfLogger.java   |  33 ++----
 .../hive/common/metrics/TestLegacyMetrics.java  | 103 ++++++-------------
 .../hive/metastore/HMSMetricsListener.java      |  52 ++--------
 .../hadoop/hive/metastore/HiveMetaStore.java    |  13 +--
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  13 +--
 .../hadoop/hive/ql/exec/mr/MapRedTask.java      |   6 +-
 .../hadoop/hive/ql/exec/mr/MapredLocalTask.java |   6 +-
 .../hadoop/hive/ql/exec/spark/SparkTask.java    |   6 +-
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |   6 +-
 .../hive/service/cli/operation/Operation.java   |  22 ++--
 15 files changed, 176 insertions(+), 310 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java b/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java
index 9be9b50..ba2267b 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java
@@ -21,11 +21,13 @@ import org.apache.hadoop.hive.common.metrics.common.Metrics;
 import org.apache.hadoop.hive.common.metrics.common.MetricsScope;
 import org.apache.hadoop.hive.common.metrics.common.MetricsVariable;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.util.HashMap;
 
+import javax.management.JMException;
 import javax.management.MBeanServer;
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
@@ -47,6 +49,8 @@ import javax.management.ObjectName;
  */
 public class LegacyMetrics implements Metrics {
 
+  private static final Logger LOG = LoggerFactory.getLogger(LegacyMetrics.class);
+
   private LegacyMetrics() {
     // block
   }
@@ -59,12 +63,12 @@ public class LegacyMetrics implements Metrics {
    */
   public static class LegacyMetricsScope implements MetricsScope {
 
-    final LegacyMetrics metrics;
+    private final LegacyMetrics metrics;
 
-    final String name;
-    final String numCounter;
-    final String timeCounter;
-    final String avgTimeCounter;
+    private final String name;
+    private final String numCounter;
+    private final String timeCounter;
+    private final String avgTimeCounter;
 
     private boolean isOpen = false;
     private Long startTime = null;
@@ -72,9 +76,8 @@ public class LegacyMetrics implements Metrics {
     /**
      * Instantiates a named scope - intended to only be called by Metrics, so locally scoped.
      * @param name - name of the variable
-     * @throws IOException
      */
-    private LegacyMetricsScope(String name, LegacyMetrics metrics) throws IOException {
+    private LegacyMetricsScope(String name, LegacyMetrics metrics) {
       this.metrics = metrics;
       this.name = name;
       this.numCounter = name + ".n";
@@ -83,33 +86,41 @@ public class LegacyMetrics implements Metrics {
       open();
     }
 
-    public Long getNumCounter() throws IOException {
-      return (Long) metrics.get(numCounter);
+    public Long getNumCounter() {
+      try {
+        return (Long) metrics.get(numCounter);
+      } catch (JMException e) {
+        LOG.warn("Could not find counter value for " + numCounter + ", returning null instead. ", e);
+        return null;
+      }
     }
 
-    public Long getTimeCounter() throws IOException {
-      return (Long) metrics.get(timeCounter);
+    public Long getTimeCounter() {
+      try {
+        return (Long) metrics.get(timeCounter);
+      } catch (JMException e) {
+        LOG.warn("Could not find timer value for " + timeCounter + ", returning null instead. ", e);
+        return null;
+      }
     }
 
     /**
      * Opens scope, and makes note of the time started, increments run counter
-     * @throws IOException
      *
      */
-    public void open() throws IOException {
+    public void open() {
       if (!isOpen) {
         isOpen = true;
         startTime = System.currentTimeMillis();
       } else {
-        throw new IOException("Scope named " + name + " is not closed, cannot be opened.");
+        LOG.warn("Scope named " + name + " is not closed, cannot be opened.");
       }
     }
 
     /**
      * Closes scope, and records the time taken
-     * @throws IOException
      */
-    public void close() throws IOException {
+    public void close() {
       if (isOpen) {
         Long endTime = System.currentTimeMillis();
         synchronized(metrics) {
@@ -120,7 +131,7 @@ public class LegacyMetrics implements Metrics {
           }
         }
       } else {
-        throw new IOException("Scope named " + name + " is not open, cannot be closed.");
+        LOG.warn("Scope named " + name + " is not open, cannot be closed.");
       }
       isOpen = false;
     }
@@ -128,9 +139,8 @@ public class LegacyMetrics implements Metrics {
 
     /**
      * Closes scope if open, and reopens it
-     * @throws IOException
      */
-    public void reopen() throws IOException {
+    public void reopen() {
       if(isOpen) {
         close();
       }
@@ -164,37 +174,47 @@ public class LegacyMetrics implements Metrics {
     mbs.registerMBean(metrics, oname);
   }
 
-  public Long incrementCounter(String name) throws IOException {
+  public Long incrementCounter(String name) {
     return incrementCounter(name,Long.valueOf(1));
   }
 
-  public Long incrementCounter(String name, long increment) throws IOException {
-    Long value;
+  public Long incrementCounter(String name, long increment) {
+    Long value = null;
     synchronized(metrics) {
       if (!metrics.hasKey(name)) {
         value = Long.valueOf(increment);
         set(name, value);
       } else {
-        value = ((Long)get(name)) + increment;
-        set(name, value);
+        try {
+          value = ((Long)get(name)) + increment;
+          set(name, value);
+        } catch (JMException e) {
+          LOG.warn("Could not find counter value for " + name
+              + ", increment operation skipped.", e);
+        }
       }
     }
     return value;
   }
 
-  public Long decrementCounter(String name) throws IOException{
+  public Long decrementCounter(String name) {
     return decrementCounter(name, Long.valueOf(1));
   }
 
-  public Long decrementCounter(String name, long decrement) throws IOException {
-    Long value;
+  public Long decrementCounter(String name, long decrement) {
+    Long value = null;
     synchronized(metrics) {
       if (!metrics.hasKey(name)) {
         value = Long.valueOf(decrement);
         set(name, -value);
       } else {
-        value = ((Long)get(name)) - decrement;
-        set(name, value);
+        try {
+          value = ((Long)get(name)) - decrement;
+          set(name, value);
+        } catch (JMException e) {
+          LOG.warn("Could not find counter value for " + name
+              + ", decrement operation skipped.", e);
+        }
       }
     }
     return value;
@@ -205,15 +225,15 @@ public class LegacyMetrics implements Metrics {
     //Not implemented.
   }
 
-  public void set(String name, Object value) throws IOException{
+  public void set(String name, Object value) {
     metrics.put(name,value);
   }
 
-  public Object get(String name) throws IOException{
+  public Object get(String name) throws JMException {
     return metrics.get(name);
   }
 
-  public void startStoredScope(String name) throws IOException{
+  public void startStoredScope(String name) {
     if (threadLocalScopes.get().containsKey(name)) {
       threadLocalScopes.get().get(name).open();
     } else {
@@ -221,25 +241,25 @@ public class LegacyMetrics implements Metrics {
     }
   }
 
-  public MetricsScope getStoredScope(String name) throws IOException {
+  public MetricsScope getStoredScope(String name) throws IllegalStateException {
     if (threadLocalScopes.get().containsKey(name)) {
       return threadLocalScopes.get().get(name);
     } else {
-      throw new IOException("No metrics scope named " + name);
+      throw new IllegalStateException("No metrics scope named " + name);
     }
   }
 
-  public void endStoredScope(String name) throws IOException{
+  public void endStoredScope(String name) {
     if (threadLocalScopes.get().containsKey(name)) {
       threadLocalScopes.get().get(name).close();
     }
   }
 
-  public MetricsScope createScope(String name) throws IOException {
+  public MetricsScope createScope(String name) {
     return new LegacyMetricsScope(name, this);
   }
 
-  public void endScope(MetricsScope scope) throws IOException {
+  public void endScope(MetricsScope scope) {
     ((LegacyMetricsScope) scope).close();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/common/src/java/org/apache/hadoop/hive/common/metrics/MetricsMBean.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/MetricsMBean.java b/common/src/java/org/apache/hadoop/hive/common/metrics/MetricsMBean.java
index 19946d9..130d8aa 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/MetricsMBean.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/MetricsMBean.java
@@ -17,9 +17,8 @@
  */
 package org.apache.hadoop.hive.common.metrics;
 
-import java.io.IOException;
-
 import javax.management.DynamicMBean;
+import javax.management.JMException;
 
 /**
  * MBean definition for metrics tracking from jmx
@@ -36,21 +35,19 @@ public interface MetricsMBean extends DynamicMBean {
      * Add a key/metric and its value to track
      * @param name Name of the key/metric
      * @param value value associated with the key
-     * @throws Exception
      */
-    public abstract void put(String name, Object value) throws IOException;
+    public abstract void put(String name, Object value);
 
     /**
      *
      * @param name
      * @return value associated with the key
-     * @throws Exception
+     * @throws JMException
      */
-    public abstract Object get(String name) throws IOException;
-    
+    public abstract Object get(String name) throws JMException;
 
     /**
-     * Removes all the keys and values from this MetricsMBean. 
+     * Removes all the keys and values from this MetricsMBean.
      */
     void clear();
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/common/src/java/org/apache/hadoop/hive/common/metrics/MetricsMBeanImpl.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/MetricsMBeanImpl.java b/common/src/java/org/apache/hadoop/hive/common/metrics/MetricsMBeanImpl.java
index a973155..9e9b85c 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/MetricsMBeanImpl.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/MetricsMBeanImpl.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.common.metrics;
 
-import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -25,6 +24,7 @@ import javax.management.Attribute;
 import javax.management.AttributeList;
 import javax.management.AttributeNotFoundException;
 import javax.management.InvalidAttributeValueException;
+import javax.management.JMException;
 import javax.management.MBeanAttributeInfo;
 import javax.management.MBeanConstructorInfo;
 import javax.management.MBeanException;
@@ -137,7 +137,7 @@ public class MetricsMBeanImpl implements  MetricsMBean {
     }
 
     @Override
-    public void put(String name, Object value) throws IOException {
+    public void put(String name, Object value) {
       synchronized(metricsMap) {
         if (!metricsMap.containsKey(name)) {
           dirtyAttributeInfoCache = true;
@@ -147,16 +147,8 @@ public class MetricsMBeanImpl implements  MetricsMBean {
     }
 
     @Override
-    public Object get(String name) throws IOException {
-        try {
-          return getAttribute(name);
-        } catch (AttributeNotFoundException e) {
-            throw new IOException(e);
-        } catch (MBeanException e) {
-            throw new IOException(e);
-        } catch (ReflectionException e) {
-            throw new IOException(e);
-        }
+    public Object get(String name) throws JMException {
+      return getAttribute(name);
     }
 
     public void reset() {

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java b/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java
index 4297233..9b263d9 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java
@@ -17,12 +17,6 @@
  */
 package org.apache.hadoop.hive.common.metrics.common;
 
-import java.io.IOException;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-
-import java.io.IOException;
-
 /**
  * Generic Metics interface.
  */
@@ -36,32 +30,28 @@ public interface Metrics {
   /**
    *
    * @param name starts a scope of a given name.  Scopes is stored as thread-local variable.
-   * @throws IOException
    */
-  public void startStoredScope(String name) throws IOException;
+  public void startStoredScope(String name);
 
   /**
    * Closes the stored scope of a given name.
    * Note that this must be called on the same thread as where the scope was started.
    * @param name
-   * @throws IOException
    */
-  public void endStoredScope(String name) throws IOException;
+  public void endStoredScope(String name);
 
   /**
    * Create scope with given name and returns it.
    * @param name
    * @return
-   * @throws IOException
    */
-  public MetricsScope createScope(String name) throws IOException;
+  public MetricsScope createScope(String name);
 
   /**
    * Close the given scope.
    * @param scope
-   * @throws IOException
    */
-  public void endScope(MetricsScope scope) throws IOException;
+  public void endScope(MetricsScope scope);
 
   //Counter-related methods
 
@@ -69,43 +59,38 @@ public interface Metrics {
    * Increments a counter of the given name by 1.
    * @param name
    * @return
-   * @throws IOException
    */
-  public Long incrementCounter(String name) throws IOException;
+  public Long incrementCounter(String name);
 
   /**
    * Increments a counter of the given name by "increment"
    * @param name
    * @param increment
    * @return
-   * @throws IOException
    */
-  public Long incrementCounter(String name, long increment) throws IOException;
+  public Long incrementCounter(String name, long increment);
 
 
   /**
    * Decrements a counter of the given name by 1.
    * @param name
    * @return
-   * @throws IOException
    */
-  public Long decrementCounter(String name) throws IOException;
+  public Long decrementCounter(String name);
 
   /**
    * Decrements a counter of the given name by "decrement"
    * @param name
    * @param decrement
    * @return
-   * @throws IOException
    */
-  public Long decrementCounter(String name, long decrement) throws IOException;
+  public Long decrementCounter(String name, long decrement);
 
 
   /**
    * Adds a metrics-gauge to track variable.  For example, number of open database connections.
    * @param name name of gauge
    * @param variable variable to track.
-   * @throws IOException
    */
   public void addGauge(String name, final MetricsVariable variable);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java b/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java
index 4c43367..9525b45 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java
@@ -74,6 +74,7 @@ import java.util.concurrent.locks.ReentrantLock;
  * Codahale-backed Metrics implementation.
  */
 public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.common.Metrics {
+
   public static final String API_PREFIX = "api_";
   public static final String ACTIVE_CALLS = "active_calls_";
   public static final Logger LOGGER = LoggerFactory.getLogger(CodahaleMetrics.class);
@@ -98,64 +99,59 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
     }
   };
 
-  public static class CodahaleMetricsScope implements MetricsScope {
+  public class CodahaleMetricsScope implements MetricsScope {
 
-    final String name;
-    final Timer timer;
-    Timer.Context timerContext;
-    CodahaleMetrics metrics;
+    private final String name;
+    private final Timer timer;
+    private Timer.Context timerContext;
 
     private boolean isOpen = false;
 
     /**
      * Instantiates a named scope - intended to only be called by Metrics, so locally scoped.
      * @param name - name of the variable
-     * @throws IOException
      */
-    private CodahaleMetricsScope(String name, CodahaleMetrics metrics) throws IOException {
+    private CodahaleMetricsScope(String name) {
       this.name = name;
-      this.metrics = metrics;
-      this.timer = metrics.getTimer(name);
+      this.timer = CodahaleMetrics.this.getTimer(name);
       open();
     }
 
     /**
      * Opens scope, and makes note of the time started, increments run counter
-     * @throws IOException
      *
      */
-    public void open() throws IOException {
+    public void open() {
       if (!isOpen) {
         isOpen = true;
         this.timerContext = timer.time();
-        metrics.incrementCounter(ACTIVE_CALLS + name);
+        CodahaleMetrics.this.incrementCounter(ACTIVE_CALLS + name);
       } else {
-        throw new IOException("Scope named " + name + " is not closed, cannot be opened.");
+        LOGGER.warn("Scope named " + name + " is not closed, cannot be opened.");
       }
     }
 
     /**
      * Closes scope, and records the time taken
-     * @throws IOException
      */
-    public void close() throws IOException {
+    public void close() {
       if (isOpen) {
         timerContext.close();
-        metrics.decrementCounter(ACTIVE_CALLS + name);
+        CodahaleMetrics.this.decrementCounter(ACTIVE_CALLS + name);
       } else {
-        throw new IOException("Scope named " + name + " is not open, cannot be closed.");
+        LOGGER.warn("Scope named " + name + " is not open, cannot be closed.");
       }
       isOpen = false;
     }
   }
 
-  public CodahaleMetrics(HiveConf conf) throws Exception {
+  public CodahaleMetrics(HiveConf conf) {
     this.conf = conf;
     //Codahale artifacts are lazily-created.
     timers = CacheBuilder.newBuilder().build(
       new CacheLoader<String, com.codahale.metrics.Timer>() {
         @Override
-        public com.codahale.metrics.Timer load(String key) throws Exception {
+        public com.codahale.metrics.Timer load(String key) {
           Timer timer = new Timer(new ExponentiallyDecayingReservoir());
           metricRegistry.register(key, timer);
           return timer;
@@ -165,7 +161,7 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
     counters = CacheBuilder.newBuilder().build(
       new CacheLoader<String, Counter>() {
         @Override
-        public Counter load(String key) throws Exception {
+        public Counter load(String key) {
           Counter counter = new Counter();
           metricRegistry.register(key, counter);
           return counter;
@@ -215,17 +211,17 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
   }
 
   @Override
-  public void startStoredScope(String name) throws IOException {
+  public void startStoredScope(String name) {
     name = API_PREFIX + name;
     if (threadLocalScopes.get().containsKey(name)) {
       threadLocalScopes.get().get(name).open();
     } else {
-      threadLocalScopes.get().put(name, new CodahaleMetricsScope(name, this));
+      threadLocalScopes.get().put(name, new CodahaleMetricsScope(name));
     }
   }
 
   @Override
-  public void endStoredScope(String name) throws IOException {
+  public void endStoredScope(String name) {
     name = API_PREFIX + name;
     if (threadLocalScopes.get().containsKey(name)) {
       threadLocalScopes.get().get(name).close();
@@ -233,56 +229,56 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
     }
   }
 
-  public MetricsScope getStoredScope(String name) throws IOException {
+  public MetricsScope getStoredScope(String name) throws IllegalArgumentException {
     if (threadLocalScopes.get().containsKey(name)) {
       return threadLocalScopes.get().get(name);
     } else {
-      throw new IOException("No metrics scope named " + name);
+      throw new IllegalArgumentException("No metrics scope named " + name);
     }
   }
 
-  public MetricsScope createScope(String name) throws IOException {
+  public MetricsScope createScope(String name) {
     name = API_PREFIX + name;
-    return new CodahaleMetricsScope(name, this);
+    return new CodahaleMetricsScope(name);
   }
 
-  public void endScope(MetricsScope scope) throws IOException {
+  public void endScope(MetricsScope scope) {
     ((CodahaleMetricsScope) scope).close();
   }
 
   @Override
-  public Long incrementCounter(String name) throws IOException {
+  public Long incrementCounter(String name) {
     return incrementCounter(name, 1L);
   }
 
   @Override
-  public Long incrementCounter(String name, long increment) throws IOException {
+  public Long incrementCounter(String name, long increment) {
     String key = name;
     try {
       countersLock.lock();
       counters.get(key).inc(increment);
       return counters.get(key).getCount();
     } catch(ExecutionException ee) {
-      throw new RuntimeException(ee);
+      throw new IllegalStateException("Error retrieving counter from the metric registry ", ee);
     } finally {
       countersLock.unlock();
     }
   }
 
   @Override
-  public Long decrementCounter(String name) throws IOException {
+  public Long decrementCounter(String name) {
     return decrementCounter(name, 1L);
   }
 
   @Override
-  public Long decrementCounter(String name, long decrement) throws IOException {
+  public Long decrementCounter(String name, long decrement) {
     String key = name;
     try {
       countersLock.lock();
       counters.get(key).dec(decrement);
       return counters.get(key).getCount();
     } catch(ExecutionException ee) {
-      throw new RuntimeException(ee);
+      throw new IllegalStateException("Error retrieving counter from the metric registry ", ee);
     } finally {
       countersLock.unlock();
     }
@@ -312,14 +308,14 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
   }
 
   // This method is necessary to synchronize lazy-creation to the timers.
-  private Timer getTimer(String name) throws IOException {
+  private Timer getTimer(String name) {
     String key = name;
     try {
       timersLock.lock();
       Timer timer = timers.get(key);
       return timer;
     } catch (ExecutionException e) {
-      throw new IOException(e);
+      throw new IllegalStateException("Error retrieving timer from the metric registry ", e);
     } finally {
       timersLock.unlock();
     }
@@ -350,7 +346,7 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
   /**
    * Should be only called once to initialize the reporters
    */
-  private void initReporting(Set<MetricsReporting> reportingSet) throws Exception {
+  private void initReporting(Set<MetricsReporting> reportingSet) {
     for (MetricsReporting reporting : reportingSet) {
       switch(reporting) {
         case CONSOLE:

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
index 6a5d22f..7658f1c 100644
--- a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
+++ b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
@@ -224,27 +224,20 @@ public class PerfLogger {
 
   private void beginMetrics(String method) {
     Metrics metrics = MetricsFactory.getInstance();
-    try {
-      if (metrics != null) {
-        MetricsScope scope = metrics.createScope(method);
-        openScopes.put(method, scope);
-      }
-    } catch (IOException e) {
-      LOG.warn("Error recording metrics", e);
+    if (metrics != null) {
+      MetricsScope scope = metrics.createScope(method);
+      openScopes.put(method, scope);
     }
+
   }
 
   private void endMetrics(String method) {
     Metrics metrics = MetricsFactory.getInstance();
-    try {
-      if (metrics != null) {
-        MetricsScope scope = openScopes.remove(method);
-        if (scope != null) {
-          metrics.endScope(scope);
-        }
+    if (metrics != null) {
+      MetricsScope scope = openScopes.remove(method);
+      if (scope != null) {
+        metrics.endScope(scope);
       }
-    } catch (IOException e) {
-      LOG.warn("Error recording metrics", e);
     }
   }
 
@@ -253,14 +246,10 @@ public class PerfLogger {
    */
   public void cleanupPerfLogMetrics() {
     Metrics metrics = MetricsFactory.getInstance();
-    try {
-      if (metrics != null) {
-        for (MetricsScope openScope : openScopes.values()) {
-          metrics.endScope(openScope);
-        }
+    if (metrics != null) {
+      for (MetricsScope openScope : openScopes.values()) {
+        metrics.endScope(openScope);
       }
-    } catch (IOException e) {
-      LOG.warn("Error cleaning up metrics", e);
     }
     openScopes.clear();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/common/src/test/org/apache/hadoop/hive/common/metrics/TestLegacyMetrics.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/metrics/TestLegacyMetrics.java b/common/src/test/org/apache/hadoop/hive/common/metrics/TestLegacyMetrics.java
index a3fb04f..2e4fff1 100644
--- a/common/src/test/org/apache/hadoop/hive/common/metrics/TestLegacyMetrics.java
+++ b/common/src/test/org/apache/hadoop/hive/common/metrics/TestLegacyMetrics.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.common.metrics;
 
-import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
@@ -32,7 +31,6 @@ import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
 import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
-import org.apache.hadoop.hive.common.metrics.common.MetricsScope;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.junit.After;
 import org.junit.Before;
@@ -113,56 +111,23 @@ public class TestLegacyMetrics {
     assertEquals(Long.valueOf(0), v);
   }
 
-  private <T> void expectIOE(Callable<T> c) throws Exception {
-    try {
-      T t = c.call();
-      fail("IOE expected but ["+t+"] was returned.");
-    } catch (IOException ioe) {
-      // ok, expected
-    }
-  }
-
   @Test
   public void testScopeSingleThread() throws Exception {
     metrics.startStoredScope(scopeName);
     final LegacyMetrics.LegacyMetricsScope fooScope = (LegacyMetrics.LegacyMetricsScope) metrics.getStoredScope(scopeName);
     // the time and number counters become available only after the 1st
     // scope close:
-    expectIOE(new Callable<Long>() {
-      @Override
-      public Long call() throws Exception {
-        Long num = fooScope.getNumCounter();
-        return num;
-      }
-    });
-    expectIOE(new Callable<Long>() {
-      @Override
-      public Long call() throws Exception {
-        Long time = fooScope.getTimeCounter();
-        return time;
-      }
-    });
-    // cannot open scope that is already open:
-    expectIOE(new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        fooScope.open();
-        return null;
-      }
-    });
+    Long num = fooScope.getNumCounter();
+    assertNull(num);
+
+    Long time = fooScope.getTimeCounter();
+    assertNull(time);
 
     assertSame(fooScope, metrics.getStoredScope(scopeName));
     Thread.sleep(periodMs+ 1);
     // 1st close:
     // closing of open scope should be ok:
     metrics.endStoredScope(scopeName);
-    expectIOE(new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        metrics.endStoredScope(scopeName); // closing of closed scope not allowed
-        return null;
-      }
-    });
 
     assertEquals(Long.valueOf(1), fooScope.getNumCounter());
     final long t1 = fooScope.getTimeCounter().longValue();
@@ -172,14 +137,6 @@ public class TestLegacyMetrics {
 
    // opening allowed after closing:
     metrics.startStoredScope(scopeName);
-    // opening of already open scope not allowed:
-    expectIOE(new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        metrics.startStoredScope(scopeName);
-        return null;
-      }
-    });
 
     assertEquals(Long.valueOf(1), fooScope.getNumCounter());
     assertEquals(t1, fooScope.getTimeCounter().longValue());
@@ -229,17 +186,34 @@ public class TestLegacyMetrics {
     metrics.endStoredScope(scopeName);
   }
 
+  @Test
+  public void testScopeIncorrectOpenOrder() throws Exception {
+    metrics.startStoredScope(scopeName);
+    LegacyMetrics.LegacyMetricsScope fooScope = (LegacyMetrics.LegacyMetricsScope) metrics.getStoredScope(scopeName);
+    assertEquals(null, fooScope.getNumCounter());
+    fooScope.close();
+    assertEquals(Long.valueOf(1), fooScope.getNumCounter());
+
+    for (int i=0; i<10; i++) {
+      fooScope.open();
+      fooScope.close();
+    }
+    // scope opened/closed 10 times
+    assertEquals(Long.valueOf(11), fooScope.getNumCounter());
+
+    for (int i=0; i<10; i++) {
+      fooScope.open();
+    }
+    for (int i=0; i<10; i++) {
+      fooScope.close();
+    }
+    // scope opened/closed once (multiple opens do not count)
+    assertEquals(Long.valueOf(12), fooScope.getNumCounter());
+  }
+
   void testScopeImpl(int n) throws Exception {
     metrics.startStoredScope(scopeName);
     final LegacyMetrics.LegacyMetricsScope fooScope = (LegacyMetrics.LegacyMetricsScope) metrics.getStoredScope(scopeName);
-      // cannot open scope that is already open:
-    expectIOE(new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        fooScope.open();
-        return null;
-      }
-    });
 
     assertSame(fooScope, metrics.getStoredScope(scopeName));
     Thread.sleep(periodMs+ 1);
@@ -250,14 +224,6 @@ public class TestLegacyMetrics {
     final long t1 = fooScope.getTimeCounter().longValue();
     assertTrue(t1 > periodMs);
 
-    expectIOE(new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        metrics.endStoredScope(scopeName); // closing of closed scope not allowed
-        return null;
-      }
-    });
-
     assertSame(fooScope, metrics.getStoredScope(scopeName));
 
    // opening allowed after closing:
@@ -266,15 +232,6 @@ public class TestLegacyMetrics {
     assertTrue(fooScope.getNumCounter().longValue() >= 1);
     assertTrue(fooScope.getTimeCounter().longValue() >= t1);
 
-   // opening of already open scope not allowed:
-    expectIOE(new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        metrics.startStoredScope(scopeName);
-        return null;
-      }
-    });
-
     assertSame(fooScope, metrics.getStoredScope(scopeName));
     Thread.sleep(periodMs + 1);
     // Reopening (close + open) allowed in opened state:

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/metastore/src/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
index 6830cf7..98288a0 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
@@ -30,8 +30,6 @@ import org.apache.hadoop.hive.metastore.events.DropTableEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-
 /**
  * Report metrics of metadata added, deleted by this Hive Metastore.
  */
@@ -47,67 +45,37 @@ public class HMSMetricsListener extends MetaStoreEventListener {
 
   @Override
   public void onCreateDatabase(CreateDatabaseEvent dbEvent) throws MetaException {
-    if (metrics != null) {
-      try {
-        metrics.incrementCounter(MetricsConstant.CREATE_TOTAL_DATABASES);
-      } catch (IOException e) {
-        LOGGER.warn("Error updating metadata metrics", e);
-      }
-    }
+    incrementCounterInternal(MetricsConstant.CREATE_TOTAL_DATABASES);
   }
 
   @Override
   public void onDropDatabase(DropDatabaseEvent dbEvent) throws MetaException {
-    if (metrics != null) {
-      try {
-        metrics.incrementCounter(MetricsConstant.DELETE_TOTAL_DATABASES);
-      } catch (IOException e) {
-        LOGGER.warn("Error updating metadata metrics", e);
-      }
-    }
+    incrementCounterInternal(MetricsConstant.DELETE_TOTAL_DATABASES);
   }
 
   @Override
   public void onCreateTable(CreateTableEvent tableEvent) throws MetaException {
-    if (metrics != null) {
-      try {
-        metrics.incrementCounter(MetricsConstant.CREATE_TOTAL_TABLES);
-      } catch (IOException e) {
-        LOGGER.warn("Error updating metadata metrics", e);
-      }
-    }
+    incrementCounterInternal(MetricsConstant.CREATE_TOTAL_TABLES);
   }
 
   @Override
   public void onDropTable(DropTableEvent tableEvent) throws MetaException {
-    if (metrics != null) {
-      try {
-        metrics.incrementCounter(MetricsConstant.DELETE_TOTAL_TABLES);
-      } catch (IOException e) {
-        LOGGER.warn("Error updating metadata metrics", e);
-      }
-    }
+    incrementCounterInternal(MetricsConstant.DELETE_TOTAL_TABLES);
   }
 
   @Override
   public void onDropPartition(DropPartitionEvent partitionEvent) throws MetaException {
-    if (metrics != null) {
-      try {
-        metrics.incrementCounter(MetricsConstant.DELETE_TOTAL_PARTITIONS);
-      } catch (IOException e) {
-        LOGGER.warn("Error updating metadata metrics", e);
-      }
-    }
+    incrementCounterInternal(MetricsConstant.DELETE_TOTAL_PARTITIONS);
   }
 
   @Override
   public void onAddPartition(AddPartitionEvent partitionEvent) throws MetaException {
+    incrementCounterInternal(MetricsConstant.CREATE_TOTAL_PARTITIONS);
+  }
+
+  private void incrementCounterInternal(String name) {
     if (metrics != null) {
-      try {
-        metrics.incrementCounter(MetricsConstant.CREATE_TOTAL_PARTITIONS);
-      } catch (IOException e) {
-        LOGGER.warn("Error updating metadata metrics", e);
-      }
+      metrics.incrementCounter(name);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 71175df..c4d03eb 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -761,12 +761,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       logInfo((getThreadLocalIpAddress() == null ? "" : "source:" + getThreadLocalIpAddress() + " ") +
           function + extraLogInfo);
       if (MetricsFactory.getInstance() != null) {
-        try {
-          MetricsFactory.getInstance().startStoredScope(function);
-        } catch (IOException e) {
-          LOG.debug("Exception when starting metrics scope"
-            + e.getClass().getName() + " " + e.getMessage(), e);
-        }
+        MetricsFactory.getInstance().startStoredScope(function);
       }
       return function;
     }
@@ -805,11 +800,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     private void endFunction(String function, MetaStoreEndFunctionContext context) {
       if (MetricsFactory.getInstance() != null) {
-        try {
-          MetricsFactory.getInstance().endStoredScope(function);
-        } catch (IOException e) {
-          LOG.debug("Exception when closing metrics scope" + e);
-        }
+        MetricsFactory.getInstance().endStoredScope(function);
       }
 
       for (MetaStoreEndFunctionListener listener : endFunctionListeners) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 03c56e1..dd55434 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -1210,12 +1210,7 @@ public class Driver implements CommandProcessor {
 
     Metrics metrics = MetricsFactory.getInstance();
     if (metrics != null) {
-      try {
-        metrics.incrementCounter(MetricsConstant.WAITING_COMPILE_OPS, 1);
-      } catch (IOException e) {
-        // This won't happen if we are using the newer CodaHale metrics. Same for below.
-        LOG.warn("Error while incrementing metrics counter.", e);
-      }
+      metrics.incrementCounter(MetricsConstant.WAITING_COMPILE_OPS, 1);
     }
 
     final ReentrantLock compileLock = tryAcquireCompileLock(isParallelEnabled,
@@ -1226,11 +1221,7 @@ public class Driver implements CommandProcessor {
 
     try {
       if (metrics != null) {
-        try {
-          metrics.decrementCounter(MetricsConstant.WAITING_COMPILE_OPS, 1);
-        } catch (IOException e) {
-          LOG.warn("Error while decrementing metrics counter.", e);
-        }
+        metrics.decrementCounter(MetricsConstant.WAITING_COMPILE_OPS, 1);
       }
       ret = compile(command);
     } finally {

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
index f48d511..55bab6c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
@@ -375,11 +375,7 @@ public class MapRedTask extends ExecDriver implements Serializable {
 
   @Override
   public void updateTaskMetrics(Metrics metrics) {
-    try {
-      metrics.incrementCounter(MetricsConstant.HIVE_MR_TASKS);
-    } catch (IOException ex) {
-      LOG.warn("Could not increment metrics for " + this, ex);
-    }
+    metrics.incrementCounter(MetricsConstant.HIVE_MR_TASKS);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
index f81fc71..c9ff191 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
@@ -126,11 +126,7 @@ public class MapredLocalTask extends Task<MapredLocalWork> implements Serializab
 
   @Override
   public void updateTaskMetrics(Metrics metrics) {
-    try {
-      metrics.incrementCounter(MetricsConstant.HIVE_MR_TASKS);
-    } catch (IOException ex) {
-      LOG.warn("Could not increment metrics for " + this, ex);
-    }
+    metrics.incrementCounter(MetricsConstant.HIVE_MR_TASKS);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
index 72c8bf7..6597a51 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
@@ -183,11 +183,7 @@ public class SparkTask extends Task<SparkWork> {
 
   @Override
   public void updateTaskMetrics(Metrics metrics) {
-    try {
-      metrics.incrementCounter(MetricsConstant.HIVE_SPARK_TASKS);
-    } catch (IOException ex) {
-      LOG.warn("Could not increment metrics for " + this, ex);
-    }
+    metrics.incrementCounter(MetricsConstant.HIVE_SPARK_TASKS);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
index c51c92f..0efca68 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
@@ -532,11 +532,7 @@ public class TezTask extends Task<TezWork> {
 
   @Override
   public void updateTaskMetrics(Metrics metrics) {
-    try {
-      metrics.incrementCounter(MetricsConstant.HIVE_TEZ_TASKS);
-    } catch (IOException ex) {
-      LOG.warn("Could not increment metrics for " + this, ex);
-    }
+    metrics.incrementCounter(MetricsConstant.HIVE_TEZ_TASKS);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/f903c4af/service/src/java/org/apache/hive/service/cli/operation/Operation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/Operation.java b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
index 6a656f9..36c6f93 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/Operation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
@@ -428,19 +428,15 @@ public abstract class Operation {
       String completedOperationPrefix, OperationState state) {
     Metrics metrics = MetricsFactory.getInstance();
     if (metrics != null) {
-      try {
-        if (stateScope != null) {
-          metrics.endScope(stateScope);
-          stateScope = null;
-        }
-        if (scopeStates.contains(state)) {
-          stateScope = metrics.createScope(operationPrefix + state);
-        }
-        if (terminalStates.contains(state)) {
-          metrics.incrementCounter(completedOperationPrefix + state);
-        }
-      } catch (IOException e) {
-        LOG.warn("Error metrics", e);
+      if (stateScope != null) {
+        metrics.endScope(stateScope);
+        stateScope = null;
+      }
+      if (scopeStates.contains(state)) {
+        stateScope = metrics.createScope(operationPrefix + state);
+      }
+      if (terminalStates.contains(state)) {
+        metrics.incrementCounter(completedOperationPrefix + state);
       }
     }
     return stateScope;


[12/44] hive git commit: HIVE-14766 : ObjectStore.initialize() needs retry mechanisms in case of connection failures (Sushanth Sowmyan, reviewed by Thejas Nair)

Posted by se...@apache.org.
HIVE-14766 : ObjectStore.initialize() needs retry mechanisms in case of connection failures (Sushanth Sowmyan, reviewed by Thejas Nair)


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

Branch: refs/heads/hive-14535
Commit: c9224d58cce6e0b0520598894e962c48ce9d97e3
Parents: ed82cfa
Author: Sushanth Sowmyan <kh...@gmail.com>
Authored: Wed Sep 21 23:25:04 2016 -0700
Committer: Thejas Nair <th...@hortonworks.com>
Committed: Wed Sep 21 23:25:19 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/metastore/FakeDerby.java | 424 +++++++++++++++++++
 .../metastore/TestObjectStoreInitRetry.java     | 127 ++++++
 .../hadoop/hive/metastore/ObjectStore.java      |  71 ++++
 3 files changed, 622 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c9224d58/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java
new file mode 100644
index 0000000..51be504
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java
@@ -0,0 +1,424 @@
+/**
+ * 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.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import java.lang.Exception;
+import java.lang.Override;
+import java.lang.RuntimeException;
+import java.lang.StackTraceElement;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.CallableStatement;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.NClob;
+import java.sql.PreparedStatement;
+import java.sql.SQLClientInfoException;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Savepoint;
+import java.sql.Statement;
+import java.sql.Struct;
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.logging.Logger;
+import java.util.Properties;
+
+import javax.jdo.JDOCanRetryException;
+
+import junit.framework.TestCase;
+import org.junit.Test;
+
+import org.apache.derby.jdbc.EmbeddedDriver;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.ObjectStore;
+
+import org.apache.hadoop.hive.metastore.TestObjectStoreInitRetry;
+
+
+/**
+ * Fake derby driver - companion class to enable testing by TestObjectStoreInitRetry
+ */
+public class FakeDerby extends org.apache.derby.jdbc.EmbeddedDriver {
+
+  public class Connection implements java.sql.Connection {
+
+    private java.sql.Connection _baseConn;
+
+    public Connection(java.sql.Connection connection) {
+      TestObjectStoreInitRetry.debugTrace();
+      this._baseConn = connection;
+    }
+
+    @Override
+    public Statement createStatement() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createStatement();
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql);
+    }
+
+    @Override
+    public CallableStatement prepareCall(String sql) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareCall(sql);
+    }
+
+    @Override
+    public String nativeSQL(String sql) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.nativeSQL(sql);
+    }
+
+    @Override
+    public void setAutoCommit(boolean autoCommit) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      TestObjectStoreInitRetry.misbehave();
+      _baseConn.setAutoCommit(autoCommit);
+    }
+
+    @Override
+    public boolean getAutoCommit() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getAutoCommit();
+    }
+
+    @Override
+    public void commit() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.commit();
+    }
+
+    @Override
+    public void rollback() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.rollback();
+    }
+
+    @Override
+    public void close() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.close();
+    }
+
+    @Override
+    public boolean isClosed() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.isClosed();
+    }
+
+    @Override
+    public DatabaseMetaData getMetaData() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getMetaData();
+    }
+
+    @Override
+    public void setReadOnly(boolean readOnly) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setReadOnly(readOnly);
+    }
+
+    @Override
+    public boolean isReadOnly() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.isReadOnly();
+    }
+
+    @Override
+    public void setCatalog(String catalog) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setCatalog(catalog);
+    }
+
+    @Override
+    public String getCatalog() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getCatalog();
+    }
+
+    @Override
+    public void setTransactionIsolation(int level) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setTransactionIsolation(level);
+    }
+
+    @Override
+    public int getTransactionIsolation() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getTransactionIsolation();
+    }
+
+    @Override
+    public SQLWarning getWarnings() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getWarnings();
+    }
+
+    @Override
+    public void clearWarnings() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.clearWarnings();
+    }
+
+    @Override
+    public Statement createStatement(int resultSetType, int resultSetConcurrency) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createStatement(resultSetType, resultSetConcurrency);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql, resultSetType, resultSetConcurrency);
+    }
+
+    @Override
+    public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareCall(sql, resultSetType, resultSetConcurrency);
+    }
+
+    @Override
+    public Map<String, Class<?>> getTypeMap() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getTypeMap();
+    }
+
+    @Override
+    public void setTypeMap(Map<String, Class<?>> map) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setTypeMap(map);
+    }
+
+    @Override
+    public void setHoldability(int holdability) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setHoldability(holdability);
+    }
+
+    @Override
+    public int getHoldability() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getHoldability();
+    }
+
+    @Override
+    public Savepoint setSavepoint() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.setSavepoint();
+    }
+
+    @Override
+    public Savepoint setSavepoint(String name) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.setSavepoint(name);
+    }
+
+    @Override
+    public void rollback(Savepoint savepoint) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.rollback(savepoint);
+    }
+
+    @Override
+    public void releaseSavepoint(Savepoint savepoint) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.releaseSavepoint(savepoint);
+    }
+
+    @Override
+    public Statement createStatement(int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createStatement(resultSetType, resultSetConcurrency, resultSetHoldability);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql, resultSetType, resultSetConcurrency, resultSetHoldability);
+    }
+
+    @Override
+    public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareCall(sql, resultSetType, resultSetConcurrency, resultSetHoldability);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql, autoGeneratedKeys);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql, columnIndexes);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, String[] columnNames) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql, columnNames);
+    }
+
+    @Override
+    public Clob createClob() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createClob();
+    }
+
+    @Override
+    public Blob createBlob() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createBlob();
+    }
+
+    @Override
+    public NClob createNClob() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createNClob();
+    }
+
+    @Override
+    public SQLXML createSQLXML() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createSQLXML();
+    }
+
+    @Override
+    public boolean isValid(int timeout) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.isValid(timeout);
+    }
+
+    @Override
+    public void setClientInfo(String name, String value) throws SQLClientInfoException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setClientInfo(name, value);
+    }
+
+    @Override
+    public void setClientInfo(Properties properties) throws SQLClientInfoException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setClientInfo(properties);
+    }
+
+    @Override
+    public String getClientInfo(String name) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getClientInfo(name);
+    }
+
+    @Override
+    public Properties getClientInfo() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getClientInfo();
+    }
+
+    @Override
+    public Array createArrayOf(String typeName, Object[] elements) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createArrayOf(typeName, elements);
+    }
+
+    @Override
+    public Struct createStruct(String typeName, Object[] attributes) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createStruct(typeName, attributes);
+    }
+
+    @Override
+    public void setSchema(String schema) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setSchema(schema);
+    }
+
+    @Override
+    public String getSchema() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getSchema();
+    }
+
+    @Override
+    public void abort(Executor executor) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.abort(executor);
+    }
+
+    @Override
+    public void setNetworkTimeout(Executor executor, int milliseconds) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setNetworkTimeout(executor, milliseconds);
+    }
+
+    @Override
+    public int getNetworkTimeout() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getNetworkTimeout();
+    }
+
+    @Override
+    public <T> T unwrap(Class<T> iface) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.unwrap(iface);
+    }
+
+    @Override
+    public boolean isWrapperFor(Class<?> iface) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.isWrapperFor(iface);
+    }
+  }
+
+  public FakeDerby(){
+  }
+
+  @Override
+  public boolean acceptsURL(String url) throws SQLException {
+    url = url.replace("fderby","derby");
+    return super.acceptsURL(url);
+  }
+
+  @Override
+  public Connection connect(java.lang.String url, java.util.Properties info) throws SQLException {
+    TestObjectStoreInitRetry.misbehave();
+    url = url.replace("fderby","derby");
+    return new FakeDerby.Connection(super.connect(url, info));
+  }
+
+  @Override
+  public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+    throw new SQLFeatureNotSupportedException(); // hope this is respected properly
+  }
+
+
+};

http://git-wip-us.apache.org/repos/asf/hive/blob/c9224d58/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java
new file mode 100644
index 0000000..1695bfd
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java
@@ -0,0 +1,127 @@
+/**
+ * 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.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+
+import javax.jdo.JDOCanRetryException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+public class TestObjectStoreInitRetry {
+
+  private static boolean noisy = true; // switch to true to see line number debug traces for FakeDerby calls
+
+  private static int injectConnectFailure = 0;
+
+  public static void setInjectConnectFailure(int x){
+    injectConnectFailure = x;
+  }
+
+  public static int getInjectConnectFailure(){
+    return injectConnectFailure;
+  }
+
+  public static void decrementInjectConnectFailure(){
+    injectConnectFailure--;
+  }
+
+  @BeforeClass
+  public static void oneTimeSetup() throws SQLException {
+    // dummy instantiation to make sure any static/ctor code blocks of that
+    // driver are loaded and ready to go.
+    DriverManager.registerDriver(new FakeDerby());
+  }
+
+  @AfterClass
+  public static void oneTimeTearDown() throws SQLException {
+    DriverManager.deregisterDriver(new FakeDerby());
+  }
+
+  public static void misbehave() throws RuntimeException{
+    TestObjectStoreInitRetry.debugTrace();
+    if (TestObjectStoreInitRetry.getInjectConnectFailure() > 0){
+      TestObjectStoreInitRetry.decrementInjectConnectFailure();
+      RuntimeException re = new JDOCanRetryException();
+      if (noisy){
+        System.err.println("MISBEHAVE:" + TestObjectStoreInitRetry.getInjectConnectFailure());
+        re.printStackTrace(System.err);
+      }
+      throw re;
+    }
+  }
+
+  // debug instrumenter - useful in finding which fns get called, and how often
+  public static void debugTrace() {
+    if (noisy){
+      Exception e = new Exception();
+      System.err.println("." + e.getStackTrace()[1].getLineNumber() + ":" + TestObjectStoreInitRetry.getInjectConnectFailure());
+    }
+  }
+
+  protected static HiveConf hiveConf;
+
+  @Test
+  public void testObjStoreRetry() throws Exception {
+    hiveConf = new HiveConf(this.getClass());
+
+    hiveConf.setIntVar(ConfVars.HMSHANDLERATTEMPTS, 4);
+    hiveConf.setVar(ConfVars.HMSHANDLERINTERVAL, "1s");
+    hiveConf.setVar(ConfVars.METASTORE_CONNECTION_DRIVER,FakeDerby.class.getName());
+    hiveConf.setBoolVar(ConfVars.METASTORE_TRY_DIRECT_SQL,true);
+    String jdbcUrl = hiveConf.get(ConfVars.METASTORECONNECTURLKEY.varname);
+    jdbcUrl = jdbcUrl.replace("derby","fderby");
+    hiveConf.setVar(ConfVars.METASTORECONNECTURLKEY,jdbcUrl);
+
+    ObjectStore objStore = new ObjectStore();
+
+    Exception savE = null;
+    try {
+      setInjectConnectFailure(5);
+      objStore.setConf(hiveConf);
+    } catch (Exception e) {
+      e.printStackTrace(System.err);
+      savE = e;
+    }
+
+    /**
+     * A note on retries.
+     *
+     * We've configured a total of 4 attempts.
+     * 5 - 4 == 1 connect failure simulation count left after this.
+     */
+
+    assertEquals(1, getInjectConnectFailure());
+    assertNotNull(savE);
+
+    setInjectConnectFailure(0);
+    objStore.setConf(hiveConf);
+    assertEquals(0, getInjectConnectFailure());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c9224d58/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 83a3e39..74bde3d 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -26,6 +26,7 @@ import java.net.InetAddress;
 import java.net.URI;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -43,6 +44,7 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.regex.Pattern;
 
+import javax.jdo.JDOCanRetryException;
 import javax.jdo.JDODataStoreException;
 import javax.jdo.JDOException;
 import javax.jdo.JDOHelper;
@@ -323,6 +325,75 @@ public class ObjectStore implements RawStore, Configurable {
 
   @SuppressWarnings("nls")
   private void initialize(Properties dsProps) {
+    int retryLimit = HiveConf.getIntVar(hiveConf,
+        HiveConf.ConfVars.HMSHANDLERATTEMPTS);
+    long retryInterval = HiveConf.getTimeVar(hiveConf,
+        HiveConf.ConfVars.HMSHANDLERINTERVAL, TimeUnit.MILLISECONDS);
+    int numTries = retryLimit;
+
+    while (numTries > 0){
+      try {
+        initializeHelper(dsProps);
+        return; // If we reach here, we succeed.
+      } catch (Exception e){
+        numTries--;
+        boolean retriable = isRetriableException(e);
+        if ((numTries > 0) && retriable){
+          LOG.info("Retriable exception while instantiating ObjectStore, retrying. "
+              + numTries + " tries left", e);
+          try {
+            Thread.sleep(retryInterval);
+          } catch (InterruptedException ie) {
+            // Restore the interrupted status, since we do not want to catch it.
+            LOG.debug("Interrupted while sleeping before retrying.",ie);
+            Thread.currentThread().interrupt();
+          }
+          // If we're here, we'll proceed down the next while loop iteration.
+        } else {
+          // we've reached our limit, throw the last one.
+          if (retriable){
+            LOG.warn("Exception retry limit reached, not retrying any longer.",
+              e);
+          } else {
+            LOG.debug("Non-retriable exception during ObjectStore initialize.", e);
+          }
+          throw e;
+        }
+      }
+    }
+  }
+
+  private static final Set<Class<? extends Throwable>> retriableExceptionClasses =
+      new HashSet<Class<? extends Throwable>>(Arrays.asList(JDOCanRetryException.class));
+  /**
+   * Helper function for initialize to determine if we should retry an exception.
+   * We return true if the exception is of a known type of retriable exceptions, or if one
+   * of its recursive .getCause returns a known type of retriable exception.
+   */
+  private boolean isRetriableException(Throwable e) {
+    if (e == null){
+      return false;
+    }
+    if (retriableExceptionClasses.contains(e.getClass())){
+      return true;
+    }
+    for (Class<? extends Throwable> c : retriableExceptionClasses){
+      if (c.isInstance(e)){
+        return true;
+      }
+    }
+
+    if (e.getCause() == null){
+      return false;
+    }
+    return isRetriableException(e.getCause());
+  }
+
+  /**
+   * private helper to do initialization routine, so we can retry if needed if it fails.
+   * @param dsProps
+   */
+  private void initializeHelper(Properties dsProps) {
     LOG.info("ObjectStore, initialize called");
     prop = dsProps;
     pm = getPersistenceManager();


[08/44] hive git commit: HIVE-14801 : improve TestPartitionNameWhitelistValidation stability (Thejas Nair, reviewed by Daniel Dai)

Posted by se...@apache.org.
HIVE-14801 : improve TestPartitionNameWhitelistValidation stability (Thejas Nair, reviewed by Daniel Dai)


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

Branch: refs/heads/hive-14535
Commit: 0c392b185d98b4fb380a33a535b5f528625a47e8
Parents: 96508d3
Author: Thejas Nair <th...@hortonworks.com>
Authored: Wed Sep 21 11:56:50 2016 -0700
Committer: Thejas Nair <th...@hortonworks.com>
Committed: Wed Sep 21 11:56:50 2016 -0700

----------------------------------------------------------------------
 .../TestPartitionNameWhitelistValidation.java   | 53 ++++++--------------
 1 file changed, 15 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0c392b18/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java
index e0a905a..e3e175b 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java
@@ -18,49 +18,38 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import static org.junit.Assert.*;
+
 import java.util.ArrayList;
 import java.util.List;
 
-import junit.framework.Assert;
-import junit.framework.TestCase;
-
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.shims.ShimLoader;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 // Validate the metastore client call validatePartitionNameCharacters to ensure it throws
 // an exception if partition fields contain Unicode characters or commas
 
-public class TestPartitionNameWhitelistValidation extends TestCase {
+public class TestPartitionNameWhitelistValidation {
 
   private static final String partitionValidationPattern = "[\\x20-\\x7E&&[^,]]*";
+  private static HiveConf hiveConf;
+  private static HiveMetaStoreClient msc;
 
-  private HiveConf hiveConf;
-  private HiveMetaStoreClient msc;
-  private Driver driver;
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
     System.setProperty(HiveConf.ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN.varname,
         partitionValidationPattern);
-    int port = MetaStoreUtils.findFreePort();
-    MetaStoreUtils.startMetaStore(port, ShimLoader.getHadoopThriftAuthBridge());
-    hiveConf = new HiveConf(this.getClass());
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-    hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
+    hiveConf = new HiveConf();
     SessionState.start(new CliSessionState(hiveConf));
     msc = new HiveMetaStoreClient(hiveConf);
-    driver = new Driver(hiveConf);
   }
 
   // Runs an instance of DisallowUnicodePreEventListener
   // Returns whether or not it succeeded
   private boolean runValidation(List<String> partVals) {
-
     try {
       msc.validatePartitionNameCharacters(partVals);
     } catch (Exception e) {
@@ -72,74 +61,62 @@ public class TestPartitionNameWhitelistValidation extends TestCase {
 
   // Sample data
   private List<String> getPartValsWithUnicode() {
-
     List<String> partVals = new ArrayList<String>();
     partVals.add("kl�wen");
     partVals.add("t�gel�ch");
 
     return partVals;
-
   }
 
   private List<String> getPartValsWithCommas() {
-
     List<String> partVals = new ArrayList<String>();
     partVals.add("a,b");
     partVals.add("c,d,e,f");
 
     return partVals;
-
   }
 
   private List<String> getPartValsWithValidCharacters() {
-
     List<String> partVals = new ArrayList<String>();
     partVals.add("part1");
     partVals.add("part2");
 
     return partVals;
-
   }
 
   @Test
   public void testAddPartitionWithCommas() {
-
-    Assert.assertFalse("Add a partition with commas in name",
+    assertFalse("Add a partition with commas in name",
         runValidation(getPartValsWithCommas()));
   }
 
   @Test
   public void testAddPartitionWithUnicode() {
-
-    Assert.assertFalse("Add a partition with unicode characters in name",
+    assertFalse("Add a partition with unicode characters in name",
         runValidation(getPartValsWithUnicode()));
   }
 
   @Test
   public void testAddPartitionWithValidPartVal() {
-
-    Assert.assertTrue("Add a partition with unicode characters in name",
+    assertTrue("Add a partition with unicode characters in name",
         runValidation(getPartValsWithValidCharacters()));
   }
 
   @Test
   public void testAppendPartitionWithUnicode() {
-
-    Assert.assertFalse("Append a partition with unicode characters in name",
+    assertFalse("Append a partition with unicode characters in name",
         runValidation(getPartValsWithUnicode()));
   }
 
   @Test
   public void testAppendPartitionWithCommas() {
-
-    Assert.assertFalse("Append a partition with unicode characters in name",
+    assertFalse("Append a partition with unicode characters in name",
         runValidation(getPartValsWithCommas()));
   }
 
   @Test
   public void testAppendPartitionWithValidCharacters() {
-
-    Assert.assertTrue("Append a partition with no unicode characters in name",
+    assertTrue("Append a partition with no unicode characters in name",
         runValidation(getPartValsWithValidCharacters()));
   }
 


[06/44] hive git commit: HIVE-14793. Allow ptest branch to be specified, PROFILE override. (Siddharth Seth, reviewed by Sergio Peña)

Posted by se...@apache.org.
HIVE-14793. Allow ptest branch to be specified, PROFILE override. (Siddharth Seth, reviewed by Sergio Pe�a)


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

Branch: refs/heads/hive-14535
Commit: 62c45de1c12b2f8db6d726462cc93382aedc0905
Parents: 1977402
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Sep 20 15:37:49 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Sep 20 15:37:49 2016 -0700

----------------------------------------------------------------------
 dev-support/jenkins-execute-build.sh | 28 +++++++++++++++++++++-------
 1 file changed, 21 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/62c45de1/dev-support/jenkins-execute-build.sh
----------------------------------------------------------------------
diff --git a/dev-support/jenkins-execute-build.sh b/dev-support/jenkins-execute-build.sh
index b2ba8e9..2142942 100644
--- a/dev-support/jenkins-execute-build.sh
+++ b/dev-support/jenkins-execute-build.sh
@@ -26,7 +26,16 @@ build_ptest_client() {
 	test -d $PTEST_BUILD_DIR || mkdir -p $PTEST_BUILD_DIR
 	cd $PTEST_BUILD_DIR &&	rm -rf hive
 
-	git clone --depth 1 https://github.com/apache/hive.git
+  unset GIT_CLONE_ARGS
+  if [ -n "${PTEST_GIT_BRANCH}" ]; then
+    GIT_CLONE_ARGS=" -b ${PTEST_GIT_BRANCH}"
+  fi
+  if [ -z "${PTEST_GIT_REPO}" ]; then
+    PTEST_GIT_REPO=https://github.com/apache/hive.git
+  fi
+  GIT_CLONE_ARGS=${GIT_CLONE_ARGS}" ${PTEST_GIT_REPO} hive"
+
+	git clone --depth 1 ${GIT_CLONE_ARGS}
 	cd hive/testutils/ptest2
 	mvn clean package -DskipTests -Drat.numUnapprovedLicenses=1000 -Dmaven.repo.local=$MVN_REPO_LOCAL
 }
@@ -99,10 +108,13 @@ if [ -n "$JIRA_ISSUE" ]; then
 		fail "attachment $attachment_id is already tested for $JIRA_ISSUE"
 	fi
 
-	BUILD_PROFILE=`get_branch_profile $JIRA_PATCH_URL $JIRA_INFO_FILE`
-	if [ -z "$BUILD_PROFILE" ]; then
-		BUILD_PROFILE="$DEFAULT_BUILD_PROFILE"
-	fi
+  # Use the BUILD_PROFILE if it is provided. 
+  if [ -z ${BUILD_PROFILE} ]; then
+	  BUILD_PROFILE=`get_branch_profile $JIRA_PATCH_URL $JIRA_INFO_FILE`
+	  if [ -z "$BUILD_PROFILE" ]; then
+	  	BUILD_PROFILE="$DEFAULT_BUILD_PROFILE"
+	  fi
+  fi
 
 	if is_clear_cache_set $JIRA_INFO_FILE; then
 		optionalArgs+=(--clearLibraryCache)
@@ -112,8 +124,10 @@ if [ -n "$JIRA_ISSUE" ]; then
 
 	echo "ISSUE: $JIRA_ISSUE PROFILE: $BUILD_PROFILE"
 else
-	# If not JIRA is specified, then use a default profile
-	BUILD_PROFILE="$DEFAULT_BUILD_PROFILE"
+	# If not JIRA is specified, and no BUILD_PROFILE provided, then use a default profile
+  if [ -z ${BUILD_PROFILE} ]; then
+  	BUILD_PROFILE="$DEFAULT_BUILD_PROFILE"
+  fi
 
 	echo "ISSUE: unspecified PROFILE: $BUILD_PROFILE"
 fi


[37/44] hive git commit: HIVE-14852. Change qtest logging to not redirect all logs to console. (Siddharth Seth, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-14852. Change qtest logging to not redirect all logs to console. (Siddharth Seth, 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/a6c60807
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a6c60807
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a6c60807

Branch: refs/heads/hive-14535
Commit: a6c60807eb1daccb940d9caaeb2f7cafa7643afe
Parents: 20304c0
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Sep 29 10:42:21 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Sep 29 10:43:01 2016 -0700

----------------------------------------------------------------------
 itests/qtest-spark/pom.xml | 2 ++
 itests/qtest/pom.xml       | 2 ++
 pom.xml                    | 3 +++
 3 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a6c60807/itests/qtest-spark/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest-spark/pom.xml b/itests/qtest-spark/pom.xml
index 1e6c3a2..240852e 100644
--- a/itests/qtest-spark/pom.xml
+++ b/itests/qtest-spark/pom.xml
@@ -30,6 +30,8 @@
 
   <properties>
     <hive.path.to.root>../..</hive.path.to.root>
+    <!-- Determines the log level of the console logger, hive.log is independent of this-->
+    <test.console.log.level>OFF</test.console.log.level>
 
     <!-- The following are to match the latest in spark project, overriding hive's versions -->
     <spark.jetty.version>8.1.14.v20131031</spark.jetty.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/a6c60807/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index e762d0e..72028f3 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -30,6 +30,8 @@
 
   <properties>
     <hive.path.to.root>../..</hive.path.to.root>
+    <!-- Determines the log level of the console logger, hive.log is independent of this-->
+    <test.console.log.level>OFF</test.console.log.level>
 <!--    <initScript></initScript>-->
     <qfile></qfile>
     <qfile_regex></qfile_regex>

http://git-wip-us.apache.org/repos/asf/hive/blob/a6c60807/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 756cc34..5d13344 100644
--- a/pom.xml
+++ b/pom.xml
@@ -78,6 +78,8 @@
     <test.log4j.scheme>file://</test.log4j.scheme>
     <test.tmp.dir>${project.build.directory}/tmp</test.tmp.dir>
     <test.tmp.dir.uri>file://${test.tmp.dir}</test.tmp.dir.uri>
+    <!-- Determines the log level of the console logger, hive.log is independent of this-->
+    <test.console.log.level>INFO</test.console.log.level>
     <test.warehouse.dir>${project.build.directory}/warehouse</test.warehouse.dir>
     <test.warehouse.scheme>pfile://</test.warehouse.scheme>
 
@@ -1028,6 +1030,7 @@
             <maven.local.repository>${maven.repo.local}</maven.local.repository>
             <mapred.job.tracker>local</mapred.job.tracker>
             <log4j.configurationFile>${test.log4j.scheme}${test.tmp.dir}/conf/hive-log4j2.properties</log4j.configurationFile>
+            <hive.test.console.log.level>${test.console.log.level}</hive.test.console.log.level>
             <log4j.debug>true</log4j.debug>
             <!-- don't diry up /tmp -->
             <java.io.tmpdir>${test.tmp.dir}</java.io.tmpdir>


[05/44] hive git commit: HIVE-14624 : LLAP: Use FQDN when submitting work to LLAP (Sergey Shelukhin, reviewed by Siddharth Seth)

Posted by se...@apache.org.
HIVE-14624 : LLAP: Use FQDN when submitting work to LLAP (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/19774029
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/19774029
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/19774029

Branch: refs/heads/hive-14535
Commit: 19774029c4c1d90982354c36840bb485d74faaf1
Parents: e297a15
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Sep 20 11:30:49 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Sep 20 11:30:59 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/conf/HiveConf.java    |  3 +++
 .../java/org/apache/hadoop/hive/llap/LlapUtil.java    | 12 ++++++++++++
 .../apache/hadoop/hive/llap/LlapBaseInputFormat.java  |  6 +++---
 .../hive/llap/tezplugins/LlapTaskCommunicator.java    | 14 ++++++++++----
 .../llap/tezplugins/TestLlapTaskCommunicator.java     |  5 +++++
 5 files changed, 33 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/19774029/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 301159e..ccdfca6 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -393,6 +393,7 @@ public class HiveConf extends Configuration {
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_CONTAINER_ID.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_VALIDATE_ACLS.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_LOGGER.varname);
+    llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_AM_USE_FQDN.varname);
   }
 
   /**
@@ -2909,6 +2910,8 @@ public class HiveConf extends Configuration {
       new TimeValidator(TimeUnit.MILLISECONDS),
       "Amount of time to wait on connection failures to the AM from an LLAP daemon before\n" +
       "considering the AM to be dead.", "llap.am.liveness.connection.timeout-millis"),
+    LLAP_DAEMON_AM_USE_FQDN("hive.llap.am.use.fqdn", false,
+        "Whether to use FQDN of the AM machine when submitting work to LLAP."),
     // Not used yet - since the Writable RPC engine does not support this policy.
     LLAP_DAEMON_AM_LIVENESS_CONNECTION_SLEEP_BETWEEN_RETRIES_MS(
       "hive.llap.am.liveness.connection.sleep.between.retries.ms", "2000ms",

http://git-wip-us.apache.org/repos/asf/hive/blob/19774029/llap-common/src/java/org/apache/hadoop/hive/llap/LlapUtil.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/LlapUtil.java b/llap-common/src/java/org/apache/hadoop/hive/llap/LlapUtil.java
index 0c04d9d..8352943 100644
--- a/llap-common/src/java/org/apache/hadoop/hive/llap/LlapUtil.java
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/LlapUtil.java
@@ -14,6 +14,8 @@
 package org.apache.hadoop.hive.llap;
 
 import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -25,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto.Builder;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
@@ -180,4 +183,13 @@ public class LlapUtil {
       return sb.toString();
     }
   }
+
+  public static String getAmHostNameFromAddress(InetSocketAddress address, Configuration conf) {
+    if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_DAEMON_AM_USE_FQDN)) {
+      return address.getHostName();
+    }
+    InetAddress ia = address.getAddress();
+    // getCanonicalHostName would either return FQDN, or an IP.
+    return (ia == null) ? address.getHostName() : ia.getCanonicalHostName();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/19774029/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 7dae4fc..288a8eb 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
@@ -160,7 +160,7 @@ public class LlapBaseInputFormat<V extends WritableComparable<?>>
     SubmitWorkRequestProto request = constructSubmitWorkRequestProto(
         submitWorkInfo, llapSplit.getSplitNum(), attemptNum, llapClient.getAddress(),
         submitWorkInfo.getToken(), llapSplit.getFragmentBytes(),
-        llapSplit.getFragmentBytesSignature());
+        llapSplit.getFragmentBytesSignature(), job);
     llapClient.submitWork(request, host, llapSubmitPort);
 
     Socket socket = new Socket(host, serviceInstance.getOutputFormatPort());
@@ -290,7 +290,7 @@ public class LlapBaseInputFormat<V extends WritableComparable<?>>
 
   private SubmitWorkRequestProto constructSubmitWorkRequestProto(SubmitWorkInfo submitWorkInfo,
       int taskNum, int attemptNum, InetSocketAddress address, Token<JobTokenIdentifier> token,
-      byte[] fragmentBytes, byte[] fragmentBytesSignature) throws IOException {
+      byte[] fragmentBytes, byte[] fragmentBytesSignature, JobConf job) throws IOException {
     ApplicationId appId = submitWorkInfo.getFakeAppId();
 
     // This works, assuming the executor is running within YARN.
@@ -325,7 +325,7 @@ public class LlapBaseInputFormat<V extends WritableComparable<?>>
     builder.setFragmentNumber(taskNum);
     builder.setAttemptNumber(attemptNum);
     builder.setContainerIdString(containerId.toString());
-    builder.setAmHost(address.getHostName());
+    builder.setAmHost(LlapUtil.getAmHostNameFromAddress(address, job));
     builder.setAmPort(address.getPort());
     builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
     builder.setFragmentRuntimeInfo(runtimeInfo.build());

http://git-wip-us.apache.org/repos/asf/hive/blob/19774029/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
index d1d2ad4..7dd778d 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.LlapNodeId;
+import org.apache.hadoop.hive.llap.LlapUtil;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteRequestProto;
@@ -108,6 +109,7 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
   private final LlapTaskUmbilicalProtocol umbilical;
   private final Token<LlapTokenIdentifier> token;
   private final String user;
+  private String amHost;
 
   // These two structures track the list of known nodes, and the list of nodes which are sending in keep-alive heartbeats.
   // Primarily for debugging purposes a.t.m, since there's some unexplained TASK_TIMEOUTS which are currently being observed.
@@ -218,9 +220,9 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
 
       server.start();
       this.address = NetUtils.getConnectAddress(server);
-      LOG.info(
-          "Started LlapUmbilical: " + umbilical.getClass().getName() + " at address: " + address +
-              " with numHandlers=" + numHandlers);
+      this.amHost = LlapUtil.getAmHostNameFromAddress(address, conf);
+      LOG.info("Started LlapUmbilical: " + umbilical.getClass().getName() + " at address: "
+          + address + " with numHandlers=" + numHandlers + " using the host name " + amHost);
     } catch (IOException e) {
       throw new TezUncheckedException(e);
     }
@@ -610,7 +612,7 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
     builder.setFragmentNumber(taskSpec.getTaskAttemptID().getTaskID().getId());
     builder.setAttemptNumber(taskSpec.getTaskAttemptID().getId());
     builder.setContainerIdString(containerId.toString());
-    builder.setAmHost(getAddress().getHostName());
+    builder.setAmHost(getAmHostString());
     builder.setAmPort(getAddress().getPort());
 
     Preconditions.checkState(currentQueryIdentifierProto.getDagIndex() ==
@@ -842,4 +844,8 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
         .setAppAttemptNumber(getContext().getApplicationAttemptId().getAttemptId())
         .build();
   }
+
+  public String getAmHostString() {
+    return amHost;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/19774029/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskCommunicator.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskCommunicator.java b/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskCommunicator.java
index 0f28f70..5efe7c6 100644
--- a/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskCommunicator.java
+++ b/llap-tez/src/test/org/apache/hadoop/hive/llap/tezplugins/TestLlapTaskCommunicator.java
@@ -417,6 +417,11 @@ public class TestLlapTaskCommunicator {
     public InetSocketAddress getAddress() {
       return InetSocketAddress.createUnresolved("localhost", 15001);
     }
+
+    @Override
+    public String getAmHostString() {
+      return "localhost";
+    }
   }
 
 }


[43/44] hive git commit: HIVE-14865 Fix comments after HIVE-14350 (Eugene Koifman, reviewed by Alan Gates)

Posted by se...@apache.org.
HIVE-14865 Fix comments after HIVE-14350 (Eugene Koifman, reviewed by Alan Gates)


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

Branch: refs/heads/hive-14535
Commit: 297b4433cd2fdfb84182668bf7b1c524e92c6593
Parents: f903c4a
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Fri Sep 30 15:10:23 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Fri Sep 30 15:10:23 2016 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java | 6 ------
 1 file changed, 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/297b4433/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index cda5f39..f1eba5d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@ -866,7 +866,6 @@ public class AcidUtils {
        * {@link txnList}.  Note that 'original' files are logically a base_Long.MIN_VALUE and thus
        * cannot have any data for an open txn.  We could check {@link deltas} has files to cover
        * [1,n] w/o gaps but this would almost never happen...*/
-      //todo: this should only care about 'open' tnxs (HIVE-14211)
       long[] exceptions = txnList.getInvalidTransactions();
       String minOpenTxn = exceptions != null && exceptions.length > 0 ?
         Long.toString(exceptions[0]) : "x";
@@ -910,11 +909,6 @@ public class AcidUtils {
    * files within the snapshot.
    */
   private static boolean isValidBase(long baseTxnId, ValidTxnList txnList) {
-    /*This implementation is suboptimal.  It considers open/aborted txns invalid while we are only
-    * concerned with 'open' ones.  (Compaction removes any data that belongs to aborted txns and
-    * reads skip anything that belongs to aborted txn, thus base_7 is still OK if the only exception
-    * is txn 5 which is aborted).  So this implementation can generate false positives. (HIVE-14211)
-    * */
     if(baseTxnId == Long.MIN_VALUE) {
       //such base is created by 1st compaction in case of non-acid to acid table conversion
       //By definition there are no open txns with id < 1.


[34/44] hive git commit: HIVE-14849: Support google-compute-engine provider on Hive ptest framework (Sergio Pena, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-14849: Support google-compute-engine provider on Hive ptest framework (Sergio Pena, 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/291f3d50
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/291f3d50
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/291f3d50

Branch: refs/heads/hive-14535
Commit: 291f3d503d5a8627f86ef5f7fdd7880d8da4760c
Parents: cf72a73
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed Sep 28 21:33:00 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed Sep 28 21:33:00 2016 -0500

----------------------------------------------------------------------
 .../ptest2/conf/cloudhost.properties.example    |  37 +++
 testutils/ptest2/pom.xml                        |   5 +
 .../execution/context/CloudComputeService.java  | 224 +++++++++++++++----
 .../context/CloudExecutionContextProvider.java  | 105 +++++++--
 4 files changed, 311 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/291f3d50/testutils/ptest2/conf/cloudhost.properties.example
----------------------------------------------------------------------
diff --git a/testutils/ptest2/conf/cloudhost.properties.example b/testutils/ptest2/conf/cloudhost.properties.example
new file mode 100644
index 0000000..c336052
--- /dev/null
+++ b/testutils/ptest2/conf/cloudhost.properties.example
@@ -0,0 +1,37 @@
+#
+# This is just an example of different cloudhost providers
+#
+
+# This context provides configurations for AWS EC2 and GCE (google compute engine)
+executionContextProvider = org.apache.hive.ptest.execution.context.CloudExecutionContextProvider$Builder
+
+# Option: GCE
+cloudProvider = google-compute-engine
+gceJsonFile = # GCE JSON KEY FILE
+instanceType = https://www.googleapis.com/compute/v1/projects/<PROJECT_ID>/zones/us-central1-a/machineTypes/n1-standard-8
+imageId = https://www.googleapis.com/compute/v1/projects/<PROJECT_ID>/global/images/hive-ptest-debian-8-20160927
+# keyPair = # UNUSED
+securityGroup = hive-ptest
+
+# Option: AWS
+cloudProvider = aws-ec2
+apiKey =    # AWS ACCESS KEY
+accessKey = # AWS SECRET ACCESS KEY
+instanceType = c3.2xlarge
+imageId = us-west-1/ami-1fa1445b
+keyPair = hive-ptest
+securityGroup = hive-ptest
+
+# Generic options
+workingDirectory = /data/hive-ptest
+profileDirectory = /usr/local/hiveptest/etc/public/
+privateKey = /home/hiveptest/.ssh/hive-ptest-user-key
+dataDir = /data/hive-ptest/data/
+numHosts = 12
+groupName = hive-ptest-slaves
+localDirs = /home/hiveptest/
+user = hiveptest
+numThreads = 2
+maxLogDirectoriesPerProfile = 30
+userMetadata.owner = # USER
+maxHostsPerCreateRequest = 12
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/291f3d50/testutils/ptest2/pom.xml
----------------------------------------------------------------------
diff --git a/testutils/ptest2/pom.xml b/testutils/ptest2/pom.xml
index cea29b6..97981fb 100644
--- a/testutils/ptest2/pom.xml
+++ b/testutils/ptest2/pom.xml
@@ -107,6 +107,11 @@ limitations under the License.
       <version>${jclouds.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.jclouds.labs</groupId>
+      <artifactId>google-compute-engine</artifactId>
+      <version>${jclouds.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.jclouds.driver</groupId>
       <artifactId>jclouds-sshj</artifactId>
       <version>${jclouds.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/291f3d50/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudComputeService.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudComputeService.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudComputeService.java
index 64ee68e..e26c5ca 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudComputeService.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudComputeService.java
@@ -18,11 +18,13 @@
  */
 package org.apache.hive.ptest.execution.context;
 
-import java.util.Collections;
-import java.util.Properties;
-import java.util.Map;
-import java.util.Set;
+import java.io.File;
+import java.io.IOException;
+import java.util.*;
 
+import com.google.common.base.Supplier;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.io.Files;
 import org.jclouds.Constants;
 import org.jclouds.ContextBuilder;
 import org.jclouds.aws.ec2.compute.AWSEC2TemplateOptions;
@@ -34,7 +36,12 @@ import org.jclouds.compute.domain.ComputeMetadata;
 import org.jclouds.compute.domain.NodeMetadata;
 import org.jclouds.compute.domain.NodeMetadata.Status;
 import org.jclouds.compute.domain.Template;
+import org.jclouds.compute.options.TemplateOptions;
+import org.jclouds.domain.Credentials;
+import org.jclouds.googlecloud.GoogleCredentialsFromJson;
+import org.jclouds.googlecomputeengine.compute.options.GoogleComputeEngineTemplateOptions;
 import org.jclouds.logging.log4j.config.Log4JLoggingModule;
+import org.jclouds.sshj.config.SshjSshClientModule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,55 +50,76 @@ import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 
+import static com.google.common.base.Charsets.UTF_8;
+
 public class CloudComputeService {
   private static final Logger LOG = LoggerFactory
       .getLogger(CloudComputeService.class);
+  private final CloudComputeConfig mConfig;
   private final ComputeServiceContext mComputeServiceContext;
   private final ComputeService mComputeService;
-  private final String mInstanceType;
-  private final String mGroupName;
+  private final Template mTemplate;
   private final String mGroupTag;
-  private final String mImageId;
-  private final String mkeyPair;
-  private final String mSecurityGroup;
-  private final Map<String, String> mUserMetadata;
-  /**
-   * JClouds requests on-demand instances when null
-   */
-  private final Float mMaxBid;
-  public CloudComputeService(String apiKey, String accessKey, String instanceType, String groupName,
-      String imageId, String keyPair, String securityGroup, Float maxBid, Map<String,String> userMetadata) {
-    mInstanceType = instanceType;
-    mGroupName = groupName;
-    mImageId = imageId;
-    mkeyPair = keyPair;
-    mSecurityGroup = securityGroup;
-    mMaxBid = maxBid;
-    mGroupTag = "group=" + mGroupName;
-    mUserMetadata = userMetadata;
+
+  public CloudComputeService(final CloudComputeConfig config) {
+    mConfig = config;
+
+    mComputeServiceContext = initComputeServiceContext(config.getmProvider(), config.getIdentity(), config.getCredential());
+    mComputeService = mComputeServiceContext.getComputeService();
+    mTemplate = mComputeService.templateBuilder().hardwareId(config.getInstanceType()).imageId(config.getImageId()).build();
+
+    TemplateOptions options = mTemplate.getOptions();
+
+    // Set generic options
+    options.blockOnPort(22, 60);
+    options.userMetadata(config.getUserMetaData());
+
+    // Set provider options
+    switch (config.getmProvider()) {
+      case AWS:
+        mGroupTag = String.format("group=%s", config.getGroupName());
+
+        options.as(AWSEC2TemplateOptions.class)
+                .keyPair(config.getKeyPairName())
+                .securityGroupIds(config.getSecurityGroup())
+                .spotPrice(config.getMaxBid())
+                .tags(Collections.singletonList(mGroupTag));
+        break;
+      case GCE:
+        mGroupTag = config.getGroupName();
+
+        options.as(GoogleComputeEngineTemplateOptions.class)
+                .tags(Arrays.asList(config.getGroupName(), config.getSecurityGroup())); // GCE firewall is set through instance tags
+        break;
+      default:
+        mGroupTag = "";
+    }
+  }
+
+  private ComputeServiceContext initComputeServiceContext(CloudComputeConfig.CloudComputeProvider provider, String identity, String credential) {
     Properties overrides = new Properties();
+
     overrides.put(ComputeServiceProperties.POLL_INITIAL_PERIOD, String.valueOf(60L * 1000L));
     overrides.put(ComputeServiceProperties.POLL_MAX_PERIOD, String.valueOf(600L * 1000L));
     overrides.put(Constants.PROPERTY_MAX_RETRIES, String.valueOf(60));
-    mComputeServiceContext = ContextBuilder.newBuilder("aws-ec2")
-        .credentials(apiKey, accessKey)
-        .modules(ImmutableSet.of(new Log4JLoggingModule()))
-        .overrides(overrides)
-        .buildView(ComputeServiceContext.class);
-    mComputeService = mComputeServiceContext.getComputeService();
+
+    return ContextBuilder.newBuilder(provider.getmJcloudsId())
+            .credentials(identity, credential)
+            .modules(ImmutableSet.of(
+                    new SshjSshClientModule(),
+                    new Log4JLoggingModule()
+            ))
+            .overrides(overrides)
+            .buildView(ComputeServiceContext.class);
   }
+
   public Set<NodeMetadata> createNodes(int count)
       throws RunNodesException {
     Set<NodeMetadata> result = Sets.newHashSet();
-    Template template = mComputeService.templateBuilder()
-        .hardwareId(mInstanceType).imageId(mImageId).build();
-    template.getOptions().as(AWSEC2TemplateOptions.class).keyPair(mkeyPair)
-    .securityGroupIds(mSecurityGroup).blockOnPort(22, 60)
-    .spotPrice(mMaxBid).tags(Collections.singletonList(mGroupTag))
-    .userMetadata(mUserMetadata);
-    result.addAll(mComputeService.createNodesInGroup(mGroupName, count, template));
+    result.addAll(mComputeService.createNodesInGroup(mConfig.getGroupName(), count, mTemplate));
     return result;
   }
+
   static Predicate<ComputeMetadata> createFilterPTestPredicate(final String groupName,
       final String groupTag) {
     return new Predicate<ComputeMetadata>() {
@@ -122,7 +150,7 @@ public class CloudComputeService {
   public Set<NodeMetadata> listRunningNodes(){
     Set<NodeMetadata> result = Sets.newHashSet();
     result.addAll(mComputeService.listNodesDetailsMatching(
-        createFilterPTestPredicate(mGroupName, mGroupTag)));
+        createFilterPTestPredicate(mConfig.getGroupName(), mGroupTag)));
     return result;
   }
   public void destroyNode(String nodeId) {
@@ -131,4 +159,124 @@ public class CloudComputeService {
   public void close() {
     mComputeServiceContext.close();
   }
+
+  public static class CloudComputeConfig {
+    public enum CloudComputeProvider {
+      AWS("aws-ec2"),
+      GCE("google-compute-engine");
+
+      private final String mJcloudsId;
+
+      CloudComputeProvider(String jcloudsId) {
+        mJcloudsId = jcloudsId;
+      }
+
+      public String getmJcloudsId() {
+        return mJcloudsId;
+      }
+    };
+
+    private final CloudComputeProvider mProvider;
+
+    private String mIdentity;
+    private String mCredential;
+    private String mInstanceType;
+    private String mImageId;
+    private String mGroupName;
+    private String mSecurityGroup;
+    private String mKeyPairName;
+    private Map<String, String> mUserMetaData;
+
+    /**
+     * JClouds requests on-demand instances when null
+     */
+    private Float mMaxBid;
+
+    public CloudComputeConfig(CloudComputeProvider provider) {
+      mProvider = provider;
+    }
+
+    public void setCredentials(String identity, String credential) {
+      mIdentity = identity;
+      mCredential = credential;
+    }
+
+    public void setInstanceType(String instanceType) {
+      mInstanceType = instanceType;
+    }
+
+    public void setImageId(String imageId) {
+      mImageId = imageId;
+    }
+
+    public void setGroupName(String groupName) {
+      mGroupName = groupName;
+    }
+
+    public void setSecurityGroup(String securityGroup) {
+      mSecurityGroup = securityGroup;
+    }
+
+    public void setMaxBid(Float maxBid) {
+      mMaxBid = maxBid;
+    }
+
+    public void setmKeyPairName(String keyPairName) {
+      mKeyPairName = keyPairName;
+    }
+
+    public void setUserMetaData(Map<String, String> userMetaData) {
+      mUserMetaData = userMetaData;
+    }
+
+    public CloudComputeProvider getmProvider() {
+      return mProvider;
+    }
+
+    public String getIdentity() {
+      return mIdentity;
+    }
+
+    public String getCredential() {
+      return mCredential;
+    }
+
+    public String getInstanceType() {
+      return mInstanceType;
+    }
+
+    public String getImageId() {
+      return mImageId;
+    }
+
+    public String getGroupName() {
+      return mGroupName;
+    }
+
+    public String getSecurityGroup() {
+      return mSecurityGroup;
+    }
+
+    public Float getMaxBid() {
+      return mMaxBid;
+    }
+
+    public String getKeyPairName() {
+      return mKeyPairName;
+    }
+
+    public Map<String, String> getUserMetaData() {
+      if (mUserMetaData == null) {
+        return ImmutableMap.of();
+      }
+
+      return mUserMetaData;
+    }
+  }
+
+  public static Credentials getCredentialsFromJsonKeyFile(String filename) throws IOException {
+    String fileContents = Files.toString(new File(filename), UTF_8);
+    Supplier<Credentials> credentialSupplier = new GoogleCredentialsFromJson(fileContents);
+    return credentialSupplier.get();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/291f3d50/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java
index 343efde..8b82497 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java
@@ -34,12 +34,14 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hive.ptest.execution.Constants;
 import org.apache.hive.ptest.execution.Dirs;
+import org.apache.hive.ptest.execution.LocalCommandFactory;
 import org.apache.hive.ptest.execution.conf.Context;
 import org.apache.hive.ptest.execution.conf.Host;
 import org.apache.hive.ptest.execution.ssh.SSHCommand;
 import org.apache.hive.ptest.execution.ssh.SSHCommandExecutor;
 import org.jclouds.compute.RunNodesException;
 import org.jclouds.compute.domain.NodeMetadata;
+import org.jclouds.domain.Credentials;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -55,16 +57,24 @@ import com.google.common.collect.Sets;
 public class CloudExecutionContextProvider implements ExecutionContextProvider {
   private static final Logger LOG = LoggerFactory
       .getLogger(CloudExecutionContextProvider.class);
-  public static final String DATA_DIR = "dataDir";
+  public static final String CLOUD_PROVIDER = "cloudProvider";
+
+  // GCE settings
+  public static final String GCE_JSON_CREDS_FILE = "gceJsonFile";
+  
+  // AWS settings
   public static final String API_KEY = "apiKey";
   public static final String ACCESS_KEY = "accessKey";
+  public static final String KEY_PAIR = "keyPair";
+  public static final String MAX_BID = "maxBid";
+  
+  // Generic settings
+  public static final String DATA_DIR = "dataDir";
   public static final String NUM_HOSTS = "numHosts";
   public static final String MAX_HOSTS_PER_CREATE_REQUEST = "maxHostsPerCreateRequest";
   public static final String GROUP_NAME = "groupName";
   public static final String IMAGE_ID = "imageId";
-  public static final String KEY_PAIR = "keyPair";
   public static final String SECURITY_GROUP = "securityGroup";
-  public static final String MAX_BID = "maxBid";
   public static final String SLAVE_LOCAL_DIRECTORIES = "localDirs";
   public static final String USERNAME = "user";
   public static final String INSTANCE_TYPE = "instanceType";
@@ -304,6 +314,7 @@ public class CloudExecutionContextProvider implements ExecutionContextProvider {
           LOG.error("Verify command still executing on a host after 10 minutes");
         }
       } catch (InterruptedException e) {
+        terminateInternal(result);
         throw new CreateHostsFailedException("Interrupted while trying to create hosts", e);
       } finally {
         if(!executorService.isShutdown()) {
@@ -418,29 +429,79 @@ public class CloudExecutionContextProvider implements ExecutionContextProvider {
       return create(context, workingDirectory);
     }
   }
+  
+  private static CloudComputeService createAwsService(final Context context) {
+    String apiKey = Preconditions.checkNotNull(context.getString(API_KEY), API_KEY + " is required");
+    String accessKey = Preconditions.checkNotNull(context.getString(ACCESS_KEY), ACCESS_KEY + " is required");
+    String imageId = Preconditions.checkNotNull(context.getString(IMAGE_ID), IMAGE_ID + " is required");
+    String keyPair = Preconditions.checkNotNull(context.getString(KEY_PAIR), KEY_PAIR + " is required");
+    String securityGroup = Preconditions.checkNotNull(context.getString(SECURITY_GROUP), SECURITY_GROUP + " is required");
+
+    Float maxBid = context.getFloat(MAX_BID);
+    Preconditions.checkArgument(maxBid == null || maxBid > 0, MAX_BID + " must be null or greater than zero");
+
+    String instanceType = context.getString(INSTANCE_TYPE, "c1.xlarge");
+    String groupName = context.getString(GROUP_NAME, "hive-ptest-slaves");
+
+    CloudComputeService.CloudComputeConfig config =
+            new CloudComputeService.CloudComputeConfig(CloudComputeService.CloudComputeConfig.CloudComputeProvider.AWS);
+
+    config.setCredentials(apiKey, accessKey);
+    config.setInstanceType(instanceType);
+    config.setGroupName(groupName);
+    config.setImageId(imageId);
+    config.setmKeyPairName(keyPair);
+    config.setSecurityGroup(securityGroup);
+    config.setMaxBid(maxBid);
+    config.setUserMetaData(context.getSubProperties(USER_METADATA + "."));
+
+    return new CloudComputeService(config);
+  }
+  
+  private static CloudComputeService createGceService(final Context context) throws IOException {
+    String gceJsonFile = Preconditions.checkNotNull(context.getString(GCE_JSON_CREDS_FILE), GCE_JSON_CREDS_FILE + " is required");
+    String imageId = Preconditions.checkNotNull(context.getString(IMAGE_ID), IMAGE_ID + " is required");
+    String securityGroup = Preconditions.checkNotNull(context.getString(SECURITY_GROUP), SECURITY_GROUP + " is required");
+    String instanceType = Preconditions.checkNotNull(context.getString(INSTANCE_TYPE, ""), INSTANCE_TYPE + " is required");
+
+    String groupName = context.getString(GROUP_NAME, "hive-ptest-slaves");
+
+    CloudComputeService.CloudComputeConfig config =
+            new CloudComputeService.CloudComputeConfig(CloudComputeService.CloudComputeConfig.CloudComputeProvider.GCE);
+
+    Credentials creds = 
+            CloudComputeService.getCredentialsFromJsonKeyFile(gceJsonFile);
+
+    config.setCredentials(creds.identity, creds.credential);
+    config.setInstanceType(instanceType);
+    config.setGroupName(groupName);
+    config.setImageId(imageId);
+    config.setSecurityGroup(securityGroup);
+    config.setUserMetaData(context.getSubProperties(USER_METADATA + "."));
+
+    return new CloudComputeService(config);
+  }
+
+  private static CloudComputeService createService(final Context context) throws IOException {
+    String cloudProvider = context.getString(CLOUD_PROVIDER, "aws-ec2");
+    
+    if (cloudProvider.equalsIgnoreCase("aws-ec2")) {
+      return createAwsService(context);
+    } else if (cloudProvider.equalsIgnoreCase("google-compute-engine")) {
+      return createGceService(context);
+    } else {
+      throw new IllegalArgumentException("Unknown cloud provider name: " + cloudProvider);
+    }
+  }
 
   private static CloudExecutionContextProvider create(Context context,
       String workingDirectory) throws IOException {
     String dataDir = Preconditions.checkNotNull(context.getString(DATA_DIR),
         DATA_DIR + " is required");
-    String apiKey = Preconditions.checkNotNull(context.getString(API_KEY),
-        API_KEY + " is required");
-    String accessKey = Preconditions.checkNotNull(
-        context.getString(ACCESS_KEY), ACCESS_KEY + " is required");
     int maxHostsPerCreateRequest = context.getInteger(MAX_HOSTS_PER_CREATE_REQUEST, 2);
     Integer numHosts = context.getInteger(NUM_HOSTS, 8);
     Preconditions.checkArgument(numHosts > 0, NUM_HOSTS
         + " must be greater than zero");
-    String groupName = context.getString(GROUP_NAME, "hive-ptest-slaves");
-    String imageId = Preconditions.checkNotNull(context.getString(IMAGE_ID),
-        IMAGE_ID + " is required");
-    String keyPair = Preconditions.checkNotNull(context.getString(KEY_PAIR),
-        KEY_PAIR + " is required");
-    String securityGroup = Preconditions.checkNotNull(
-        context.getString(SECURITY_GROUP), SECURITY_GROUP + " is required");
-    Float maxBid = context.getFloat(MAX_BID);
-    Preconditions.checkArgument(maxBid == null || maxBid > 0, MAX_BID
-        + " must be null or greater than zero");
     String privateKey = Preconditions.checkNotNull(
         context.getString(PRIVATE_KEY), PRIVATE_KEY + " is required");
     String user = context.getString(USERNAME, "hiveptest");
@@ -448,12 +509,12 @@ public class CloudExecutionContextProvider implements ExecutionContextProvider {
         .split(context.getString(SLAVE_LOCAL_DIRECTORIES, "/home/hiveptest/")),
         String.class);
     Integer numThreads = context.getInteger(NUM_THREADS, 3);
-    String instanceType = context.getString(INSTANCE_TYPE, "c1.xlarge");
-    CloudComputeService cloudComputeService = new CloudComputeService(apiKey, accessKey,
-        instanceType, groupName, imageId, keyPair, securityGroup, maxBid, context.getSubProperties(USER_METADATA + "."));
+    
+    CloudComputeService cloudComputeService = createService(context);
     CloudExecutionContextProvider service = new CloudExecutionContextProvider(
-        dataDir, numHosts, cloudComputeService, new SSHCommandExecutor(LOG), workingDirectory,
-        privateKey, user, localDirs, numThreads, 60, maxHostsPerCreateRequest);
+        dataDir, numHosts, cloudComputeService,
+        new SSHCommandExecutor(LOG, new LocalCommandFactory(LOG), "-o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no"),
+        workingDirectory, privateKey, user, localDirs, numThreads, 60, maxHostsPerCreateRequest);
     return service;
   }
 }


[15/44] hive git commit: HIVE-14751: Add support for date truncation (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-14751: Add support for date truncation (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: e532549f162c29e6f621c9cb0ba7588e5882e4bc
Parents: 3237bb2
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri Sep 23 10:19:55 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri Sep 23 10:19:55 2016 +0100

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   3 +
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |  30 ++-
 .../apache/hadoop/hive/ql/udf/UDFDateFloor.java |   8 +-
 .../ql/udf/TestUDFDateFormatGranularity.java    |  22 +-
 ql/src/test/queries/clientpositive/floor_time.q |  46 ++++
 .../results/clientpositive/floor_time.q.out     | 211 +++++++++++++++++++
 6 files changed, 306 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e532549f/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index 7ceb005..af659ad 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -302,7 +302,9 @@ KW_CONF: 'CONF';
 KW_VALUES: 'VALUES';
 KW_RELOAD: 'RELOAD';
 KW_YEAR: 'YEAR';
+KW_QUARTER: 'QUARTER';
 KW_MONTH: 'MONTH';
+KW_WEEK: 'WEEK';
 KW_DAY: 'DAY';
 KW_HOUR: 'HOUR';
 KW_MINUTE: 'MINUTE';
@@ -329,6 +331,7 @@ KW_RELY: 'RELY';
 KW_NORELY: 'NORELY';
 KW_KEY: 'KEY';
 KW_ABORT: 'ABORT';
+KW_FLOOR: 'FLOOR';
 
 // Operators
 // NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.

http://git-wip-us.apache.org/repos/asf/hive/blob/e532549f/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index 9ba1865..7842d50 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -241,6 +241,30 @@ whenExpression
     KW_END -> ^(TOK_FUNCTION KW_WHEN expression*)
     ;
 
+floorExpression
+    :
+    KW_FLOOR
+    LPAREN
+          expression
+          (KW_TO
+          (floorUnit=floorDateQualifiers))?
+    RPAREN
+    -> {floorUnit != null}? ^(TOK_FUNCTION $floorUnit expression)
+    -> ^(TOK_FUNCTION Identifier["floor"] expression)
+    ;
+
+floorDateQualifiers
+    :
+    KW_YEAR -> Identifier["floor_year"]
+    | KW_QUARTER -> Identifier["floor_quarter"]
+    | KW_MONTH -> Identifier["floor_month"]
+    | KW_WEEK -> Identifier["floor_week"]
+    | KW_DAY -> Identifier["floor_day"]
+    | KW_HOUR -> Identifier["floor_hour"]
+    | KW_MINUTE -> Identifier["floor_minute"]
+    | KW_SECOND -> Identifier["floor_second"]
+    ;
+
 constant
 @init { gParent.pushMsg("constant", state); }
 @after { gParent.popMsg(state); }
@@ -323,6 +347,7 @@ atomExpression
     (KW_NULL) => KW_NULL -> TOK_NULL
     | (constant) => constant
     | castExpression
+    | floorExpression
     | caseExpression
     | whenExpression
     | (functionName LPAREN) => function
@@ -566,6 +591,7 @@ sysFuncNames
     | KW_IF
     | KW_CASE
     | KW_WHEN
+    | KW_FLOOR
     | KW_TINYINT
     | KW_SMALLINT
     | KW_INT
@@ -654,13 +680,13 @@ nonReserved
     | KW_KEYS | KW_KEY_TYPE | KW_LAST | KW_LIMIT | KW_OFFSET | KW_LINES | KW_LOAD | KW_LOCATION | KW_LOCK | KW_LOCKS | KW_LOGICAL | KW_LONG
     | KW_MAPJOIN | KW_MATERIALIZED | KW_METADATA | KW_MINUS | KW_MINUTE | KW_MONTH | KW_MSCK | KW_NOSCAN | KW_NO_DROP | KW_NULLS | KW_OFFLINE
     | KW_OPTION | KW_OUTPUTDRIVER | KW_OUTPUTFORMAT | KW_OVERWRITE | KW_OWNER | KW_PARTITIONED | KW_PARTITIONS | KW_PLUS | KW_PRETTY
-    | KW_PRINCIPALS | KW_PROTECTION | KW_PURGE | KW_READ | KW_READONLY | KW_REBUILD | KW_RECORDREADER | KW_RECORDWRITER
+    | KW_PRINCIPALS | KW_PROTECTION | KW_PURGE | KW_QUARTER | KW_READ | KW_READONLY | KW_REBUILD | KW_RECORDREADER | KW_RECORDWRITER
     | KW_RELOAD | KW_RENAME | KW_REPAIR | KW_REPLACE | KW_REPLICATION | KW_RESTRICT | KW_REWRITE
     | KW_ROLE | KW_ROLES | KW_SCHEMA | KW_SCHEMAS | KW_SECOND | KW_SEMI | KW_SERDE | KW_SERDEPROPERTIES | KW_SERVER | KW_SETS | KW_SHARED
     | KW_SHOW | KW_SHOW_DATABASE | KW_SKEWED | KW_SORT | KW_SORTED | KW_SSL | KW_STATISTICS | KW_STORED
     | KW_STREAMTABLE | KW_STRING | KW_STRUCT | KW_TABLES | KW_TBLPROPERTIES | KW_TEMPORARY | KW_TERMINATED
     | KW_TINYINT | KW_TOUCH | KW_TRANSACTIONS | KW_UNARCHIVE | KW_UNDO | KW_UNIONTYPE | KW_UNLOCK | KW_UNSET
-    | KW_UNSIGNED | KW_URI | KW_USE | KW_UTC | KW_UTCTIMESTAMP | KW_VALUE_TYPE | KW_VIEW | KW_WHILE | KW_YEAR
+    | KW_UNSIGNED | KW_URI | KW_USE | KW_UTC | KW_UTCTIMESTAMP | KW_VALUE_TYPE | KW_VIEW | KW_WEEK | KW_WHILE | KW_YEAR
     | KW_WORK
     | KW_TRANSACTION
     | KW_WRITE

http://git-wip-us.apache.org/repos/asf/hive/blob/e532549f/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java
index 08ed9fd..126ed44 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java
@@ -40,7 +40,6 @@ import com.google.common.collect.ImmutableMap;
 public abstract class UDFDateFloor extends UDF {
 
   private final QueryGranularity granularity;
-
   private final TimestampWritable result;
 
   public UDFDateFloor(String granularity) {
@@ -52,7 +51,12 @@ public abstract class UDFDateFloor extends UDF {
     if (t == null) {
       return null;
     }
-    long newTimestamp = granularity.truncate(t.getTimestamp().getTime());
+    final long originalTimestamp = t.getTimestamp().getTime(); // default
+    final long originalTimestampUTC = new DateTime(originalTimestamp)
+        .withZoneRetainFields(DateTimeZone.UTC).getMillis(); // default -> utc
+    final long newTimestampUTC = granularity.truncate(originalTimestampUTC); // utc
+    final long newTimestamp = new DateTime(newTimestampUTC, DateTimeZone.UTC)
+        .withZoneRetainFields(DateTimeZone.getDefault()).getMillis(); // utc -> default
     result.setTime(newTimestamp);
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/e532549f/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
index f871de2..03b2d42 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
@@ -25,61 +25,63 @@ import org.junit.Test;
 import junit.framework.TestCase;
 
 public class TestUDFDateFormatGranularity extends TestCase {
-  
+
+  // Timestamp values are PST (timezone for tests is set to PST by default)
+
   @Test
   public void testTimestampToTimestampWithGranularity() throws Exception {
     // Running example
     // Friday 30th August 1985 02:47:02 AM
-    final TimestampWritable t = new TimestampWritable(new Timestamp(494218022082L));
+    final TimestampWritable t = new TimestampWritable(new Timestamp(494243222000L));
     UDFDateFloor g;
 
     // Year granularity
     // Tuesday 1st January 1985 12:00:00 AM
     g = new UDFDateFloorYear();
     TimestampWritable i1 = g.evaluate(t);
-    assertEquals(473385600000L, i1.getTimestamp().getTime());
+    assertEquals(473414400000L, i1.getTimestamp().getTime());
     
     // Quarter granularity
     // Monday 1st July 1985 12:00:00 AM
     g = new UDFDateFloorQuarter();
     TimestampWritable i2 = g.evaluate(t);
-    assertEquals(489024000000L, i2.getTimestamp().getTime());
+    assertEquals(489049200000L, i2.getTimestamp().getTime());
 
     // Month granularity
     // Thursday 1st August 1985 12:00:00 AM
     g = new UDFDateFloorMonth();
     TimestampWritable i3 = g.evaluate(t);
-    assertEquals(491702400000L, i3.getTimestamp().getTime());
+    assertEquals(491727600000L, i3.getTimestamp().getTime());
 
     // Week granularity
     // Monday 26th August 1985 12:00:00 AM
     g = new UDFDateFloorWeek();
     TimestampWritable i4 = g.evaluate(t);
-    assertEquals(493862400000L, i4.getTimestamp().getTime());
+    assertEquals(493887600000L, i4.getTimestamp().getTime());
 
     // Day granularity
     // Friday 30th August 1985 12:00:00 AM
     g = new UDFDateFloorDay();
     TimestampWritable i5 = g.evaluate(t);
-    assertEquals(494208000000L, i5.getTimestamp().getTime());
+    assertEquals(494233200000L, i5.getTimestamp().getTime());
 
     // Hour granularity
     // Friday 30th August 1985 02:00:00 AM
     g = new UDFDateFloorHour();
     TimestampWritable i6 = g.evaluate(t);
-    assertEquals(494215200000L, i6.getTimestamp().getTime());
+    assertEquals(494240400000L, i6.getTimestamp().getTime());
 
     // Minute granularity
     // Friday 30th August 1985 02:47:00 AM
     g = new UDFDateFloorMinute();
     TimestampWritable i7 = g.evaluate(t);
-    assertEquals(494218020000L, i7.getTimestamp().getTime());
+    assertEquals(494243220000L, i7.getTimestamp().getTime());
 
     // Second granularity
     // Friday 30th August 1985 02:47:02 AM
     g = new UDFDateFloorSecond();
     TimestampWritable i8 = g.evaluate(t);
-    assertEquals(494218022000L, i8.getTimestamp().getTime());
+    assertEquals(494243222000L, i8.getTimestamp().getTime());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/e532549f/ql/src/test/queries/clientpositive/floor_time.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/floor_time.q b/ql/src/test/queries/clientpositive/floor_time.q
new file mode 100644
index 0000000..b0f6f90
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/floor_time.q
@@ -0,0 +1,46 @@
+drop table extract_udf;
+
+create table extract_udf (t timestamp);
+from (select * from src tablesample (1 rows)) s
+  insert overwrite table extract_udf 
+    select '2011-05-06 07:08:09.1234567';
+
+select t
+from extract_udf;
+
+explain
+select floor_day(t)
+from extract_udf;
+
+select floor_day(t)
+from extract_udf;
+
+-- new syntax
+explain
+select floor(t to day)
+from extract_udf;
+
+select floor(t to day)
+from extract_udf;
+
+
+select floor(t to second)
+from extract_udf;
+
+select floor(t to minute)
+from extract_udf;
+
+select floor(t to hour)
+from extract_udf;
+
+select floor(t to week)
+from extract_udf;
+
+select floor(t to month)
+from extract_udf;
+
+select floor(t to quarter)
+from extract_udf;
+
+select floor(t to year)
+from extract_udf;

http://git-wip-us.apache.org/repos/asf/hive/blob/e532549f/ql/src/test/results/clientpositive/floor_time.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/floor_time.q.out b/ql/src/test/results/clientpositive/floor_time.q.out
new file mode 100644
index 0000000..f0cb324
--- /dev/null
+++ b/ql/src/test/results/clientpositive/floor_time.q.out
@@ -0,0 +1,211 @@
+PREHOOK: query: drop table extract_udf
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table extract_udf
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table extract_udf (t timestamp)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@extract_udf
+POSTHOOK: query: create table extract_udf (t timestamp)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@extract_udf
+PREHOOK: query: from (select * from src tablesample (1 rows)) s
+  insert overwrite table extract_udf 
+    select '2011-05-06 07:08:09.1234567'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@extract_udf
+POSTHOOK: query: from (select * from src tablesample (1 rows)) s
+  insert overwrite table extract_udf 
+    select '2011-05-06 07:08:09.1234567'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@extract_udf
+POSTHOOK: Lineage: extract_udf.t EXPRESSION []
+PREHOOK: query: select t
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select t
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011-05-06 07:08:09.1234567
+PREHOOK: query: explain
+select floor_day(t)
+from extract_udf
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select floor_day(t)
+from extract_udf
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: extract_udf
+            Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: floor_day(t) (type: timestamp)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select floor_day(t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select floor_day(t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011-05-06 00:00:00
+PREHOOK: query: -- new syntax
+explain
+select floor(t to day)
+from extract_udf
+PREHOOK: type: QUERY
+POSTHOOK: query: -- new syntax
+explain
+select floor(t to day)
+from extract_udf
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: extract_udf
+            Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: floor_day(t) (type: timestamp)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select floor(t to day)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select floor(t to day)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011-05-06 00:00:00
+PREHOOK: query: select floor(t to second)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select floor(t to second)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011-05-06 07:08:09
+PREHOOK: query: select floor(t to minute)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select floor(t to minute)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011-05-06 07:08:00
+PREHOOK: query: select floor(t to hour)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select floor(t to hour)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011-05-06 07:00:00
+PREHOOK: query: select floor(t to week)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select floor(t to week)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011-05-02 00:00:00
+PREHOOK: query: select floor(t to month)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select floor(t to month)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011-05-01 00:00:00
+PREHOOK: query: select floor(t to quarter)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select floor(t to quarter)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011-04-01 00:00:00
+PREHOOK: query: select floor(t to year)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select floor(t to year)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011-01-01 00:00:00


[24/44] hive git commit: HIVE-3173 Add tests for JDBC getTypeInfo method (Xiu Gao via gates)

Posted by se...@apache.org.
HIVE-3173 Add tests for JDBC getTypeInfo method (Xiu Gao via gates)


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

Branch: refs/heads/hive-14535
Commit: 19fd56137caa23fbe8ef1e452a11603fc14f4325
Parents: 4ce5fe1
Author: Alan Gates <ga...@hortonworks.com>
Authored: Mon Sep 26 10:47:48 2016 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Mon Sep 26 10:47:48 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   | 32 ++++++++++++++++++++
 1 file changed, 32 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/19fd5613/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
index fc91f9d..ff4d63f 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
@@ -1145,6 +1145,38 @@ public class TestJdbcDriver2 {
     assertFalse("Unexpected table", rs.next());
   }
 
+  @Test
+  public void testMetaDataGetTypeInfo() throws SQLException {
+    HiveBaseResultSet rs = (HiveBaseResultSet) con.getMetaData().getTypeInfo();
+    Set<String> typeInfos = new HashSet<String>();
+    typeInfos.add("BOOLEAN");
+    typeInfos.add("TINYINT");
+    typeInfos.add("SMALLINT");
+    typeInfos.add("INT");
+    typeInfos.add("BIGINT");
+    typeInfos.add("FLOAT");
+    typeInfos.add("DOUBLE");
+    typeInfos.add("STRING");
+    typeInfos.add("TIMESTAMP");
+    typeInfos.add("BINARY");
+    typeInfos.add("DECIMAL");
+    typeInfos.add("ARRAY");
+    typeInfos.add("MAP");
+    typeInfos.add("STRUCT");
+    typeInfos.add("UNIONTYPE");
+
+    int cnt = 0;
+    while (rs.next()) {
+      String typeInfo = rs.getString("TYPE_NAME");
+      assertEquals("Get by index different from get by name", rs.getString(1), typeInfo);
+      typeInfos.remove(typeInfo);
+      cnt++;
+    }
+    rs.close();
+    assertEquals("Incorrect typeInfo count.", 0, typeInfos.size());
+    assertTrue("Found less typeInfos than we test for.", cnt >= typeInfos.size());
+  }
+
   /**
    * Test the type returned for pre-created table type table and view type table
    * @param tableTypeNames expected table types


[22/44] hive git commit: HIVE-14818. Reduce number of retries while starting HiveServer for tests. (Siddharth Seth, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-14818. Reduce number of retries while starting HiveServer for tests. (Siddharth Seth, 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/a2131154
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a2131154
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a2131154

Branch: refs/heads/hive-14535
Commit: a21311544a4935785aa964e6f2c66b1669982e8a
Parents: e08d94e
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Sep 23 14:58:20 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Sep 23 14:58:20 2016 -0700

----------------------------------------------------------------------
 common/src/java/org/apache/hadoop/hive/conf/HiveConf.java    | 8 ++++++--
 .../src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java  | 4 ++++
 .../src/java/org/apache/hive/service/server/HiveServer2.java | 7 +++++--
 3 files changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a2131154/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 ccdfca6..43a16d7 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2172,9 +2172,13 @@ public class HiveConf extends Configuration {
         "SSL Versions to disable for all Hive Servers"),
 
      // HiveServer2 specific configs
+    HIVE_SERVER2_SLEEP_INTERVAL_BETWEEN_START_ATTEMPTS("hive.server2.sleep.interval.between.start.attempts",
+        "60s", new TimeValidator(TimeUnit.MILLISECONDS, 0l, true, Long.MAX_VALUE, true),
+        "Amount of time to sleep between HiveServer2 start attempts. Primarily meant for tests"),
     HIVE_SERVER2_MAX_START_ATTEMPTS("hive.server2.max.start.attempts", 30L, new RangeValidator(0L, null),
-        "Number of times HiveServer2 will attempt to start before exiting, sleeping 60 seconds " +
-        "between retries. \n The default of 30 will keep trying for 30 minutes."),
+        "Number of times HiveServer2 will attempt to start before exiting. The sleep interval between retries" +
+        " is determined by " + ConfVars.HIVE_SERVER2_SLEEP_INTERVAL_BETWEEN_START_ATTEMPTS.varname +
+        "\n The default of 30 will keep trying for 30 minutes."),
     HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY("hive.server2.support.dynamic.service.discovery", false,
         "Whether HiveServer2 supports dynamic service discovery for its clients. " +
         "To support this, each instance of HiveServer2 currently uses ZooKeeper to register itself, " +

http://git-wip-us.apache.org/repos/asf/hive/blob/a2131154/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
index abb80a2..176761f 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
@@ -22,6 +22,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -187,6 +188,9 @@ public class MiniHS2 extends AbstractHiveService {
     super(hiveConf, "localhost",
         (usePortsFromConf ? hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT) : MetaStoreUtils.findFreePort()),
         (usePortsFromConf ? hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT) : MetaStoreUtils.findFreePort()));
+    hiveConf.setLongVar(ConfVars.HIVE_SERVER2_MAX_START_ATTEMPTS, 3l);
+    hiveConf.setTimeVar(ConfVars.HIVE_SERVER2_SLEEP_INTERVAL_BETWEEN_START_ATTEMPTS, 10,
+        TimeUnit.SECONDS);
     this.miniClusterType = miniClusterType;
     this.useMiniKdc = useMiniKdc;
     this.serverPrincipal = serverPrincipal;

http://git-wip-us.apache.org/repos/asf/hive/blob/a2131154/service/src/java/org/apache/hive/service/server/HiveServer2.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index 58e9aeb..590b1f3 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -544,6 +544,9 @@ public class HiveServer2 extends CompositeService {
       LOG.info("Starting HiveServer2");
       HiveConf hiveConf = new HiveConf();
       maxAttempts = hiveConf.getLongVar(HiveConf.ConfVars.HIVE_SERVER2_MAX_START_ATTEMPTS);
+      long retrySleepIntervalMs = hiveConf
+          .getTimeVar(ConfVars.HIVE_SERVER2_SLEEP_INTERVAL_BETWEEN_START_ATTEMPTS,
+              TimeUnit.MILLISECONDS);
       HiveServer2 server = null;
       try {
         // Initialize the pool before we start the server; don't start yet.
@@ -589,9 +592,9 @@ public class HiveServer2 extends CompositeService {
           throw new Error("Max start attempts " + maxAttempts + " exhausted", throwable);
         } else {
           LOG.warn("Error starting HiveServer2 on attempt " + attempts
-              + ", will retry in 60 seconds", throwable);
+              + ", will retry in " + retrySleepIntervalMs + "ms", throwable);
           try {
-            Thread.sleep(60L * 1000L);
+            Thread.sleep(retrySleepIntervalMs);
           } catch (InterruptedException e) {
             Thread.currentThread().interrupt();
           }


[14/44] hive git commit: HIVE-14774: Canceling query using Ctrl-C in beeline might lead to stale locks (Chaoyu Tang, reviewed by Jimmy Xiang, Mohit Sabharwal)

Posted by se...@apache.org.
HIVE-14774: Canceling query using Ctrl-C in beeline might lead to stale locks (Chaoyu Tang, reviewed by Jimmy Xiang, Mohit Sabharwal)


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

Branch: refs/heads/hive-14535
Commit: 3237bb276b79d2a54d0a5d8a5ede400147017618
Parents: 1a3e4be
Author: ctang <ct...@cloudera.com>
Authored: Thu Sep 22 14:47:47 2016 -0400
Committer: ctang <ct...@cloudera.com>
Committed: Thu Sep 22 14:47:47 2016 -0400

----------------------------------------------------------------------
 .../hive/service/cli/operation/SQLOperation.java  | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3237bb27/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 6f2daf3..abdf8cd 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -399,6 +399,17 @@ public class SQLOperation extends ExecuteStatementOperation {
 
   private synchronized void cleanup(OperationState state) throws HiveSQLException {
     setState(state);
+
+    if (shouldRunAsync()) {
+      Future<?> backgroundHandle = getBackgroundHandle();
+      if (backgroundHandle != null) {
+        boolean success = backgroundHandle.cancel(true);
+        if (success) {
+          LOG.info("The running operation has been successfully interrupted.");
+        }
+      }
+    }
+
     if (driver != null) {
       driver.close();
       driver.destroy();
@@ -413,13 +424,6 @@ public class SQLOperation extends ExecuteStatementOperation {
       ss.deleteTmpErrOutputFile();
     }
 
-    if (shouldRunAsync()) {
-      Future<?> backgroundHandle = getBackgroundHandle();
-      if (backgroundHandle != null) {
-        backgroundHandle.cancel(true);
-      }
-    }
-
     // Shutdown the timeout thread if any, while closing this operation
     if ((timeoutExecutor != null) && (state != OperationState.TIMEDOUT) && (state.isTerminal())) {
       timeoutExecutor.shutdownNow();


[04/44] hive git commit: HIVE-14651. Add a local cluster for Tez and LLAP. (Siddharth Seth, reviewed by Prasanth Jayachandran, Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14651. Add a local cluster for Tez and LLAP. (Siddharth Seth,reviewed by Prasanth Jayachandran, Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: e297a157cfa57f0bd08843bf770856b2f168da75
Parents: 3c55115
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Sep 20 11:09:11 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Sep 20 11:09:11 2016 -0700

----------------------------------------------------------------------
 data/conf/llap/tez-site.xml                     |  12 +
 data/conf/tez/hive-site.xml                     |  10 -
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 281 ++++++++++++++-----
 .../tezplugins/LlapTaskSchedulerService.java    |   2 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |  73 +++--
 .../apache/hadoop/hive/shims/HadoopShims.java   |   2 +
 6 files changed, 274 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e297a157/data/conf/llap/tez-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/llap/tez-site.xml b/data/conf/llap/tez-site.xml
index 940f390..6f1b9d2 100644
--- a/data/conf/llap/tez-site.xml
+++ b/data/conf/llap/tez-site.xml
@@ -1,6 +1,18 @@
 <configuration>
+
   <property>
     <name>tez.am.dag.scheduler.class</name>
     <value>org.apache.tez.dag.app.dag.impl.DAGSchedulerNaturalOrderControlled</value>
   </property>
+
+  <!-- Fail fast during tests -->
+  <property>
+    <name>tez.am.task.max.failed.attempts</name>
+    <value>2</value>
+  </property>
+  <property>
+    <name>tez.runtime.shuffle.connect.timeout</name>
+    <value>20000</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/e297a157/data/conf/tez/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/tez/hive-site.xml b/data/conf/tez/hive-site.xml
index f3e4dae..dbff10c 100644
--- a/data/conf/tez/hive-site.xml
+++ b/data/conf/tez/hive-site.xml
@@ -269,16 +269,6 @@
 </property>
 
 <property>
-  <name>hive.metastore.fastpath</name>
-  <value>true</value>
-</property>
-
-<property>
-  <name>hive.metastore.rawstore.impl</name>
-  <value>org.apache.hadoop.hive.metastore.hbase.HBaseStore</value>
-</property>
-
-<property>
   <name>hive.orc.splits.ms.footer.cache.enabled</name>
   <value>true</value>
 </property>

http://git-wip-us.apache.org/repos/asf/hive/blob/e297a157/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index f456dfb..0dfd727 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -63,6 +63,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
@@ -91,6 +92,7 @@ import org.apache.hadoop.hive.llap.daemon.impl.LlapDaemon;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.hbase.HBaseStore;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -117,6 +119,7 @@ import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.util.Shell;
 import org.apache.hive.common.util.StreamPrinter;
+import org.apache.logging.log4j.util.Strings;
 import org.apache.tools.ant.BuildException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
@@ -147,8 +150,13 @@ public class QTestUtil {
   private final static String defaultCleanupScript = "q_test_cleanup.sql";
   private final String[] testOnlyCommands = new String[]{"crypto"};
 
+  private static final String TEST_TMP_DIR_PROPERTY = "test.tmp.dir"; // typically target/tmp
+  private static final String BUILD_DIR_PROPERTY = "build.dir"; // typically target
+
   private String testWarehouse;
   private final String testFiles;
+  private final boolean useLocalFs;
+  private final boolean localMode;
   protected final String outDir;
   protected final String logDir;
   private final TreeMap<String, String> qMap;
@@ -161,7 +169,7 @@ public class QTestUtil {
   private final Set<String> qJavaVersionSpecificOutput;
   private static final String SORT_SUFFIX = ".sorted";
   private final HashSet<String> srcTables;
-  private static MiniClusterType clusterType = MiniClusterType.none;
+  private final MiniClusterType clusterType;
   private ParseDriver pd;
   protected Hive db;
   protected QueryState queryState;
@@ -172,6 +180,7 @@ public class QTestUtil {
   private CliDriver cliDriver;
   private HadoopShims.MiniMrShim mr = null;
   private HadoopShims.MiniDFSShim dfs = null;
+  private FileSystem fs;
   private HadoopShims.HdfsEncryptionShim hes = null;
   private MiniLlapCluster llapCluster = null;
   private String hadoopVer = null;
@@ -308,20 +317,24 @@ public class QTestUtil {
       // Plug verifying metastore in for testing DirectSQL.
       conf.setVar(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL,
         "org.apache.hadoop.hive.metastore.VerifyingObjectStore");
+    } else {
+      conf.setVar(ConfVars.METASTORE_RAW_STORE_IMPL, HBaseStore.class.getName());
+      conf.setBoolVar(ConfVars.METASTORE_FASTPATH, true);
     }
 
     if (mr != null) {
-      assert dfs != null;
-
       mr.setupConfiguration(conf);
 
-      // set fs.default.name to the uri of mini-dfs
-      String dfsUriString = WindowsPathUtil.getHdfsUriString(dfs.getFileSystem().getUri().toString());
-      conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, dfsUriString);
-      // hive.metastore.warehouse.dir needs to be set relative to the mini-dfs
-      conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE,
-                  (new Path(dfsUriString,
-                            "/build/ql/test/data/warehouse/")).toString());
+      // TODO Ideally this should be done independent of whether mr is setup or not.
+      setFsRelatedProperties(conf, fs.getScheme().equals("file"),fs);
+    }
+
+    if (llapCluster != null) {
+      Configuration clusterSpecificConf = llapCluster.getClusterSpecificConfiguration();
+      for (Map.Entry<String, String> confEntry : clusterSpecificConf) {
+        // Conf.get takes care of parameter replacement, iterator.value does not.
+        conf.set(confEntry.getKey(), clusterSpecificConf.get(confEntry.getKey()));
+      }
     }
 
     // Windows paths should be converted after MiniMrShim.setupConfiguration()
@@ -331,6 +344,76 @@ public class QTestUtil {
     }
   }
 
+  private void setFsRelatedProperties(HiveConf conf, boolean isLocalFs, FileSystem fs) {
+    String fsUriString = WindowsPathUtil.getHdfsUriString(fs.getUri().toString());
+
+    // Different paths if running locally vs a remote fileSystem. Ideally this difference should not exist.
+    Path warehousePath;
+    Path jarPath;
+    Path userInstallPath;
+    if (isLocalFs) {
+      String buildDir = System.getProperty(BUILD_DIR_PROPERTY);
+      Preconditions.checkState(Strings.isNotBlank(buildDir));
+      Path path = new Path(fsUriString, buildDir);
+
+      // Create a fake fs root for local fs
+      Path localFsRoot  = new Path(path, "localfs");
+      warehousePath = new Path(localFsRoot, "warehouse");
+      jarPath = new Path(localFsRoot, "jar");
+      userInstallPath = new Path(localFsRoot, "user_install");
+    } else {
+      // TODO Why is this changed from the default in hive-conf?
+      warehousePath = new Path(fsUriString, "/build/ql/test/data/warehouse/");
+      jarPath = new Path(new Path(fsUriString, "/user"), "hive");
+      userInstallPath = new Path(fsUriString, "/user");
+    }
+
+    warehousePath = fs.makeQualified(warehousePath);
+    jarPath = fs.makeQualified(jarPath);
+    userInstallPath = fs.makeQualified(userInstallPath);
+
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsUriString);
+
+    // Remote dirs
+    conf.setVar(ConfVars.METASTOREWAREHOUSE, warehousePath.toString());
+    conf.setVar(ConfVars.HIVE_JAR_DIRECTORY, jarPath.toString());
+    conf.setVar(ConfVars.HIVE_USER_INSTALL_DIR, userInstallPath.toString());
+    // ConfVars.SCRATCHDIR - {test.tmp.dir}/scratchdir
+
+    // Local dirs
+    // ConfVars.LOCALSCRATCHDIR - {test.tmp.dir}/localscratchdir
+
+    // TODO Make sure to cleanup created dirs.
+  }
+
+  private void createRemoteDirs() {
+    assert fs != null;
+    Path warehousePath = fs.makeQualified(new Path(conf.getVar(ConfVars.METASTOREWAREHOUSE)));
+    assert warehousePath != null;
+    Path hiveJarPath = fs.makeQualified(new Path(conf.getVar(ConfVars.HIVE_JAR_DIRECTORY)));
+    assert hiveJarPath != null;
+    Path userInstallPath = fs.makeQualified(new Path(conf.getVar(ConfVars.HIVE_USER_INSTALL_DIR)));
+    assert userInstallPath != null;
+    try {
+      fs.mkdirs(warehousePath);
+    } catch (IOException e) {
+      LOG.error("Failed to create path={}. Continuing. Exception message={}", warehousePath,
+          e.getMessage());
+    }
+    try {
+      fs.mkdirs(hiveJarPath);
+    } catch (IOException e) {
+      LOG.error("Failed to create path={}. Continuing. Exception message={}", warehousePath,
+          e.getMessage());
+    }
+    try {
+      fs.mkdirs(userInstallPath);
+    } catch (IOException e) {
+      LOG.error("Failed to create path={}. Continuing. Exception message={}", warehousePath,
+          e.getMessage());
+    }
+  }
+
   public enum MiniClusterType {
     mr,
     tez,
@@ -382,16 +465,28 @@ public class QTestUtil {
         "org.apache.hadoop.hive.metastore.hbase.HBaseStoreTestUtil")
         .getMethod("initHBaseMetastore", HBaseAdmin.class, HiveConf.class);
     initHBaseMetastoreMethod.invoke(null, admin, conf);
+    conf.setVar(ConfVars.METASTORE_RAW_STORE_IMPL, HBaseStore.class.getName());
+    conf.setBoolVar(ConfVars.METASTORE_FASTPATH, true);
+  }
+
+  public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
+                   String confDir, String hadoopVer, String initScript, String cleanupScript,
+                   boolean useHBaseMetastore, boolean withLlapIo) throws Exception {
+    this(outDir, logDir, clusterType, confDir, hadoopVer, initScript, cleanupScript,
+        useHBaseMetastore, withLlapIo, false, false);
   }
 
   public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
       String confDir, String hadoopVer, String initScript, String cleanupScript,
-      boolean useHBaseMetastore, boolean withLlapIo)
+      boolean useHBaseMetastore, boolean withLlapIo, boolean localMode, boolean useLocalFs)
     throws Exception {
-    LOG.info("Setting up QtestUtil with outDir={}, logDir={}, clusterType={}, confDir={}," +
-        " hadoopVer={}, initScript={}, cleanupScript={}, useHbaseMetaStore={}, withLlapIo={}"
+    LOG.info("Setting up QTestUtil with outDir={}, logDir={}, clusterType={}, confDir={}," +
+        " hadoopVer={}, initScript={}, cleanupScript={}, useHbaseMetaStore={}, withLlapIo={}," +
+            " localMode={}, useLocalFs={}"
         , outDir, logDir, clusterType, confDir, hadoopVer, initScript, cleanupScript,
-        useHBaseMetastore, withLlapIo);
+        useHBaseMetastore, withLlapIo, localMode, useLocalFs);
+    this.useLocalFs = useLocalFs;
+    this.localMode = localMode;
     this.outDir = outDir;
     this.logDir = logDir;
     this.useHBaseMetastore = useHBaseMetastore;
@@ -418,57 +513,20 @@ public class QTestUtil {
     qSortNHashQuerySet = new HashSet<String>();
     qNoSessionReuseQuerySet = new HashSet<String>();
     qJavaVersionSpecificOutput = new HashSet<String>();
-    QTestUtil.clusterType = clusterType;
+    this.clusterType = clusterType;
 
     HadoopShims shims = ShimLoader.getHadoopShims();
-    int numberOfDataNodes = 4;
-
-    if (clusterType != MiniClusterType.none && clusterType != MiniClusterType.spark) {
-      FileSystem fs = null;
-
-      if (clusterType == MiniClusterType.encrypted) {
-        // Set the security key provider so that the MiniDFS cluster is initialized
-        // with encryption
-        conf.set(SECURITY_KEY_PROVIDER_URI_NAME, getKeyProviderURI());
-        conf.setInt("fs.trash.interval", 50);
 
-        dfs = shims.getMiniDfs(conf, numberOfDataNodes, true, null);
-        fs = dfs.getFileSystem();
-
-        // set up the java key provider for encrypted hdfs cluster
-        hes = shims.createHdfsEncryptionShim(fs, conf);
+    setupFileSystem(shims);
 
-        LOG.info("key provider is initialized");
-      } else {
-        dfs = shims.getMiniDfs(conf, numberOfDataNodes, true, null);
-        fs = dfs.getFileSystem();
-      }
-
-      setup = new QTestSetup();
-      setup.preTest(conf);
+    setup = new QTestSetup();
+    setup.preTest(conf);
 
-      String uriString = WindowsPathUtil.getHdfsUriString(fs.getUri().toString());
-      if (clusterType == MiniClusterType.tez) {
-        if (confDir != null && !confDir.isEmpty()) {
-          conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
-              + "/tez-site.xml"));
-        }
-        mr = shims.getMiniTezCluster(conf, 4, uriString);
-      } else if (clusterType == MiniClusterType.llap) {
-        llapCluster = LlapItUtils.startAndGetMiniLlapCluster(conf, setup.zooKeeperCluster, confDir);
-        mr = shims.getMiniTezCluster(conf, 2, uriString);
-      } else if (clusterType == MiniClusterType.miniSparkOnYarn) {
-        mr = shims.getMiniSparkCluster(conf, 4, uriString, 1);
-      } else {
-        mr = shims.getMiniMrCluster(conf, 4, uriString, 1);
-      }
-    } else {
-      setup = new QTestSetup();
-      setup.preTest(conf);
-    }
+    setupMiniCluster(shims, confDir);
 
     initConf();
-    if (withLlapIo && clusterType == MiniClusterType.none) {
+
+    if (withLlapIo && (clusterType == MiniClusterType.none)) {
       LOG.info("initializing llap IO");
       LlapProxy.initializeLlapIo(conf);
     }
@@ -495,6 +553,80 @@ public class QTestUtil {
     init();
   }
 
+  private void setupFileSystem(HadoopShims shims) throws IOException {
+
+    if (useLocalFs) {
+      Preconditions
+          .checkState(clusterType == MiniClusterType.tez || clusterType == MiniClusterType.llap,
+              "useLocalFs can currently only be set for tez or llap");
+    }
+
+    if (clusterType != MiniClusterType.none && clusterType != MiniClusterType.spark) {
+      int numDataNodes = 4;
+
+      if (clusterType == MiniClusterType.encrypted) {
+        // Set the security key provider so that the MiniDFS cluster is initialized
+        // with encryption
+        conf.set(SECURITY_KEY_PROVIDER_URI_NAME, getKeyProviderURI());
+        conf.setInt("fs.trash.interval", 50);
+
+        dfs = shims.getMiniDfs(conf, numDataNodes, true, null);
+        fs = dfs.getFileSystem();
+
+        // set up the java key provider for encrypted hdfs cluster
+        hes = shims.createHdfsEncryptionShim(fs, conf);
+
+        LOG.info("key provider is initialized");
+      } else {
+        if (!useLocalFs) {
+          dfs = shims.getMiniDfs(conf, numDataNodes, true, null);
+          fs = dfs.getFileSystem();
+        } else {
+          fs = FileSystem.getLocal(conf);
+        }
+      }
+    } else {
+      // Setup local file system
+      fs = FileSystem.getLocal(conf);
+    }
+  }
+
+  private void setupMiniCluster(HadoopShims shims, String confDir) throws
+      IOException {
+
+    if (localMode) {
+      Preconditions
+          .checkState(clusterType == MiniClusterType.tez || clusterType == MiniClusterType.llap,
+              "localMode can currently only be set for tez or llap");
+    }
+
+    String uriString = WindowsPathUtil.getHdfsUriString(fs.getUri().toString());
+
+    if (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.llap) {
+      if (confDir != null && !confDir.isEmpty()) {
+        conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
+            + "/tez-site.xml"));
+      }
+      int numTrackers;
+      if (clusterType == MiniClusterType.tez) {
+        numTrackers = 4;
+      } else {
+        llapCluster = LlapItUtils.startAndGetMiniLlapCluster(conf, setup.zooKeeperCluster, confDir);
+        numTrackers = 2;
+      }
+      if (localMode) {
+        mr = shims.getLocalMiniTezCluster(conf, clusterType == MiniClusterType.llap);
+      } else {
+        mr = shims.getMiniTezCluster(conf, numTrackers, uriString);
+      }
+    } else if (clusterType == MiniClusterType.miniSparkOnYarn) {
+      mr = shims.getMiniSparkCluster(conf, 4, uriString, 1);
+    } else if (clusterType == MiniClusterType.mr || clusterType == MiniClusterType.encrypted) {
+      mr = shims.getMiniMrCluster(conf, 4, uriString, 1);
+    }
+  }
+
+
   public void shutdown() throws Exception {
     if (System.getenv(QTEST_LEAVE_FILES) == null) {
       cleanUp();
@@ -881,6 +1013,8 @@ public class QTestUtil {
       // Best effort
     }
 
+    // TODO: Clean up all the other paths that are created.
+
     FunctionRegistry.unregisterTemporaryUDF("test_udaf");
     FunctionRegistry.unregisterTemporaryUDF("test_error");
   }
@@ -932,8 +1066,9 @@ public class QTestUtil {
     LOG.info("Initial setup (" + initScript + "):\n" + initCommands);
 
     int result = cliDriver.processLine(initCommands);
+    LOG.info("Result from cliDrriver.processLine in createSources=" + result);
     if (result != 0) {
-      Assert.fail("Failed during createSurces processLine with code=" + result);
+      Assert.fail("Failed during createSources processLine with code=" + result);
     }
 
     conf.setBoolean("hive.test.init.phase", false);
@@ -941,6 +1076,11 @@ public class QTestUtil {
 
   public void init() throws Exception {
 
+    // Create remote dirs once.
+    if (mr != null) {
+      createRemoteDirs();
+    }
+
     testWarehouse = conf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE);
     String execEngine = conf.get("hive.execution.engine");
     conf.set("hive.execution.engine", "mr");
@@ -1721,7 +1861,7 @@ public class QTestUtil {
 
       if (zooKeeperCluster == null) {
         //create temp dir
-        String tmpBaseDir =  System.getProperty("test.tmp.dir");
+        String tmpBaseDir =  System.getProperty(TEST_TMP_DIR_PROPERTY);
         File tmpDir = Utilities.createTempDir(tmpBaseDir);
 
         zooKeeperCluster = new MiniZooKeeperCluster();
@@ -1978,19 +2118,26 @@ public class QTestUtil {
 
   public void failed(int ecode, String fname, String debugHint) {
     String command = SessionState.get() != null ? SessionState.get().getLastCommand() : null;
-    Assert.fail("Client Execution failed with error code = " + ecode +
-        (command != null ? " running " + command : "") + (debugHint != null ? debugHint : ""));
+    String message = "Client execution failed with error code = " + ecode +
+        (command != null ? " running " + command : "") + "fname=" + fname +
+        (debugHint != null ? debugHint : "");
+    LOG.error(message);
+    Assert.fail(message);
   }
 
   // for negative tests, which is succeeded.. no need to print the query string
   public void failed(String fname, String debugHint) {
-    Assert.fail("Client Execution was expected to fail, but succeeded with error code 0 " +
-        (debugHint != null ? debugHint : ""));
+    Assert.fail(
+        "Client Execution was expected to fail, but succeeded with error code 0 for fname=" +
+            fname + (debugHint != null ? (" " + debugHint) : ""));
   }
 
   public void failedDiff(int ecode, String fname, String debugHint) {
-    Assert.fail("Client Execution results failed with error code = " + ecode +
-        (debugHint != null ? debugHint : ""));
+    String message =
+        "Client Execution results failed with error code = " + ecode + " while executing fname=" +
+            fname + (debugHint != null ? (" " + debugHint) : "");
+    LOG.error(message);
+    Assert.fail(message);
   }
 
   public void failed(Throwable e, String fname, String debugHint) {
@@ -2092,7 +2239,7 @@ public class QTestUtil {
       File tabParamsCsv = new File(mdbPath+"csv/TABLE_PARAMS.txt");
 
       // Set up the foreign key constraints properly in the TAB_COL_STATS data
-      String tmpBaseDir =  System.getProperty("test.tmp.dir");
+      String tmpBaseDir =  System.getProperty(TEST_TMP_DIR_PROPERTY);
       File tmpFileLoc1 = new File(tmpBaseDir+"/TAB_COL_STATS.txt");
       File tmpFileLoc2 = new File(tmpBaseDir+"/TABLE_PARAMS.txt");
       FileUtils.copyFile(tabColStatsCsv, tmpFileLoc1);

http://git-wip-us.apache.org/repos/asf/hive/blob/e297a157/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
index 9fc43b3..3f0dde5 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
@@ -267,7 +267,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
         new ThreadFactoryBuilder().setDaemon(true).setNameFormat("LlapScheduler").build());
     schedulerExecutor = MoreExecutors.listeningDecorator(schedulerExecutorServiceRaw);
 
-    if (initMetrics) {
+    if (initMetrics && !conf.getBoolean(ConfVars.HIVE_IN_TEST.varname, false)) {
       // Initialize the metrics system
       LlapMetricsSystem.initialize("LlapTaskScheduler");
       this.pauseMonitor = new JvmPauseMonitor(conf);

http://git-wip-us.apache.org/repos/asf/hive/blob/e297a157/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
----------------------------------------------------------------------
diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
index 84b1174..341b1e5 100644
--- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
+++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
@@ -98,6 +98,8 @@ import org.apache.hadoop.tools.DistCpOptions;
 import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
 import org.apache.tez.test.MiniTezCluster;
 
 /**
@@ -326,6 +328,49 @@ public class Hadoop23Shims extends HadoopShimsSecure {
     }
   }
 
+  @Override
+  public HadoopShims.MiniMrShim getLocalMiniTezCluster(Configuration conf, boolean usingLlap) {
+    return new MiniTezLocalShim(conf, usingLlap);
+  }
+
+  public class MiniTezLocalShim extends Hadoop23Shims.MiniMrShim {
+    private final Configuration conf;
+    private final boolean isLlap;
+
+    public MiniTezLocalShim(Configuration conf, boolean usingLlap) {
+      this.conf = conf;
+      this.isLlap = usingLlap;
+      setupConfiguration(conf);
+    }
+
+    @Override
+    public int getJobTrackerPort() throws UnsupportedOperationException {
+      throw new UnsupportedOperationException("No JobTracker port for local mode");
+    }
+
+    @Override
+    public void setupConfiguration(Configuration conf) {
+      conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true);
+
+      conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true);
+
+      conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true);
+
+      // TODO Force fs to file://, setup staging dir?
+      //      conf.set("fs.defaultFS", "file:///");
+      //      conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, "/tmp");
+
+      if (!isLlap) {
+        conf.setBoolean("hive.llap.io.enabled", false);
+      }
+    }
+
+    @Override
+    public void shutdown() throws IOException {
+      // Nothing to do
+    }
+  }
+
   /**
    * Returns a shim to wrap MiniMrTez
    */
@@ -376,20 +421,6 @@ public class Hadoop23Shims extends HadoopShimsSecure {
       for (Map.Entry<String, String> pair: config) {
         conf.set(pair.getKey(), pair.getValue());
       }
-
-      Path jarPath = new Path("hdfs:///user/hive");
-      Path hdfsPath = new Path("hdfs:///user/");
-      try {
-        FileSystem fs = cluster.getFileSystem();
-        jarPath = fs.makeQualified(jarPath);
-        conf.set("hive.jar.directory", jarPath.toString());
-        fs.mkdirs(jarPath);
-        hdfsPath = fs.makeQualified(hdfsPath);
-        conf.set("hive.user.install.directory", hdfsPath.toString());
-        fs.mkdirs(hdfsPath);
-      } catch (Exception e) {
-        e.printStackTrace();
-      }
     }
   }
 
@@ -458,20 +489,6 @@ public class Hadoop23Shims extends HadoopShimsSecure {
       for (Map.Entry<String, String> pair : config) {
         conf.set(pair.getKey(), pair.getValue());
       }
-
-      Path jarPath = new Path("hdfs:///user/hive");
-      Path hdfsPath = new Path("hdfs:///user/");
-      try {
-        FileSystem fs = cluster.getFileSystem();
-        jarPath = fs.makeQualified(jarPath);
-        conf.set("hive.jar.directory", jarPath.toString());
-        fs.mkdirs(jarPath);
-        hdfsPath = fs.makeQualified(hdfsPath);
-        conf.set("hive.user.install.directory", hdfsPath.toString());
-        fs.mkdirs(hdfsPath);
-      } catch (Exception e) {
-        e.printStackTrace();
-      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/e297a157/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
index 04f075b..f987814 100644
--- a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
+++ b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
@@ -92,6 +92,8 @@ public interface HadoopShims {
   public MiniMrShim getMiniTezCluster(Configuration conf, int numberOfTaskTrackers,
       String nameNode) throws IOException;
 
+  public MiniMrShim getLocalMiniTezCluster(Configuration conf, boolean usingLlap);
+
   public MiniMrShim getMiniSparkCluster(Configuration conf, int numberOfTaskTrackers,
       String nameNode, int numDir) throws IOException;
 


[25/44] hive git commit: HIVE-5867: JDBC driver and beeline should support executing an initial SQL script(Jianguo Tian, via Ferdinand Xu)

Posted by se...@apache.org.
HIVE-5867: JDBC driver and beeline should support executing an initial SQL script(Jianguo Tian, via Ferdinand Xu)


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

Branch: refs/heads/hive-14535
Commit: 737fd09a20dbb4b728307f84825f7ddc4294ae02
Parents: 19fd561
Author: Jianguo Tian <ji...@intel.com>
Authored: Tue Sep 27 04:01:49 2016 +0800
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Tue Sep 27 04:01:49 2016 +0800

----------------------------------------------------------------------
 .../org/apache/hive/jdbc/HiveConnection.java    | 79 ++++++++++++++++
 jdbc/src/java/org/apache/hive/jdbc/Utils.java   |  1 +
 .../org/apache/hive/jdbc/TestJdbcDriver.java    | 98 ++++++++++++++++++++
 3 files changed, 178 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/737fd09a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java b/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
index ad96a64..ce85320 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
@@ -67,8 +67,11 @@ import javax.net.ssl.TrustManagerFactory;
 import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
 
+import java.io.BufferedReader;
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -92,6 +95,7 @@ import java.sql.SQLXML;
 import java.sql.Savepoint;
 import java.sql.Statement;
 import java.sql.Struct;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -125,6 +129,7 @@ public class HiveConnection implements java.sql.Connection {
   private int loginTimeout = 0;
   private TProtocolVersion protocol;
   private int fetchSize = HiveStatement.DEFAULT_FETCH_SIZE;
+  private String initFile = null;
 
   public HiveConnection(String uri, Properties info) throws SQLException {
     setupLoginTimeout();
@@ -147,6 +152,9 @@ public class HiveConnection implements java.sql.Connection {
     if (sessConfMap.containsKey(JdbcConnectionParams.FETCH_SIZE)) {
       fetchSize = Integer.parseInt(sessConfMap.get(JdbcConnectionParams.FETCH_SIZE));
     }
+    if (sessConfMap.containsKey(JdbcConnectionParams.INIT_FILE)) {
+      initFile = sessConfMap.get(JdbcConnectionParams.INIT_FILE);
+    }
 
     // add supported protocols
     supportedProtocols.add(TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1);
@@ -166,6 +174,7 @@ public class HiveConnection implements java.sql.Connection {
 
       // open client session
       openSession();
+      executeInitSql();
     } else {
       int maxRetries = 1;
       try {
@@ -184,6 +193,7 @@ public class HiveConnection implements java.sql.Connection {
           client = new TCLIService.Client(new TBinaryProtocol(transport));
           // open client session
           openSession();
+          executeInitSql();
 
           break;
         } catch (Exception e) {
@@ -218,6 +228,75 @@ public class HiveConnection implements java.sql.Connection {
     client = newSynchronizedClient(client);
   }
 
+  private void executeInitSql() throws SQLException {
+    if (initFile != null) {
+      try {
+        List<String> sqlList = parseInitFile(initFile);
+        Statement st = createStatement();
+        for(String sql : sqlList) {
+          boolean hasResult = st.execute(sql);
+          if (hasResult) {
+            ResultSet rs = st.getResultSet();
+            while (rs.next()) {
+              System.out.println(rs.getString(1));
+            }
+          }
+        }
+      } catch(Exception e) {
+        LOG.error("Failed to execute initial SQL");
+        throw new SQLException(e.getMessage());
+      }
+    }
+  }
+
+  public static List<String> parseInitFile(String initFile) throws IOException {
+    File file = new File(initFile);
+    BufferedReader br = null;
+    List<String> initSqlList = null;
+    try {
+      FileInputStream input = new FileInputStream(file);
+      br = new BufferedReader(new InputStreamReader(input, "UTF-8"));
+      String line;
+      StringBuilder sb = new StringBuilder("");
+      while ((line = br.readLine()) != null) {
+        line = line.trim();
+        if (line.length() != 0) {
+          if (line.startsWith("#") || line.startsWith("--")) {
+            continue;
+          } else {
+            line = line.concat(" ");
+            sb.append(line);
+          }
+        }
+      }
+      initSqlList = getInitSql(sb.toString());
+    } catch(IOException e) {
+      LOG.error("Failed to read initial SQL file", e);
+      throw new IOException(e);
+    } finally {
+      if (br != null) {
+        br.close();
+      }
+    }
+    return initSqlList;
+  }
+
+  private static List<String> getInitSql(String sbLine) {
+    char[] sqlArray = sbLine.toCharArray();
+    List<String> initSqlList = new ArrayList();
+    int index = 0;
+    int beginIndex = 0;
+    for (; index < sqlArray.length; index++) {
+      if (sqlArray[index] == ';') {
+        String sql = sbLine.substring(beginIndex, index).trim();
+        initSqlList.add(sql);
+        beginIndex = index + 1;
+      }
+    }
+    return initSqlList;
+  }
+
+
   private void openTransport() throws Exception {
       assumeSubject =
           JdbcConnectionParams.AUTH_KERBEROS_AUTH_TYPE_FROM_SUBJECT.equals(sessConfMap

http://git-wip-us.apache.org/repos/asf/hive/blob/737fd09a/jdbc/src/java/org/apache/hive/jdbc/Utils.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/Utils.java b/jdbc/src/java/org/apache/hive/jdbc/Utils.java
index 3161566..bfae8b9 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/Utils.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/Utils.java
@@ -120,6 +120,7 @@ public class Utils {
     static final String HTTP_HEADER_PREFIX = "http.header.";
     // Set the fetchSize
     static final String FETCH_SIZE = "fetchSize";
+    static final String INIT_FILE = "initFile";
 
     // --------------- Begin 2 way ssl options -------------------------
     // Use two way ssl. This param will take effect only when ssl=true

http://git-wip-us.apache.org/repos/asf/hive/blob/737fd09a/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver.java
----------------------------------------------------------------------
diff --git a/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver.java b/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver.java
new file mode 100644
index 0000000..162e42f
--- /dev/null
+++ b/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver.java
@@ -0,0 +1,98 @@
+/*
+ * 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.
+ */
+
+package org.apache.hive.jdbc;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.junit.Assert.assertEquals;
+
+@RunWith(Parameterized.class)
+public class TestJdbcDriver {
+  private static File file = null;
+  private String input;
+  private String expected;
+
+  public TestJdbcDriver(String input, String expected) throws Exception {
+    this.input = input;
+    this.expected = expected;
+  }
+
+  @Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][]{
+      // Here are some positive cases which can be executed as below :
+      {"show databases;show tables;", "show databases,show tables"},
+      {" show\n\r  tables;", "show tables"},
+      {"show databases; show\ntables;", "show databases,show tables"},
+      {"show    tables;", "show    tables"},
+      {"show tables ;", "show tables"},
+      // Here are some negative cases as below :
+      {"show tables", ","},
+      {"show tables show tables;", "show tables show tables"},
+      {"show tab les;", "show tab les"},
+      {"#show tables; show\n tables;", "tables"},
+      {"show tab les;show tables;", "show tab les,show tables"}
+    });
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    file = new File(System.getProperty("user.dir") + File.separator + "Init.sql");
+    if (!file.exists()) {
+      file.createNewFile();
+    }
+  }
+
+  @AfterClass
+  public static void cleanUpAfterClass() throws Exception {
+    if (file != null) {
+      file.delete();
+    }
+  }
+
+  @Test
+  public void testParseInitFile() throws IOException {
+    BufferedWriter bw = null;
+    try {
+      bw = new BufferedWriter(new FileWriter(file));
+      bw.write(input);
+      bw.flush();
+      assertEquals(Arrays.asList(expected.split(",")), HiveConnection.parseInitFile(file.toString()));
+    } catch(Exception e) {
+      Assert.fail("Test was failed due to " + e);
+    } finally {
+      if (bw != null) {
+        bw.close();
+      }
+    }
+  }
+}


[09/44] hive git commit: HIVE-14782: Improve runtime of NegativeMinimrCliDriver (Prasanth Jayachandran reviewed by Siddharth Seth)

Posted by se...@apache.org.
HIVE-14782: Improve runtime of NegativeMinimrCliDriver (Prasanth Jayachandran 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/66af7643
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/66af7643
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/66af7643

Branch: refs/heads/hive-14535
Commit: 66af76435b5a9852f3fa1b8ce9a12c01128737fa
Parents: 0c392b1
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Wed Sep 21 12:01:40 2016 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Wed Sep 21 12:01:40 2016 -0700

----------------------------------------------------------------------
 itests/src/test/resources/testconfiguration.properties |  2 --
 .../org/apache/hadoop/hive/cli/control/CliConfigs.java |  4 ++--
 .../clientnegative/mapreduce_stack_trace_hadoop20.q    | 13 -------------
 .../mapreduce_stack_trace_turnoff_hadoop20.q           | 13 -------------
 .../mapreduce_stack_trace_hadoop20.q.out               | 13 -------------
 .../mapreduce_stack_trace_turnoff_hadoop20.q.out       |  5 -----
 .../src/test/resources/test-configuration2.properties  |  3 +--
 7 files changed, 3 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/66af7643/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 4597ace..e810a58 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -698,9 +698,7 @@ minimr.query.negative.files=cluster_tasklog_retrieval.q,\
   file_with_header_footer_negative.q,\
   local_mapred_error_cache.q,\
   mapreduce_stack_trace.q,\
-  mapreduce_stack_trace_hadoop20.q,\
   mapreduce_stack_trace_turnoff.q,\
-  mapreduce_stack_trace_turnoff_hadoop20.q,\
   minimr_broken_pipe.q,\
   table_nonprintable_negative.q,\
   udf_local_resource.q

http://git-wip-us.apache.org/repos/asf/hive/blob/66af7643/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
index 69c4974..d74f51a 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
@@ -276,8 +276,8 @@ public class CliConfigs {
         setResultsDir("ql/src/test/results/clientnegative");
         setLogDir("itests/qtest/target/qfile-results/clientnegative");
 
-        setInitScript("q_test_init.sql");
-        setCleanupScript("q_test_cleanup.sql");
+        setInitScript("q_test_init_for_encryption.sql");
+        setCleanupScript("q_test_cleanup_for_encryption.sql");
 
         setHiveConfDir("");
         setClusterType(MiniClusterType.mr);

http://git-wip-us.apache.org/repos/asf/hive/blob/66af7643/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q b/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q
deleted file mode 100644
index 9d0548c..0000000
--- a/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q
+++ /dev/null
@@ -1,13 +0,0 @@
-set hive.exec.mode.local.auto=false;
-set hive.exec.job.debug.capture.stacktraces=true;
-set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook;
-
-FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value);
-
--- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
--- Hadoop 0.23 changes the getTaskDiagnostics behavior
--- The Error Code of hive failure MapReduce job changes
--- In Hadoop 0.20
--- Hive failure MapReduce job gets 20000 as Error Code
--- In Hadoop 0.23
--- Hive failure MapReduce job gets 2 as Error Code

http://git-wip-us.apache.org/repos/asf/hive/blob/66af7643/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q b/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q
deleted file mode 100644
index e319944..0000000
--- a/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q
+++ /dev/null
@@ -1,13 +0,0 @@
-set hive.exec.mode.local.auto=false;
-set hive.exec.job.debug.capture.stacktraces=false;
-set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook;
-
-FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value);
-
--- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
--- Hadoop 0.23 changes the getTaskDiagnostics behavior
--- The Error Code of hive failure MapReduce job changes
--- In Hadoop 0.20
--- Hive failure MapReduce job gets 20000 as Error Code
--- In Hadoop 0.23
--- Hive failure MapReduce job gets 2 as Error Code

http://git-wip-us.apache.org/repos/asf/hive/blob/66af7643/ql/src/test/results/clientnegative/mapreduce_stack_trace_hadoop20.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/mapreduce_stack_trace_hadoop20.q.out b/ql/src/test/results/clientnegative/mapreduce_stack_trace_hadoop20.q.out
deleted file mode 100644
index dda4216..0000000
--- a/ql/src/test/results/clientnegative/mapreduce_stack_trace_hadoop20.q.out
+++ /dev/null
@@ -1,13 +0,0 @@
-PREHOOK: query: FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value)
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-#### A masked pattern was here ####
-FATAL org.apache.hadoop.hive.ql.exec.mr.ExecMapper: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"key":"238","value":"val_238"}
-Hive Runtime Error while processing row {"key":"238","value":"val_238"}
-FATAL org.apache.hadoop.hive.ql.exec.mr.ExecMapper: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"key":"238","value":"val_238"}
-Hive Runtime Error while processing row {"key":"238","value":"val_238"}
-FATAL org.apache.hadoop.hive.ql.exec.mr.ExecMapper: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"key":"238","value":"val_238"}
-Hive Runtime Error while processing row {"key":"238","value":"val_238"}
-FATAL org.apache.hadoop.hive.ql.exec.mr.ExecMapper: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"key":"238","value":"val_238"}
-Hive Runtime Error while processing row {"key":"238","value":"val_238"}
-FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. Unable to initialize custom script.

http://git-wip-us.apache.org/repos/asf/hive/blob/66af7643/ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q.out b/ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q.out
deleted file mode 100644
index dfc8f54..0000000
--- a/ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q.out
+++ /dev/null
@@ -1,5 +0,0 @@
-PREHOOK: query: FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value)
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-#### A masked pattern was here ####
-FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. Unable to initialize custom script.

http://git-wip-us.apache.org/repos/asf/hive/blob/66af7643/testutils/ptest2/src/test/resources/test-configuration2.properties
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/resources/test-configuration2.properties b/testutils/ptest2/src/test/resources/test-configuration2.properties
index 19e1ac2..bb4e77c 100644
--- a/testutils/ptest2/src/test/resources/test-configuration2.properties
+++ b/testutils/ptest2/src/test/resources/test-configuration2.properties
@@ -107,7 +107,6 @@ qFileTest.miniMrNegative.include = normal
 qFileTest.miniMrNegative.isolate = flaky
 # normal are tests that run in minimr mode via build-common.xml
 qFileTest.miniMrNegative.groups.normal = mainProperties.${minimr.query.negative.files}
-qFileTest.miniMrNegative.groups.flaky = mapreduce_stack_trace_hadoop20.q
 
 qFileTest.clientNegative.driver = TestNegativeCliDriver
 qFileTest.clientNegative.directory = ql/src/test/queries/clientnegative
@@ -118,7 +117,7 @@ qFileTest.clientNegative.exclude = miniMrNormal failing
 # Disable for HIVE-4941 as this tests runs via ant test
 #qFileTest.clientNegative.groups.failing = stats_aggregator_error_1.q
 # normal are run via minimr
-qFileTest.clientNegative.groups.miniMrNormal = cluster_tasklog_retrieval.q minimr_broken_pipe.q mapreduce_stack_trace.q mapreduce_stack_trace_turnoff.q mapreduce_stack_trace_hadoop20.q mapreduce_stack_trace_turnoff_hadoop20.q
+qFileTest.clientNegative.groups.miniMrNormal = cluster_tasklog_retrieval.q minimr_broken_pipe.q mapreduce_stack_trace.q mapreduce_stack_trace_turnoff.q
 
 qFileTest.hbasePositive.driver = TestHBaseCliDriver
 qFileTest.hbasePositive.directory = hbase-handler/src/test/queries/positive


[44/44] hive git commit: HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: 6d9144835cbdda27fea172f0fb268f4f4aac1a0c
Parents: 70299dc 297b443
Author: Sergey Shelukhin <se...@apache.org>
Authored: Fri Sep 30 18:07:43 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Fri Sep 30 18:07:43 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   |  20 +-
 .../org/apache/hive/beeline/BeeLineOpts.java    |   2 +-
 beeline/src/main/resources/BeeLine.properties   |  11 +-
 .../beeline/TestBeeLineExceptionHandling.java   |  72 ++
 .../hive/common/metrics/LegacyMetrics.java      |  96 +--
 .../hive/common/metrics/MetricsMBean.java       |  13 +-
 .../hive/common/metrics/MetricsMBeanImpl.java   |  16 +-
 .../hive/common/metrics/common/Metrics.java     |  31 +-
 .../common/metrics/common/MetricsConstant.java  |   7 +
 .../metrics/metrics2/CodahaleMetrics.java       |  70 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  14 +-
 .../apache/hadoop/hive/ql/log/PerfLogger.java   |  33 +-
 .../hive/common/metrics/TestLegacyMetrics.java  | 103 +--
 data/conf/llap/tez-site.xml                     |  12 +
 data/conf/tez/hive-site.xml                     |  10 -
 dev-support/jenkins-execute-build.sh            |  30 +-
 druid-handler/pom.xml                           |  11 +-
 .../src/test/queries/positive/hbase_bulk.m      |  62 --
 .../src/test/queries/positive/hbase_bulk.q      |  62 ++
 .../src/test/results/positive/hbase_bulk.m.out  | 133 ----
 .../src/test/results/positive/hbase_bulk.q.out  | 133 ++++
 .../hcatalog/streaming/StreamingConnection.java |   2 +
 .../hcatalog/streaming/TransactionBatch.java    |   8 +-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |   4 +
 .../apache/hadoop/hive/metastore/FakeDerby.java | 424 ++++++++++++
 .../metastore/TestObjectStoreInitRetry.java     | 127 ++++
 .../TestPartitionNameWhitelistValidation.java   |  53 +-
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |  32 +
 itests/qtest-spark/pom.xml                      |   2 +
 itests/qtest/pom.xml                            |   2 +
 .../hive/cli/TestHBaseMinimrCliDriver.java      |  62 --
 .../test/resources/testconfiguration.properties |   2 -
 .../hive/cli/control/AbstractCliConfig.java     |  23 +-
 .../hadoop/hive/cli/control/CliConfigs.java     |  38 +-
 .../hadoop/hive/cli/control/CoreCliDriver.java  |   3 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 344 +++++++---
 .../org/apache/hive/jdbc/HiveConnection.java    |  79 +++
 jdbc/src/java/org/apache/hive/jdbc/Utils.java   |   1 +
 .../org/apache/hive/jdbc/TestJdbcDriver.java    |  98 +++
 .../hive/llap/registry/ServiceInstanceSet.java  |   7 +-
 .../registry/impl/InactiveServiceInstance.java  |  77 +++
 .../registry/impl/LlapFixedRegistryImpl.java    |   2 +-
 .../impl/LlapZookeeperRegistryImpl.java         |  34 +-
 .../org/apache/hadoop/hive/llap/LlapUtil.java   |  12 +
 .../hadoop/hive/llap/LlapBaseInputFormat.java   |   6 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |   5 +-
 .../daemon/services/impl/LlapWebServices.java   |   2 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |  14 +-
 .../tezplugins/LlapTaskSchedulerService.java    |   2 +-
 .../tezplugins/TestLlapTaskCommunicator.java    |   5 +
 .../hive/metastore/HMSMetricsListener.java      |  52 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |  13 +-
 .../hadoop/hive/metastore/ObjectStore.java      |  71 ++
 pom.xml                                         |  15 +-
 ql/pom.xml                                      |  26 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  18 +-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |  10 +-
 .../hadoop/hive/ql/exec/FunctionInfo.java       |  20 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   4 +-
 .../hadoop/hive/ql/exec/ReduceSinkOperator.java |   9 +
 .../apache/hadoop/hive/ql/exec/Registry.java    |  96 ++-
 .../org/apache/hadoop/hive/ql/exec/Task.java    |   9 +
 .../hadoop/hive/ql/exec/WindowFunctionInfo.java |   4 +-
 .../hadoop/hive/ql/exec/mr/MapRedTask.java      |   7 +
 .../hadoop/hive/ql/exec/mr/MapredLocalTask.java |   7 +
 .../exec/spark/HiveBaseFunctionResultList.java  |  96 ++-
 .../hive/ql/exec/spark/HiveMapFunction.java     |   2 +-
 .../hive/ql/exec/spark/HiveReduceFunction.java  |   2 +-
 .../hive/ql/exec/spark/SortByShuffler.java      |  84 ++-
 .../hadoop/hive/ql/exec/spark/SparkTask.java    |   7 +
 .../spark/status/impl/JobMetricsListener.java   |   4 +-
 .../tez/HostAffinitySplitLocationProvider.java  |  80 ++-
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |   7 +
 .../apache/hadoop/hive/ql/exec/tez/Utils.java   |   8 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |   6 -
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |   3 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   2 +-
 .../optimizer/SortedDynPartitionOptimizer.java  |   4 +-
 .../translator/SqlFunctionConverter.java        |   2 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   4 +
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |  57 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   2 +-
 .../apache/hadoop/hive/ql/udf/UDFDateFloor.java |   8 +-
 .../ql/udf/generic/GenericUDFLoggedInUser.java  |  82 +++
 .../hive/ql/exec/TestFunctionRegistry.java      | 100 +++
 .../hadoop/hive/ql/exec/mr/TestMapRedTask.java  |  47 ++
 .../hive/ql/exec/mr/TestMapredLocalTask.java    |  46 ++
 .../ql/exec/spark/TestHiveKVResultCache.java    |   5 +-
 .../hive/ql/exec/spark/TestSparkTask.java       |  46 ++
 .../TestHostAffinitySplitLocationProvider.java  | 150 ++++-
 .../hadoop/hive/ql/exec/tez/TestTezTask.java    |  17 +
 .../hadoop/hive/ql/plan/TestViewEntity.java     |  31 +
 .../ql/udf/TestUDFDateFormatGranularity.java    |  22 +-
 .../mapreduce_stack_trace_hadoop20.q            |  13 -
 .../mapreduce_stack_trace_turnoff_hadoop20.q    |  13 -
 ql/src/test/queries/clientpositive/extract.q    |  43 ++
 ql/src/test/queries/clientpositive/floor_time.q |  46 ++
 .../queries/clientpositive/udf_logged_in_user.q |   5 +
 .../mapreduce_stack_trace_hadoop20.q.out        |  13 -
 ...mapreduce_stack_trace_turnoff_hadoop20.q.out |   5 -
 .../results/clientpositive/cbo_union_view.q.out |  12 +
 .../dynpart_sort_optimization_acid.q.out        |  20 +-
 .../test/results/clientpositive/extract.q.out   | 200 ++++++
 .../results/clientpositive/floor_time.q.out     | 211 ++++++
 .../results/clientpositive/ppd_union_view.q.out |  12 +
 .../results/clientpositive/show_functions.q.out |   5 +
 .../clientpositive/udf_logged_in_user.q.out     |  22 +
 .../results/clientpositive/union_view.q.out     |  54 ++
 .../apache/hadoop/hive/serde2/SerDeUtils.java   |  11 +
 service/pom.xml                                 |   7 +
 .../auth/LdapAuthenticationProviderImpl.java    | 657 ++-----------------
 .../service/auth/ldap/ChainFilterFactory.java   |  78 +++
 .../auth/ldap/CustomQueryFilterFactory.java     |  84 +++
 .../hive/service/auth/ldap/DirSearch.java       |  52 ++
 .../service/auth/ldap/DirSearchFactory.java     |  37 ++
 .../apache/hive/service/auth/ldap/Filter.java   |  36 +
 .../hive/service/auth/ldap/FilterFactory.java   |  33 +
 .../service/auth/ldap/GroupFilterFactory.java   |  90 +++
 .../hive/service/auth/ldap/LdapSearch.java      | 155 +++++
 .../service/auth/ldap/LdapSearchFactory.java    |  64 ++
 .../hive/service/auth/ldap/LdapUtils.java       | 228 +++++++
 .../apache/hive/service/auth/ldap/Query.java    | 154 +++++
 .../hive/service/auth/ldap/QueryFactory.java    | 135 ++++
 .../service/auth/ldap/SearchResultHandler.java  | 163 +++++
 .../service/auth/ldap/UserFilterFactory.java    |  75 +++
 .../auth/ldap/UserSearchFilterFactory.java      |  65 ++
 .../hive/service/cli/operation/Operation.java   |  33 +-
 .../service/cli/operation/SQLOperation.java     |  18 +-
 .../service/cli/session/SessionManager.java     |  33 +-
 .../apache/hive/service/server/HiveServer2.java |   7 +-
 .../auth/TestLdapAtnProviderWithMiniDS.java     |   3 +-
 .../TestLdapAuthenticationProviderImpl.java     | 277 +++++++-
 .../hive/service/auth/ldap/Credentials.java     |  41 ++
 .../hive/service/auth/ldap/LdapTestUtils.java   | 126 ++++
 .../hive/service/auth/ldap/TestChainFilter.java | 103 +++
 .../auth/ldap/TestCustomQueryFilter.java        |  85 +++
 .../hive/service/auth/ldap/TestGroupFilter.java | 101 +++
 .../hive/service/auth/ldap/TestLdapSearch.java  | 209 ++++++
 .../hive/service/auth/ldap/TestLdapUtils.java   | 103 +++
 .../hive/service/auth/ldap/TestQuery.java       |  59 ++
 .../service/auth/ldap/TestQueryFactory.java     |  79 +++
 .../auth/ldap/TestSearchResultHandler.java      | 222 +++++++
 .../hive/service/auth/ldap/TestUserFilter.java  |  75 +++
 .../service/auth/ldap/TestUserSearchFilter.java |  94 +++
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |  73 ++-
 .../apache/hadoop/hive/shims/HadoopShims.java   |   2 +
 spark-client/pom.xml                            |  15 +-
 .../hive/spark/client/MetricsCollection.java    |   8 +-
 .../apache/hive/spark/client/RemoteDriver.java  |   4 +-
 .../hive/spark/client/SparkClientImpl.java      |   8 +
 .../hive/spark/client/metrics/InputMetrics.java |   9 +-
 .../hive/spark/client/metrics/Metrics.java      |   6 +-
 .../client/metrics/ShuffleReadMetrics.java      |  18 +-
 .../client/metrics/ShuffleWriteMetrics.java     |   4 +-
 .../hive/spark/client/rpc/RpcConfiguration.java |  42 +-
 .../apache/hive/spark/client/rpc/RpcServer.java |  44 +-
 .../spark/client/TestMetricsCollection.java     |   8 +-
 .../apache/hive/spark/client/rpc/TestRpc.java   |  66 +-
 .../ptest2/conf/cloudhost.properties.example    |  37 ++
 testutils/ptest2/pom.xml                        |   5 +
 .../execution/context/CloudComputeService.java  | 224 +++++--
 .../context/CloudExecutionContextProvider.java  | 105 ++-
 .../resources/test-configuration2.properties    |   3 +-
 163 files changed, 7018 insertions(+), 1825 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6d914483/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/6d914483/itests/qtest-spark/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/6d914483/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/6d914483/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --cc metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 32e4daf,74bde3d..6db28a5
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@@ -25,9 -25,8 +25,10 @@@ import java.lang.reflect.Field
  import java.net.InetAddress;
  import java.net.URI;
  import java.nio.ByteBuffer;
 +import java.sql.Connection;
 +import java.sql.SQLException;
  import java.util.ArrayList;
+ import java.util.Arrays;
  import java.util.Collection;
  import java.util.Collections;
  import java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/hive/blob/6d914483/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/6d914483/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/6d914483/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/6d914483/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/6d914483/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------


[20/44] hive git commit: HIVE-14713: LDAP Authentication Provider should be covered with unit tests (Illya Yalovyy, reviewed by Chaoyu Tang, Szehon Ho)

Posted by se...@apache.org.
HIVE-14713: LDAP Authentication Provider should be covered with unit tests (Illya Yalovyy, reviewed by Chaoyu Tang, Szehon Ho)


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

Branch: refs/heads/hive-14535
Commit: 990927e3dcddcc7c82a16437d55d9f7ea9a1a447
Parents: 421d97a
Author: ctang <ct...@cloudera.com>
Authored: Fri Sep 23 15:50:32 2016 -0400
Committer: ctang <ct...@cloudera.com>
Committed: Fri Sep 23 15:50:32 2016 -0400

----------------------------------------------------------------------
 service/pom.xml                                 |   7 +
 .../auth/LdapAuthenticationProviderImpl.java    | 657 ++-----------------
 .../service/auth/ldap/ChainFilterFactory.java   |  78 +++
 .../auth/ldap/CustomQueryFilterFactory.java     |  84 +++
 .../hive/service/auth/ldap/DirSearch.java       |  52 ++
 .../service/auth/ldap/DirSearchFactory.java     |  37 ++
 .../apache/hive/service/auth/ldap/Filter.java   |  36 +
 .../hive/service/auth/ldap/FilterFactory.java   |  33 +
 .../service/auth/ldap/GroupFilterFactory.java   |  90 +++
 .../hive/service/auth/ldap/LdapSearch.java      | 155 +++++
 .../service/auth/ldap/LdapSearchFactory.java    |  64 ++
 .../hive/service/auth/ldap/LdapUtils.java       | 228 +++++++
 .../apache/hive/service/auth/ldap/Query.java    | 154 +++++
 .../hive/service/auth/ldap/QueryFactory.java    | 135 ++++
 .../service/auth/ldap/SearchResultHandler.java  | 163 +++++
 .../service/auth/ldap/UserFilterFactory.java    |  75 +++
 .../auth/ldap/UserSearchFilterFactory.java      |  65 ++
 .../auth/TestLdapAtnProviderWithMiniDS.java     |   3 +-
 .../TestLdapAuthenticationProviderImpl.java     | 277 +++++++-
 .../hive/service/auth/ldap/Credentials.java     |  41 ++
 .../hive/service/auth/ldap/LdapTestUtils.java   | 126 ++++
 .../hive/service/auth/ldap/TestChainFilter.java | 103 +++
 .../auth/ldap/TestCustomQueryFilter.java        |  85 +++
 .../hive/service/auth/ldap/TestGroupFilter.java | 101 +++
 .../hive/service/auth/ldap/TestLdapSearch.java  | 209 ++++++
 .../hive/service/auth/ldap/TestLdapUtils.java   | 103 +++
 .../hive/service/auth/ldap/TestQuery.java       |  59 ++
 .../service/auth/ldap/TestQueryFactory.java     |  79 +++
 .../auth/ldap/TestSearchResultHandler.java      | 222 +++++++
 .../hive/service/auth/ldap/TestUserFilter.java  |  75 +++
 .../service/auth/ldap/TestUserSearchFilter.java |  94 +++
 31 files changed, 3062 insertions(+), 628 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/pom.xml
----------------------------------------------------------------------
diff --git a/service/pom.xml b/service/pom.xml
index ecea719..9306739 100644
--- a/service/pom.xml
+++ b/service/pom.xml
@@ -164,6 +164,13 @@
     </dependency>
 
     <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>${mockito-all.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
       <groupId>org.apache.directory.client.ldap</groupId>
       <artifactId>ldap-client-api</artifactId>
       <version>${apache-directory-clientapi.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java b/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
index efd5393..c21da28 100644
--- a/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
+++ b/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
@@ -17,633 +17,106 @@
  */
 package org.apache.hive.service.auth;
 
-import java.util.ArrayList;
-import java.util.Hashtable;
-import java.util.List;
-import java.util.ListIterator;
-
-import javax.naming.Context;
-import javax.naming.NamingEnumeration;
-import javax.naming.NamingException;
-import javax.naming.directory.Attribute;
-import javax.naming.directory.Attributes;
-import javax.naming.directory.DirContext;
-import javax.naming.directory.InitialDirContext;
-import javax.naming.directory.SearchControls;
-import javax.naming.directory.SearchResult;
 import javax.security.sasl.AuthenticationException;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hive.service.ServiceUtils;
+import org.apache.hive.service.auth.ldap.ChainFilterFactory;
+import org.apache.hive.service.auth.ldap.CustomQueryFilterFactory;
+import org.apache.hive.service.auth.ldap.LdapSearchFactory;
+import org.apache.hive.service.auth.ldap.Filter;
+import org.apache.hive.service.auth.ldap.DirSearch;
+import org.apache.hive.service.auth.ldap.DirSearchFactory;
+import org.apache.hive.service.auth.ldap.FilterFactory;
+import org.apache.hive.service.auth.ldap.GroupFilterFactory;
+import org.apache.hive.service.auth.ldap.LdapUtils;
+import org.apache.hive.service.auth.ldap.UserFilterFactory;
+import org.apache.hive.service.auth.ldap.UserSearchFilterFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvider {
 
-  private static final Logger LOG     = LoggerFactory.getLogger(LdapAuthenticationProviderImpl.class);
+  private static final Logger LOG = LoggerFactory.getLogger(LdapAuthenticationProviderImpl.class);
 
-  private String ldapURL;
-  private String baseDN;
-  private String ldapDomain;
-  private static List<String> groupBases;
-  private static List<String> userBases;
-  private static List<String> userFilter;
-  private static List<String> groupFilter;
-  private String customQuery;
-  private static String guid_attr;
-  private static String groupMembership_attr;
-  private static String groupClass_attr;
-
-  LdapAuthenticationProviderImpl(HiveConf conf) {
-    init(conf);
-  }
-
-  protected void init(HiveConf conf) {
-    ldapURL     = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_URL);
-    baseDN      = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN);
-    ldapDomain  = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN);
-    customQuery = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY);
-    guid_attr   = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY);
-    groupBases  = new ArrayList<String>();
-    userBases   = new ArrayList<String>();
-    userFilter  = new ArrayList<String>();
-    groupFilter = new ArrayList<String>();
-
-    String groupDNPatterns = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN);
-    String groupFilterVal  = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER);
-    String userDNPatterns  = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN);
-    String userFilterVal   = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER);
-    groupMembership_attr   = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPMEMBERSHIP_KEY);
-    groupClass_attr        = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPCLASS_KEY);
-
-    // parse COLON delimited root DNs for users/groups that may or may not be under BaseDN.
-    // Expect the root DNs be fully qualified including the baseDN
-    if (groupDNPatterns != null && groupDNPatterns.trim().length() > 0) {
-      String[] groupTokens = groupDNPatterns.split(":");
-      for (int i = 0; i < groupTokens.length; i++) {
-        if (groupTokens[i].contains(",") && groupTokens[i].contains("=")) {
-          groupBases.add(groupTokens[i]);
-        } else {
-          LOG.warn("Unexpected format for " + HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN
-                       + "..ignoring " + groupTokens[i]);
-        }
-      }
-    } else if (baseDN != null) {
-      groupBases.add(guid_attr + "=%s," + baseDN);
-    }
+  private static final List<FilterFactory> FILTER_FACTORIES = ImmutableList.<FilterFactory>of(
+      new CustomQueryFilterFactory(),
+      new ChainFilterFactory(new UserSearchFilterFactory(), new UserFilterFactory(),
+          new GroupFilterFactory())
+  );
 
-    if (groupFilterVal != null && groupFilterVal.trim().length() > 0) {
-      String[] groups = groupFilterVal.split(",");
-      for (int i = 0; i < groups.length; i++) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Filtered group: " + groups[i]);
-        }
-        groupFilter.add(groups[i]);
-      }
-    }
+  private final HiveConf conf;
+  private final Filter filter;
+  private final DirSearchFactory searchFactory;
 
-    if (userDNPatterns != null && userDNPatterns.trim().length() > 0) {
-      String[] userTokens = userDNPatterns.split(":");
-      for (int i = 0; i < userTokens.length; i++) {
-        if (userTokens[i].contains(",") && userTokens[i].contains("=")) {
-          userBases.add(userTokens[i]);
-        } else {
-          LOG.warn("Unexpected format for " + HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN
-                       + "..ignoring " + userTokens[i]);
-        }
-      }
-    } else if (baseDN != null) {
-      userBases.add(guid_attr + "=%s," + baseDN);
-    }
+  public LdapAuthenticationProviderImpl(HiveConf conf) {
+    this(conf, new LdapSearchFactory());
+  }
 
-    if (userFilterVal != null && userFilterVal.trim().length() > 0) {
-      String[] users = userFilterVal.split(",");
-      for (int i = 0; i < users.length; i++) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Filtered user: " + users[i]);
-        }
-        userFilter.add(users[i]);
-      }
-    }
+  @VisibleForTesting
+  LdapAuthenticationProviderImpl(HiveConf conf, DirSearchFactory searchFactory) {
+    this.conf = conf;
+    this.searchFactory = searchFactory;
+    filter = resolveFilter(conf);
   }
 
   @Override
   public void Authenticate(String user, String password) throws AuthenticationException {
-
-    Hashtable<String, Object> env = new Hashtable<String, Object>();
-    env.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.ldap.LdapCtxFactory");
-    env.put(Context.PROVIDER_URL, ldapURL);
-
-    // If the domain is available in the config, then append it unless domain is
-    // already part of the username. LDAP providers like Active Directory use a
-    // fully qualified user name like foo@bar.com.
-    if (!hasDomain(user) && ldapDomain != null) {
-      user  = user + "@" + ldapDomain;
-    }
-
-    if (password == null || password.isEmpty() || password.getBytes()[0] == 0) {
-      throw new AuthenticationException("Error validating LDAP user:" +
-          " a null or blank password has been provided");
-    }
-
-    env.put(Context.SECURITY_AUTHENTICATION, "simple");
-    env.put(Context.SECURITY_CREDENTIALS, password);
-
-    // setup the security principal
-    String bindDN   = null;
-    DirContext ctx  = null;
-    String userDN   = null;
-    String userName = null;
-    Exception ex    = null;
-
-    if (!isDN(user) && !hasDomain(user) && userBases.size() > 0) {
-      ListIterator<String> listIter = userBases.listIterator();
-      while (listIter.hasNext()) {
-        try {
-          bindDN = listIter.next().replaceAll("%s", user);
-          env.put(Context.SECURITY_PRINCIPAL, bindDN);
-          LOG.debug("Connecting using DN " + bindDN + " at url " + ldapURL);
-          ctx = new InitialDirContext(env);
-          break;
-        } catch (NamingException e) {
-          ex = e;
-        }
-      }
-    } else {
-      env.put(Context.SECURITY_PRINCIPAL, user);
-      LOG.debug("Connecting using principal " + user + " at url " + ldapURL);
-      try {
-        ctx = new InitialDirContext(env);
-      } catch (NamingException e) {
-        ex = e;
-      }
-    }
-
-    if (ctx == null) {
-      LOG.debug("Could not connect to the LDAP Server:Authentication failed for " + user);
-      throw new AuthenticationException("LDAP Authentication failed for user", ex);
-    }
-
-    LOG.debug("Connected using principal=" + user + " at url=" + ldapURL);
+    DirSearch search = null;
     try {
-      if (isDN(user) || hasDomain(user)) {
-        userName = extractName(user);
-      } else {
-        userName = user;
-      }
-
-      // if a custom LDAP query is specified, it takes precedence over other configuration properties.
-      // if the user being authenticated is part of the resultset from the custom query, it succeeds.
-      if (customQuery != null) {
-        List<String> resultList = executeLDAPQuery(ctx, customQuery, baseDN);
-        if (resultList != null) {
-          for (String matchedDN : resultList) {
-            LOG.info("<queried user=" + matchedDN.split(",",2)[0].split("=",2)[1] + ",user=" + user + ">");
-            if (matchedDN.split(",",2)[0].split("=",2)[1].equalsIgnoreCase(user) ||
-                matchedDN.equalsIgnoreCase(user)) {
-              LOG.info("Authentication succeeded based on result set from LDAP query");
-              return;
-            }
-          }
-        }
-        LOG.info("Authentication failed based on result set from custom LDAP query");
-        throw new AuthenticationException("Authentication failed: LDAP query " +
-            "from property returned no data");
-      } else if (userBases.size() > 0) {
-        if (isDN(user)) {
-          userDN = findUserDNByDN(ctx, user);
-        } else {
-          if (userDN == null) {
-            userDN = findUserDNByPattern(ctx, userName);
-          }
-
-          if (userDN == null) {
-            userDN = findUserDNByName(ctx, userName);
-          }
-        }
-
-        // This should not be null because we were allowed to bind with this username
-        // safe check in case we were able to bind anonymously.
-        if (userDN == null) {
-          throw new AuthenticationException("Authentication failed: User search failed");
-        }
-
-        // This section checks if the user satisfies the specified user filter.
-        if (userFilter.size() > 0) {
-          LOG.info("Authenticating user " + user + " using user filter");
-
-          if (userDN != null) {
-            LOG.info("User filter partially satisfied");
-          }
-
-          boolean success = false;
-          for (String filteredUser : userFilter) {
-            if (filteredUser.equalsIgnoreCase(userName)) {
-              LOG.debug("User filter entirely satisfied");
-              success = true;
-              break;
-            }
-          }
-
-          if (!success) {
-            LOG.info("Authentication failed based on user membership");
-            throw new AuthenticationException("Authentication failed: User not a member " +
-                "of specified list");
-          }
-        }
-
-        // This section checks if the user satisfies the specified user filter.
-        if (groupFilter.size() > 0) {
-          LOG.debug("Authenticating user " + user + " using group membership");
-          List<String> userGroups = getGroupsForUser(ctx, userDN);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("User member of :");
-            prettyPrint(userGroups);
-          }
-
-          if (userGroups != null) {
-            for (String elem : userGroups) {
-              String shortName = ((elem.split(","))[0].split("="))[1];
-              if (groupFilter.contains(shortName)) {
-                LOG.info("Authentication succeeded based on group membership");
-                return;
-              }
-            }
-          }
-
-          LOG.debug("Authentication failed: User is not a member of configured groups");
-          throw new AuthenticationException("Authentication failed: User not a member of " +
-              "listed groups");
-        }
-        LOG.info("Authentication succeeded using ldap user search");
-        return;
-      }
-      // Ideally we should not be here. Indicates partially configured LDAP Service.
-      // We allow it for now for backward compatibility.
-      LOG.info("Simple password authentication succeeded");
-    } catch (NamingException e) {
-      throw new AuthenticationException("LDAP Authentication failed for user", e);
+      search = createDirSearch(user, password);
+      applyFilter(search, user);
     } finally {
-      try {
-        if (ctx != null) {
-          ctx.close();
-        }
-      } catch(Exception e) {
-        LOG.warn("Exception when closing LDAP context:" + e.getMessage());
-      }
+      ServiceUtils.cleanup(LOG, search);
     }
   }
 
-  private boolean hasDomain(String userName) {
-    return (ServiceUtils.indexOfDomainMatch(userName) > 0);
-  }
-
-  private static void prettyPrint(List<String> list) {
-    for (String elem : list) {
-      LOG.debug("    " + elem);
+  private DirSearch createDirSearch(String user, String password) throws AuthenticationException {
+    if (StringUtils.isBlank(user)) {
+      throw new AuthenticationException("Error validating LDAP user:"
+          + " a null or blank user name has been provided");
     }
-  }
-
-  private static void prettyPrint(Attributes attrs) {
-    NamingEnumeration<? extends Attribute> set = attrs.getAll();
-    try {
-      NamingEnumeration<?> list = null;
-      while (set.hasMore()) {
-        Attribute attr = set.next();
-        list = attr.getAll();
-        String attrVals = "";
-        while (list.hasMore()) {
-          attrVals += list.next() + "+";
-        }
-        LOG.debug(attr.getID() + ":::" + attrVals);
-      }
-    } catch (Exception e) {
-      System.out.println("Error occurred when reading ldap data:" + e.getMessage());
+    if (StringUtils.isBlank(password) || password.getBytes()[0] == 0) {
+      throw new AuthenticationException("Error validating LDAP user:"
+          + " a null or blank password has been provided");
     }
-  }
-
-  /**
-   * This helper method attempts to find a DN given a unique groupname.
-   * Various LDAP implementations have different keys/properties that store this unique ID.
-   * So the first attempt is to find an entity with objectClass=group && CN=groupName
-   * @param ctx DirContext for the LDAP Connection.
-   * @param baseDN BaseDN for this LDAP directory where the search is to be performed.
-   * @param groupName A unique groupname that is to be located in the LDAP.
-   * @return LDAP DN if the group is found in LDAP, null otherwise.
-   */
-  public static String findGroupDNByName(DirContext ctx, String baseDN, String groupName)
-    throws NamingException {
-    String searchFilter  = "(&(objectClass=" + groupClass_attr + ")(" + guid_attr + "=" + groupName + "))";
-    List<String> results = null;
-
-    results = findDNByName(ctx, baseDN, searchFilter, 2);
-
-    if (results == null) {
-      return null;
-    } else if (results.size() > 1) {
-      //make sure there is not another item available, there should be only 1 match
-      LOG.info("Matched multiple groups for the group: " + groupName + ",returning null");
-      return null;
-    }
-    return results.get(0);
-  }
-
-  /**
-   * This helper method attempts to find an LDAP group entity given a unique name using a
-   * user-defined pattern for GROUPBASE.The list of group bases is defined by the user via property
-   * "hive.server2.authentication.ldap.groupDNPattern" in the hive-site.xml.
-   * Users can use %s where the actual groupname is to be substituted in the LDAP Query.
-   * @param ctx DirContext for the LDAP Connection.
-   * @param groupName A unique groupname that is to be located in the LDAP.
-   * @return LDAP DN of given group if found in the directory, null otherwise.
-   */
-  public static String findGroupDNByPattern(DirContext ctx, String groupName)
-      throws NamingException {
-    return findDNByPattern(ctx, groupName, groupBases);
-  }
-
-  public static String findDNByPattern(DirContext ctx, String name, List<String> nodes)
-      throws NamingException {
-    String searchFilter;
-    String searchBase;
-    SearchResult searchResult = null;
-    NamingEnumeration<SearchResult> results;
-
-    String[] returnAttributes     = new String[0]; // empty set
-    SearchControls searchControls = new SearchControls();
-
-    searchControls.setSearchScope(SearchControls.SUBTREE_SCOPE);
-    searchControls.setReturningAttributes(returnAttributes);
-
-    for (String node : nodes) {
-      searchFilter = "(" + (node.substring(0,node.indexOf(","))).replaceAll("%s", name) + ")";
-      searchBase   = node.split(",",2)[1];
-      results      = ctx.search(searchBase, searchFilter, searchControls);
-
-      if(results.hasMoreElements()) {
-        searchResult = results.nextElement();
-        //make sure there is not another item available, there should be only 1 match
-        if(results.hasMoreElements()) {
-          LOG.warn("Matched multiple entities for the name: " + name);
-          return null;
+    List<String> principals = LdapUtils.createCandidatePrincipals(conf, user);
+    for (Iterator<String> iterator = principals.iterator(); iterator.hasNext();) {
+      String principal = iterator.next();
+      try {
+        return searchFactory.getInstance(conf, principal, password);
+      } catch (AuthenticationException ex) {
+        if (!iterator.hasNext()) {
+          throw ex;
         }
-        return searchResult.getNameInNamespace();
       }
     }
-    return null;
+    throw new AuthenticationException(
+        String.format("No candidate principals for %s was found.", user));
   }
 
-  /**
-   * This helper method attempts to find a DN given a unique username.
-   * Various LDAP implementations have different keys/properties that store this unique userID.
-   * Active Directory has a "sAMAccountName" that appears reliable,openLDAP uses "uid"
-   * So the first attempt is to find an entity with objectClass=person||user where
-   * (uid||sAMAccountName) matches the given username.
-   * The second attempt is to use CN attribute for wild card matching and then match the
-   * username in the DN.
-   * @param ctx DirContext for the LDAP Connection.
-   * @param baseDN BaseDN for this LDAP directory where the search is to be performed.
-   * @param userName A unique userid that is to be located in the LDAP.
-   * @return LDAP DN if the user is found in LDAP, null otherwise.
-   */
-  public static String findUserDNByName(DirContext ctx, String userName)
-      throws NamingException {
-    if (userBases.size() == 0) {
-      return null;
-    }
-
-    String baseFilter    = "(&(|(objectClass=person)(objectClass=user)(objectClass=inetOrgPerson))";
-    String suffix[]      = new String[] {
-                             "(|(uid=" + userName + ")(sAMAccountName=" + userName + ")))",
-                             "(|(cn=*" + userName + "*)))"
-                           };
-
-    String searchFilter           = null;
-    List<String> results          = null;
-    ListIterator<String> listIter = userBases.listIterator();
-
-    for (int i = 0; i < suffix.length; i++) {
-      searchFilter = baseFilter + suffix[i];
-
-      while (listIter.hasNext()) {
-        results = findDNByName(ctx, listIter.next().split(",",2)[1], searchFilter, 2);
-
-        if(results == null) {
-          continue;
-        }
-
-        if(results != null && results.size() > 1) {
-          //make sure there is not another item available, there should be only 1 match
-          LOG.info("Matched multiple users for the user: " + userName + ",returning null");
-          return null;
-        }
-        return results.get(0);
+  private static Filter resolveFilter(HiveConf conf) {
+    for (FilterFactory filterProvider : FILTER_FACTORIES) {
+      Filter filter = filterProvider.getInstance(conf);
+      if (filter != null) {
+        return filter;
       }
     }
     return null;
   }
 
-  /**
-   * This helper method attempts to find a username given a DN.
-   * Various LDAP implementations have different keys/properties that store this unique userID.
-   * Active Directory has a "sAMAccountName" that appears reliable,openLDAP uses "uid"
-   * So the first attempt is to find an entity with objectClass=person||user where
-   * (uid||sAMAccountName) matches the given username.
-   * The second attempt is to use CN attribute for wild card matching and then match the
-   * username in the DN.
-   * @param ctx DirContext for the LDAP Connection.
-   * @param baseDN BaseDN for this LDAP directory where the search is to be performed.
-   * @param userName A unique userid that is to be located in the LDAP.
-   * @return LDAP DN if the user is found in LDAP, null otherwise.
-   */
-  public static String findUserDNByDN(DirContext ctx, String userDN)
-      throws NamingException {
-    if (!isDN(userDN)) {
-      return null;
-    }
-
-    String baseDN        = extractBaseDN(userDN);
-    List<String> results = null;
-    // we are using the first part of the userDN in the search criteria.
-    // We know the DN is legal as we are able to bind with it, this is to confirm that its a user.
-    String searchFilter  = "(&(|(objectClass=person)(objectClass=user)(objectClass=inetOrgPerson))("
-                             +  userDN.substring(0,userDN.indexOf(",")) + "))";
-
-    results = findDNByName(ctx, baseDN, searchFilter, 2);
-
-    if (results == null) {
-      return null;
-    }
-
-    if(results.size() > 1) {
-      //make sure there is not another item available, there should be only 1 match
-      LOG.info("Matched multiple users for the user: " + userDN + ",returning null");
-      return null;
-    }
-    return results.get(0);
-  }
-
-  public static List<String> findDNByName(DirContext ctx, String baseDN,
-      String searchString, int limit) throws NamingException {
-    SearchResult searchResult     = null;
-    List<String> retValues        = null;
-    String matchedDN              = null;
-    SearchControls searchControls = new SearchControls();
-    String[] returnAttributes     = new String[0]; //empty set
-
-    searchControls.setSearchScope(SearchControls.SUBTREE_SCOPE);
-    searchControls.setReturningAttributes(returnAttributes);
-    if (limit > 0) {
-      searchControls.setCountLimit(limit); // limit the result set to limit the size of resultset
-    }
-
-    NamingEnumeration<SearchResult> results = ctx.search(baseDN, searchString, searchControls);
-    while(results.hasMoreElements()) {
-      searchResult = results.nextElement();
-      matchedDN    = searchResult.getNameInNamespace();
-
-      if (retValues == null) {
-        retValues = new ArrayList<String>();
-      }
-      retValues.add(matchedDN);
-    }
-    return retValues;
-  }
-
-  /**
-   * This helper method attempts to find a UserDN given a unique username from a
-   * user-defined pattern for USERBASE. The list of user bases is defined by the user
-   * via property "hive.server2.authentication.ldap.userDNPattern" in the hive-site.xml.
-   * Users can use %s where the actual username is to be subsituted in the LDAP Query.
-   * @param ctx DirContext for the LDAP Connection.
-   * @param userName A unique userid that is to be located in the LDAP.
-   * @return LDAP DN of given user if found in the directory, null otherwise.
-   */
-  public static String findUserDNByPattern(DirContext ctx, String userName)
-      throws NamingException {
-    return findDNByPattern(ctx, userName, userBases);
-  }
-
-  /**
-   * This helper method finds all the groups a given user belongs to.
-   * This method relies on the attribute,configurable via HIVE_SERVER2_PLAIN_LDAP_GROUPMEMBERSHIP_KEY,
-   * being set on the user entry that references the group. The returned list ONLY includes direct
-   * groups the user belongs to. Parent groups of these direct groups are NOT included.
-   * @param ctx DirContext for the LDAP Connection.
-   * @param userDN A unique userDN that is to be located in the LDAP.
-   * @return List of Group DNs the user belongs to, emptylist otherwise.
-   */
-  public static List<String> getGroupsForUser(DirContext ctx, String userDN)
-      throws NamingException {
-    List<String> groupList        = new ArrayList<String>();
-    String user                   = extractName(userDN);
-    String searchFilter           = "(&(objectClass=" + groupClass_attr + ")(|(" +
-                                      groupMembership_attr + "=" + userDN + ")(" +
-                                      groupMembership_attr + "=" + user + ")))";
-    SearchControls searchControls = new SearchControls();
-    NamingEnumeration<SearchResult> results = null;
-    SearchResult result = null;
-    String groupBase = null;
-
-    LOG.debug("getGroupsForUser:searchFilter=" + searchFilter);
-    String[] attrIDs = new String[0];
-    searchControls.setSearchScope(SearchControls.SUBTREE_SCOPE);
-    searchControls.setReturningAttributes(attrIDs);
-
-    ListIterator<String> listIter = groupBases.listIterator();
-    while (listIter.hasNext()) {
-      try {
-        groupBase = listIter.next().split(",", 2)[1];
-        LOG.debug("Searching for groups under " + groupBase);
-        results   = ctx.search(groupBase, searchFilter, searchControls);
-
-        while(results.hasMoreElements()) {
-          result = results.nextElement();
-          LOG.debug("Found Group:" + result.getNameInNamespace());
-          groupList.add(result.getNameInNamespace());
-        }
-      } catch (NamingException e) {
-        LOG.warn("Exception searching for user groups", e);
-      }
-    }
-
-    return groupList;
-  }
-
-  /**
-   * This method helps execute a LDAP query defined by the user via property
-   * "hive.server2.authentication.ldap.customLDAPQuery"
-   * A full LDAP query that LDAP Atn provider uses to execute against LDAP Server.
-   * If this query return a null resultset, the LDAP Provider fails the authentication request.
-   * If the LDAP query returns a list of DNs, a check is performed to confirm one
-   * of the entries is for the user being authenticated.
-   * For example: (&(objectClass=group)(objectClass=top)(instanceType=4)(cn=Domain*))
-   * (&(objectClass=person)(|(sAMAccountName=admin)
-   *                       (|(memberOf=CN=Domain Admins,CN=Users,DC=domain,DC=com)
-   *                         (memberOf=CN=Administrators,CN=Builtin,DC=domain,DC=com))))
-   * @param ctx DirContext to execute this query within.
-   * @param query User-defined LDAP Query string to be used to authenticate users.
-   * @param rootDN BaseDN at which to execute the LDAP query, typically rootDN for the LDAP.
-   * @return List of LDAP DNs returned from executing the LDAP Query.
-   */
-  public static List<String> executeLDAPQuery(DirContext ctx, String query, String rootDN)
-      throws NamingException {
-    if (rootDN == null) {
-      return null;
-    }
-
-    SearchControls searchControls = new SearchControls();
-    List<String> list             = new ArrayList<String>();
-    String[] returnAttributes;
-    if (groupMembership_attr != null) {
-      // retrieve the attributes that are meant to desginate user DNs
-      returnAttributes = new String[] { groupMembership_attr };
-    } else {
-      returnAttributes = new String[0]; //empty set
-    }
-
-    searchControls.setSearchScope(SearchControls.SUBTREE_SCOPE);
-    searchControls.setReturningAttributes(returnAttributes);
-
-    LOG.info("Using a user specified LDAP query for adjudication:" + query + ",baseDN=" + rootDN);
-    NamingEnumeration<SearchResult> results = ctx.search(rootDN, query, searchControls);
-    SearchResult searchResult = null;
-    while(results.hasMoreElements()) {
-      searchResult = results.nextElement();
-      if (groupMembership_attr != null) {
-        Attribute userAttribute = searchResult.getAttributes().get(groupMembership_attr);
-        if (userAttribute != null) {
-          list.add((String)userAttribute.get());
-          continue;
-        }
+  private void applyFilter(DirSearch client, String user) throws AuthenticationException {
+    if (filter != null) {
+      if (LdapUtils.hasDomain(user)) {
+        filter.apply(client, LdapUtils.extractUserName(user));
+      } else {
+        filter.apply(client, user);
       }
-
-      list.add(searchResult.getNameInNamespace());
-      LOG.debug("LDAPAtn:executeLDAPQuery()::Return set size " + list.get(list.size() - 1));
     }
-    return list;
-  }
-
-  public static boolean isDN(String name) {
-    return (name.indexOf("=") > -1);
-  }
-
-  public static String extractName(String dn) {
-    int domainIdx = ServiceUtils.indexOfDomainMatch(dn);
-    if (domainIdx > 0) {
-      return dn.substring(0, domainIdx);
-    }
-
-    if (dn.indexOf("=") > -1) {
-      return dn.substring(dn.indexOf("=") + 1, dn.indexOf(","));
-    }
-    return dn;
-  }
-
-  public static String extractBaseDN(String dn) {
-    if (dn.indexOf(",") > -1) {
-      return dn.substring(dn.indexOf(",") + 1);
-    }
-    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/ChainFilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/ChainFilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/ChainFilterFactory.java
new file mode 100644
index 0000000..e6255e8
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/ChainFilterFactory.java
@@ -0,0 +1,78 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.List;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * A factory that produces a {@link Filter} that is implemented as a chain of other filters.
+ * The chain of filters are created as a result of
+ * {@link #getInstance(org.apache.hadoop.hive.conf.HiveConf) }
+ * method call. The resulting object filters out all users that don't pass <b>all</b>
+ * chained filters. The filters will be applied in the order they are mentioned in the factory
+ * constructor.
+ */
+public class ChainFilterFactory  implements FilterFactory {
+
+  private final List<FilterFactory> chainedFactories;
+
+  /**
+   * Constructs a factory for a chain of filters.
+   *
+   * @param factories The array of factories that will be used to construct a chain of filters.
+   */
+  public ChainFilterFactory(FilterFactory... factories) {
+    this.chainedFactories = ImmutableList.copyOf(factories);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Filter getInstance(HiveConf conf) {
+    List<Filter> filters = new ArrayList<>();
+    for (FilterFactory factory : chainedFactories) {
+      Filter filter = factory.getInstance(conf);
+      if (filter != null) {
+        filters.add(filter);
+      }
+    }
+
+    return filters.isEmpty() ? null : new ChainFilter(ImmutableList.copyOf(filters));
+  }
+
+  private static final class ChainFilter implements Filter {
+
+    private final List<Filter> chainedFilters;
+
+    public ChainFilter(List<Filter> chainedFilters) {
+      this.chainedFilters = chainedFilters;
+    }
+
+    @Override
+    public void apply(DirSearch client, String user) throws AuthenticationException {
+      for (Filter filter : chainedFilters) {
+        filter.apply(client, user);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/CustomQueryFilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/CustomQueryFilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/CustomQueryFilterFactory.java
new file mode 100644
index 0000000..a0708c3
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/CustomQueryFilterFactory.java
@@ -0,0 +1,84 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import com.google.common.base.Strings;
+import java.util.List;
+import javax.naming.NamingException;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A factory for a {@link Filter} based on a custom query.
+ * <br>
+ * The produced filter object filters out all users that are not found in the search result
+ * of the query provided in Hive configuration.
+ * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY
+ */
+public class CustomQueryFilterFactory implements FilterFactory {
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Filter getInstance(HiveConf conf) {
+    String customQuery = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY);
+
+    if (Strings.isNullOrEmpty(customQuery)) {
+      return null;
+    }
+
+    return new CustomQueryFilter(customQuery);
+  }
+
+  private static final class CustomQueryFilter implements Filter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CustomQueryFilter.class);
+
+    private final String query;
+
+    public CustomQueryFilter(String query) {
+      this.query = query;
+    }
+
+    @Override
+    public void apply(DirSearch client, String user) throws AuthenticationException {
+      List<String> resultList;
+      try {
+        resultList = client.executeCustomQuery(query);
+      } catch (NamingException e) {
+        throw new AuthenticationException("LDAP Authentication failed for user", e);
+      }
+      if (resultList != null) {
+        for (String matchedDn : resultList) {
+          String shortUserName = LdapUtils.getShortName(matchedDn);
+          LOG.info("<queried user=" + shortUserName + ",user=" + user + ">");
+          if (shortUserName.equalsIgnoreCase(user) || matchedDn.equalsIgnoreCase(user)) {
+            LOG.info("Authentication succeeded based on result set from LDAP query");
+            return;
+          }
+        }
+      }
+      LOG.info("Authentication failed based on result set from custom LDAP query");
+      throw new AuthenticationException("Authentication failed: LDAP query "
+          + "from property returned no data");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/DirSearch.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/DirSearch.java b/service/src/java/org/apache/hive/service/auth/ldap/DirSearch.java
new file mode 100644
index 0000000..33b6088
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/DirSearch.java
@@ -0,0 +1,52 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.io.Closeable;
+import java.util.List;
+import javax.naming.NamingException;
+
+/**
+ * The object used for executing queries on the Directory Service.
+ */
+public interface DirSearch extends Closeable {
+
+  /**
+   * Finds user's distinguished name.
+   * @param user username
+   * @return DN for the specified username
+   * @throws NamingException
+   */
+  String findUserDn(String user) throws NamingException;
+
+  /**
+   * Finds groups that contain the specified user.
+   * @param userDn user's distinguished name
+   * @return list of groups
+   * @throws NamingException
+   */
+  List<String> findGroupsForUser(String userDn) throws NamingException;
+
+  /**
+   * Executes an arbitrary query.
+   * @param query any query
+   * @return list of names in the namespace
+   * @throws NamingException
+   */
+  List<String> executeCustomQuery(String query) throws NamingException;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/DirSearchFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/DirSearchFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/DirSearchFactory.java
new file mode 100644
index 0000000..e6d9ff9
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/DirSearchFactory.java
@@ -0,0 +1,37 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * A factory for {@code DirSearch}.
+ */
+public interface DirSearchFactory {
+
+  /**
+   * Returns an instance of {@code DirSearch}.
+   * @param conf Hive configuration
+   * @param user username
+   * @param password user password
+   * @return instance of {@code DirSearch}
+   * @throws AuthenticationException
+   */
+  DirSearch getInstance(HiveConf conf, String user, String password) throws AuthenticationException;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/Filter.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/Filter.java b/service/src/java/org/apache/hive/service/auth/ldap/Filter.java
new file mode 100644
index 0000000..fa72ced
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/Filter.java
@@ -0,0 +1,36 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import javax.security.sasl.AuthenticationException;
+
+/**
+ * The object that filters LDAP users.
+ * <br>
+ * The assumption is that this user was already authenticated by a previous bind operation.
+ */
+public interface Filter {
+
+  /**
+   * Applies this filter to the authenticated user.
+   * @param client LDAP client that will be used for execution of LDAP queries.
+   * @param user username
+   * @throws AuthenticationException
+   */
+  void apply(DirSearch client, String user) throws AuthenticationException;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/FilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/FilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/FilterFactory.java
new file mode 100644
index 0000000..0708ccd
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/FilterFactory.java
@@ -0,0 +1,33 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * Factory for the filter.
+ */
+public interface FilterFactory {
+
+  /**
+   * Returns an instance of the corresponding filter.
+   * @param conf Hive properties used to configure the filter.
+   * @return the filter or {@code null} if this filter doesn't support provided set of properties
+   */
+  Filter getInstance(HiveConf conf);
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/GroupFilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/GroupFilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/GroupFilterFactory.java
new file mode 100644
index 0000000..152c4b2
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/GroupFilterFactory.java
@@ -0,0 +1,90 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import javax.naming.NamingException;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A factory for a {@link Filter} based on a list of allowed groups.
+ * <br>
+ * The produced filter object filters out all users that are not members of at least one of
+ * the groups provided in Hive configuration.
+ * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER
+ */
+public final class GroupFilterFactory implements FilterFactory {
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Filter getInstance(HiveConf conf) {
+    Collection<String> groupFilter = conf.getStringCollection(
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER.varname);
+
+    if (groupFilter.isEmpty()) {
+      return null;
+    }
+
+    return new GroupFilter(groupFilter);
+  }
+
+  private static final class GroupFilter implements Filter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(GroupFilter.class);
+
+    private final Set<String> groupFilter = new HashSet<>();
+
+    GroupFilter(Collection<String> groupFilter) {
+      this.groupFilter.addAll(groupFilter);
+    }
+
+    @Override
+    public void apply(DirSearch ldap, String user) throws AuthenticationException {
+      LOG.info("Authenticating user '{}' using group membership", user);
+
+      List<String> memberOf = null;
+
+      try {
+        String userDn = ldap.findUserDn(user);
+        memberOf = ldap.findGroupsForUser(userDn);
+        LOG.debug("User {} member of : {}", userDn, memberOf);
+      } catch (NamingException e) {
+        throw new AuthenticationException("LDAP Authentication failed for user", e);
+      }
+
+      for (String groupDn : memberOf) {
+        String shortName = LdapUtils.getShortName(groupDn);
+        if (groupFilter.contains(shortName)) {
+          LOG.info("Authentication succeeded based on group membership");
+          return;
+        }
+      }
+      LOG.info("Authentication failed based on user membership");
+      throw new AuthenticationException("Authentication failed: "
+          + "User not a member of specified list");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/LdapSearch.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/LdapSearch.java b/service/src/java/org/apache/hive/service/auth/ldap/LdapSearch.java
new file mode 100644
index 0000000..65076ea
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/LdapSearch.java
@@ -0,0 +1,155 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.SearchResult;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements search for LDAP.
+ */
+public final class LdapSearch implements DirSearch {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LdapSearch.class);
+
+  private final String baseDn;
+  private final List<String> groupBases;
+  private final List<String> userBases;
+  private final List<String> userPatterns;
+
+  private final QueryFactory queries;
+
+  private final DirContext ctx;
+
+  /**
+   * Construct an instance of {@code LdapSearch}.
+   * @param conf Hive configuration
+   * @param ctx Directory service that will be used for the queries.
+   * @throws NamingException
+   */
+  public LdapSearch(HiveConf conf, DirContext ctx) throws NamingException {
+    baseDn = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN);
+    userPatterns = LdapUtils.parseDnPatterns(conf,
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN);
+    groupBases = LdapUtils.patternsToBaseDns(LdapUtils.parseDnPatterns(conf,
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN));
+    userBases = LdapUtils.patternsToBaseDns(userPatterns);
+    this.ctx = ctx;
+    queries = new QueryFactory(conf);
+  }
+
+  /**
+   * Closes this search object and releases any system resources associated
+   * with it. If the search object is already closed then invoking this
+   * method has no effect.
+   */
+  @Override
+  public void close() {
+    try {
+      ctx.close();
+    } catch (NamingException e) {
+      LOG.warn("Exception when closing LDAP context:", e);
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public String findUserDn(String user) throws NamingException {
+    List<String> allLdapNames;
+    if (LdapUtils.isDn(user)) {
+      String userBaseDn = LdapUtils.extractBaseDn(user);
+      String userRdn = LdapUtils.extractFirstRdn(user);
+      allLdapNames = execute(Collections.singletonList(userBaseDn),
+          queries.findUserDnByRdn(userRdn)).getAllLdapNames();
+    } else {
+      allLdapNames = findDnByPattern(userPatterns, user);
+      if (allLdapNames.isEmpty()) {
+        allLdapNames = execute(userBases, queries.findUserDnByName(user)).getAllLdapNames();
+      }
+    }
+
+    if (allLdapNames.size() == 1) {
+      return allLdapNames.get(0);
+    } else {
+      LOG.info("Expected exactly one user result for the user: {}, but got {}. Returning null",
+          user, allLdapNames.size());
+      LOG.debug("Matched users: {}", allLdapNames);
+      return null;
+    }
+  }
+
+  private List<String> findDnByPattern(List<String> patterns, String name) throws NamingException {
+    for (String pattern : patterns) {
+      String baseDnFromPattern = LdapUtils.extractBaseDn(pattern);
+      String rdn = LdapUtils.extractFirstRdn(pattern).replaceAll("%s", name);
+      List<String> list = execute(Collections.singletonList(baseDnFromPattern),
+          queries.findDnByPattern(rdn)).getAllLdapNames();
+      if (!list.isEmpty()) {
+        return list;
+      }
+    }
+    return Collections.emptyList();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public List<String> findGroupsForUser(String userDn) throws NamingException {
+    String userName = LdapUtils.extractUserName(userDn);
+    return execute(groupBases, queries.findGroupsForUser(userName, userDn)).getAllLdapNames();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public List<String> executeCustomQuery(String query) throws NamingException {
+    return execute(Collections.singletonList(baseDn), queries.customQuery(query))
+        .getAllLdapNamesAndAttributes();
+  }
+
+  private SearchResultHandler execute(Collection<String> baseDns, Query query) {
+    List<NamingEnumeration<SearchResult>> searchResults = new ArrayList<>();
+    LOG.debug("Executing a query: '{}' with base DNs {}.", query.getFilter(), baseDns);
+    for (String aBaseDn : baseDns) {
+      try {
+        NamingEnumeration<SearchResult> searchResult = ctx.search(aBaseDn, query.getFilter(),
+            query.getControls());
+        if (searchResult != null) {
+          searchResults.add(searchResult);
+        }
+      } catch (NamingException ex) {
+        LOG.debug("Exception happened for query '" + query.getFilter() +
+            "' with base DN '" + aBaseDn + "'", ex);
+      }
+    }
+    return new SearchResultHandler(searchResults);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/LdapSearchFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/LdapSearchFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/LdapSearchFactory.java
new file mode 100644
index 0000000..71c3bfe
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/LdapSearchFactory.java
@@ -0,0 +1,64 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.Hashtable;
+import javax.naming.Context;
+import javax.naming.NamingException;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.InitialDirContext;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A factory for LDAP search objects.
+ */
+public final class LdapSearchFactory implements DirSearchFactory {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LdapSearchFactory.class);
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public DirSearch getInstance(HiveConf conf, String principal, String password)
+      throws AuthenticationException {
+    try {
+      DirContext ctx = createDirContext(conf, principal, password);
+      return new LdapSearch(conf, ctx);
+    } catch (NamingException e) {
+      LOG.debug("Could not connect to the LDAP Server:Authentication failed for {}", principal);
+      throw new AuthenticationException("Error validating LDAP user", e);
+    }
+  }
+
+  private static DirContext createDirContext(HiveConf conf, String principal, String password)
+      throws NamingException {
+    Hashtable<String, Object> env = new Hashtable<String, Object>();
+    String ldapUrl = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_URL);
+    env.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.ldap.LdapCtxFactory");
+    env.put(Context.PROVIDER_URL, ldapUrl);
+    env.put(Context.SECURITY_AUTHENTICATION, "simple");
+    env.put(Context.SECURITY_CREDENTIALS, password);
+    env.put(Context.SECURITY_PRINCIPAL, principal);
+    LOG.debug("Connecting using principal {} to ldap url {}", principal, ldapUrl);
+    return new InitialDirContext(env);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/LdapUtils.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/LdapUtils.java b/service/src/java/org/apache/hive/service/auth/ldap/LdapUtils.java
new file mode 100644
index 0000000..df2ba6b
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/LdapUtils.java
@@ -0,0 +1,228 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.service.ServiceUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Static utility methods related to LDAP authentication module.
+ */
+public final class LdapUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LdapUtils.class);
+
+  /**
+   * Extracts a base DN from the provided distinguished name.
+   * <br>
+   * <b>Example:</b>
+   * <br>
+   * "ou=CORP,dc=mycompany,dc=com" is the base DN for "cn=user1,ou=CORP,dc=mycompany,dc=com"
+   *
+   * @param dn distinguished name
+   * @return base DN
+   */
+  public static String extractBaseDn(String dn) {
+    final int indexOfFirstDelimiter = dn.indexOf(",");
+    if (indexOfFirstDelimiter > -1) {
+      return dn.substring(indexOfFirstDelimiter + 1);
+    }
+    return null;
+  }
+
+  /**
+   * Extracts the first Relative Distinguished Name (RDN).
+   * <br>
+   * <b>Example:</b>
+   * <br>
+   * For DN "cn=user1,ou=CORP,dc=mycompany,dc=com" this method will return "cn=user1"
+   * @param dn distinguished name
+   * @return first RDN
+   */
+  public static String extractFirstRdn(String dn) {
+    return dn.substring(0, dn.indexOf(","));
+  }
+
+  /**
+   * Extracts username from user DN.
+   * <br>
+   * <b>Examples:</b>
+   * <pre>
+   * LdapUtils.extractUserName("UserName")                        = "UserName"
+   * LdapUtils.extractUserName("UserName@mycorp.com")             = "UserName"
+   * LdapUtils.extractUserName("cn=UserName,dc=mycompany,dc=com") = "UserName"
+   * </pre>
+   * @param userDn
+   * @return
+   */
+  public static String extractUserName(String userDn) {
+    if (!isDn(userDn) && !hasDomain(userDn)) {
+      return userDn;
+    }
+
+    int domainIdx = ServiceUtils.indexOfDomainMatch(userDn);
+    if (domainIdx > 0) {
+      return userDn.substring(0, domainIdx);
+    }
+
+    if (userDn.contains("=")) {
+      return userDn.substring(userDn.indexOf("=") + 1, userDn.indexOf(","));
+    }
+    return userDn;
+  }
+
+  /**
+   * Gets value part of the first attribute in the provided RDN.
+   * <br>
+   * <b>Example:</b>
+   * <br>
+   * For RDN "cn=user1,ou=CORP" this method will return "user1"
+   * @param rdn Relative Distinguished Name
+   * @return value part of the first attribute
+   */
+  public static String getShortName(String rdn) {
+    return ((rdn.split(","))[0].split("="))[1];
+  }
+
+  /**
+   * Check for a domain part in the provided username.
+   * <br>
+   * <b>Example:</b>
+   * <br>
+   * <pre>
+   * LdapUtils.hasDomain("user1@mycorp.com") = true
+   * LdapUtils.hasDomain("user1")            = false
+   * </pre>
+   * @param userName username
+   * @return true if {@code userName} contains {@code @<domain>} part
+   */
+  public static boolean hasDomain(String userName) {
+    return (ServiceUtils.indexOfDomainMatch(userName) > 0);
+  }
+
+  /**
+   * Detects DN names.
+   * <br>
+   * <b>Example:</b>
+   * <br>
+   * <pre>
+   * LdapUtils.isDn("cn=UserName,dc=mycompany,dc=com") = true
+   * LdapUtils.isDn("user1")                           = false
+   * </pre>
+   * @param name name to be checked
+   * @return true if the provided name is a distinguished name
+   */
+  public static boolean isDn(String name) {
+    return name.contains("=");
+  }
+
+  /**
+   * Reads and parses DN patterns from Hive configuration.
+   * <br>
+   * If no patterns are provided in the configuration, then the base DN will be used.
+   * @param conf Hive configuration
+   * @param var variable to be read
+   * @return a list of DN patterns
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN
+   */
+  public static List<String> parseDnPatterns(HiveConf conf, HiveConf.ConfVars var) {
+    String patternsString = conf.getVar(var);
+    List<String> result = new ArrayList<>();
+    if (StringUtils.isBlank(patternsString)) {
+      String defaultBaseDn = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN);
+      String guidAttr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY);
+      if (StringUtils.isNotBlank(defaultBaseDn)) {
+        result.add(guidAttr + "=%s," + defaultBaseDn);
+      }
+    } else {
+      String[] patterns = patternsString.split(":");
+      for (String pattern : patterns) {
+        if (pattern.contains(",") && pattern.contains("=")) {
+          result.add(pattern);
+        } else {
+          LOG.warn("Unexpected format for " + var + "..ignoring " + pattern);
+        }
+      }
+    }
+    return result;
+  }
+
+  private static String patternToBaseDn(String pattern) {
+    if (pattern.contains("=%s")) {
+      return pattern.split(",", 2)[1];
+    }
+    return pattern;
+  }
+
+  /**
+   * Converts a collection of Distinguished Name patterns to a collection of base DNs.
+   * @param patterns Distinguished Name patterns
+   * @return a list of base DNs
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN
+   */
+  public static List<String> patternsToBaseDns(Collection<String> patterns) {
+    List<String> result = new ArrayList<>();
+    for (String pattern : patterns) {
+      result.add(patternToBaseDn(pattern));
+    }
+    return result;
+  }
+
+  /**
+   * Creates a list of principals to be used for user authentication.
+   * @param conf Hive configuration
+   * @param user username
+   * @return a list of user's principals
+   */
+  public static List<String> createCandidatePrincipals(HiveConf conf, String user) {
+    if (hasDomain(user) || isDn(user)) {
+      return Collections.singletonList(user);
+    }
+
+    String ldapDomain = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN);
+    if (StringUtils.isNotBlank(ldapDomain)) {
+      return Collections.singletonList(user + "@" + ldapDomain);
+    }
+
+    List<String> userPatterns = parseDnPatterns(conf,
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN);
+    if (userPatterns.isEmpty()) {
+      return Collections.singletonList(user);
+    }
+
+    List<String> candidatePrincipals = new ArrayList<>();
+    for (String userPattern : userPatterns) {
+      candidatePrincipals.add(userPattern.replaceAll("%s", user));
+    }
+    return candidatePrincipals;
+  }
+
+  private LdapUtils() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/Query.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/Query.java b/service/src/java/org/apache/hive/service/auth/ldap/Query.java
new file mode 100644
index 0000000..b8bf938
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/Query.java
@@ -0,0 +1,154 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.List;
+import javax.naming.directory.SearchControls;
+import org.stringtemplate.v4.ST;
+
+/**
+ * The object that encompasses all components of a Directory Service search query.
+ * <br>
+ * @see LdapSearch
+ */
+public final class Query {
+
+  private final String filter;
+  private final SearchControls controls;
+
+  /**
+   * Constructs an instance of Directory Service search query.
+   * @param filter search filter
+   * @param controls search controls
+   */
+  public Query(String filter, SearchControls controls) {
+    this.filter = filter;
+    this.controls = controls;
+  }
+
+  /**
+   * Returns search filter.
+   * @return search filter
+   */
+  public String getFilter() {
+    return filter;
+  }
+
+  /**
+   * Returns search controls.
+   * @return search controls
+   */
+  public SearchControls getControls() {
+    return controls;
+  }
+
+  /**
+   * Creates Query Builder.
+   * @return query builder.
+   */
+  public static QueryBuilder builder() {
+    return new QueryBuilder();
+  }
+
+  /**
+   * A builder of the {@link Query}.
+   */
+  public static final class QueryBuilder {
+
+    private ST filterTemplate;
+    private final SearchControls controls = new SearchControls();
+    private final List<String> returningAttributes = new ArrayList<>();
+
+    private QueryBuilder() {
+      controls.setSearchScope(SearchControls.SUBTREE_SCOPE);
+      controls.setReturningAttributes(new String[0]);
+    }
+
+    /**
+     * Sets search filter template.
+     * @param filterTemplate search filter template
+     * @return the current instance of the builder
+     */
+    public QueryBuilder filter(String filterTemplate) {
+      this.filterTemplate = new ST(filterTemplate);
+      return this;
+    }
+
+    /**
+     * Sets mapping between names in the search filter template and actual values.
+     * @param key marker in the search filter template.
+     * @param value actual value
+     * @return the current instance of the builder
+     */
+    public QueryBuilder map(String key, String value) {
+      filterTemplate.add(key, value);
+      return this;
+    }
+
+    /**
+     * Sets attribute that should be returned in results for the query.
+     * @param attributeName attribute name
+     * @return the current instance of the builder
+     */
+    public QueryBuilder returnAttribute(String attributeName) {
+      returningAttributes.add(attributeName);
+      return this;
+    }
+
+    /**
+     * Sets the maximum number of entries to be returned as a result of the search.
+     * <br>
+     * 0 indicates no limit: all entries will be returned.
+     * @param limit The maximum number of entries that will be returned.
+     * @return the current instance of the builder
+     */
+    public QueryBuilder limit(int limit) {
+      controls.setCountLimit(limit);
+      return this;
+    }
+
+    private void validate() {
+      Preconditions.checkArgument(filterTemplate != null,
+          "filter is required for LDAP search query");
+    }
+
+    private String createFilter() {
+      return filterTemplate.render();
+    }
+
+    private void updateControls() {
+      if (!returningAttributes.isEmpty()) {
+        controls.setReturningAttributes(returningAttributes
+            .toArray(new String[returningAttributes.size()]));
+      }
+    }
+
+    /**
+     * Builds an instance of {@link Query}.
+     * @return configured directory service query
+     */
+    public Query build() {
+      validate();
+      String filter = createFilter();
+      updateControls();
+      return new Query(filter, controls);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/QueryFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/QueryFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/QueryFactory.java
new file mode 100644
index 0000000..e9172d3
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/QueryFactory.java
@@ -0,0 +1,135 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import com.google.common.base.Strings;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * A factory for common types of directory service search queries.
+ */
+public final class QueryFactory {
+
+  private final String guidAttr;
+  private final String groupClassAttr;
+  private final String groupMembershipAttr;
+
+  /**
+   * Constructs the factory based on provided Hive configuration.
+   * @param conf Hive configuration
+   */
+  public QueryFactory(HiveConf conf) {
+    guidAttr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY);
+    groupClassAttr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPCLASS_KEY);
+    groupMembershipAttr = conf.getVar(
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPMEMBERSHIP_KEY);
+  }
+
+  /**
+   * Returns a query for finding Group DN based on group unique ID.
+   * @param groupId group unique identifier
+   * @return an instance of {@link Query}
+   */
+  public Query findGroupDnById(String groupId) {
+    return Query.builder()
+        .filter("(&(objectClass=<groupClassAttr>)(<guidAttr>=<groupID>))")
+        .map("guidAttr", guidAttr)
+        .map("groupClassAttr", groupClassAttr)
+        .map("groupID", groupId)
+        .limit(2)
+        .build();
+  }
+
+  /**
+   * Returns a query for finding user DN based on user RDN.
+   * @param userRdn user RDN
+   * @return an instance of {@link Query}
+   */
+  public Query findUserDnByRdn(String userRdn) {
+    return Query.builder()
+        .filter("(&(|(objectClass=person)(objectClass=user)(objectClass=inetOrgPerson))"
+            + "(<userRdn>))")
+        .limit(2)
+        .map("userRdn", userRdn)
+        .build();
+  }
+
+  /**
+   * Returns a query for finding user DN based on DN pattern.
+   * <br>
+   * Name of this method was derived from the original implementation of LDAP authentication.
+   * This method should be replaced by {@link QueryFactory#findUserDnByRdn(java.lang.String).
+   *
+   * @param rdn user RDN
+   * @return an instance of {@link Query}
+   */
+  public Query findDnByPattern(String rdn) {
+    return Query.builder()
+        .filter("(<rdn>)")
+        .map("rdn", rdn)
+        .limit(2)
+        .build();
+  }
+
+  /**
+   * Returns a query for finding user DN based on user unique name.
+   * @param userName user unique name (uid or sAMAccountName)
+   * @return an instance of {@link Query}
+   */
+  public Query findUserDnByName(String userName) {
+    return Query.builder()
+        .filter("(&(|(objectClass=person)(objectClass=user)(objectClass=inetOrgPerson))"
+            + "(|(uid=<userName>)(sAMAccountName=<userName>)))")
+        .map("userName", userName)
+        .limit(2)
+        .build();
+  }
+
+  /**
+   * Returns a query for finding groups to which the user belongs.
+   * @param userName username
+   * @param userDn user DN
+   * @return an instance of {@link Query}
+   */
+  public Query findGroupsForUser(String userName, String userDn) {
+    return Query.builder()
+        .filter("(&(objectClass=<groupClassAttr>)(|(<groupMembershipAttr>=<userDn>)"
+            + "(<groupMembershipAttr>=<userName>)))")
+        .map("groupClassAttr", groupClassAttr)
+        .map("groupMembershipAttr", groupMembershipAttr)
+        .map("userName", userName)
+        .map("userDn", userDn)
+        .build();
+  }
+
+  /**
+   * Returns a query object created for the custom filter.
+   * <br>
+   * This query is configured to return a group membership attribute as part of the search result.
+   * @param searchFilter custom search filter
+   * @return an instance of {@link Query}
+   */
+  public Query customQuery(String searchFilter) {
+    Query.QueryBuilder builder = Query.builder();
+    builder.filter(searchFilter);
+    if (!Strings.isNullOrEmpty(groupMembershipAttr)) {
+      builder.returnAttribute(groupMembershipAttr);
+    }
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/SearchResultHandler.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/SearchResultHandler.java b/service/src/java/org/apache/hive/service/auth/ldap/SearchResultHandler.java
new file mode 100644
index 0000000..1b391f8
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/SearchResultHandler.java
@@ -0,0 +1,163 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.SearchResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The object that handles Directory Service search results.
+ * In most cases it converts search results into a list of names in the namespace.
+ */
+public final class SearchResultHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SearchResultHandler.class);
+
+  private final Collection<NamingEnumeration<SearchResult>> searchResults;
+
+  /**
+   * Constructs a search result handler object for the provided search results.
+   * @param searchResults directory service search results
+   */
+  public SearchResultHandler(Collection<NamingEnumeration<SearchResult>> searchResults) {
+    this.searchResults = searchResults;
+  }
+
+  /**
+   * Returns all entries from the search result.
+   * @return a list of names in the namespace
+   * @throws NamingException
+   */
+  public List<String> getAllLdapNames() throws NamingException {
+    final List<String> result = new ArrayList<>();
+    handle(new RecordProcessor() {
+      @Override
+      public boolean process(SearchResult record) throws NamingException {
+        result.add(record.getNameInNamespace());
+        return true;
+      }
+    });
+    return result;
+  }
+
+  /**
+   * Checks whether search result contains exactly one entry.
+   * @return true if the search result contains a single entry.
+   * @throws NamingException
+   */
+  public boolean hasSingleResult() throws NamingException {
+    List<String> allResults = getAllLdapNames();
+    return allResults != null && allResults.size() == 1;
+  }
+
+  /**
+   * Returns a single entry from the search result.
+   * Throws {@code NamingException} if the search result doesn't contain exactly one entry.
+   * @return name in the namespace
+   * @throws NamingException
+   */
+  public String getSingleLdapName() throws NamingException {
+    List<String> allLdapNames = getAllLdapNames();
+    if (allLdapNames.size() == 1) {
+      return allLdapNames.get(0);
+    }
+    throw new NamingException("Single result was expected");
+  }
+
+  /**
+   * Returns all entries and all attributes for these entries.
+   * @return a list that includes all entries and all attributes from these entries.
+   * @throws NamingException
+   */
+  public List<String> getAllLdapNamesAndAttributes() throws NamingException {
+    final List<String> result = new ArrayList<>();
+    handle(new RecordProcessor() {
+      @Override
+      public boolean process(SearchResult record) throws NamingException {
+        result.add(record.getNameInNamespace());
+        NamingEnumeration<? extends Attribute> allAttributes = record.getAttributes().getAll();
+        while(allAttributes.hasMore()) {
+          Attribute attribute = allAttributes.next();
+          addAllAttributeValuesToResult(attribute.getAll());
+        }
+        return true;
+      }
+
+      private void addAllAttributeValuesToResult(NamingEnumeration values) throws NamingException {
+        while(values.hasMore()) {
+          result.add(String.valueOf(values.next()));
+        }
+      }
+
+    });
+    return result;
+  }
+
+  /**
+   * Allows for custom processing of the search results.
+   * @param processor {@link RecordProcessor} implementation
+   * @throws NamingException
+   */
+  public void handle(RecordProcessor processor) throws NamingException {
+    try {
+      for (NamingEnumeration<SearchResult> searchResult : searchResults) {
+        while (searchResult.hasMore()) {
+          if (!processor.process(searchResult.next())) {
+            return;
+          }
+        }
+      }
+    } finally {
+      for (NamingEnumeration<SearchResult> searchResult : searchResults) {
+        try {
+          searchResult.close();
+        } catch (NamingException ex) {
+          LOG.warn("Failed to close LDAP search result", ex);
+        }
+      }
+    }
+  }
+
+  /**
+   * An interface used by {@link SearchResultHandler} for processing records of
+   * a {@link SearchResult} on a per-record basis.
+   * <br>
+   * Implementations of this interface perform the actual work of processing each record,
+   * but don't need to worry about exception handling, closing underlying data structures,
+   * and combining results from several search requests.
+   * {@see SearchResultHandler}
+   */
+  public interface RecordProcessor {
+
+    /**
+     * Implementations must implement this method to process each record in {@link SearchResult}.
+     * @param record the {@code SearchResult} to precess
+     * @return {@code true} to continue processing, {@code false} to stop iterating
+     * over search results
+     * @throws NamingException
+     */
+    boolean process(SearchResult record) throws NamingException;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/UserFilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/UserFilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/UserFilterFactory.java
new file mode 100644
index 0000000..c8a6c88
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/UserFilterFactory.java
@@ -0,0 +1,75 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A factory for a {@link Filter} based on a list of allowed users.
+ * <br>
+ * The produced filter object filters out all users that are not on the provided in
+ * Hive configuration list.
+ * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER
+ */
+public final class UserFilterFactory implements FilterFactory {
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Filter getInstance(HiveConf conf) {
+    Collection<String> userFilter = conf.getStringCollection(
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER.varname);
+
+    if (userFilter.isEmpty()) {
+      return null;
+    }
+
+    return new UserFilter(userFilter);
+  }
+
+  private static final class UserFilter implements Filter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(UserFilter.class);
+
+    private final Set<String> userFilter = new HashSet<>();
+
+    UserFilter(Collection<String> userFilter) {
+      for (String userFilterItem : userFilter) {
+        this.userFilter.add(userFilterItem.toLowerCase());
+      }
+    }
+
+    @Override
+    public void apply(DirSearch ldap, String user) throws AuthenticationException {
+      LOG.info("Authenticating user '{}' using user filter", user);
+      String userName = LdapUtils.extractUserName(user).toLowerCase();
+      if (!userFilter.contains(userName)) {
+        LOG.info("Authentication failed based on user membership");
+        throw new AuthenticationException("Authentication failed: "
+            + "User not a member of specified list");
+      }
+    }
+  }
+}


[40/44] hive git commit: HIVE-14784: Operation logs are disabled automatically if the parent directory does not exist. (Naveen Gangam via Yongzhi Chen)

Posted by se...@apache.org.
HIVE-14784: Operation logs are disabled automatically if the parent directory does not exist. (Naveen Gangam via Yongzhi Chen)


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

Branch: refs/heads/hive-14535
Commit: 0562efce642e70f1ac69eae6cca8c0a63230bafd
Parents: 74a6ff6
Author: Yongzhi Chen <yc...@apache.org>
Authored: Fri Sep 30 10:39:11 2016 -0400
Committer: Yongzhi Chen <yc...@apache.org>
Committed: Fri Sep 30 10:41:37 2016 -0400

----------------------------------------------------------------------
 .../org/apache/hive/service/cli/operation/Operation.java | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0562efce/service/src/java/org/apache/hive/service/cli/operation/Operation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/Operation.java b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
index 90fe76d..6a656f9 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/Operation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
@@ -234,6 +234,17 @@ public abstract class Operation {
               operationLogFile.getAbsolutePath());
           operationLogFile.delete();
         }
+        if (!operationLogFile.getParentFile().exists()) {
+          LOG.warn("Operations log directory for this session does not exist, it could have been deleted " +
+              "externally. Recreating the directory for future queries in this session but the older operation " +
+              "logs for this session are no longer available");
+          if (!operationLogFile.getParentFile().mkdir()) {
+            LOG.warn("Log directory for this session could not be created, disabling " +
+                "operation logs: " + operationLogFile.getParentFile().getAbsolutePath());
+            isOperationLogEnabled = false;
+            return;
+          }
+        }
         if (!operationLogFile.createNewFile()) {
           // the log file already exists and cannot be deleted.
           // If it can be read/written, keep its contents and use it.


[02/44] hive git commit: HIVE-14790: Jenkins is not displaying test results because 'set -e' is aborting the script too soon (Sergio Pena)

Posted by se...@apache.org.
HIVE-14790: Jenkins is not displaying test results because 'set -e' is aborting the script too soon (Sergio Pena)


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

Branch: refs/heads/hive-14535
Commit: eab7b40c7d197f3b4a7ea97fe5424a1100ef2ad5
Parents: 83ef6f9
Author: Sergio Pena <se...@cloudera.com>
Authored: Mon Sep 19 18:26:35 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Mon Sep 19 18:27:21 2016 -0500

----------------------------------------------------------------------
 dev-support/jenkins-execute-build.sh | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/eab7b40c/dev-support/jenkins-execute-build.sh
----------------------------------------------------------------------
diff --git a/dev-support/jenkins-execute-build.sh b/dev-support/jenkins-execute-build.sh
index a9935e1..b2ba8e9 100644
--- a/dev-support/jenkins-execute-build.sh
+++ b/dev-support/jenkins-execute-build.sh
@@ -118,6 +118,8 @@ else
 	echo "ISSUE: unspecified PROFILE: $BUILD_PROFILE"
 fi
 
+set +e
+
 call_ptest_server --testHandle "$TEST_HANDLE" --endpoint "$PTEST_API_ENDPOINT" --logsEndpoint "$PTEST_LOG_ENDPOINT" \
 	--profile "$BUILD_PROFILE" ${optionalArgs[@]} "$@"
 


[26/44] hive git commit: HIVE-14358: Add metrics for number of queries executed for each execution engine (Barna Zsombor Klara, reviewed by Gabor Szadovszky, Yongzhi Chen)

Posted by se...@apache.org.
HIVE-14358: Add metrics for number of queries executed for each execution engine (Barna Zsombor Klara, reviewed by Gabor Szadovszky, Yongzhi Chen)


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

Branch: refs/heads/hive-14535
Commit: 0c55d46f2afdc7c282304839a10ac39221520316
Parents: 737fd09
Author: Yongzhi Chen <yc...@apache.org>
Authored: Mon Sep 26 13:55:28 2016 -0400
Committer: Yongzhi Chen <yc...@apache.org>
Committed: Tue Sep 27 09:23:16 2016 -0400

----------------------------------------------------------------------
 .../common/metrics/common/MetricsConstant.java  |  7 +++
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  5 +++
 .../org/apache/hadoop/hive/ql/exec/Task.java    |  9 ++++
 .../hadoop/hive/ql/exec/mr/MapRedTask.java      | 11 +++++
 .../hadoop/hive/ql/exec/mr/MapredLocalTask.java | 11 +++++
 .../hadoop/hive/ql/exec/spark/SparkTask.java    | 11 +++++
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java | 11 +++++
 .../hadoop/hive/ql/exec/mr/TestMapRedTask.java  | 47 ++++++++++++++++++++
 .../hive/ql/exec/mr/TestMapredLocalTask.java    | 46 +++++++++++++++++++
 .../hive/ql/exec/spark/TestSparkTask.java       | 46 +++++++++++++++++++
 .../hadoop/hive/ql/exec/tez/TestTezTask.java    | 17 +++++++
 11 files changed, 221 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
index 9dc96f9..c9d4087 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
@@ -61,4 +61,11 @@ public class MetricsConstant {
   // The number of Hive operations that are waiting to enter the compile block
   public static final String WAITING_COMPILE_OPS = "waiting_compile_ops";
 
+  // The number of map reduce tasks executed by the HiveServer2 since the last restart
+  public static final String HIVE_MR_TASKS = "hive_mapred_tasks";
+  // The number of spark tasks executed by the HiveServer2 since the last restart
+  public static final String HIVE_SPARK_TASKS = "hive_spark_tasks";
+  // The number of tez tasks executed by the HiveServer2 since the last restart
+  public static final String HIVE_TEZ_TASKS = "hive_tez_tasks";
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 42d398d..03c56e1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -1671,6 +1671,11 @@ public class Driver implements CommandProcessor {
         // incorrect results.
         assert tsk.getParentTasks() == null || tsk.getParentTasks().isEmpty();
         driverCxt.addToRunnable(tsk);
+
+        Metrics metrics = MetricsFactory.getInstance();
+        if (metrics != null) {
+          tsk.updateTaskMetrics(metrics);
+        }
       }
 
       perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.RUN_TASKS);

http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
index eeaa543..e1bd291 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
@@ -27,6 +27,8 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
@@ -534,6 +536,13 @@ public abstract class Task<T extends Serializable> implements Serializable, Node
     }
   }
 
+  /**
+   * Provide metrics on the type and number of tasks executed by the HiveServer
+   * @param metrics
+   */
+  public void updateTaskMetrics(Metrics metrics) {
+    // no metrics gathered by default
+   }
 
   public int getTaskTag() {
     return taskTag;

http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
index ce1106d9..f48d511 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.io.CachingPrintStream;
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.Context;
@@ -371,6 +373,15 @@ public class MapRedTask extends ExecDriver implements Serializable {
     return runningViaChild ? done() : b;
   }
 
+  @Override
+  public void updateTaskMetrics(Metrics metrics) {
+    try {
+      metrics.incrementCounter(MetricsConstant.HIVE_MR_TASKS);
+    } catch (IOException ex) {
+      LOG.warn("Could not increment metrics for " + this, ex);
+    }
+  }
+
   /**
    * Set the number of reducers for the mapred work.
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
index 48d2540..f81fc71 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
@@ -43,6 +43,8 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.LogUtils;
 import org.apache.hadoop.hive.common.io.CachingPrintStream;
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
@@ -123,6 +125,15 @@ public class MapredLocalTask extends Task<MapredLocalWork> implements Serializab
   }
 
   @Override
+  public void updateTaskMetrics(Metrics metrics) {
+    try {
+      metrics.incrementCounter(MetricsConstant.HIVE_MR_TASKS);
+    } catch (IOException ex) {
+      LOG.warn("Could not increment metrics for " + this, ex);
+    }
+  }
+
+  @Override
   public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext driverContext,
       CompilationOpContext opContext) {
     super.initialize(queryState, queryPlan, driverContext, opContext);

http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
index 0b494aa..72c8bf7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
@@ -26,6 +26,8 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -180,6 +182,15 @@ public class SparkTask extends Task<SparkWork> {
   }
 
   @Override
+  public void updateTaskMetrics(Metrics metrics) {
+    try {
+      metrics.incrementCounter(MetricsConstant.HIVE_SPARK_TASKS);
+    } catch (IOException ex) {
+      LOG.warn("Could not increment metrics for " + this, ex);
+    }
+  }
+
+  @Override
   public boolean isMapRedTask() {
     return true;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
index 25c4514..c51c92f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
@@ -35,6 +35,8 @@ import javax.annotation.Nullable;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
@@ -529,6 +531,15 @@ public class TezTask extends Task<TezWork> {
   }
 
   @Override
+  public void updateTaskMetrics(Metrics metrics) {
+    try {
+      metrics.incrementCounter(MetricsConstant.HIVE_TEZ_TASKS);
+    } catch (IOException ex) {
+      LOG.warn("Could not increment metrics for " + this, ex);
+    }
+  }
+
+  @Override
   public boolean isMapRedTask() {
     return true;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java
new file mode 100644
index 0000000..5ec7c0d
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java
@@ -0,0 +1,47 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hive.ql.exec.mr;
+
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
+import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestMapRedTask {
+
+  @Test
+  public void mrTask_updates_Metrics() throws IOException {
+
+    Metrics mockMetrics = Mockito.mock(Metrics.class);
+
+    MapRedTask mapRedTask = new MapRedTask();
+    mapRedTask.updateTaskMetrics(mockMetrics);
+
+    verify(mockMetrics, times(1)).incrementCounter(MetricsConstant.HIVE_MR_TASKS);
+    verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_TEZ_TASKS);
+    verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_SPARK_TASKS);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapredLocalTask.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapredLocalTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapredLocalTask.java
new file mode 100644
index 0000000..4a0fafe
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapredLocalTask.java
@@ -0,0 +1,46 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hive.ql.exec.mr;
+
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestMapredLocalTask {
+
+  @Test
+  public void localMRTask_updates_Metrics() throws IOException {
+
+    Metrics mockMetrics = Mockito.mock(Metrics.class);
+
+    MapredLocalTask localMrTask = new MapredLocalTask();
+    localMrTask.updateTaskMetrics(mockMetrics);
+
+    verify(mockMetrics, times(1)).incrementCounter(MetricsConstant.HIVE_MR_TASKS);
+    verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_TEZ_TASKS);
+    verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_SPARK_TASKS);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java
new file mode 100644
index 0000000..4c7ec76
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java
@@ -0,0 +1,46 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hive.ql.exec.spark;
+
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestSparkTask {
+
+  @Test
+  public void sparkTask_updates_Metrics() throws IOException {
+
+    Metrics mockMetrics = Mockito.mock(Metrics.class);
+
+    SparkTask sparkTask = new SparkTask();
+    sparkTask.updateTaskMetrics(mockMetrics);
+
+    verify(mockMetrics, times(1)).incrementCounter(MetricsConstant.HIVE_SPARK_TASKS);
+    verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_TEZ_TASKS);
+    verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_MR_TASKS);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
index 53672a9..5c012f3 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
@@ -27,6 +27,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.never;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -39,6 +40,8 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.exec.Operator;
@@ -67,6 +70,7 @@ import org.apache.tez.dag.api.client.DAGClient;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -326,4 +330,17 @@ public class TestTezTask {
     heapSize = DagUtils.parseRightmostXmx(javaOpts);
     assertEquals("Unexpected maximum heap size", -1, heapSize);
   }
+
+  @Test
+  public void tezTask_updates_Metrics() throws IOException {
+
+    Metrics mockMetrics = Mockito.mock(Metrics.class);
+
+    TezTask tezTask = new TezTask();
+    tezTask.updateTaskMetrics(mockMetrics);
+
+    verify(mockMetrics, times(1)).incrementCounter(MetricsConstant.HIVE_TEZ_TASKS);
+    verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_SPARK_TASKS);
+    verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_MR_TASKS);
+  }
 }


[33/44] hive git commit: HIVE-14824. Separate fstype from cluster type in QTestUtil. (Siddharth Seth, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-14824. Separate fstype from cluster type in QTestUtil. (Siddharth Seth, 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/cf72a737
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/cf72a737
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/cf72a737

Branch: refs/heads/hive-14535
Commit: cf72a73708b00b2e585d101258d95eb9cbd3791f
Parents: e2bd513
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Sep 28 13:40:04 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed Sep 28 13:40:04 2016 -0700

----------------------------------------------------------------------
 .../hive/cli/control/AbstractCliConfig.java     | 10 +++
 .../hadoop/hive/cli/control/CliConfigs.java     | 12 ++-
 .../hadoop/hive/cli/control/CoreCliDriver.java  |  3 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 84 +++++++++++---------
 4 files changed, 69 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cf72a737/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
index efbd465..03d4075 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
@@ -35,6 +35,7 @@ import java.util.regex.Pattern;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.ql.QTestUtil;
+import org.apache.hadoop.hive.ql.QTestUtil.FsType;
 import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
 import com.google.common.base.Splitter;
 import com.google.common.collect.Sets;
@@ -63,6 +64,7 @@ public abstract class AbstractCliConfig {
   private String initScript;
   private String hiveConfDir;
   private MiniClusterType clusterType;
+  private FsType fsType;
 
   // FIXME: null value is treated differently on the other end..when those filter will be
   // moved...this may change
@@ -380,6 +382,14 @@ public abstract class AbstractCliConfig {
     }
   }
 
+  protected FsType getFsType() {
+    return this.fsType;
+  }
+
+  protected void setFsType(FsType fsType) {
+    this.fsType = fsType;
+  }
+
   private String getSysPropValue(String propName) {
     String propValue = System.getProperty(propName);
     if (propValue == null || propValue.trim().length() == 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/cf72a737/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
index 0068b95..ca72282 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
@@ -21,6 +21,7 @@ import java.io.File;
 import java.net.MalformedURLException;
 import java.net.URL;
 
+import org.apache.hadoop.hive.ql.QTestUtil;
 import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
 import org.apache.hadoop.hive.ql.parse.CoreParseNegative;
 
@@ -171,8 +172,15 @@ public class CliConfigs {
         setInitScript("q_test_init_for_encryption.sql");
         setCleanupScript("q_test_cleanup_for_encryption.sql");
 
-        setHiveConfDir("data/conf");
-        setClusterType(MiniClusterType.encrypted);
+
+        setClusterType(MiniClusterType.mr);
+        setFsType(QTestUtil.FsType.encrypted_hdfs);
+        if (getClusterType() == MiniClusterType.tez) {
+          setHiveConfDir("data/conf/tez");
+        } else {
+          setHiveConfDir("data/conf");
+        }
+
       } catch (Exception e) {
         throw new RuntimeException("can't construct cliconfig", e);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/cf72a737/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
index db58f1d..d83ff45 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
@@ -61,7 +61,8 @@ public class CoreCliDriver extends CliAdapter {
         @Override
         public QTestUtil invokeInternal() throws Exception {
           return new QTestUtil((cliConfig.getResultsDir()), (cliConfig.getLogDir()), miniMR,
-              hiveConfDir, hadoopVer, initScript, cleanupScript, useHBaseMetastore, true);
+              hiveConfDir, hadoopVer, initScript, cleanupScript, useHBaseMetastore, true, false,
+              cliConfig.getFsType());
         }
       }.invoke("QtestUtil instance created", LOG, true);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/cf72a737/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 0dfd727..909d7f6 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -51,7 +51,6 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.Deque;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -86,9 +85,7 @@ import org.apache.hadoop.hive.common.io.SortPrintStream;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.LlapItUtils;
-import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration;
 import org.apache.hadoop.hive.llap.daemon.MiniLlapCluster;
-import org.apache.hadoop.hive.llap.daemon.impl.LlapDaemon;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.Index;
@@ -155,7 +152,6 @@ public class QTestUtil {
 
   private String testWarehouse;
   private final String testFiles;
-  private final boolean useLocalFs;
   private final boolean localMode;
   protected final String outDir;
   protected final String logDir;
@@ -170,6 +166,7 @@ public class QTestUtil {
   private static final String SORT_SUFFIX = ".sorted";
   private final HashSet<String> srcTables;
   private final MiniClusterType clusterType;
+  private final FsType fsType;
   private ParseDriver pd;
   protected Hive db;
   protected QueryState queryState;
@@ -414,14 +411,32 @@ public class QTestUtil {
     }
   }
 
+
+  public enum FsType {
+    local,
+    hdfs,
+    encrypted_hdfs,
+  }
+
   public enum MiniClusterType {
-    mr,
-    tez,
-    spark,
-    encrypted,
-    miniSparkOnYarn,
-    llap,
-    none;
+
+    mr(FsType.hdfs),
+    tez(FsType.hdfs),
+    spark(FsType.local),
+    miniSparkOnYarn(FsType.hdfs),
+    llap(FsType.hdfs),
+    none(FsType.local);
+
+
+    private final FsType defaultFsType;
+
+    MiniClusterType(FsType defaultFsType) {
+      this.defaultFsType = defaultFsType;
+    }
+
+    public FsType getDefaultFsType() {
+      return defaultFsType;
+    }
 
     public static MiniClusterType valueForString(String type) {
       if (type.equals("miniMR")) {
@@ -430,8 +445,6 @@ public class QTestUtil {
         return tez;
       } else if (type.equals("spark")) {
         return spark;
-      } else if (type.equals("encrypted")) {
-        return encrypted;
       } else if (type.equals("miniSparkOnYarn")) {
         return miniSparkOnYarn;
       } else if (type.equals("llap")) {
@@ -442,6 +455,7 @@ public class QTestUtil {
     }
   }
 
+
   private String getKeyProviderURI() {
     // Use the target directory if it is not specified
     String HIVE_ROOT = AbstractCliConfig.HIVE_ROOT;
@@ -473,19 +487,24 @@ public class QTestUtil {
                    String confDir, String hadoopVer, String initScript, String cleanupScript,
                    boolean useHBaseMetastore, boolean withLlapIo) throws Exception {
     this(outDir, logDir, clusterType, confDir, hadoopVer, initScript, cleanupScript,
-        useHBaseMetastore, withLlapIo, false, false);
+        useHBaseMetastore, withLlapIo, false, null);
   }
 
   public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
       String confDir, String hadoopVer, String initScript, String cleanupScript,
-      boolean useHBaseMetastore, boolean withLlapIo, boolean localMode, boolean useLocalFs)
+      boolean useHBaseMetastore, boolean withLlapIo, boolean localMode, FsType fsType)
     throws Exception {
     LOG.info("Setting up QTestUtil with outDir={}, logDir={}, clusterType={}, confDir={}," +
         " hadoopVer={}, initScript={}, cleanupScript={}, useHbaseMetaStore={}, withLlapIo={}," +
-            " localMode={}, useLocalFs={}"
+            " localMode={}, fsType={}"
         , outDir, logDir, clusterType, confDir, hadoopVer, initScript, cleanupScript,
-        useHBaseMetastore, withLlapIo, localMode, useLocalFs);
-    this.useLocalFs = useLocalFs;
+        useHBaseMetastore, withLlapIo, localMode, fsType);
+    Preconditions.checkNotNull(clusterType, "ClusterType cannot be null");
+    if (fsType != null) {
+      this.fsType = fsType;
+    } else {
+      this.fsType = clusterType.getDefaultFsType();
+    }
     this.localMode = localMode;
     this.outDir = outDir;
     this.logDir = logDir;
@@ -555,16 +574,12 @@ public class QTestUtil {
 
   private void setupFileSystem(HadoopShims shims) throws IOException {
 
-    if (useLocalFs) {
-      Preconditions
-          .checkState(clusterType == MiniClusterType.tez || clusterType == MiniClusterType.llap,
-              "useLocalFs can currently only be set for tez or llap");
-    }
-
-    if (clusterType != MiniClusterType.none && clusterType != MiniClusterType.spark) {
+    if (fsType == FsType.local) {
+      fs = FileSystem.getLocal(conf);
+    } else if (fsType == FsType.hdfs || fsType == FsType.encrypted_hdfs) {
       int numDataNodes = 4;
 
-      if (clusterType == MiniClusterType.encrypted) {
+      if (fsType == FsType.encrypted_hdfs) {
         // Set the security key provider so that the MiniDFS cluster is initialized
         // with encryption
         conf.set(SECURITY_KEY_PROVIDER_URI_NAME, getKeyProviderURI());
@@ -578,16 +593,11 @@ public class QTestUtil {
 
         LOG.info("key provider is initialized");
       } else {
-        if (!useLocalFs) {
-          dfs = shims.getMiniDfs(conf, numDataNodes, true, null);
-          fs = dfs.getFileSystem();
-        } else {
-          fs = FileSystem.getLocal(conf);
-        }
+        dfs = shims.getMiniDfs(conf, numDataNodes, true, null);
+        fs = dfs.getFileSystem();
       }
     } else {
-      // Setup local file system
-      fs = FileSystem.getLocal(conf);
+      throw new IllegalArgumentException("Unknown or unhandled fsType [" + fsType + "]");
     }
   }
 
@@ -621,7 +631,7 @@ public class QTestUtil {
       }
     } else if (clusterType == MiniClusterType.miniSparkOnYarn) {
       mr = shims.getMiniSparkCluster(conf, 4, uriString, 1);
-    } else if (clusterType == MiniClusterType.mr || clusterType == MiniClusterType.encrypted) {
+    } else if (clusterType == MiniClusterType.mr) {
       mr = shims.getMiniMrCluster(conf, 4, uriString, 1);
     }
   }
@@ -900,7 +910,7 @@ public class QTestUtil {
           if(tblObj.isIndexTable()) {
             continue;
           }
-          db.dropTable(dbName, tblName, true, true, clusterType == MiniClusterType.encrypted);
+          db.dropTable(dbName, tblName, true, true, fsType == FsType.encrypted_hdfs);
         } else {
           // this table is defined in srcTables, drop all indexes on it
          List<Index> indexes = db.getIndexes(dbName, tblName, (short)-1);
@@ -1563,7 +1573,7 @@ public class QTestUtil {
     boolean partialMaskWasMatched = false;
     Matcher matcher;
     while (null != (line = in.readLine())) {
-      if (clusterType == MiniClusterType.encrypted) {
+      if (fsType == FsType.encrypted_hdfs) {
         for (Pattern pattern : partialReservedPlanMask) {
           matcher = pattern.matcher(line);
           if (matcher.find()) {


[35/44] hive git commit: HiveServer2: Provide the user with different error messages depending on the Thrift client exception code (Peter Vary via Chaoyu Tang)

Posted by se...@apache.org.
HiveServer2: Provide the user with different error messages depending on the Thrift client exception code (Peter Vary via Chaoyu Tang)


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

Branch: refs/heads/hive-14535
Commit: d16d4f1bcc43d6ebcab0eaf5bc635fb88b60be5f
Parents: 291f3d5
Author: ctang <ct...@cloudera.com>
Authored: Thu Sep 29 11:25:21 2016 -0400
Committer: ctang <ct...@cloudera.com>
Committed: Thu Sep 29 11:25:21 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   | 20 +++++-
 beeline/src/main/resources/BeeLine.properties   | 11 ++-
 .../beeline/TestBeeLineExceptionHandling.java   | 72 ++++++++++++++++++++
 3 files changed, 101 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d16d4f1b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
index 5322ca6..79922d2 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -1778,7 +1778,25 @@ public class BeeLine implements Closeable {
     }
 
     if (e.getCause() instanceof TTransportException) {
-      error(loc("hs2-unavailable"));
+      switch (((TTransportException)e.getCause()).getType()) {
+        case TTransportException.ALREADY_OPEN:
+          error(loc("hs2-connection-already-open"));
+          break;
+        case TTransportException.END_OF_FILE:
+          error(loc("hs2-unexpected-end-of-file"));
+          break;
+        case TTransportException.NOT_OPEN:
+          error(loc("hs2-could-not-open-connection"));
+          break;
+        case TTransportException.TIMED_OUT:
+          error(loc("hs2-connection-timed-out"));
+          break;
+        case TTransportException.UNKNOWN:
+          error(loc("hs2-unknown-connection-problem"));
+          break;
+        default:
+          error(loc("hs2-unexpected-error"));
+      }
     }
 
     error(loc(e instanceof SQLWarning ? "Warning" : "Error",

http://git-wip-us.apache.org/repos/asf/hive/blob/d16d4f1b/beeline/src/main/resources/BeeLine.properties
----------------------------------------------------------------------
diff --git a/beeline/src/main/resources/BeeLine.properties b/beeline/src/main/resources/BeeLine.properties
index 13321d2..ad79c01 100644
--- a/beeline/src/main/resources/BeeLine.properties
+++ b/beeline/src/main/resources/BeeLine.properties
@@ -142,7 +142,16 @@ active-connections: 0#No active connections|1#{0} active connection:|1<{0} activ
 
 time-ms: ({0,number,#.###} seconds)
 
-hs2-unavailable: HS2 may be unavailable, check server status
+hs2-connection-already-open: Socket already connected.
+hs2-unexpected-end-of-file: Unexpected end of file when reading from HS2 server. The root \
+cause might be too many concurrent connections. Please ask the administrator to check the number \
+of active connections, and adjust hive.server2.thrift.max.worker.threads if applicable.
+hs2-could-not-open-connection: Could not open connection to the HS2 server. Please check the \
+server URI and if the URI is correct, then ask the administrator to check the server status.\
+hs2-connection-timed-out: Connection timeout when communicating with HS2 server.
+hs2-unknown-connection-problem: Unknown HS2 problem when communicating with Thrift server.
+hs2-unexpected-error: Unexpected HS2 error when communicating with the Thrift server.
+
 
 cmd-usage: Usage: java org.apache.hive.cli.beeline.BeeLine \n \
 \  -u <database url>               the JDBC URL to connect to\n \

http://git-wip-us.apache.org/repos/asf/hive/blob/d16d4f1b/beeline/src/test/org/apache/hive/beeline/TestBeeLineExceptionHandling.java
----------------------------------------------------------------------
diff --git a/beeline/src/test/org/apache/hive/beeline/TestBeeLineExceptionHandling.java b/beeline/src/test/org/apache/hive/beeline/TestBeeLineExceptionHandling.java
new file mode 100644
index 0000000..08579e8
--- /dev/null
+++ b/beeline/src/test/org/apache/hive/beeline/TestBeeLineExceptionHandling.java
@@ -0,0 +1,72 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.hive.beeline;
+
+import junit.framework.Assert;
+import org.apache.thrift.transport.TTransportException;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestBeeLineExceptionHandling {
+
+  public class TestBeeline extends BeeLine {
+    private String expectedLoc;
+    private int logCount;
+    public TestBeeline(String expectedLoc) {
+      this.expectedLoc = expectedLoc;
+      this.logCount = 0;
+    }
+
+    @Override
+    boolean error(String log) {
+      if (logCount == 0) {
+        Assert.assertEquals(loc(expectedLoc), log);
+      } else {
+        Assert.assertEquals("Error: org.apache.thrift.transport.TTransportException "
+            + "(state=,code=0)", log);
+      }
+      logCount++;
+      return false;
+    }
+  }
+
+  @Test
+  public void testHandleSQLExceptionLog() throws Exception {
+    checkException(TTransportException.ALREADY_OPEN, "hs2-connection-already-open");
+    checkException(TTransportException.END_OF_FILE, "hs2-unexpected-end-of-file");
+    checkException(TTransportException.NOT_OPEN, "hs2-could-not-open-connection");
+    checkException(TTransportException.TIMED_OUT, "hs2-connection-timed-out");
+    checkException(TTransportException.UNKNOWN, "hs2-unknown-connection-problem");
+    checkException(-1, "hs2-unexpected-error");
+  }
+
+  private void checkException(int type, String loc) {
+    BeeLine testBeeLine = new TestBeeline(loc);
+    TTransportException tTransportException = new TTransportException(type);
+    SQLException sqlException = new SQLException(tTransportException);
+    testBeeLine.handleSQLException(sqlException);
+  }
+}


[21/44] hive git commit: HIVE-14817. Shutdown the SessionManager timeoutChecker thread properly upon shutdown. (Siddharth Seth, reviewed by Thejas Nair)

Posted by se...@apache.org.
HIVE-14817. Shutdown the SessionManager timeoutChecker thread properly upon shutdown. (Siddharth Seth, reviewed by Thejas Nair)


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

Branch: refs/heads/hive-14535
Commit: e08d94e57d99245ebaa90c4be69dade84ba27172
Parents: 990927e
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Sep 23 14:56:57 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Sep 23 14:56:57 2016 -0700

----------------------------------------------------------------------
 .../service/cli/session/SessionManager.java     | 33 +++++++++++++++-----
 1 file changed, 25 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e08d94e5/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
index 15bab06..76e759f 100644
--- a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
+++ b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
@@ -185,14 +185,20 @@ public class SessionManager extends CompositeService {
     }
   }
 
+  private final Object timeoutCheckerLock = new Object();
+
   private void startTimeoutChecker() {
     final long interval = Math.max(checkInterval, 3000l);  // minimum 3 seconds
-    Runnable timeoutChecker = new Runnable() {
+    final Runnable timeoutChecker = new Runnable() {
       @Override
       public void run() {
-        for (sleepInterval(interval); !shutdown; sleepInterval(interval)) {
+        sleepFor(interval);
+        while (!shutdown) {
           long current = System.currentTimeMillis();
           for (HiveSession session : new ArrayList<HiveSession>(handleToSession.values())) {
+            if (shutdown) {
+              break;
+            }
             if (sessionTimeout > 0 && session.getLastAccessTime() + sessionTimeout <= current
                 && (!checkOperation || session.getNoOperationTime() > sessionTimeout)) {
               SessionHandle handle = session.getSessionHandle();
@@ -207,24 +213,35 @@ public class SessionManager extends CompositeService {
               session.closeExpiredOperations();
             }
           }
+          sleepFor(interval);
         }
       }
 
-      private void sleepInterval(long interval) {
-        try {
-          Thread.sleep(interval);
-        } catch (InterruptedException e) {
-          // ignore
+      private void sleepFor(long interval) {
+        synchronized (timeoutCheckerLock) {
+          try {
+            timeoutCheckerLock.wait(interval);
+          } catch (InterruptedException e) {
+            // Ignore, and break.
+          }
         }
       }
     };
     backgroundOperationPool.execute(timeoutChecker);
   }
 
+  private void shutdownTimeoutChecker() {
+    shutdown = true;
+    synchronized (timeoutCheckerLock) {
+      timeoutCheckerLock.notify();
+    }
+  }
+
+
   @Override
   public synchronized void stop() {
     super.stop();
-    shutdown = true;
+    shutdownTimeoutChecker();
     if (backgroundOperationPool != null) {
       backgroundOperationPool.shutdown();
       long timeout = hiveConf.getTimeVar(


[19/44] hive git commit: HIVE-14713: LDAP Authentication Provider should be covered with unit tests (Illya Yalovyy, reviewed by Chaoyu Tang, Szehon Ho)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/UserSearchFilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/UserSearchFilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/UserSearchFilterFactory.java
new file mode 100644
index 0000000..3218875
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/UserSearchFilterFactory.java
@@ -0,0 +1,65 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.Collection;
+import javax.naming.NamingException;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * A factory for a {@link Filter} that check whether provided user could be found in the directory.
+ * <br>
+ * The produced filter object filters out all users that are not found in the directory.
+ */
+public final class UserSearchFilterFactory implements FilterFactory {
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Filter getInstance(HiveConf conf) {
+    Collection<String> groupFilter = conf.getStringCollection(
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER.varname);
+    Collection<String> userFilter = conf.getStringCollection(
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER.varname);
+
+    if (groupFilter.isEmpty() && userFilter.isEmpty()) {
+      return null;
+    }
+
+    return new UserSearchFilter();
+  }
+
+  private static final class UserSearchFilter implements Filter {
+    @Override
+    public void apply(DirSearch client, String user) throws AuthenticationException {
+      try {
+        String userDn = client.findUserDn(user);
+
+        // This should not be null because we were allowed to bind with this username
+        // safe check in case we were able to bind anonymously.
+        if (userDn == null) {
+          throw new AuthenticationException("Authentication failed: User search failed");
+        }
+      } catch (NamingException e) {
+        throw new AuthenticationException("LDAP Authentication failed for user", e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java b/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
index 089a059..23a048a 100644
--- a/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
+++ b/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
@@ -225,7 +225,6 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
     hiveConf = new HiveConf();
 
     ldapProvider = new LdapAuthenticationProviderImpl(hiveConf);
-    ldapProvider.init(hiveConf);
   }
 
   @AfterClass
@@ -259,7 +258,7 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
       }
     }
 
-    ldapProvider.init(hiveConf);
+    ldapProvider = new LdapAuthenticationProviderImpl(hiveConf);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/TestLdapAuthenticationProviderImpl.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/TestLdapAuthenticationProviderImpl.java b/service/src/test/org/apache/hive/service/auth/TestLdapAuthenticationProviderImpl.java
index f276906..4fad755 100644
--- a/service/src/test/org/apache/hive/service/auth/TestLdapAuthenticationProviderImpl.java
+++ b/service/src/test/org/apache/hive/service/auth/TestLdapAuthenticationProviderImpl.java
@@ -15,51 +15,260 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hive.service.auth;
 
+import java.io.IOException;
+import java.util.Arrays;
+import javax.naming.NamingException;
 import javax.security.sasl.AuthenticationException;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-
-import junit.framework.TestCase;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.service.auth.ldap.DirSearch;
+import org.apache.hive.service.auth.ldap.DirSearchFactory;
+import org.apache.hive.service.auth.ldap.LdapSearchFactory;
+import org.junit.Test;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestLdapAuthenticationProviderImpl {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
 
-public class TestLdapAuthenticationProviderImpl extends TestCase {
+  public HiveConf conf;
+  public LdapAuthenticationProviderImpl auth;
 
-  private static HiveConf hiveConf;
-  private static byte[] hiveConfBackup;
+  @Mock
+  public DirSearchFactory factory;
 
-  @Override
-  public void setUp() throws Exception {
-      hiveConf = new HiveConf();
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
-      hiveConf.writeXml(baos);
-      baos.close();
-      hiveConfBackup = baos.toByteArray();
-      hiveConf.set("hive.server2.authentication.ldap.url", "localhost");
-      FileOutputStream fos = new FileOutputStream(new File(hiveConf.getHiveSiteLocation().toURI()));
-      hiveConf.writeXml(fos);
-      fos.close();
+  @Mock
+  public DirSearch search;
+
+  @Before
+  public void setup() throws AuthenticationException {
+    conf = new HiveConf();
+    conf.set("hive.root.logger", "DEBUG,console");
+    conf.set("hive.server2.authentication.ldap.url", "localhost");
+    when(factory.getInstance(any(HiveConf.class), anyString(), anyString())).thenReturn(search);
   }
 
-  public void testLdapEmptyPassword() {
-    LdapAuthenticationProviderImpl ldapImpl = new LdapAuthenticationProviderImpl(hiveConf);
-    try {
-      ldapImpl.Authenticate("user", "");
-      assertFalse(true);
-    } catch (AuthenticationException e) {
-      assertTrue(e.getMessage(), e.getMessage().contains("a null or blank password has been provided"));
-    }
+  @Test
+  public void authenticateGivenBlankPassword() throws Exception {
+    auth = new LdapAuthenticationProviderImpl(conf, new LdapSearchFactory());
+    expectAuthenticationExceptionForInvalidPassword();
+    auth.Authenticate("user", "");
+  }
+
+  @Test
+  public void authenticateGivenStringWithNullCharacterForPassword() throws Exception {
+    auth = new LdapAuthenticationProviderImpl(conf, new LdapSearchFactory());
+    expectAuthenticationExceptionForInvalidPassword();
+    auth.Authenticate("user", "\0");
+  }
+
+  @Test
+  public void authenticateGivenNullForPassword() throws Exception {
+    auth = new LdapAuthenticationProviderImpl(conf, new LdapSearchFactory());
+    expectAuthenticationExceptionForInvalidPassword();
+    auth.Authenticate("user", null);
+  }
+
+  @Test
+  public void testAuthenticateNoUserOrGroupFilter() throws NamingException, AuthenticationException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN,
+        "cn=%s,ou=Users,dc=mycorp,dc=com:cn=%s,ou=PowerUsers,dc=mycorp,dc=com");
+
+    DirSearchFactory factory = mock(DirSearchFactory.class);
+
+    when(search.findUserDn("user1")).thenReturn("cn=user1,ou=PowerUsers,dc=mycorp,dc=com");
+
+    when(factory.getInstance(conf, "cn=user1,ou=PowerUsers,dc=mycorp,dc=com", "Blah")).thenReturn(search);
+    when(factory.getInstance(conf, "cn=user1,ou=Users,dc=mycorp,dc=com", "Blah")).thenThrow(AuthenticationException.class);
+
+    auth = new LdapAuthenticationProviderImpl(conf, factory);
+    auth.Authenticate("user1", "Blah");
+
+    verify(factory, times(2)).getInstance(isA(HiveConf.class), anyString(), eq("Blah"));
+    verify(search, atLeastOnce()).close();
   }
 
-  @Override
-  public void tearDown() throws Exception {
-    if(hiveConf != null && hiveConfBackup != null) {
-      FileOutputStream fos = new FileOutputStream(new File(hiveConf.getHiveSiteLocation().toURI()));
-      fos.write(hiveConfBackup);
-      fos.close();
+  @Test
+  public void testAuthenticateWhenUserFilterPasses() throws NamingException, AuthenticationException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER,
+        "user1,user2");
+
+    when(search.findUserDn("user1")).thenReturn("cn=user1,ou=PowerUsers,dc=mycorp,dc=com");
+    when(search.findUserDn("user2")).thenReturn("cn=user2,ou=PowerUsers,dc=mycorp,dc=com");
+
+    authenticateUserAndCheckSearchIsClosed("user1");
+    authenticateUserAndCheckSearchIsClosed("user2");
+  }
+
+  @Test
+  public void testAuthenticateWhenLoginWithDomainAndUserFilterPasses() throws NamingException, AuthenticationException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER,
+        "user1");
+
+    when(search.findUserDn("user1")).thenReturn("cn=user1,ou=PowerUsers,dc=mycorp,dc=com");
+
+    authenticateUserAndCheckSearchIsClosed("user1@mydomain.com");
+  }
+
+  @Test
+  public void testAuthenticateWhenLoginWithDnAndUserFilterPasses() throws NamingException, AuthenticationException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER,
+        "user1");
+
+    when(search.findUserDn("cn=user1,ou=PowerUsers,dc=mycorp,dc=com")).thenReturn("cn=user1,ou=PowerUsers,dc=mycorp,dc=com");
+
+    authenticateUserAndCheckSearchIsClosed("cn=user1,ou=PowerUsers,dc=mycorp,dc=com");
+  }
+
+  @Test
+  public void testAuthenticateWhenUserSearchFails() throws NamingException, AuthenticationException, IOException {
+    thrown.expect(AuthenticationException.class);
+
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "user1,user2");
+
+    when(search.findUserDn("user1")).thenReturn(null);
+
+    authenticateUserAndCheckSearchIsClosed("user1");
+  }
+
+  @Test
+  public void testAuthenticateWhenUserFilterFails() throws NamingException, AuthenticationException, IOException {
+    thrown.expect(AuthenticationException.class);
+
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "user1,user2");
+
+    when(search.findUserDn("user3")).thenReturn("cn=user3,ou=PowerUsers,dc=mycorp,dc=com");
+
+    authenticateUserAndCheckSearchIsClosed("user3");
+  }
+
+  @Test
+  public void testAuthenticateWhenGroupFilterPasses() throws NamingException, AuthenticationException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER, "group1,group2");
+
+    when(search.findUserDn("user1")).thenReturn("cn=user1,ou=PowerUsers,dc=mycorp,dc=com");
+    when(search.findUserDn("user2")).thenReturn("cn=user2,ou=PowerUsers,dc=mycorp,dc=com");
+
+    when(search.findGroupsForUser("cn=user1,ou=PowerUsers,dc=mycorp,dc=com"))
+        .thenReturn(Arrays.asList(
+            "cn=testGroup,ou=Groups,dc=mycorp,dc=com",
+            "cn=group1,ou=Groups,dc=mycorp,dc=com"));
+    when(search.findGroupsForUser("cn=user2,ou=PowerUsers,dc=mycorp,dc=com"))
+        .thenReturn(Arrays.asList(
+            "cn=testGroup,ou=Groups,dc=mycorp,dc=com",
+            "cn=group2,ou=Groups,dc=mycorp,dc=com"));
+
+    authenticateUserAndCheckSearchIsClosed("user1");
+    authenticateUserAndCheckSearchIsClosed("user2");
+  }
+
+  @Test
+  public void testAuthenticateWhenUserAndGroupFiltersPass() throws NamingException, AuthenticationException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER, "group1,group2");
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "user1,user2");
+
+    when(search.findUserDn("user1")).thenReturn("cn=user1,ou=PowerUsers,dc=mycorp,dc=com");
+    when(search.findUserDn("user2")).thenReturn("cn=user2,ou=PowerUsers,dc=mycorp,dc=com");
+
+    when(search.findGroupsForUser("cn=user1,ou=PowerUsers,dc=mycorp,dc=com"))
+        .thenReturn(Arrays.asList(
+            "cn=testGroup,ou=Groups,dc=mycorp,dc=com",
+            "cn=group1,ou=Groups,dc=mycorp,dc=com"));
+    when(search.findGroupsForUser("cn=user2,ou=PowerUsers,dc=mycorp,dc=com"))
+        .thenReturn(Arrays.asList(
+            "cn=testGroup,ou=Groups,dc=mycorp,dc=com",
+            "cn=group2,ou=Groups,dc=mycorp,dc=com"));
+
+    authenticateUserAndCheckSearchIsClosed("user1");
+    authenticateUserAndCheckSearchIsClosed("user2");
+  }
+
+  @Test
+  public void testAuthenticateWhenUserFilterPassesAndGroupFilterFails()
+      throws NamingException, AuthenticationException, IOException {
+    thrown.expect(AuthenticationException.class);
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER, "group1,group2");
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "user1,user2");
+
+    when(search.findUserDn("user1")).thenReturn("cn=user1,ou=PowerUsers,dc=mycorp,dc=com");
+
+    when(search.findGroupsForUser("cn=user1,ou=PowerUsers,dc=mycorp,dc=com"))
+        .thenReturn(Arrays.asList(
+            "cn=testGroup,ou=Groups,dc=mycorp,dc=com",
+            "cn=OtherGroup,ou=Groups,dc=mycorp,dc=com"));
+
+    authenticateUserAndCheckSearchIsClosed("user1");
+  }
+
+  @Test
+  public void testAuthenticateWhenUserFilterFailsAndGroupFilterPasses()
+      throws NamingException, AuthenticationException, IOException {
+    thrown.expect(AuthenticationException.class);
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER, "group3");
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "user1,user2");
+
+    when(search.findUserDn("user3")).thenReturn("cn=user3,ou=PowerUsers,dc=mycorp,dc=com");
+
+    when(search.findGroupsForUser("cn=user3,ou=PowerUsers,dc=mycorp,dc=com"))
+        .thenReturn(Arrays.asList(
+            "cn=testGroup,ou=Groups,dc=mycorp,dc=com",
+            "cn=group3,ou=Groups,dc=mycorp,dc=com"));
+
+    authenticateUserAndCheckSearchIsClosed("user3");
+  }
+
+  @Test
+  public void testAuthenticateWhenCustomQueryFilterPasses() throws NamingException, AuthenticationException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN, "dc=mycorp,dc=com");
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY,
+        "(&(objectClass=person)(|(memberOf=CN=Domain Admins,CN=Users,DC=apache,DC=org)(memberOf=CN=Administrators,CN=Builtin,DC=apache,DC=org)))");
+
+    when(search.executeCustomQuery(anyString())).thenReturn(Arrays.asList(
+        "cn=user1,ou=PowerUsers,dc=mycorp,dc=com",
+        "cn=user2,ou=PowerUsers,dc=mycorp,dc=com"));
+
+    authenticateUserAndCheckSearchIsClosed("user1");
+  }
+
+  @Test
+  public void testAuthenticateWhenCustomQueryFilterFailsAndUserFilterPasses() throws NamingException, AuthenticationException, IOException {
+    thrown.expect(AuthenticationException.class);
+
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN, "dc=mycorp,dc=com");
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY,
+        "(&(objectClass=person)(|(memberOf=CN=Domain Admins,CN=Users,DC=apache,DC=org)(memberOf=CN=Administrators,CN=Builtin,DC=apache,DC=org)))");
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "user3");
+
+    when(search.findUserDn("user3")).thenReturn("cn=user3,ou=PowerUsers,dc=mycorp,dc=com");
+    when(search.executeCustomQuery(anyString())).thenReturn(Arrays.asList(
+        "cn=user1,ou=PowerUsers,dc=mycorp,dc=com",
+        "cn=user2,ou=PowerUsers,dc=mycorp,dc=com"));
+
+    authenticateUserAndCheckSearchIsClosed("user3");
+  }
+
+  private void expectAuthenticationExceptionForInvalidPassword() {
+    thrown.expect(AuthenticationException.class);
+    thrown.expectMessage("a null or blank password has been provided");
+  }
+
+  private void authenticateUserAndCheckSearchIsClosed(String user) throws IOException {
+    auth = new LdapAuthenticationProviderImpl(conf, factory);
+    try {
+      auth.Authenticate(user, "password doesn't matter");
+    } finally {
+      verify(search, atLeastOnce()).close();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/Credentials.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/Credentials.java b/service/src/test/org/apache/hive/service/auth/ldap/Credentials.java
new file mode 100644
index 0000000..ce22b8e
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/Credentials.java
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+public final class Credentials {
+
+  private final String user;
+  private final String password;
+
+  private Credentials(String user, String password) {
+    this.user = user;
+    this.password = password;
+  }
+
+  public static Credentials of(String user, String password) {
+    return new Credentials(user, password);
+  }
+
+  public String getUser() {
+    return user;
+  }
+
+  public String getPassword() {
+    return password;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/LdapTestUtils.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/LdapTestUtils.java b/service/src/test/org/apache/hive/service/auth/ldap/LdapTestUtils.java
new file mode 100644
index 0000000..d4e034f
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/LdapTestUtils.java
@@ -0,0 +1,126 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.Attributes;
+import javax.naming.directory.BasicAttribute;
+import javax.naming.directory.BasicAttributes;
+import javax.naming.directory.SearchResult;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import org.mockito.stubbing.OngoingStubbing;
+
+public final class LdapTestUtils {
+
+  private LdapTestUtils() {
+  }
+
+  public static NamingEnumeration<SearchResult> mockEmptyNamingEnumeration() throws NamingException {
+    return mockNamingEnumeration(new SearchResult[0]);
+  }
+
+  public static NamingEnumeration<SearchResult> mockNamingEnumeration(String... dns) throws NamingException {
+    return mockNamingEnumeration(mockSearchResults(dns).toArray(new SearchResult[0]));
+  }
+
+  public static NamingEnumeration<SearchResult> mockNamingEnumeration(SearchResult... searchResults) throws NamingException {
+    NamingEnumeration<SearchResult> ne =
+        (NamingEnumeration<SearchResult>) mock(NamingEnumeration.class);
+    mockHasMoreMethod(ne, searchResults.length);
+    if (searchResults.length > 0) {
+      List<SearchResult> mockedResults = Arrays.asList(searchResults);
+      mockNextMethod(ne, mockedResults);
+    }
+    return ne;
+  }
+
+  public static void mockHasMoreMethod(NamingEnumeration<SearchResult> ne, int length) throws NamingException {
+    OngoingStubbing<Boolean> hasMoreStub = when(ne.hasMore());
+    for (int i = 0; i < length; i++) {
+      hasMoreStub = hasMoreStub.thenReturn(true);
+    }
+    hasMoreStub.thenReturn(false);
+  }
+
+  public static void mockNextMethod(NamingEnumeration<SearchResult> ne, List<SearchResult> searchResults) throws NamingException {
+    OngoingStubbing<SearchResult> nextStub = when(ne.next());
+    for (SearchResult searchResult : searchResults) {
+      nextStub = nextStub.thenReturn(searchResult);
+    }
+  }
+
+  public static List<SearchResult> mockSearchResults(String[] dns) {
+    List<SearchResult> list = new ArrayList<>();
+    for (String dn : dns) {
+      list.add(mockSearchResult(dn, null));
+    }
+    return list;
+  }
+
+  public static SearchResult mockSearchResult(String dn, Attributes attributes) {
+    SearchResult searchResult = mock(SearchResult.class);
+    when(searchResult.getNameInNamespace()).thenReturn(dn);
+    when(searchResult.getAttributes()).thenReturn(attributes);
+    return searchResult;
+  }
+
+  public static Attributes mockEmptyAttributes() throws NamingException {
+    return mockAttributes();
+  }
+
+  public static Attributes mockAttributes(String name, String value) throws NamingException {
+    return mockAttributes(new NameValues(name, value));
+  }
+
+  public static Attributes mockAttributes(String name1, String value1, String name2, String value2) throws NamingException {
+    if (name1.equals(name2)) {
+      return mockAttributes(new NameValues(name1, value1, value2));
+    } else {
+      return mockAttributes(new NameValues(name1, value1), new NameValues(name2, value2));
+    }
+  }
+
+  private static Attributes mockAttributes(NameValues... namedValues) throws NamingException {
+    Attributes attributes =  new BasicAttributes();
+    for (NameValues namedValue : namedValues) {
+      Attribute attr = new BasicAttribute(namedValue.name);
+      for (String value : namedValue.values) {
+        attr.add(value);
+      }
+      attributes.put(attr);
+    }
+    return attributes;
+  }
+
+  private static final class NameValues {
+    final String name;
+    final String[] values;
+
+    public NameValues(String name, String... values) {
+      this.name = name;
+      this.values = values;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/TestChainFilter.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/TestChainFilter.java b/service/src/test/org/apache/hive/service/auth/ldap/TestChainFilter.java
new file mode 100644
index 0000000..9caa233
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/TestChainFilter.java
@@ -0,0 +1,103 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.io.IOException;
+import javax.naming.NamingException;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import org.junit.Before;
+import org.mockito.Mock;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestChainFilter {
+
+  private FilterFactory factory;
+  private HiveConf conf;
+
+  @Mock
+  public Filter filter1;
+
+  @Mock
+  public Filter filter2;
+
+  @Mock
+  public Filter filter3;
+
+  @Mock
+  public FilterFactory factory1;
+
+  @Mock
+  public FilterFactory factory2;
+
+  @Mock
+  public FilterFactory factory3;
+
+  @Mock
+  private DirSearch search;
+
+  @Before
+  public void setup() {
+    conf = new HiveConf();
+    factory = new ChainFilterFactory(factory1, factory2, factory3);
+  }
+
+  @Test
+  public void testFactoryAllNull() {
+    assertNull(factory.getInstance(conf));
+  }
+
+  @Test
+  public void testFactoryAllEmpty() {
+    FilterFactory emptyFactory = new ChainFilterFactory();
+    assertNull(emptyFactory.getInstance(conf));
+  }
+
+  @Test
+  public void testFactory() throws AuthenticationException {
+    when(factory1.getInstance(any(HiveConf.class))).thenReturn(filter1);
+    when(factory2.getInstance(any(HiveConf.class))).thenReturn(filter2);
+    when(factory3.getInstance(any(HiveConf.class))).thenReturn(filter3);
+
+    Filter filter = factory.getInstance(conf);
+
+    filter.apply(search, "User");
+    verify(filter1, times(1)).apply(search, "User");
+    verify(filter2, times(1)).apply(search, "User");
+    verify(filter3, times(1)).apply(search, "User");
+  }
+
+  @Test(expected = AuthenticationException.class)
+  public void testApplyNegative() throws AuthenticationException, NamingException, IOException {
+    doThrow(AuthenticationException.class).when(filter3).apply((DirSearch) anyObject(), anyString());
+
+    when(factory1.getInstance(any(HiveConf.class))).thenReturn(filter1);
+    when(factory3.getInstance(any(HiveConf.class))).thenReturn(filter3);
+
+    Filter filter = factory.getInstance(conf);
+
+    filter.apply(search, "User");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/TestCustomQueryFilter.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/TestCustomQueryFilter.java b/service/src/test/org/apache/hive/service/auth/ldap/TestCustomQueryFilter.java
new file mode 100644
index 0000000..fd4b898
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/TestCustomQueryFilter.java
@@ -0,0 +1,85 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.io.IOException;
+import java.util.Arrays;
+import javax.naming.NamingException;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import org.junit.Before;
+import org.mockito.Mock;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestCustomQueryFilter {
+
+  private static final String USER2_DN = "uid=user2,ou=People,dc=example,dc=com";
+  private static final String USER1_DN = "uid=user1,ou=People,dc=example,dc=com";
+  private static final String CUSTOM_QUERY = "(&(objectClass=person)(|(uid=user1)(uid=user2)))";
+
+  private FilterFactory factory;
+  private HiveConf conf;
+
+  @Mock
+  private DirSearch search;
+
+  @Before
+  public void setup() {
+    conf = new HiveConf();
+    conf.set("hive.root.logger", "DEBUG,console");
+    factory = new CustomQueryFilterFactory();
+  }
+
+  @Test
+  public void testFactory() {
+    conf.unset(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY.varname);
+    assertNull(factory.getInstance(conf));
+
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY, CUSTOM_QUERY);
+    assertNotNull(factory.getInstance(conf));
+  }
+
+  @Test
+  public void testApplyPositive() throws AuthenticationException, NamingException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY, CUSTOM_QUERY);
+
+    when(search.executeCustomQuery(eq(CUSTOM_QUERY))).thenReturn(Arrays.asList(USER1_DN, USER2_DN));
+
+    Filter filter = factory.getInstance(conf);
+    filter.apply(search, "user1");
+    filter.apply(search, "user2");
+  }
+
+
+  @Test(expected = AuthenticationException.class)
+  public void testApplyNegative() throws AuthenticationException, NamingException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY, CUSTOM_QUERY);
+
+    when(search.executeCustomQuery(eq(CUSTOM_QUERY))).thenReturn(Arrays.asList(USER1_DN, USER2_DN));
+
+    Filter filter = factory.getInstance(conf);
+    filter.apply(search, "user3");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/TestGroupFilter.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/TestGroupFilter.java b/service/src/test/org/apache/hive/service/auth/ldap/TestGroupFilter.java
new file mode 100644
index 0000000..0cc2ead
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/TestGroupFilter.java
@@ -0,0 +1,101 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.io.IOException;
+import java.util.Arrays;
+import javax.naming.NamingException;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import org.junit.Before;
+import org.mockito.Mock;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestGroupFilter {
+
+  private FilterFactory factory;
+  private HiveConf conf;
+
+  @Mock
+  private DirSearch search;
+
+  @Before
+  public void setup() {
+    conf = new HiveConf();
+    conf.set("hive.root.logger", "DEBUG,console");
+    factory = new GroupFilterFactory();
+  }
+
+  @Test
+  public void testFactory() {
+    conf.unset(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER.varname);
+    assertNull(factory.getInstance(conf));
+
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER, "G1");
+    assertNotNull(factory.getInstance(conf));
+  }
+
+  @Test
+  public void testApplyPositive() throws AuthenticationException, NamingException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER, "HiveUsers");
+
+    when(search.findUserDn(eq("user1")))
+        .thenReturn("cn=user1,ou=People,dc=example,dc=com");
+    when(search.findUserDn(eq("cn=user2,dc=example,dc=com")))
+        .thenReturn("cn=user2,ou=People,dc=example,dc=com");
+    when(search.findUserDn(eq("user3@mydomain.com")))
+        .thenReturn("cn=user3,ou=People,dc=example,dc=com");
+
+    when(search.findGroupsForUser(eq("cn=user1,ou=People,dc=example,dc=com")))
+        .thenReturn(Arrays.asList(
+            "cn=SuperUsers,ou=Groups,dc=example,dc=com",
+            "cn=Office1,ou=Groups,dc=example,dc=com",
+            "cn=HiveUsers,ou=Groups,dc=example,dc=com",
+            "cn=G1,ou=Groups,dc=example,dc=com"));
+    when(search.findGroupsForUser(eq("cn=user2,ou=People,dc=example,dc=com")))
+        .thenReturn(Arrays.asList(
+            "cn=HiveUsers,ou=Groups,dc=example,dc=com"));
+    when(search.findGroupsForUser(eq("cn=user3,ou=People,dc=example,dc=com")))
+        .thenReturn(Arrays.asList(
+            "cn=HiveUsers,ou=Groups,dc=example,dc=com",
+            "cn=G1,ou=Groups,dc=example,dc=com",
+            "cn=G2,ou=Groups,dc=example,dc=com"));
+
+    Filter filter = factory.getInstance(conf);
+    filter.apply(search, "user1");
+    filter.apply(search, "cn=user2,dc=example,dc=com");
+    filter.apply(search, "user3@mydomain.com");
+  }
+
+  @Test(expected = AuthenticationException.class)
+  public void testApplyNegative() throws AuthenticationException, NamingException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER, "HiveUsers");
+
+    when(search.findGroupsForUser(eq("user1"))).thenReturn(Arrays.asList("SuperUsers", "Office1", "G1", "G2"));
+
+    Filter filter = factory.getInstance(conf);
+    filter.apply(search, "user1");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/TestLdapSearch.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/TestLdapSearch.java b/service/src/test/org/apache/hive/service/auth/ldap/TestLdapSearch.java
new file mode 100644
index 0000000..499b624
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/TestLdapSearch.java
@@ -0,0 +1,209 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.SearchControls;
+import javax.naming.directory.SearchResult;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+import static org.apache.hive.service.auth.ldap.LdapTestUtils.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestLdapSearch {
+
+  @Mock
+  private DirContext ctx;
+
+  private HiveConf conf;
+  private LdapSearch search;
+
+  @Before
+  public void setup() {
+    conf = new HiveConf();
+  }
+
+  @Test
+  public void testClose() throws NamingException {
+    search = new LdapSearch(conf, ctx);
+    search.close();
+    verify(ctx, atLeastOnce()).close();
+  }
+
+  @Test
+  public void testFindUserDnWhenUserDnPositive() throws NamingException {
+    NamingEnumeration<SearchResult> searchResult = mockNamingEnumeration("CN=User1,OU=org1,DC=foo,DC=bar");
+    when(ctx.search(anyString(), anyString(), any(SearchControls.class)))
+        .thenReturn(searchResult)
+        .thenThrow(NamingException.class);
+    search = new LdapSearch(conf, ctx);
+    String expected = "CN=User1,OU=org1,DC=foo,DC=bar";
+    String actual = search.findUserDn("CN=User1,OU=org1");
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testFindUserDnWhenUserDnNegativeDuplicates() throws NamingException {
+    NamingEnumeration<SearchResult> searchResult = mockNamingEnumeration(
+            "CN=User1,OU=org1,DC=foo,DC=bar",
+            "CN=User1,OU=org2,DC=foo,DC=bar");
+    when(ctx.search(anyString(), anyString(), any(SearchControls.class))).thenReturn(searchResult);
+    search = new LdapSearch(conf, ctx);
+    assertNull(search.findUserDn("CN=User1,DC=foo,DC=bar"));
+  }
+
+  @Test
+  public void testFindUserDnWhenUserDnNegativeNone() throws NamingException {
+    NamingEnumeration<SearchResult> searchResult = mockEmptyNamingEnumeration();
+    when(ctx.search(anyString(), anyString(), any(SearchControls.class))).thenReturn(searchResult);
+    search = new LdapSearch(conf, ctx);
+    assertNull(search.findUserDn("CN=User1,DC=foo,DC=bar"));
+  }
+
+  @Test
+  public void testFindUserDnWhenUserPatternFoundBySecondPattern() throws NamingException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN,
+        "CN=%s,OU=org1,DC=foo,DC=bar:CN=%s,OU=org2,DC=foo,DC=bar");
+    NamingEnumeration<SearchResult> emptyResult = mockEmptyNamingEnumeration();
+    NamingEnumeration<SearchResult> validResult = mockNamingEnumeration("CN=User1,OU=org2,DC=foo,DC=bar");
+    when(ctx.search(anyString(), anyString(), any(SearchControls.class)))
+        .thenReturn(emptyResult)
+        .thenReturn(validResult);
+    search = new LdapSearch(conf, ctx);
+    String expected = "CN=User1,OU=org2,DC=foo,DC=bar";
+    String actual = search.findUserDn("User1");
+    assertEquals(expected, actual);
+    verify(ctx).search(eq("OU=org1,DC=foo,DC=bar"), contains("CN=User1"), any(SearchControls.class));
+    verify(ctx).search(eq("OU=org2,DC=foo,DC=bar"), contains("CN=User1"), any(SearchControls.class));
+  }
+
+  @Test
+  public void testFindUserDnWhenUserPatternFoundByFirstPattern() throws NamingException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN,
+        "CN=%s,OU=org1,DC=foo,DC=bar:CN=%s,OU=org2,DC=foo,DC=bar");
+    NamingEnumeration<SearchResult> emptyResult = mockEmptyNamingEnumeration();
+    NamingEnumeration<SearchResult> validResult = mockNamingEnumeration("CN=User1,OU=org2,DC=foo,DC=bar");
+    when(ctx.search(anyString(), anyString(), any(SearchControls.class)))
+        .thenReturn(validResult)
+        .thenReturn(emptyResult);
+    search = new LdapSearch(conf, ctx);
+    String expected = "CN=User1,OU=org2,DC=foo,DC=bar";
+    String actual = search.findUserDn("User1");
+    assertEquals(expected, actual);
+    verify(ctx).search(eq("OU=org1,DC=foo,DC=bar"), contains("CN=User1"), any(SearchControls.class));
+  }
+
+  @Test
+  public void testFindUserDnWhenUserPatternFoundByUniqueIdentifier() throws NamingException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN,
+        "CN=%s,OU=org1,DC=foo,DC=bar");
+    NamingEnumeration<SearchResult> validResult = mockNamingEnumeration("CN=User1,OU=org1,DC=foo,DC=bar");
+    when(ctx.search(anyString(), anyString(), any(SearchControls.class)))
+        .thenReturn(null)
+        .thenReturn(validResult);
+    search = new LdapSearch(conf, ctx);
+    String expected = "CN=User1,OU=org1,DC=foo,DC=bar";
+    String actual = search.findUserDn("User1");
+    assertEquals(expected, actual);
+    verify(ctx).search(eq("OU=org1,DC=foo,DC=bar"), contains("CN=User1"), any(SearchControls.class));
+    verify(ctx).search(eq("OU=org1,DC=foo,DC=bar"), contains("uid=User1"), any(SearchControls.class));
+  }
+
+  @Test
+  public void testFindUserDnWhenUserPatternFoundByUniqueIdentifierNegativeNone() throws NamingException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN,
+        "CN=%s,OU=org1,DC=foo,DC=bar");
+    when(ctx.search(anyString(), anyString(), any(SearchControls.class)))
+        .thenReturn(null)
+        .thenReturn(null);
+    search = new LdapSearch(conf, ctx);
+    assertNull(search.findUserDn("User1"));
+  }
+
+  @Test
+  public void testFindUserDnWhenUserPatternFoundByUniqueIdentifierNegativeMany() throws NamingException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN,
+        "CN=%s,OU=org1,DC=foo,DC=bar");
+    NamingEnumeration<SearchResult> manyResult = mockNamingEnumeration(
+        "CN=User1,OU=org1,DC=foo,DC=bar",
+        "CN=User12,OU=org1,DC=foo,DC=bar");
+    when(ctx.search(anyString(), anyString(), any(SearchControls.class)))
+        .thenReturn(null)
+        .thenReturn(manyResult);
+    search = new LdapSearch(conf, ctx);
+    assertNull(search.findUserDn("User1"));
+  }
+
+  @Test
+  public void testFindGroupsForUser() throws NamingException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN,
+        "CN=%s,OU=org1,DC=foo,DC=bar");
+
+    NamingEnumeration<SearchResult> groupsResult = mockNamingEnumeration("CN=Group1,OU=org1,DC=foo,DC=bar");
+    when(ctx.search(eq("OU=org1,DC=foo,DC=bar"), contains("User1"), any(SearchControls.class)))
+        .thenReturn(groupsResult);
+
+    search = new LdapSearch(conf, ctx);
+
+    List<String> expected = Arrays.asList("CN=Group1,OU=org1,DC=foo,DC=bar");
+    List<String> actual = search.findGroupsForUser("CN=User1,OU=org1,DC=foo,DC=bar");
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testExecuteCustomQuery() throws NamingException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN, "dc=example,dc=com");
+
+    NamingEnumeration<SearchResult> customQueryResult = mockNamingEnumeration(
+        mockSearchResult(
+            "uid=group1,ou=Groups,dc=example,dc=com",
+            mockAttributes("member", "uid=user1,ou=People,dc=example,dc=com")),
+        mockSearchResult(
+            "uid=group2,ou=Groups,dc=example,dc=com",
+            mockAttributes("member", "uid=user2,ou=People,dc=example,dc=com"))
+        );
+
+    when(ctx.search(eq("dc=example,dc=com"), anyString(), any(SearchControls.class)))
+        .thenReturn(customQueryResult);
+
+    search = new LdapSearch(conf, ctx);
+
+    List<String> expected = Arrays.asList(
+        "uid=group1,ou=Groups,dc=example,dc=com",
+        "uid=user1,ou=People,dc=example,dc=com",
+        "uid=group2,ou=Groups,dc=example,dc=com",
+        "uid=user2,ou=People,dc=example,dc=com");
+    List<String> actual = search.executeCustomQuery("(&(objectClass=groupOfNames)(|(cn=group1)(cn=group2)))");
+    Collections.sort(expected);
+    Collections.sort(actual);
+    assertEquals(expected, actual);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/TestLdapUtils.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/TestLdapUtils.java b/service/src/test/org/apache/hive/service/auth/ldap/TestLdapUtils.java
new file mode 100644
index 0000000..661aff4
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/TestLdapUtils.java
@@ -0,0 +1,103 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestLdapUtils {
+
+  @Test
+  public void testCreateCandidatePrincipalsForUserDn() {
+    HiveConf conf = new HiveConf();
+    String userDn = "cn=user1,ou=CORP,dc=mycompany,dc=com";
+    List<String> expected = Arrays.asList(userDn);
+    List<String> actual = LdapUtils.createCandidatePrincipals(conf, userDn);
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testCreateCandidatePrincipalsForUserWithDomain() {
+    HiveConf conf = new HiveConf();
+    String userWithDomain = "user1@mycompany.com";
+    List<String> expected = Arrays.asList(userWithDomain);
+    List<String> actual = LdapUtils.createCandidatePrincipals(conf, userWithDomain);
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testCreateCandidatePrincipalsLdapDomain() {
+    HiveConf conf = new HiveConf();
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN, "mycompany.com");
+    List<String> expected = Arrays.asList("user1@mycompany.com");
+    List<String> actual = LdapUtils.createCandidatePrincipals(conf, "user1");
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testCreateCandidatePrincipalsUserPatternsDefaultBaseDn() {
+    HiveConf conf = new HiveConf();
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY, "sAMAccountName");
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN, "dc=mycompany,dc=com");
+    List<String> expected = Arrays.asList("sAMAccountName=user1,dc=mycompany,dc=com");
+    List<String> actual = LdapUtils.createCandidatePrincipals(conf, "user1");
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testCreateCandidatePrincipals() {
+    HiveConf conf = new HiveConf();
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN, "dc=mycompany,dc=com");
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN,
+        "cn=%s,ou=CORP1,dc=mycompany,dc=com:cn=%s,ou=CORP2,dc=mycompany,dc=com");
+    List<String> expected = Arrays.asList(
+        "cn=user1,ou=CORP1,dc=mycompany,dc=com",
+        "cn=user1,ou=CORP2,dc=mycompany,dc=com");
+    List<String> actual = LdapUtils.createCandidatePrincipals(conf, "user1");
+    Collections.sort(expected);
+    Collections.sort(actual);
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testExtractFirstRdn() {
+    String dn = "cn=user1,ou=CORP1,dc=mycompany,dc=com";
+    String expected = "cn=user1";
+    String actual = LdapUtils.extractFirstRdn(dn);
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testExtractBaseDn() {
+    String dn = "cn=user1,ou=CORP1,dc=mycompany,dc=com";
+    String expected = "ou=CORP1,dc=mycompany,dc=com";
+    String actual = LdapUtils.extractBaseDn(dn);
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testExtractBaseDnNegative() {
+    String dn = "cn=user1";
+    assertNull(LdapUtils.extractBaseDn(dn));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/TestQuery.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/TestQuery.java b/service/src/test/org/apache/hive/service/auth/ldap/TestQuery.java
new file mode 100644
index 0000000..1f4bb1a
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/TestQuery.java
@@ -0,0 +1,59 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestQuery {
+
+  @Test
+  public void testQueryBuilderFilter() {
+    Query q = Query.builder()
+        .filter("test <uid_attr>=<value> query")
+        .map("uid_attr", "uid")
+        .map("value", "Hello!")
+        .build();
+    assertEquals("test uid=Hello! query", q.getFilter());
+    assertEquals(0, q.getControls().getCountLimit());
+  }
+
+  @Test
+  public void testQueryBuilderLimit() {
+    Query q = Query.builder()
+        .filter("<key1>,<key2>")
+        .map("key1", "value1")
+        .map("key2", "value2")
+        .limit(8)
+        .build();
+    assertEquals("value1,value2", q.getFilter());
+    assertEquals(8, q.getControls().getCountLimit());
+  }
+
+  @Test
+  public void testQueryBuilderReturningAttributes() {
+    Query q = Query.builder()
+        .filter("(query)")
+        .returnAttribute("attr1")
+        .returnAttribute("attr2")
+        .build();
+    assertEquals("(query)", q.getFilter());
+    assertArrayEquals(new String[] {"attr1", "attr2"}, q.getControls().getReturningAttributes());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/TestQueryFactory.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/TestQueryFactory.java b/service/src/test/org/apache/hive/service/auth/ldap/TestQueryFactory.java
new file mode 100644
index 0000000..3054e33
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/TestQueryFactory.java
@@ -0,0 +1,79 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestQueryFactory {
+
+  private QueryFactory queries;
+  private HiveConf conf;
+
+  @Before
+  public void setup() {
+    conf = new HiveConf();
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY, "guid");
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPCLASS_KEY, "superGroups");
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPMEMBERSHIP_KEY, "member");
+    queries = new QueryFactory(conf);
+  }
+
+  @Test
+  public void testFindGroupDnById() {
+    Query q = queries.findGroupDnById("unique_group_id");
+    String expected = "(&(objectClass=superGroups)(guid=unique_group_id))";
+    String actual = q.getFilter();
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testFindUserDnByRdn() {
+    Query q = queries.findUserDnByRdn("cn=User1");
+    String expected = "(&(|(objectClass=person)(objectClass=user)(objectClass=inetOrgPerson))(cn=User1))";
+    String actual = q.getFilter();
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testFindDnByPattern() {
+    Query q = queries.findDnByPattern("cn=User1");
+    String expected = "(cn=User1)";
+    String actual = q.getFilter();
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testFindUserDnByName() {
+    Query q = queries.findUserDnByName("unique_user_id");
+    String expected = "(&(|(objectClass=person)(objectClass=user)(objectClass=inetOrgPerson))(|(uid=unique_user_id)(sAMAccountName=unique_user_id)))";
+    String actual = q.getFilter();
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testFindGroupsForUser() {
+    Query q = queries.findGroupsForUser("user_name", "user_Dn");
+    String expected = "(&(objectClass=superGroups)(|(member=user_Dn)(member=user_name)))";
+    String actual = q.getFilter();
+    assertEquals(expected, actual);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/TestSearchResultHandler.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/TestSearchResultHandler.java b/service/src/test/org/apache/hive/service/auth/ldap/TestSearchResultHandler.java
new file mode 100644
index 0000000..2615680
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/TestSearchResultHandler.java
@@ -0,0 +1,222 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.util.AbstractCollection;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.SearchResult;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+import static org.apache.hive.service.auth.ldap.LdapTestUtils.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestSearchResultHandler {
+
+  SearchResultHandler handler;
+
+  @Test
+  public void testHandle() throws NamingException {
+    MockResultCollection resultCollection = MockResultCollection.create()
+        .addSearchResultWithDns("1")
+        .addSearchResultWithDns("2", "3");
+    handler = new SearchResultHandler(resultCollection);
+    List<String> expected = Arrays.asList("1", "2");
+    final List<String> actual = new ArrayList<>();
+    handler.handle(new SearchResultHandler.RecordProcessor() {
+      @Override
+      public boolean process(SearchResult record) throws NamingException {
+        actual.add(record.getNameInNamespace());
+        return actual.size() < 2;
+      }
+    });
+    assertEquals(expected, actual);
+    assertAllNamingEnumerationsClosed(resultCollection);
+  }
+
+  @Test
+  public void testGetAllLdapNamesNoRecords() throws NamingException {
+    MockResultCollection resultCollection = MockResultCollection.create()
+        .addEmptySearchResult();
+    handler = new SearchResultHandler(resultCollection);
+    List<String> actual = handler.getAllLdapNames();
+    assertEquals("Resultset size", 0, actual.size());
+    assertAllNamingEnumerationsClosed(resultCollection);
+  }
+
+  @Test
+  public void testGetAllLdapNamesWithExceptionInNamingEnumerationClose() throws NamingException {
+    MockResultCollection resultCollection = MockResultCollection.create()
+        .addSearchResultWithDns("1")
+        .addSearchResultWithDns("2");
+    doThrow(NamingException.class).when(resultCollection.iterator().next()).close();
+    handler = new SearchResultHandler(resultCollection);
+    List<String> actual = handler.getAllLdapNames();
+    assertEquals("Resultset size", 2, actual.size());
+    assertAllNamingEnumerationsClosed(resultCollection);
+  }
+
+  @Test
+  public void testGetAllLdapNames() throws NamingException {
+    String objectDn1 = "cn=a1,dc=b,dc=c";
+    String objectDn2 = "cn=a2,dc=b,dc=c";
+    String objectDn3 = "cn=a3,dc=b,dc=c";
+    MockResultCollection resultCollection = MockResultCollection.create()
+        .addSearchResultWithDns(objectDn1)
+        .addSearchResultWithDns(objectDn2, objectDn3);
+    handler = new SearchResultHandler(resultCollection);
+    List<String> expected = Arrays.asList(objectDn1, objectDn2, objectDn3);
+    Collections.sort(expected);
+    List<String> actual = handler.getAllLdapNames();
+    Collections.sort(actual);
+    assertEquals(expected, actual);
+    assertAllNamingEnumerationsClosed(resultCollection);
+  }
+
+  @Test
+  public void testGetAllLdapNamesAndAttributes() throws NamingException {
+    SearchResult searchResult1 = mockSearchResult("cn=a1,dc=b,dc=c",
+        mockAttributes("attr1", "attr1value1"));
+    SearchResult searchResult2 = mockSearchResult("cn=a2,dc=b,dc=c",
+        mockAttributes("attr1", "attr1value2", "attr2", "attr2value1"));
+    SearchResult searchResult3 = mockSearchResult("cn=a3,dc=b,dc=c",
+        mockAttributes("attr1", "attr1value3", "attr1", "attr1value4"));
+    SearchResult searchResult4 = mockSearchResult("cn=a4,dc=b,dc=c",
+        mockEmptyAttributes());
+
+    MockResultCollection resultCollection = MockResultCollection.create()
+        .addSearchResults(searchResult1)
+        .addSearchResults(searchResult2, searchResult3)
+        .addSearchResults(searchResult4);
+
+    handler = new SearchResultHandler(resultCollection);
+    List<String> expected = Arrays.asList(
+        "cn=a1,dc=b,dc=c", "attr1value1",
+        "cn=a2,dc=b,dc=c", "attr1value2", "attr2value1",
+        "cn=a3,dc=b,dc=c", "attr1value3", "attr1value4",
+        "cn=a4,dc=b,dc=c");
+    Collections.sort(expected);
+    List<String> actual = handler.getAllLdapNamesAndAttributes();
+    Collections.sort(actual);
+    assertEquals(expected, actual);
+    assertAllNamingEnumerationsClosed(resultCollection);
+  }
+
+  @Test
+  public void testHasSingleResultNoRecords() throws NamingException {
+    MockResultCollection resultCollection = MockResultCollection.create()
+        .addEmptySearchResult();
+    handler = new SearchResultHandler(resultCollection);
+    assertFalse(handler.hasSingleResult());
+    assertAllNamingEnumerationsClosed(resultCollection);
+  }
+
+  @Test
+  public void testHasSingleResult() throws NamingException {
+    MockResultCollection resultCollection = MockResultCollection.create()
+        .addSearchResultWithDns("1");
+    handler = new SearchResultHandler(resultCollection);
+    assertTrue(handler.hasSingleResult());
+    assertAllNamingEnumerationsClosed(resultCollection);
+  }
+
+  @Test
+  public void testHasSingleResultManyRecords() throws NamingException {
+    MockResultCollection resultCollection = MockResultCollection.create()
+        .addSearchResultWithDns("1")
+        .addSearchResultWithDns("2");
+    handler = new SearchResultHandler(resultCollection);
+    assertFalse(handler.hasSingleResult());
+    assertAllNamingEnumerationsClosed(resultCollection);
+  }
+
+  @Test(expected = NamingException.class)
+  public void testGetSingleLdapNameNoRecords() throws NamingException {
+    MockResultCollection resultCollection = MockResultCollection.create()
+        .addEmptySearchResult();
+    handler = new SearchResultHandler(resultCollection);
+    try {
+      handler.getSingleLdapName();
+    } finally {
+      assertAllNamingEnumerationsClosed(resultCollection);
+    }
+  }
+
+  @Test
+  public void testGetSingleLdapName() throws NamingException {
+    String objectDn = "cn=a,dc=b,dc=c";
+    MockResultCollection resultCollection = MockResultCollection.create()
+        .addEmptySearchResult()
+        .addSearchResultWithDns(objectDn);
+
+    handler = new SearchResultHandler(resultCollection);
+    String expected = objectDn;
+    String actual = handler.getSingleLdapName();
+    assertEquals(expected, actual);
+    assertAllNamingEnumerationsClosed(resultCollection);
+  }
+
+  private void assertAllNamingEnumerationsClosed(MockResultCollection resultCollection) throws NamingException {
+    for (NamingEnumeration<SearchResult> namingEnumeration : resultCollection) {
+      verify(namingEnumeration, atLeastOnce()).close();
+    }
+  }
+
+  private static final class MockResultCollection extends AbstractCollection<NamingEnumeration<SearchResult>> {
+
+    List<NamingEnumeration<SearchResult>> results = new ArrayList<>();
+
+    static MockResultCollection create() {
+      return new MockResultCollection();
+    }
+
+    MockResultCollection addSearchResultWithDns(String... dns) throws NamingException {
+      results.add(mockNamingEnumeration(dns));
+      return this;
+    }
+
+    MockResultCollection addSearchResults(SearchResult... dns) throws NamingException {
+      results.add(mockNamingEnumeration(dns));
+      return this;
+    }
+
+    MockResultCollection addEmptySearchResult() throws NamingException {
+      addSearchResults();
+      return this;
+    }
+
+    @Override
+    public Iterator<NamingEnumeration<SearchResult>> iterator() {
+      return results.iterator();
+    }
+
+    @Override
+    public int size() {
+      return results.size();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/TestUserFilter.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/TestUserFilter.java b/service/src/test/org/apache/hive/service/auth/ldap/TestUserFilter.java
new file mode 100644
index 0000000..f941c9c
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/TestUserFilter.java
@@ -0,0 +1,75 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.io.IOException;
+import javax.naming.NamingException;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import org.junit.Before;
+import org.mockito.Mock;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestUserFilter {
+
+  private FilterFactory factory;
+  private HiveConf conf;
+
+  @Mock
+  private DirSearch search;
+
+  @Before
+  public void setup() {
+    conf = new HiveConf();
+    factory = new UserFilterFactory();
+  }
+
+  @Test
+  public void testFactory() {
+    conf.unset(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER.varname);
+    assertNull(factory.getInstance(conf));
+
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "User1");
+    assertNotNull(factory.getInstance(conf));
+  }
+
+  @Test
+  public void testApplyPositive() throws AuthenticationException, NamingException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "User1,User2,uSeR3");
+
+    Filter filter = factory.getInstance(conf);
+    filter.apply(search, "User1");
+    filter.apply(search, "uid=user2,ou=People,dc=example,dc=com");
+    filter.apply(search, "User3@mydomain.com");
+  }
+
+  @Test(expected = AuthenticationException.class)
+  public void testApplyNegative() throws AuthenticationException, NamingException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "User1,User2");
+
+    Filter filter = factory.getInstance(conf);
+    filter.apply(search, "User3");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/test/org/apache/hive/service/auth/ldap/TestUserSearchFilter.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/ldap/TestUserSearchFilter.java b/service/src/test/org/apache/hive/service/auth/ldap/TestUserSearchFilter.java
new file mode 100644
index 0000000..0f2b509
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/ldap/TestUserSearchFilter.java
@@ -0,0 +1,94 @@
+/**
+ * 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.
+ */
+package org.apache.hive.service.auth.ldap;
+
+import java.io.IOException;
+import javax.naming.NamingException;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestUserSearchFilter {
+
+  private FilterFactory factory;
+  private HiveConf conf;
+
+  @Mock
+  private DirSearch search;
+
+  @Before
+  public void setup() {
+    conf = new HiveConf();
+    factory = new UserSearchFilterFactory();
+  }
+
+  @Test
+  public void testFactoryWhenNoGroupOrUserFilters() {
+    assertNull(factory.getInstance(conf));
+  }
+
+  @Test
+  public void testFactoryWhenGroupFilter() {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER, "Grp1,Grp2");
+    assertNotNull(factory.getInstance(conf));
+  }
+
+  @Test
+  public void testFactoryWhenUserFilter() {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "User1,User2");
+    assertNotNull(factory.getInstance(conf));
+  }
+
+  @Test
+  public void testApplyPositive() throws AuthenticationException, NamingException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "User1");
+    Filter filter = factory.getInstance(conf);
+
+    when(search.findUserDn(anyString())).thenReturn("cn=User1,ou=People,dc=example,dc=com");
+
+    filter.apply(search, "User1");
+  }
+
+  @Test(expected = AuthenticationException.class)
+  public void testApplyWhenNamingException() throws AuthenticationException, NamingException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "User1");
+    Filter filter = factory.getInstance(conf);
+
+    when(search.findUserDn(anyString())).thenThrow(NamingException.class);
+
+    filter.apply(search, "User3");
+  }
+
+  @Test(expected = AuthenticationException.class)
+  public void testApplyWhenNotFound() throws AuthenticationException, NamingException, IOException {
+    conf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER, "User1");
+    Filter filter = factory.getInstance(conf);
+
+    when(search.findUserDn(anyString())).thenReturn(null);
+
+    filter.apply(search, "User3");
+  }
+}


[39/44] hive git commit: HIVE-14819: FunctionInfo for permanent functions shows TEMPORARY FunctionType (Jason Dere, reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14819: FunctionInfo for permanent functions shows TEMPORARY FunctionType (Jason Dere, reviewed by Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: 74a6ff678f9312d946a7d55d73bf4a60127de763
Parents: 474425a
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Sep 29 13:35:17 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Sep 29 13:35:17 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FunctionInfo.java       |  20 ++--
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   3 +-
 .../apache/hadoop/hive/ql/exec/Registry.java    |  96 ++++++++++++++----
 .../hadoop/hive/ql/exec/WindowFunctionInfo.java |   4 +-
 .../translator/SqlFunctionConverter.java        |   2 +-
 .../hive/ql/exec/TestFunctionRegistry.java      | 100 +++++++++++++++++++
 6 files changed, 194 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
index 30ba996..8014dab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
@@ -73,36 +73,36 @@ public class FunctionInfo {
     this.discarded = new AtomicBoolean(false);  // shared to all session functions
   }
 
-  public FunctionInfo(boolean isNative, String displayName,
+  public FunctionInfo(FunctionType functionType, String displayName,
       GenericUDF genericUDF, FunctionResource... resources) {
-    this.functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    this.functionType = functionType;
     this.displayName = displayName;
     this.genericUDF = genericUDF;
     this.isInternalTableFunction = false;
     this.resources = resources;
   }
 
-  public FunctionInfo(boolean isNative, String displayName,
+  public FunctionInfo(FunctionType functionType, String displayName,
       GenericUDAFResolver genericUDAFResolver, FunctionResource... resources) {
-    this.functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    this.functionType = functionType;
     this.displayName = displayName;
     this.genericUDAFResolver = genericUDAFResolver;
     this.isInternalTableFunction = false;
     this.resources = resources;
   }
 
-  public FunctionInfo(boolean isNative, String displayName,
+  public FunctionInfo(FunctionType functionType, String displayName,
       GenericUDTF genericUDTF, FunctionResource... resources) {
-    this.functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    this.functionType = functionType;
     this.displayName = displayName;
     this.genericUDTF = genericUDTF;
     this.isInternalTableFunction = false;
     this.resources = resources;
   }
 
-  public FunctionInfo(boolean isNative, String displayName, Class<? extends TableFunctionResolver> tFnCls,
+  public FunctionInfo(FunctionType functionType, String displayName, Class<? extends TableFunctionResolver> tFnCls,
       FunctionResource... resources) {
-    this.functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    this.functionType = functionType;
     this.displayName = displayName;
     this.tableFunctionResolver = tFnCls;
     PartitionTableFunctionDescription def = AnnotationUtils.getAnnotation(
@@ -263,6 +263,10 @@ public class FunctionInfo {
     }
   }
 
+  public FunctionType getFunctionType() {
+    return functionType;
+  }
+
   public static class FunctionResource {
     private final SessionState.ResourceType resourceType;
     private final String resourceURI;

http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index de74c3e..b277f5e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -1535,7 +1535,8 @@ public final class FunctionRegistry {
     }
 
     if (clazz != null) {
-      return system.isPermanentFunc(clazz);
+      // Use session registry - see Registry.isPermanentFunc()
+      return SessionState.getRegistryForWrite().isPermanentFunc(clazz);
     }
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java
index 05926b5..c274933 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionResource;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionType;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -104,31 +105,37 @@ public class Registry {
    * @return true if udfClass's type was recognized (so registration
    *         succeeded); false otherwise
    */
-  @SuppressWarnings("unchecked")
   public FunctionInfo registerFunction(
       String functionName, Class<?> udfClass, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerFunction(functionName, functionType, udfClass, resources);
+  }
+
+  @SuppressWarnings("unchecked")
+  private FunctionInfo registerFunction(
+      String functionName, FunctionType functionType, Class<?> udfClass, FunctionResource... resources) {
 
     FunctionUtils.UDFClassType udfClassType = FunctionUtils.getUDFClassType(udfClass);
     switch (udfClassType) {
       case UDF:
         return registerUDF(
-            functionName, (Class<? extends UDF>) udfClass, false, resources);
+            functionName, functionType, (Class<? extends UDF>) udfClass, false, functionName.toLowerCase(), resources);
       case GENERIC_UDF:
         return registerGenericUDF(
-            functionName, (Class<? extends GenericUDF>) udfClass, resources);
+            functionName, functionType, (Class<? extends GenericUDF>) udfClass, resources);
       case GENERIC_UDTF:
         return registerGenericUDTF(
-            functionName, (Class<? extends GenericUDTF>) udfClass, resources);
+            functionName, functionType, (Class<? extends GenericUDTF>) udfClass, resources);
       case UDAF:
         return registerUDAF(
-            functionName, (Class<? extends UDAF>) udfClass, resources);
+            functionName, functionType, (Class<? extends UDAF>) udfClass, resources);
       case GENERIC_UDAF_RESOLVER:
         return registerGenericUDAF(
-            functionName, (GenericUDAFResolver)
-            ReflectionUtil.newInstance(udfClass, null), resources);
+            functionName, functionType,
+            (GenericUDAFResolver) ReflectionUtil.newInstance(udfClass, null), resources);
       case TABLE_FUNCTION_RESOLVER:
         // native or not would be decided by annotation. need to evaluate that first
-        return registerTableFunction(functionName,
+        return registerTableFunction(functionName, functionType,
             (Class<? extends TableFunctionResolver>) udfClass, resources);
     }
     return null;
@@ -143,8 +150,15 @@ public class Registry {
   public FunctionInfo registerUDF(String functionName,
       Class<? extends UDF> UDFClass, boolean isOperator, String displayName,
       FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerUDF(functionName, functionType, UDFClass, isOperator, displayName);
+  }
+
+  private FunctionInfo registerUDF(String functionName, FunctionType functionType,
+      Class<? extends UDF> UDFClass, boolean isOperator, String displayName,
+      FunctionResource... resources) {
     validateClass(UDFClass, UDF.class);
-    FunctionInfo fI = new FunctionInfo(isNative, displayName,
+    FunctionInfo fI = new FunctionInfo(functionType, displayName,
         new GenericUDFBridge(displayName, isOperator, UDFClass.getName()), resources);
     addFunction(functionName, fI);
     return fI;
@@ -152,8 +166,14 @@ public class Registry {
 
   public FunctionInfo registerGenericUDF(String functionName,
       Class<? extends GenericUDF> genericUDFClass, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerGenericUDF(functionName, functionType, genericUDFClass, resources);
+  }
+
+  private FunctionInfo registerGenericUDF(String functionName, FunctionType functionType,
+      Class<? extends GenericUDF> genericUDFClass, FunctionResource... resources) {
     validateClass(genericUDFClass, GenericUDF.class);
-    FunctionInfo fI = new FunctionInfo(isNative, functionName,
+    FunctionInfo fI = new FunctionInfo(functionType, functionName,
         ReflectionUtil.newInstance(genericUDFClass, null), resources);
     addFunction(functionName, fI);
     return fI;
@@ -177,8 +197,14 @@ public class Registry {
 
   public FunctionInfo registerGenericUDTF(String functionName,
       Class<? extends GenericUDTF> genericUDTFClass, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerGenericUDTF(functionName, functionType, genericUDTFClass, resources);
+  }
+
+  private FunctionInfo registerGenericUDTF(String functionName, FunctionType functionType,
+      Class<? extends GenericUDTF> genericUDTFClass, FunctionResource... resources) {
     validateClass(genericUDTFClass, GenericUDTF.class);
-    FunctionInfo fI = new FunctionInfo(isNative, functionName,
+    FunctionInfo fI = new FunctionInfo(functionType, functionName,
         ReflectionUtil.newInstance(genericUDTFClass, null), resources);
     addFunction(functionName, fI);
     return fI;
@@ -186,8 +212,14 @@ public class Registry {
 
   public FunctionInfo registerGenericUDAF(String functionName,
       GenericUDAFResolver genericUDAFResolver, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerGenericUDAF(functionName, functionType, genericUDAFResolver, resources);
+  }
+
+  private FunctionInfo registerGenericUDAF(String functionName, FunctionType functionType,
+      GenericUDAFResolver genericUDAFResolver, FunctionResource... resources) {
     FunctionInfo function =
-        new WindowFunctionInfo(isNative, functionName, genericUDAFResolver, resources);
+        new WindowFunctionInfo(functionType, functionName, genericUDAFResolver, resources);
     addFunction(functionName, function);
     addFunction(WINDOW_FUNC_PREFIX + functionName, function);
     return function;
@@ -195,8 +227,14 @@ public class Registry {
 
   public FunctionInfo registerUDAF(String functionName,
       Class<? extends UDAF> udafClass, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerUDAF(functionName, functionType, udafClass, resources);
+  }
+
+  private FunctionInfo registerUDAF(String functionName, FunctionType functionType,
+      Class<? extends UDAF> udafClass, FunctionResource... resources) {
     validateClass(udafClass, UDAF.class);
-    FunctionInfo function = new WindowFunctionInfo(isNative, functionName,
+    FunctionInfo function = new WindowFunctionInfo(functionType, functionName,
         new GenericUDAFBridge(ReflectionUtil.newInstance(udafClass, null)), resources);
     addFunction(functionName, function);
     addFunction(WINDOW_FUNC_PREFIX + functionName, function);
@@ -205,8 +243,14 @@ public class Registry {
 
   public FunctionInfo registerTableFunction(String functionName,
       Class<? extends TableFunctionResolver> tFnCls, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerTableFunction(functionName, functionType, tFnCls, resources);
+  }
+
+  private FunctionInfo registerTableFunction(String functionName, FunctionType functionType,
+      Class<? extends TableFunctionResolver> tFnCls, FunctionResource... resources) {
     validateClass(tFnCls, TableFunctionResolver.class);
-    FunctionInfo function = new FunctionInfo(isNative, functionName, tFnCls, resources);
+    FunctionInfo function = new FunctionInfo(functionType, functionName, tFnCls, resources);
     addFunction(functionName, function);
     return function;
   }
@@ -219,7 +263,7 @@ public class Registry {
   public FunctionInfo registerMacro(String macroName, ExprNodeDesc body,
       List<String> colNames, List<TypeInfo> colTypes, FunctionResource... resources) {
     GenericUDFMacro macro = new GenericUDFMacro(macroName, body, colNames, colTypes);
-    FunctionInfo fI = new FunctionInfo(isNative, macroName, macro, resources);
+    FunctionInfo fI = new FunctionInfo(FunctionType.TEMPORARY, macroName, macro, resources);
     addFunction(macroName, fI);
     return fI;
   }
@@ -252,7 +296,8 @@ public class Registry {
    * @param wFn
    */
   void registerWindowFunction(String name, GenericUDAFResolver wFn) {
-    addFunction(WINDOW_FUNC_PREFIX + name, new WindowFunctionInfo(isNative, name, wFn, null));
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    addFunction(WINDOW_FUNC_PREFIX + name, new WindowFunctionInfo(functionType, name, wFn, null));
   }
 
   private void validateClass(Class input, Class expected) {
@@ -310,6 +355,12 @@ public class Registry {
   }
 
   public boolean isPermanentFunc(Class<?> udfClass) {
+    // Note that permanent functions can only be properly checked from the session registry.
+    // If permanent functions are read from the metastore during Hive initialization,
+    // the JARs are not loaded for the UDFs during that time and so Hive is unable to instantiate
+    // the UDf classes to add to the persistent functions set.
+    // Once a permanent UDF has been referenced in a session its FunctionInfo should be registered
+    // in the session registry (and persistent set updated), so it can be looked up there.
     return udfClass != null && persistent.containsKey(udfClass);
   }
 
@@ -448,7 +499,10 @@ public class Registry {
   private void addFunction(String functionName, FunctionInfo function) {
     lock.lock();
     try {
-      if (isNative != function.isNative()) {
+      // Built-in functions shouldn't go in the session registry,
+      // and temp functions shouldn't go in the system registry.
+      // Persistent functions can be in either registry.
+      if ((!isNative && function.isBuiltIn()) || (isNative && !function.isNative())) {
         throw new RuntimeException("Function " + functionName + " is not for this registry");
       }
       functionName = functionName.toLowerCase();
@@ -463,7 +517,8 @@ public class Registry {
       mFunctions.put(functionName, function);
       if (function.isBuiltIn()) {
         builtIns.add(function.getFunctionClass());
-      } else if (function.isPersistent()) {
+      } else if (function.isPersistent() && !isNative) {
+        // System registry should not be used to check persistent functions - see isPermanentFunc()
         Class<?> functionClass = getPermanentUdfClass(function);
         Integer refCount = persistent.get(functionClass);
         persistent.put(functionClass, Integer.valueOf(refCount == null ? 1 : refCount + 1));
@@ -597,7 +652,10 @@ public class Registry {
       ClassLoader loader = Utilities.getSessionSpecifiedClassLoader();
       Class<?> udfClass = Class.forName(function.getClassName(), true, loader);
 
-      ret = FunctionRegistry.registerTemporaryUDF(qualifiedName, udfClass, resources);
+      // Make sure the FunctionInfo is listed as PERSISTENT (rather than TEMPORARY)
+      // when it is registered to the system registry.
+      ret = SessionState.getRegistryForWrite().registerFunction(
+          qualifiedName, FunctionType.PERSISTENT, udfClass, resources);
       if (ret == null) {
         LOG.error(function.getClassName() + " is not a valid UDF class and was not registered.");
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java
index a16d9e5..5e57ccd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java
@@ -28,9 +28,9 @@ public class WindowFunctionInfo extends FunctionInfo {
   private final boolean pivotResult;
   private final boolean impliesOrder;
 
-  public WindowFunctionInfo(boolean isNative, String functionName,
+  public WindowFunctionInfo(FunctionType functionType, String functionName,
       GenericUDAFResolver resolver, FunctionResource[] resources) {
-    super(isNative, functionName, resolver, resources);
+    super(functionType, functionName, resolver, resources);
     WindowFunctionDescription def =
         AnnotationUtils.getAnnotation(resolver.getClass(), WindowFunctionDescription.class);
     supportsWindow = def == null ? true : def.supportsWindow();

http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
index 911b86b..53e4a2a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
@@ -191,7 +191,7 @@ public class SqlFunctionConverter {
       throw new RuntimeException(e);
     }
     return new FunctionInfo(
-        fi.isNative(), fi.getDisplayName(), (GenericUDF) udf, fi.getResources());
+        fi.getFunctionType(), fi.getDisplayName(), (GenericUDF) udf, fi.getResources());
   }
 
   // TODO: 1) handle Agg Func Name translation 2) is it correct to add func

http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
index d2d5a1b..2192a89 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
@@ -27,7 +27,17 @@ import junit.framework.TestCase;
 
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionResource;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionType;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFLn;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFConcat;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCurrentTimestamp;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFExplode;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
@@ -406,4 +416,94 @@ public class TestFunctionRegistry extends TestCase {
     Assert.assertTrue(FunctionRegistry.impliesOrder("lag"));
     Assert.assertFalse(FunctionRegistry.impliesOrder("min"));
   }
+
+  public void testRegisterTemporaryFunctions() throws Exception {
+    FunctionResource[] emptyResources = new FunctionResource[] {};
+
+    // UDF
+    FunctionRegistry.registerTemporaryUDF("tmp_ln", UDFLn.class, emptyResources);
+    FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo("tmp_ln");
+    assertFalse(functionInfo.isNative());
+
+    // GenericUDF
+    FunctionRegistry.registerTemporaryUDF("tmp_concat", GenericUDFConcat.class, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("tmp_concat");
+    assertFalse(functionInfo.isNative());
+
+    // GenericUDAF
+    FunctionRegistry.registerTemporaryUDF("tmp_max",GenericUDAFMax.class, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("tmp_max");
+    assertFalse(functionInfo.isNative());
+    functionInfo = FunctionRegistry.getWindowFunctionInfo("tmp_max");
+    assertFalse(functionInfo.isNative());
+
+    // UDTF
+    FunctionRegistry.registerTemporaryUDF("tmp_explode", GenericUDTFExplode.class, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("tmp_explode");
+    assertFalse(functionInfo.isNative());
+  }
+
+  public void testRegisterPermanentFunction() throws Exception {
+    FunctionResource[] emptyResources = new FunctionResource[] {};
+
+    // UDF
+    FunctionRegistry.registerPermanentFunction("perm_ln", UDFLn.class.getName(), true, emptyResources);
+    FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo("perm_ln");
+    assertTrue(functionInfo.isPersistent());
+    assertTrue(functionInfo.isNative());
+    assertFalse(functionInfo.isBuiltIn());
+    functionInfo = FunctionRegistry.getFunctionInfo("default.perm_ln");
+    assertTrue(functionInfo.isPersistent());
+    assertTrue(functionInfo.isNative());
+    assertFalse(functionInfo.isBuiltIn());
+
+    // GenericUDF
+    FunctionRegistry.registerPermanentFunction("default.perm_concat",
+        GenericUDFConcat.class.getName(), true, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("default.perm_concat");
+    assertTrue(functionInfo.isPersistent());
+    assertTrue(functionInfo.isNative());
+    assertFalse(functionInfo.isBuiltIn());
+
+    // GenericUDAF
+    FunctionRegistry.registerPermanentFunction("default.perm_max",
+        GenericUDAFMax.class.getName(), true, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("default.perm_max");
+    assertTrue(functionInfo.isPersistent());
+    functionInfo = FunctionRegistry.getWindowFunctionInfo("default.perm_max");
+    assertTrue(functionInfo.isPersistent());
+    assertTrue(functionInfo.isNative());
+    assertFalse(functionInfo.isBuiltIn());
+
+    // UDTF
+    FunctionRegistry.registerPermanentFunction("default.perm_explode",
+        GenericUDTFExplode.class.getName(), true, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("default.perm_explode");
+    assertTrue(functionInfo.isPersistent());
+    assertTrue(functionInfo.isNative());
+    assertFalse(functionInfo.isBuiltIn());
+  }
+
+  public void testBuiltInFunction() throws Exception {
+    FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo("ln");
+    assertTrue(functionInfo.isBuiltIn());
+    assertTrue(functionInfo.isNative());
+  }
+
+  public void testIsPermanentFunction() throws Exception {
+    // Setup exprNode
+    GenericUDF udf = new GenericUDFCurrentTimestamp();
+    List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+    ExprNodeGenericFuncDesc fnExpr =
+        new ExprNodeGenericFuncDesc(TypeInfoFactory.timestampTypeInfo, udf, children);
+
+    assertFalse("Function not added as permanent yet", FunctionRegistry.isPermanentFunction(fnExpr));
+
+    // Now register as permanent function
+    FunctionResource[] emptyResources = new FunctionResource[] {};
+    FunctionRegistry.registerPermanentFunction("default.perm_current_timestamp",
+        GenericUDFCurrentTimestamp.class.getName(), true, emptyResources);
+
+    assertTrue("Function should now be recognized as permanent function", FunctionRegistry.isPermanentFunction(fnExpr));
+  }
 }


[07/44] hive git commit: HIVE-14714: Avoid misleading "java.io.IOException: Stream closed" when shutting down HoS (Gabor Szadovszky via Rui)

Posted by se...@apache.org.
HIVE-14714: Avoid misleading "java.io.IOException: Stream closed" when shutting down HoS (Gabor Szadovszky via Rui)


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

Branch: refs/heads/hive-14535
Commit: 96508d341fbae4d9ff4268bdcff4849893d9c277
Parents: 62c45de
Author: Gabor Szadovszky <ga...@cloudera.com>
Authored: Wed Sep 21 16:17:07 2016 +0800
Committer: Rui Li <sh...@cn.ibm.com>
Committed: Wed Sep 21 16:17:07 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/hive/spark/client/SparkClientImpl.java   | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/96508d34/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
index e8ca42a..936fdaf 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
@@ -621,6 +621,14 @@ class SparkClientImpl implements SparkClient {
             }
           }
         }
+      } catch (IOException e) {
+        if (isAlive) {
+          LOG.warn("I/O error in redirector thread.", e);
+        } else {
+          // When stopping the remote driver the process might be destroyed during reading from the stream.
+          // We should not log the related exceptions in a visible level as they might mislead the user.
+          LOG.debug("I/O error in redirector thread while stopping the remote driver", e);
+        }
       } catch (Exception e) {
         LOG.warn("Error in redirector thread.", e);
       }


[10/44] hive git commit: HIVE-14461. Move hbase_bulk to run via TestCliDriver. (Siddharth Seth, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-14461. Move hbase_bulk to run via TestCliDriver. (Siddharth Seth, 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/91082e5f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/91082e5f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/91082e5f

Branch: refs/heads/hive-14535
Commit: 91082e5fffbce87029e0a0280d50693a0ffdb1f8
Parents: 66af764
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Sep 21 14:35:53 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed Sep 21 14:35:53 2016 -0700

----------------------------------------------------------------------
 .../src/test/queries/positive/hbase_bulk.m      |  62 ---------
 .../src/test/queries/positive/hbase_bulk.q      |  62 +++++++++
 .../src/test/results/positive/hbase_bulk.m.out  | 133 -------------------
 .../src/test/results/positive/hbase_bulk.q.out  | 133 +++++++++++++++++++
 .../hive/cli/TestHBaseMinimrCliDriver.java      |  62 ---------
 .../hadoop/hive/cli/control/CliConfigs.java     |  22 ---
 6 files changed, 195 insertions(+), 279 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/91082e5f/hbase-handler/src/test/queries/positive/hbase_bulk.m
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_bulk.m b/hbase-handler/src/test/queries/positive/hbase_bulk.m
deleted file mode 100644
index f8bb47d..0000000
--- a/hbase-handler/src/test/queries/positive/hbase_bulk.m
+++ /dev/null
@@ -1,62 +0,0 @@
-drop table hbsort;
-drop table hbpartition;
-
-set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
-
--- this is a dummy table used for controlling how the HFiles are
--- created
-create table hbsort(key string, val string, val2 string)
-stored as
-INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
-OUTPUTFORMAT 'org.apache.hadoop.hive.hbase.HiveHFileOutputFormat'
-TBLPROPERTIES ('hfile.family.path' = '/tmp/hbsort/cf');
-
--- this is a dummy table used for controlling how the input file
--- for TotalOrderPartitioner is created
-create table hbpartition(part_break string)
-row format serde
-'org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe'
-stored as
-inputformat
-'org.apache.hadoop.mapred.TextInputFormat'
-outputformat
-'org.apache.hadoop.hive.ql.io.HiveNullValueSequenceFileOutputFormat'
-location '/tmp/data/hbpartition';
-
--- this should produce one file, but we do not
--- know what it will be called, so we will copy it to a well known
--- filename /tmp/hbpartition.lst
-insert overwrite table hbpartition
-select distinct value
-from src
-where value='val_100' or value='val_200';
-
-dfs -count /tmp/data/hbpartition;
-dfs -cp /tmp/data/hbpartition/* /tmp/hbpartition.lst;
-
-set mapred.reduce.tasks=3;
-set hive.mapred.partitioner=org.apache.hadoop.mapred.lib.TotalOrderPartitioner;
-set total.order.partitioner.natural.order=false;
-set total.order.partitioner.path=/tmp/hbpartition.lst;
-set mapreduce.totalorderpartitioner.naturalorder=false;
-set mapreduce.totalorderpartitioner.path=/tmp/hbpartition.lst;
-
--- this should produce three files in /tmp/hbsort/cf
--- include some trailing blanks and nulls to make sure we handle them correctly
-insert overwrite table hbsort
-select distinct value,
-  case when key=103 then cast(null as string) else key end,
-  case when key=103 then ''
-       else cast(key+1 as string) end
-from src
-cluster by value;
-
-dfs -count /tmp/hbsort/cf;
-
--- To get the files out to your local filesystem for loading into
--- HBase, run mkdir -p /tmp/blah/cf, then uncomment and
--- semicolon-terminate the line below before running this test:
--- dfs -copyToLocal /tmp/hbsort/cf/* /tmp/blah/cf
-
-drop table hbsort;
-drop table hbpartition;

http://git-wip-us.apache.org/repos/asf/hive/blob/91082e5f/hbase-handler/src/test/queries/positive/hbase_bulk.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_bulk.q b/hbase-handler/src/test/queries/positive/hbase_bulk.q
new file mode 100644
index 0000000..f8bb47d
--- /dev/null
+++ b/hbase-handler/src/test/queries/positive/hbase_bulk.q
@@ -0,0 +1,62 @@
+drop table hbsort;
+drop table hbpartition;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+-- this is a dummy table used for controlling how the HFiles are
+-- created
+create table hbsort(key string, val string, val2 string)
+stored as
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+OUTPUTFORMAT 'org.apache.hadoop.hive.hbase.HiveHFileOutputFormat'
+TBLPROPERTIES ('hfile.family.path' = '/tmp/hbsort/cf');
+
+-- this is a dummy table used for controlling how the input file
+-- for TotalOrderPartitioner is created
+create table hbpartition(part_break string)
+row format serde
+'org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe'
+stored as
+inputformat
+'org.apache.hadoop.mapred.TextInputFormat'
+outputformat
+'org.apache.hadoop.hive.ql.io.HiveNullValueSequenceFileOutputFormat'
+location '/tmp/data/hbpartition';
+
+-- this should produce one file, but we do not
+-- know what it will be called, so we will copy it to a well known
+-- filename /tmp/hbpartition.lst
+insert overwrite table hbpartition
+select distinct value
+from src
+where value='val_100' or value='val_200';
+
+dfs -count /tmp/data/hbpartition;
+dfs -cp /tmp/data/hbpartition/* /tmp/hbpartition.lst;
+
+set mapred.reduce.tasks=3;
+set hive.mapred.partitioner=org.apache.hadoop.mapred.lib.TotalOrderPartitioner;
+set total.order.partitioner.natural.order=false;
+set total.order.partitioner.path=/tmp/hbpartition.lst;
+set mapreduce.totalorderpartitioner.naturalorder=false;
+set mapreduce.totalorderpartitioner.path=/tmp/hbpartition.lst;
+
+-- this should produce three files in /tmp/hbsort/cf
+-- include some trailing blanks and nulls to make sure we handle them correctly
+insert overwrite table hbsort
+select distinct value,
+  case when key=103 then cast(null as string) else key end,
+  case when key=103 then ''
+       else cast(key+1 as string) end
+from src
+cluster by value;
+
+dfs -count /tmp/hbsort/cf;
+
+-- To get the files out to your local filesystem for loading into
+-- HBase, run mkdir -p /tmp/blah/cf, then uncomment and
+-- semicolon-terminate the line below before running this test:
+-- dfs -copyToLocal /tmp/hbsort/cf/* /tmp/blah/cf
+
+drop table hbsort;
+drop table hbpartition;

http://git-wip-us.apache.org/repos/asf/hive/blob/91082e5f/hbase-handler/src/test/results/positive/hbase_bulk.m.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_bulk.m.out b/hbase-handler/src/test/results/positive/hbase_bulk.m.out
deleted file mode 100644
index 0dc5802..0000000
--- a/hbase-handler/src/test/results/positive/hbase_bulk.m.out
+++ /dev/null
@@ -1,133 +0,0 @@
-PREHOOK: query: drop table hbsort
-PREHOOK: type: DROPTABLE
-POSTHOOK: query: drop table hbsort
-POSTHOOK: type: DROPTABLE
-PREHOOK: query: drop table hbpartition
-PREHOOK: type: DROPTABLE
-POSTHOOK: query: drop table hbpartition
-POSTHOOK: type: DROPTABLE
-PREHOOK: query: -- this is a dummy table used for controlling how the HFiles are
--- created
-create table hbsort(key string, val string, val2 string)
-stored as
-INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
-OUTPUTFORMAT 'org.apache.hadoop.hive.hbase.HiveHFileOutputFormat'
-#### A masked pattern was here ####
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@hbsort
-POSTHOOK: query: -- this is a dummy table used for controlling how the HFiles are
--- created
-create table hbsort(key string, val string, val2 string)
-stored as
-INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
-OUTPUTFORMAT 'org.apache.hadoop.hive.hbase.HiveHFileOutputFormat'
-#### A masked pattern was here ####
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@hbsort
-PREHOOK: query: -- this is a dummy table used for controlling how the input file
--- for TotalOrderPartitioner is created
-create table hbpartition(part_break string)
-row format serde
-'org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe'
-stored as
-inputformat
-'org.apache.hadoop.mapred.TextInputFormat'
-outputformat
-'org.apache.hadoop.hive.ql.io.HiveNullValueSequenceFileOutputFormat'
-#### A masked pattern was here ####
-PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
-PREHOOK: Output: database:default
-PREHOOK: Output: default@hbpartition
-POSTHOOK: query: -- this is a dummy table used for controlling how the input file
--- for TotalOrderPartitioner is created
-create table hbpartition(part_break string)
-row format serde
-'org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe'
-stored as
-inputformat
-'org.apache.hadoop.mapred.TextInputFormat'
-outputformat
-'org.apache.hadoop.hive.ql.io.HiveNullValueSequenceFileOutputFormat'
-#### A masked pattern was here ####
-POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@hbpartition
-PREHOOK: query: -- this should produce one file, but we do not
--- know what it will be called, so we will copy it to a well known
-#### A masked pattern was here ####
-insert overwrite table hbpartition
-select distinct value
-from src
-where value='val_100' or value='val_200'
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@hbpartition
-POSTHOOK: query: -- this should produce one file, but we do not
--- know what it will be called, so we will copy it to a well known
-#### A masked pattern was here ####
-insert overwrite table hbpartition
-select distinct value
-from src
-where value='val_100' or value='val_200'
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@hbpartition
-POSTHOOK: Lineage: hbpartition.part_break SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-#### A masked pattern was here ####
--- include some trailing blanks and nulls to make sure we handle them correctly
-insert overwrite table hbsort
-select distinct value,
-  case when key=103 then cast(null as string) else key end,
-  case when key=103 then ''
-       else cast(key+1 as string) end
-from src
-cluster by value
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@hbsort
-#### A masked pattern was here ####
--- include some trailing blanks and nulls to make sure we handle them correctly
-insert overwrite table hbsort
-select distinct value,
-  case when key=103 then cast(null as string) else key end,
-  case when key=103 then ''
-       else cast(key+1 as string) end
-from src
-cluster by value
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@hbsort
-POSTHOOK: Lineage: hbsort.key SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: hbsort.val EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: hbsort.val2 EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-#### A masked pattern was here ####
-PREHOOK: query: -- To get the files out to your local filesystem for loading into
-#### A masked pattern was here ####
--- semicolon-terminate the line below before running this test:
-#### A masked pattern was here ####
-
-drop table hbsort
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@hbsort
-PREHOOK: Output: default@hbsort
-POSTHOOK: query: -- To get the files out to your local filesystem for loading into
-#### A masked pattern was here ####
--- semicolon-terminate the line below before running this test:
-#### A masked pattern was here ####
-
-drop table hbsort
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@hbsort
-POSTHOOK: Output: default@hbsort
-PREHOOK: query: drop table hbpartition
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@hbpartition
-PREHOOK: Output: default@hbpartition
-POSTHOOK: query: drop table hbpartition
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@hbpartition
-POSTHOOK: Output: default@hbpartition

http://git-wip-us.apache.org/repos/asf/hive/blob/91082e5f/hbase-handler/src/test/results/positive/hbase_bulk.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_bulk.q.out b/hbase-handler/src/test/results/positive/hbase_bulk.q.out
new file mode 100644
index 0000000..0dc5802
--- /dev/null
+++ b/hbase-handler/src/test/results/positive/hbase_bulk.q.out
@@ -0,0 +1,133 @@
+PREHOOK: query: drop table hbsort
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table hbsort
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table hbpartition
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table hbpartition
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: -- this is a dummy table used for controlling how the HFiles are
+-- created
+create table hbsort(key string, val string, val2 string)
+stored as
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+OUTPUTFORMAT 'org.apache.hadoop.hive.hbase.HiveHFileOutputFormat'
+#### A masked pattern was here ####
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hbsort
+POSTHOOK: query: -- this is a dummy table used for controlling how the HFiles are
+-- created
+create table hbsort(key string, val string, val2 string)
+stored as
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+OUTPUTFORMAT 'org.apache.hadoop.hive.hbase.HiveHFileOutputFormat'
+#### A masked pattern was here ####
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hbsort
+PREHOOK: query: -- this is a dummy table used for controlling how the input file
+-- for TotalOrderPartitioner is created
+create table hbpartition(part_break string)
+row format serde
+'org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe'
+stored as
+inputformat
+'org.apache.hadoop.mapred.TextInputFormat'
+outputformat
+'org.apache.hadoop.hive.ql.io.HiveNullValueSequenceFileOutputFormat'
+#### A masked pattern was here ####
+PREHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hbpartition
+POSTHOOK: query: -- this is a dummy table used for controlling how the input file
+-- for TotalOrderPartitioner is created
+create table hbpartition(part_break string)
+row format serde
+'org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe'
+stored as
+inputformat
+'org.apache.hadoop.mapred.TextInputFormat'
+outputformat
+'org.apache.hadoop.hive.ql.io.HiveNullValueSequenceFileOutputFormat'
+#### A masked pattern was here ####
+POSTHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hbpartition
+PREHOOK: query: -- this should produce one file, but we do not
+-- know what it will be called, so we will copy it to a well known
+#### A masked pattern was here ####
+insert overwrite table hbpartition
+select distinct value
+from src
+where value='val_100' or value='val_200'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@hbpartition
+POSTHOOK: query: -- this should produce one file, but we do not
+-- know what it will be called, so we will copy it to a well known
+#### A masked pattern was here ####
+insert overwrite table hbpartition
+select distinct value
+from src
+where value='val_100' or value='val_200'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@hbpartition
+POSTHOOK: Lineage: hbpartition.part_break SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+#### A masked pattern was here ####
+-- include some trailing blanks and nulls to make sure we handle them correctly
+insert overwrite table hbsort
+select distinct value,
+  case when key=103 then cast(null as string) else key end,
+  case when key=103 then ''
+       else cast(key+1 as string) end
+from src
+cluster by value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@hbsort
+#### A masked pattern was here ####
+-- include some trailing blanks and nulls to make sure we handle them correctly
+insert overwrite table hbsort
+select distinct value,
+  case when key=103 then cast(null as string) else key end,
+  case when key=103 then ''
+       else cast(key+1 as string) end
+from src
+cluster by value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@hbsort
+POSTHOOK: Lineage: hbsort.key SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: hbsort.val EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: hbsort.val2 EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+#### A masked pattern was here ####
+PREHOOK: query: -- To get the files out to your local filesystem for loading into
+#### A masked pattern was here ####
+-- semicolon-terminate the line below before running this test:
+#### A masked pattern was here ####
+
+drop table hbsort
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@hbsort
+PREHOOK: Output: default@hbsort
+POSTHOOK: query: -- To get the files out to your local filesystem for loading into
+#### A masked pattern was here ####
+-- semicolon-terminate the line below before running this test:
+#### A masked pattern was here ####
+
+drop table hbsort
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@hbsort
+POSTHOOK: Output: default@hbsort
+PREHOOK: query: drop table hbpartition
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@hbpartition
+PREHOOK: Output: default@hbpartition
+POSTHOOK: query: drop table hbpartition
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@hbpartition
+POSTHOOK: Output: default@hbpartition

http://git-wip-us.apache.org/repos/asf/hive/blob/91082e5f/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseMinimrCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseMinimrCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseMinimrCliDriver.java
deleted file mode 100644
index 934af16..0000000
--- a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseMinimrCliDriver.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * 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.
- */
-package org.apache.hadoop.hive.cli;
-
-import java.io.File;
-import java.util.List;
-
-import org.apache.hadoop.hive.cli.control.CliAdapter;
-import org.apache.hadoop.hive.cli.control.CliConfigs;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-@RunWith(Parameterized.class)
-public class TestHBaseMinimrCliDriver {
-
-  static CliAdapter adapter = new CliConfigs.HBaseMinimrCliConfig().getCliAdapter();
-
-  @Parameters(name = "{0}")
-  public static List<Object[]> getParameters() throws Exception {
-    return adapter.getParameters();
-  }
-
-  @ClassRule
-  public static TestRule cliClassRule = adapter.buildClassRule();
-
-  @Rule
-  public TestRule cliTestRule = adapter.buildTestRule();
-
-  private String name;
-  private File qfile;
-
-  public TestHBaseMinimrCliDriver(String name, File qfile) {
-    this.name = name;
-    this.qfile = qfile;
-  }
-
-  @Test
-  public void testCliDriver() throws Exception {
-    adapter.runTest(name, qfile);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/91082e5f/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
index d74f51a..0068b95 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
@@ -307,28 +307,6 @@ public class CliConfigs {
     }
   }
 
-  public static class HBaseMinimrCliConfig extends AbstractCliConfig {
-    public HBaseMinimrCliConfig() {
-      super(CoreHBaseCliDriver.class);
-      try {
-        setQueryDir("hbase-handler/src/test/queries/positive");
-        // XXX: i think this was non intentionally set to run only hbase_bulk.m???
-        // includeQuery("hbase_bulk.m"); => will be filter out because not ends with .q
-        // to keep existing behaviour i added this method
-        overrideUserQueryFile("hbase_bulk.m");
-
-        setResultsDir("hbase-handler/src/test/results/positive");
-        setLogDir("itests/qtest/target/qfile-results/hbase-handler/minimrpositive");
-        setInitScript("q_test_init_for_minimr.sql");
-        setCleanupScript("q_test_cleanup.sql");
-        setHiveConfDir("");
-        setClusterType(MiniClusterType.mr);
-      } catch (Exception e) {
-        throw new RuntimeException("can't construct cliconfig", e);
-      }
-    }
-  }
-
   public static class DummyConfig extends AbstractCliConfig {
     public DummyConfig() {
       super(CoreDummy.class);


[18/44] hive git commit: HIVE-14820: RPC server for spark inside HS2 is not getting server address properly (Aihua Xu, reviewed by Yongzhi Chen)

Posted by se...@apache.org.
HIVE-14820: RPC server for spark inside HS2 is not getting server address properly (Aihua Xu, reviewed by Yongzhi Chen)


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

Branch: refs/heads/hive-14535
Commit: 421d97a8d75490ca8ec698ef67f7ed8739e394f8
Parents: f284b6d
Author: Aihua Xu <ai...@apache.org>
Authored: Thu Sep 22 15:46:21 2016 -0400
Committer: Aihua Xu <ai...@apache.org>
Committed: Fri Sep 23 09:15:36 2016 -0400

----------------------------------------------------------------------
 .../hive/spark/client/rpc/RpcConfiguration.java |  4 +--
 .../apache/hive/spark/client/rpc/TestRpc.java   | 29 +++++++++++++++++++-
 2 files changed, 30 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/421d97a8/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java
index e387659..210f8a4 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java
@@ -97,11 +97,11 @@ public final class RpcConfiguration {
    * @throws IOException
    */
   String getServerAddress() throws IOException {
-    String hiveHost = config.get(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS);
+    String hiveHost = config.get(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS.varname);
     if(StringUtils.isEmpty(hiveHost)) {
       hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST");
       if (hiveHost == null) {
-        hiveHost = config.get(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST);
+        hiveHost = config.get(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname);
       }
     }
     return ServerUtils.getHostAddress(hiveHost).getHostName();

http://git-wip-us.apache.org/repos/asf/hive/blob/421d97a8/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
----------------------------------------------------------------------
diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
index d7969c9..7bcf1df 100644
--- a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
+++ b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
@@ -18,26 +18,30 @@
 package org.apache.hive.spark.client.rpc;
 
 import java.io.Closeable;
+import java.net.InetAddress;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
+
 import javax.security.sasl.SaslException;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
+
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.embedded.EmbeddedChannel;
 import io.netty.channel.nio.NioEventLoopGroup;
 import io.netty.util.concurrent.Future;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -117,6 +121,29 @@ public class TestRpc {
   }
 
   @Test
+  public void testServerAddress() throws Exception {
+    String hostAddress = InetAddress.getLocalHost().getHostName();
+    Map<String, String> config = new HashMap<String, String>();
+
+    // Test if rpc_server_address is configured
+    config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS.varname, hostAddress);
+    RpcServer server1 = autoClose(new RpcServer(config));
+    assertTrue("Host address should match the expected one", server1.getAddress() == hostAddress);
+
+    // Test if rpc_server_address is not configured but HS2 server host is configured
+    config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS.varname, "");
+    config.put(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, hostAddress);
+    RpcServer server2 = autoClose(new RpcServer(config));
+    assertTrue("Host address should match the expected one", server2.getAddress() == hostAddress);
+
+    // Test if both are not configured
+    config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS.varname, "");
+    config.put(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, "");
+    RpcServer server3 = autoClose(new RpcServer(config));
+    assertTrue("Host address should match the expected one", server3.getAddress() == InetAddress.getLocalHost().getHostName());
+  }
+
+  @Test
   public void testBadHello() throws Exception {
     RpcServer server = autoClose(new RpcServer(emptyConfig));
 


[11/44] hive git commit: HIVE-14783 : bucketing column should be part of sorting for delete/update operation when spdo is on (Ashutosh Chauhan via Prasanth J) Addendum patch

Posted by se...@apache.org.
HIVE-14783 : bucketing column should be part of sorting for delete/update operation when spdo is on (Ashutosh Chauhan via Prasanth J)
Addendum patch

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/hive-14535
Commit: ed82cfa914769cfabfc7460b7b5abbdae71e562a
Parents: 91082e5
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Wed Sep 21 15:18:37 2016 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Wed Sep 21 15:19:18 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FileSinkOperator.java   | 10 +++++-----
 .../hadoop/hive/ql/exec/ReduceSinkOperator.java |  9 +++++++++
 .../optimizer/SortedDynPartitionOptimizer.java  |  4 ++--
 .../dynpart_sort_optimization_acid.q.out        | 20 ++++++++++----------
 4 files changed, 26 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ed82cfa9/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index e386717..eeba6cd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -766,19 +766,19 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
         if (fpaths.acidLastBucket != bucketNum) {
           fpaths.acidLastBucket = bucketNum;
           // Switch files
-          fpaths.updaters[++fpaths.acidFileOffset] = HiveFileFormatUtils.getAcidRecordUpdater(
-              jc, conf.getTableInfo(), bucketNum, conf, fpaths.outPaths[fpaths.acidFileOffset],
+          fpaths.updaters[conf.getDpSortState().equals(DPSortState.PARTITION_BUCKET_SORTED) ? 0 : ++fpaths.acidFileOffset] = HiveFileFormatUtils.getAcidRecordUpdater(
+              jc, conf.getTableInfo(), bucketNum, conf, fpaths.outPaths[conf.getDpSortState().equals(DPSortState.PARTITION_BUCKET_SORTED) ? 0 :fpaths.acidFileOffset],
               rowInspector, reporter, 0);
           if (isDebugEnabled) {
             LOG.debug("Created updater for bucket number " + bucketNum + " using file " +
-                fpaths.outPaths[fpaths.acidFileOffset]);
+                fpaths.outPaths[conf.getDpSortState().equals(DPSortState.PARTITION_BUCKET_SORTED) ? 0 :fpaths.acidFileOffset]);
           }
         }
 
         if (conf.getWriteType() == AcidUtils.Operation.UPDATE) {
-          fpaths.updaters[fpaths.acidFileOffset].update(conf.getTransactionId(), row);
+          fpaths.updaters[conf.getDpSortState().equals(DPSortState.PARTITION_BUCKET_SORTED) ? 0 :fpaths.acidFileOffset].update(conf.getTransactionId(), row);
         } else if (conf.getWriteType() == AcidUtils.Operation.DELETE) {
-          fpaths.updaters[fpaths.acidFileOffset].delete(conf.getTransactionId(), row);
+          fpaths.updaters[conf.getDpSortState().equals(DPSortState.PARTITION_BUCKET_SORTED) ? 0 :fpaths.acidFileOffset].delete(conf.getTransactionId(), row);
         } else {
           throw new HiveException("Unknown write type " + conf.getWriteType().toString());
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/ed82cfa9/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
index a9885d8..4eea6b9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils;
 import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
@@ -78,6 +80,7 @@ public class ReduceSinkOperator extends TerminalOperator<ReduceSinkDesc>
   private transient ObjectInspector[] partitionObjectInspectors;
   private transient ObjectInspector[] bucketObjectInspectors;
   private transient int buckColIdxInKey;
+  private transient int buckColIdxInKeyForAcid = -1;
   private boolean firstRow;
   private transient int tag;
   private boolean skipTag = false;
@@ -183,6 +186,9 @@ public class ReduceSinkOperator extends TerminalOperator<ReduceSinkDesc>
       keyEval = new ExprNodeEvaluator[keys.size()];
       int i = 0;
       for (ExprNodeDesc e : keys) {
+        if (e instanceof ExprNodeConstantDesc && ("_bucket_number").equals(((ExprNodeConstantDesc)e).getValue())) {
+          buckColIdxInKeyForAcid = i;
+        }
         keyEval[i++] = ExprNodeEvaluatorFactory.get(e);
       }
 
@@ -359,6 +365,9 @@ public class ReduceSinkOperator extends TerminalOperator<ReduceSinkDesc>
         // In the non-partitioned case we still want to compute the bucket number for updates and
         // deletes.
         bucketNumber = computeBucketNumber(row, conf.getNumBuckets());
+        if (buckColIdxInKeyForAcid != -1) {
+          cachedKeys[0][buckColIdxInKeyForAcid] = new Text(String.valueOf(bucketNumber));
+        }
       }
 
       HiveKey firstKey = toHiveKey(cachedKeys[0], tag, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/ed82cfa9/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
index 8b4af72..926386b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
@@ -247,7 +247,7 @@ public class SortedDynPartitionOptimizer extends Transform {
         }
       }
       RowSchema selRS = new RowSchema(fsParent.getSchema());
-      if (!bucketColumns.isEmpty()) {
+      if (!bucketColumns.isEmpty() || fsOp.getConf().getWriteType() == Operation.DELETE || fsOp.getConf().getWriteType() == Operation.UPDATE) {
         descs.add(new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, ReduceField.KEY.toString()+".'"+BUCKET_NUMBER_COL_NAME+"'", null, false));
         colNames.add("'"+BUCKET_NUMBER_COL_NAME+"'");
         ColumnInfo ci = new ColumnInfo(BUCKET_NUMBER_COL_NAME, TypeInfoFactory.stringTypeInfo, selRS.getSignature().get(0).getTabAlias(), true, true);
@@ -268,7 +268,7 @@ public class SortedDynPartitionOptimizer extends Transform {
 
       // Set if partition sorted or partition bucket sorted
       fsOp.getConf().setDpSortState(FileSinkDesc.DPSortState.PARTITION_SORTED);
-      if (bucketColumns.size() > 0) {
+      if (bucketColumns.size() > 0 || fsOp.getConf().getWriteType() == Operation.DELETE || fsOp.getConf().getWriteType() == Operation.UPDATE) {
         fsOp.getConf().setDpSortState(FileSinkDesc.DPSortState.PARTITION_BUCKET_SORTED);
       }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ed82cfa9/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
index 1838d6a..111ce18 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
@@ -422,8 +422,8 @@ STAGE PLANS:
                   Statistics: Num rows: 892 Data size: 2676 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), 'foo' (type: string), 'bar' (type: string), KEY._col3 (type: string)
-          outputColumnNames: _col0, _col1, _col2, _col3
+          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), 'foo' (type: string), 'bar' (type: string), KEY._col3 (type: string), KEY.'_bucket_number' (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, '_bucket_number'
           Statistics: Num rows: 892 Data size: 2676 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
@@ -1042,8 +1042,8 @@ STAGE PLANS:
                   Statistics: Num rows: 1517 Data size: 4551 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), KEY._col4 (type: int)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4
+          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), KEY._col4 (type: int), KEY.'_bucket_number' (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, '_bucket_number'
           Statistics: Num rows: 1517 Data size: 4551 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
@@ -1152,8 +1152,8 @@ STAGE PLANS:
                   Statistics: Num rows: 2979 Data size: 8937 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), KEY._col1 (type: string), KEY._col2 (type: int)
-          outputColumnNames: _col0, _col1, _col2
+          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), KEY._col1 (type: string), KEY._col2 (type: int), KEY.'_bucket_number' (type: string)
+          outputColumnNames: _col0, _col1, _col2, '_bucket_number'
           Statistics: Num rows: 2979 Data size: 8937 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
@@ -1327,8 +1327,8 @@ STAGE PLANS:
                   value expressions: _col1 (type: string), 'bar' (type: string)
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4
+          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int), KEY.'_bucket_number' (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, '_bucket_number'
           Statistics: Num rows: 23 Data size: 2322 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
@@ -1407,8 +1407,8 @@ STAGE PLANS:
                   value expressions: _col1 (type: string), 'bar' (type: string)
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4
+          expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int), KEY.'_bucket_number' (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, '_bucket_number'
           Statistics: Num rows: 45 Data size: 4550 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false


[41/44] hive git commit: HIVE-14100: Adding a new logged_in_user() UDF which returns the user provided when connecting (Peter Vary, reviewed by Mohit Sabharwal)

Posted by se...@apache.org.
HIVE-14100: Adding a new logged_in_user() UDF which returns the user provided when connecting (Peter Vary, reviewed by Mohit Sabharwal)


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

Branch: refs/heads/hive-14535
Commit: 45c1a09b7b76e41f05520de4bb0e26bb6fadc21f
Parents: 0562efc
Author: Mohit Sabharwal <mo...@cloudera.com>
Authored: Fri Sep 30 13:54:31 2016 -0400
Committer: Mohit Sabharwal <mo...@cloudera.com>
Committed: Fri Sep 30 13:57:10 2016 -0400

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |  1 +
 .../ql/udf/generic/GenericUDFLoggedInUser.java  | 82 ++++++++++++++++++++
 .../queries/clientpositive/udf_logged_in_user.q |  5 ++
 .../results/clientpositive/show_functions.q.out |  5 ++
 .../clientpositive/udf_logged_in_user.q.out     | 22 ++++++
 5 files changed, 115 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/45c1a09b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index b277f5e..6870dfa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -344,6 +344,7 @@ public final class FunctionRegistry {
     system.registerGenericUDF("current_date", GenericUDFCurrentDate.class);
     system.registerGenericUDF("current_timestamp", GenericUDFCurrentTimestamp.class);
     system.registerGenericUDF("current_user", GenericUDFCurrentUser.class);
+    system.registerGenericUDF("logged_in_user", GenericUDFLoggedInUser.class);
 
     system.registerGenericUDF("isnull", GenericUDFOPNull.class);
     system.registerGenericUDF("isnotnull", GenericUDFOPNotNull.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/45c1a09b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLoggedInUser.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLoggedInUser.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLoggedInUser.java
new file mode 100644
index 0000000..2915b86
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLoggedInUser.java
@@ -0,0 +1,82 @@
+/**
+ * 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.
+ */
+
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFType;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.Text;
+
+@UDFType(deterministic = true)
+@Description(name = "logged_in_user", value = "_FUNC_() - Returns logged in user name",
+        extended = "SessionState GetUserName - the username provided at session initialization")
+@NDV(maxNdv = 1)
+public class GenericUDFLoggedInUser extends GenericUDF {
+  protected Text loggedInUser;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+    if (arguments.length != 0) {
+      throw new UDFArgumentLengthException(
+          "The function LOGGED_IN_USER does not take any arguments, but found " + arguments.length);
+    }
+
+    if (loggedInUser == null) {
+      String loggedInUserName = SessionState.get().getUserName();
+      if (loggedInUserName != null) {
+        loggedInUser = new Text(loggedInUserName);
+      }
+    }
+
+    return PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    return loggedInUser;
+  }
+
+  public Text getLoggedInUser() {
+    return loggedInUser;
+  }
+
+  public void setLoggedInUser(Text loggedInUser) {
+    this.loggedInUser = loggedInUser;
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return "LOGGED_IN_USER()";
+  }
+
+  @Override
+  public void copyToNewInstance(Object newInstance) throws UDFArgumentException {
+    super.copyToNewInstance(newInstance);
+    // Need to preserve loggedInUser
+    GenericUDFLoggedInUser other = (GenericUDFLoggedInUser) newInstance;
+    if (this.loggedInUser != null) {
+      other.loggedInUser = new Text(this.loggedInUser);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/45c1a09b/ql/src/test/queries/clientpositive/udf_logged_in_user.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_logged_in_user.q b/ql/src/test/queries/clientpositive/udf_logged_in_user.q
new file mode 100644
index 0000000..4814c72
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/udf_logged_in_user.q
@@ -0,0 +1,5 @@
+DESCRIBE FUNCTION logged_in_user;
+DESCRIBE FUNCTION EXTENDED logged_in_user;
+
+select logged_in_user()
+FROM src tablesample (1 rows);

http://git-wip-us.apache.org/repos/asf/hive/blob/45c1a09b/ql/src/test/results/clientpositive/show_functions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_functions.q.out b/ql/src/test/results/clientpositive/show_functions.q.out
index 18c7b0e..4a40094 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -130,6 +130,7 @@ locate
 log
 log10
 log2
+logged_in_user
 lower
 lpad
 ltrim
@@ -328,6 +329,7 @@ POSTHOOK: type: SHOWFUNCTIONS
 log
 log10
 log2
+logged_in_user
 PREHOOK: query: SHOW FUNCTIONS '.*date.*'
 PREHOOK: type: SHOWFUNCTIONS
 POSTHOOK: query: SHOW FUNCTIONS '.*date.*'
@@ -408,6 +410,7 @@ POSTHOOK: type: SHOWFUNCTIONS
 log
 log10
 log2
+logged_in_user
 PREHOOK: query: SHOW FUNCTIONS LIKE "log*"
 PREHOOK: type: SHOWFUNCTIONS
 POSTHOOK: query: SHOW FUNCTIONS LIKE "log*"
@@ -415,6 +418,7 @@ POSTHOOK: type: SHOWFUNCTIONS
 log
 log10
 log2
+logged_in_user
 PREHOOK: query: SHOW FUNCTIONS LIKE `log*`
 PREHOOK: type: SHOWFUNCTIONS
 POSTHOOK: query: SHOW FUNCTIONS LIKE `log*`
@@ -422,3 +426,4 @@ POSTHOOK: type: SHOWFUNCTIONS
 log
 log10
 log2
+logged_in_user

http://git-wip-us.apache.org/repos/asf/hive/blob/45c1a09b/ql/src/test/results/clientpositive/udf_logged_in_user.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_logged_in_user.q.out b/ql/src/test/results/clientpositive/udf_logged_in_user.q.out
new file mode 100644
index 0000000..ffb19ca
--- /dev/null
+++ b/ql/src/test/results/clientpositive/udf_logged_in_user.q.out
@@ -0,0 +1,22 @@
+PREHOOK: query: DESCRIBE FUNCTION logged_in_user
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION logged_in_user
+POSTHOOK: type: DESCFUNCTION
+logged_in_user() - Returns logged in user name
+PREHOOK: query: DESCRIBE FUNCTION EXTENDED logged_in_user
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION EXTENDED logged_in_user
+POSTHOOK: type: DESCFUNCTION
+logged_in_user() - Returns logged in user name
+SessionState GetUserName - the username provided at session initialization
+PREHOOK: query: select logged_in_user()
+FROM src tablesample (1 rows)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select logged_in_user()
+FROM src tablesample (1 rows)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+NULL


[13/44] hive git commit: HIVE-14814: metastoreClient is used directly in Hive cause NPE (Prasanth Jayachandran reviewed by Eugene Koifman)

Posted by se...@apache.org.
HIVE-14814: metastoreClient is used directly in Hive cause NPE (Prasanth Jayachandran reviewed by Eugene Koifman)


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

Branch: refs/heads/hive-14535
Commit: 1a3e4be3dbd485f2630c7249254727ce58374d1c
Parents: c9224d5
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Thu Sep 22 10:50:30 2016 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Thu Sep 22 10:50:30 2016 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1a3e4be3/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index da46854..de6adb5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -1864,7 +1864,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
         for (Partition p : partitionsMap.values()) {
           partNames.add(p.getName());
         }
-        metaStoreClient.addDynamicPartitions(txnId, tbl.getDbName(), tbl.getTableName(),
+        getMSC().addDynamicPartitions(txnId, tbl.getDbName(), tbl.getTableName(),
           partNames, AcidUtils.toDataOperationType(operation));
       }
       LOG.info("Loaded " + partitionsMap.size() + " partitions");


[16/44] hive git commit: HIVE-14579: Add support for date extract (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-14579: Add support for date extract (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: da376eba44ab090d693c2f9eb3f94ade04942b52
Parents: e532549
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri Sep 23 10:28:45 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri Sep 23 10:28:45 2016 +0100

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   1 +
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |  23 +++
 ql/src/test/queries/clientpositive/extract.q    |  43 ++++
 .../test/results/clientpositive/extract.q.out   | 200 +++++++++++++++++++
 4 files changed, 267 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/da376eba/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index af659ad..b623187 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -331,6 +331,7 @@ KW_RELY: 'RELY';
 KW_NORELY: 'NORELY';
 KW_KEY: 'KEY';
 KW_ABORT: 'ABORT';
+KW_EXTRACT: 'EXTRACT';
 KW_FLOOR: 'FLOOR';
 
 // Operators

http://git-wip-us.apache.org/repos/asf/hive/blob/da376eba/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index 7842d50..e6b70a0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -265,6 +265,28 @@ floorDateQualifiers
     | KW_SECOND -> Identifier["floor_second"]
     ;
 
+extractExpression
+    :
+    KW_EXTRACT
+    LPAREN
+          (timeUnit=timeQualifiers)
+          KW_FROM
+          expression
+    RPAREN -> ^(TOK_FUNCTION $timeUnit expression)
+    ;
+
+timeQualifiers
+    :
+    KW_YEAR -> Identifier["year"]
+    | KW_QUARTER -> Identifier["quarter"]
+    | KW_MONTH -> Identifier["month"]
+    | KW_WEEK -> Identifier["weekofyear"]
+    | KW_DAY -> Identifier["day"]
+    | KW_HOUR -> Identifier["hour"]
+    | KW_MINUTE -> Identifier["minute"]
+    | KW_SECOND -> Identifier["second"]
+    ;
+
 constant
 @init { gParent.pushMsg("constant", state); }
 @after { gParent.popMsg(state); }
@@ -347,6 +369,7 @@ atomExpression
     (KW_NULL) => KW_NULL -> TOK_NULL
     | (constant) => constant
     | castExpression
+    | extractExpression
     | floorExpression
     | caseExpression
     | whenExpression

http://git-wip-us.apache.org/repos/asf/hive/blob/da376eba/ql/src/test/queries/clientpositive/extract.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/extract.q b/ql/src/test/queries/clientpositive/extract.q
new file mode 100644
index 0000000..c09574b
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/extract.q
@@ -0,0 +1,43 @@
+drop table extract_udf;
+
+create table extract_udf (t timestamp);
+from (select * from src tablesample (1 rows)) s
+  insert overwrite table extract_udf 
+    select '2011-05-06 07:08:09.1234567';
+
+explain
+select day(t)
+from extract_udf;
+
+select day(t)
+from extract_udf;
+
+-- new syntax
+explain
+select extract(day from t)
+from extract_udf;
+
+select extract(day from t)
+from extract_udf;
+
+
+select extract(second from t)
+from extract_udf;
+
+select extract(minute from t)
+from extract_udf;
+
+select extract(hour from t)
+from extract_udf;
+
+select extract(week from t)
+from extract_udf;
+
+select extract(month from t)
+from extract_udf;
+
+select extract(quarter from t)
+from extract_udf;
+
+select extract(year from t)
+from extract_udf;

http://git-wip-us.apache.org/repos/asf/hive/blob/da376eba/ql/src/test/results/clientpositive/extract.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/extract.q.out b/ql/src/test/results/clientpositive/extract.q.out
new file mode 100644
index 0000000..73c9bc6
--- /dev/null
+++ b/ql/src/test/results/clientpositive/extract.q.out
@@ -0,0 +1,200 @@
+PREHOOK: query: drop table extract_udf
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table extract_udf
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table extract_udf (t timestamp)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@extract_udf
+POSTHOOK: query: create table extract_udf (t timestamp)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@extract_udf
+PREHOOK: query: from (select * from src tablesample (1 rows)) s
+  insert overwrite table extract_udf 
+    select '2011-05-06 07:08:09.1234567'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@extract_udf
+POSTHOOK: query: from (select * from src tablesample (1 rows)) s
+  insert overwrite table extract_udf 
+    select '2011-05-06 07:08:09.1234567'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@extract_udf
+POSTHOOK: Lineage: extract_udf.t EXPRESSION []
+PREHOOK: query: explain
+select day(t)
+from extract_udf
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select day(t)
+from extract_udf
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: extract_udf
+            Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: day(t) (type: int)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select day(t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select day(t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+6
+PREHOOK: query: -- new syntax
+explain
+select extract(day from t)
+from extract_udf
+PREHOOK: type: QUERY
+POSTHOOK: query: -- new syntax
+explain
+select extract(day from t)
+from extract_udf
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: extract_udf
+            Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: day(t) (type: int)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select extract(day from t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(day from t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+6
+PREHOOK: query: select extract(second from t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(second from t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+9
+PREHOOK: query: select extract(minute from t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(minute from t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+8
+PREHOOK: query: select extract(hour from t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(hour from t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+7
+PREHOOK: query: select extract(week from t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(week from t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+18
+PREHOOK: query: select extract(month from t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(month from t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+5
+PREHOOK: query: select extract(quarter from t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(quarter from t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2
+PREHOOK: query: select extract(year from t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(year from t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+2011