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 2015/10/05 21:41:20 UTC

[01/23] hive git commit: HIVE-12004 : SDPO doesnt set colExprMap correctly on new RS (Ashutosh Chauhan via Prasanth J)

Repository: hive
Updated Branches:
  refs/heads/llap c5ccf6694 -> edd3079f7


HIVE-12004 : SDPO doesnt set colExprMap correctly on new RS (Ashutosh Chauhan via Prasanth J)

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/ff9822eb
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ff9822eb
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ff9822eb

Branch: refs/heads/llap
Commit: ff9822eb33610d483dd075af82b9b4f97b314e30
Parents: 5074423
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Tue Sep 29 08:14:40 2015 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Oct 2 08:43:35 2015 -0700

----------------------------------------------------------------------
 .../optimizer/SortedDynPartitionOptimizer.java  |  7 +++--
 .../correlation/CorrelationUtilities.java       | 33 --------------------
 .../dynpart_sort_opt_vectorization.q            |  2 --
 .../clientpositive/dynpart_sort_optimization.q  |  2 --
 .../clientpositive/dynpart_sort_optimization2.q |  2 --
 5 files changed, 5 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ff9822eb/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 7bcb797..d58c24d 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
@@ -205,10 +205,8 @@ public class SortedDynPartitionOptimizer implements Transform {
       RowSchema outRS = new RowSchema(fsParent.getSchema());
       ArrayList<ColumnInfo> valColInfo = Lists.newArrayList(fsParent.getSchema().getSignature());
       ArrayList<ExprNodeDesc> newValueCols = Lists.newArrayList();
-      Map<String, ExprNodeDesc> colExprMap = Maps.newHashMap();
       for (ColumnInfo ci : valColInfo) {
         newValueCols.add(new ExprNodeColumnDesc(ci));
-        colExprMap.put(ci.getInternalName(), newValueCols.get(newValueCols.size() - 1));
       }
       ReduceSinkDesc rsConf = getReduceSinkDesc(partitionPositions, sortPositions, sortOrder,
           newValueCols, bucketColumns, numBuckets, fsParent, fsOp.getConf().getWriteType());
@@ -223,6 +221,11 @@ public class SortedDynPartitionOptimizer implements Transform {
       // Create ReduceSink operator
       ReduceSinkOperator rsOp = (ReduceSinkOperator) OperatorFactory.getAndMakeChild(
               rsConf, new RowSchema(outRS.getSignature()), fsParent);
+      List<String> valueColNames = rsConf.getOutputValueColumnNames();
+      Map<String, ExprNodeDesc> colExprMap = Maps.newHashMap();
+      for (int i = 0 ; i < valueColNames.size(); i++) {
+        colExprMap.put(Utilities.ReduceField.VALUE + "." + valueColNames.get(i), newValueCols.get(i));
+      }
       rsOp.setColumnExprMap(colExprMap);
 
       List<ExprNodeDesc> valCols = rsConf.getValueCols();

http://git-wip-us.apache.org/repos/asf/hive/blob/ff9822eb/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationUtilities.java
index 7bb49be..388399c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationUtilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationUtilities.java
@@ -354,45 +354,12 @@ public final class CorrelationUtilities {
       ch.replaceParent(childRS, sel);
     }
 
-    removeChildSelIfApplicable(getSingleChild(childRS), sel, context, procCtx);
     childRS.setChildOperators(null);
     childRS.setParentOperators(null);
     procCtx.addRemovedOperator(childRS);
     return sel;
   }
 
-  //TODO: ideally this method should be removed in future, as in we need not to rely on removing
-  // this select operator which likely is introduced by SortedDynPartitionOptimizer.
-  // NonblockingdedupOptimizer should be able to merge this select Operator with its
-  // parent. But, that is not working at the moment. See: dynpart_sort_optimization2.q
-
-  private static void removeChildSelIfApplicable(Operator<?> child, SelectOperator sel,
-      ParseContext context, AbstractCorrelationProcCtx procCtx) throws SemanticException {
-
-    if (!(child instanceof SelectOperator)) {
-     return;
-   }
-   if (child.getColumnExprMap() != null) {
-     return;
-   }
-
-   SelectOperator selOp = (SelectOperator) child;
-
-   for (ExprNodeDesc desc : selOp.getConf().getColList()) {
-     if (!(desc instanceof ExprNodeColumnDesc)) {
-       return;
-     }
-     ExprNodeColumnDesc col = (ExprNodeColumnDesc) desc;
-     if(!col.getColumn().startsWith(ReduceField.VALUE.toString()+".") ||
-         col.getTabAlias() != null || col.getIsPartitionColOrVirtualCol()){
-       return;
-     }
-   }
-
-   removeOperator(child, getSingleChild(child), sel, context);
-   procCtx.addRemovedOperator(child);
-  }
-
   protected static void removeReduceSinkForGroupBy(ReduceSinkOperator cRS, GroupByOperator cGBYr,
       ParseContext context, AbstractCorrelationProcCtx procCtx) throws SemanticException {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ff9822eb/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q b/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
index 7e94f23..3d0cdcd 100644
--- a/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
+++ b/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
@@ -7,8 +7,6 @@ set hive.exec.dynamic.partition.mode=nonstrict;
 set hive.vectorized.execution.enabled=true;
 set hive.enforce.bucketing=false;
 set hive.enforce.sorting=false;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
 
 create table over1k(
            t tinyint,

http://git-wip-us.apache.org/repos/asf/hive/blob/ff9822eb/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q b/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
index ea670e9..a1a87d8 100644
--- a/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
+++ b/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
@@ -6,8 +6,6 @@ set hive.exec.max.dynamic.partitions.pernode=1000;
 set hive.exec.dynamic.partition.mode=nonstrict;
 set hive.enforce.bucketing=false;
 set hive.enforce.sorting=false;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
 
 create table over1k(
            t tinyint,

http://git-wip-us.apache.org/repos/asf/hive/blob/ff9822eb/ql/src/test/queries/clientpositive/dynpart_sort_optimization2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/dynpart_sort_optimization2.q b/ql/src/test/queries/clientpositive/dynpart_sort_optimization2.q
index 5a504ec..c18f1cc 100644
--- a/ql/src/test/queries/clientpositive/dynpart_sort_optimization2.q
+++ b/ql/src/test/queries/clientpositive/dynpart_sort_optimization2.q
@@ -6,8 +6,6 @@ set hive.exec.max.dynamic.partitions.pernode=1000;
 set hive.exec.dynamic.partition.mode=nonstrict;
 set hive.enforce.bucketing=false;
 set hive.enforce.sorting=false;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
 
 -- SORT_QUERY_RESULTS
 


[09/23] hive git commit: HIVE-11913 : Verify existence of tests for new changes in HiveQA (Szehon, reviewed by Sergio Pena)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/test/resources/HIVE-10761.6.patch
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/resources/HIVE-10761.6.patch b/testutils/ptest2/src/test/resources/HIVE-10761.6.patch
new file mode 100644
index 0000000..5b41850
--- /dev/null
+++ b/testutils/ptest2/src/test/resources/HIVE-10761.6.patch
@@ -0,0 +1,2539 @@
+diff --git a/common/pom.xml b/common/pom.xml
+index a615c1e..8d4b1ea 100644
+--- a/common/pom.xml
++++ b/common/pom.xml
+@@ -98,6 +98,26 @@
+       <artifactId>json</artifactId>
+       <version>${json.version}</version>
+     </dependency>
++    <dependency>
++      <groupId>io.dropwizard.metrics</groupId>
++      <artifactId>metrics-core</artifactId>
++      <version>${dropwizard.version}</version>
++    </dependency>
++    <dependency>
++      <groupId>io.dropwizard.metrics</groupId>
++      <artifactId>metrics-jvm</artifactId>
++      <version>${dropwizard.version}</version>
++    </dependency>
++    <dependency>
++      <groupId>io.dropwizard.metrics</groupId>
++      <artifactId>metrics-json</artifactId>
++      <version>${dropwizard.version}</version>
++    </dependency>
++    <dependency>
++      <groupId>com.fasterxml.jackson.core</groupId>
++      <artifactId>jackson-databind</artifactId>
++      <version>${jackson.new.version}</version>
++    </dependency>
+   </dependencies>
+ 
+   <profiles>
+diff --git a/common/src/java/org/apache/hadoop/hive/common/JvmPauseMonitor.java b/common/src/java/org/apache/hadoop/hive/common/JvmPauseMonitor.java
+new file mode 100644
+index 0000000..c3949f2
+--- /dev/null
++++ b/common/src/java/org/apache/hadoop/hive/common/JvmPauseMonitor.java
+@@ -0,0 +1,225 @@
++/**
++ * 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.common;
++
++import com.google.common.base.Joiner;
++import com.google.common.base.Preconditions;
++import com.google.common.base.Stopwatch;
++import com.google.common.collect.Lists;
++import com.google.common.collect.Maps;
++import com.google.common.collect.Sets;
++import org.apache.commons.logging.Log;
++import org.apache.commons.logging.LogFactory;
++import org.apache.hadoop.conf.Configuration;
++import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
++import org.apache.hadoop.util.Daemon;
++
++import java.lang.management.GarbageCollectorMXBean;
++import java.lang.management.ManagementFactory;
++import java.util.List;
++import java.util.Map;
++import java.util.Set;
++
++/**
++ * Based on the JvmPauseMonitor from Hadoop.
++ */
++public class JvmPauseMonitor {
++  private static final Log LOG = LogFactory.getLog(
++    JvmPauseMonitor.class);
++
++  /** The target sleep time */
++  private static final long SLEEP_INTERVAL_MS = 500;
++
++  /** log WARN if we detect a pause longer than this threshold */
++  private final long warnThresholdMs;
++  private static final String WARN_THRESHOLD_KEY =
++    "jvm.pause.warn-threshold.ms";
++  private static final long WARN_THRESHOLD_DEFAULT = 10000;
++
++  /** log INFO if we detect a pause longer than this threshold */
++  private final long infoThresholdMs;
++  private static final String INFO_THRESHOLD_KEY =
++    "jvm.pause.info-threshold.ms";
++  private static final long INFO_THRESHOLD_DEFAULT = 1000;
++
++  private long numGcWarnThresholdExceeded = 0;
++  private long numGcInfoThresholdExceeded = 0;
++  private long totalGcExtraSleepTime = 0;
++
++  private Thread monitorThread;
++  private volatile boolean shouldRun = true;
++
++  public JvmPauseMonitor(Configuration conf) {
++    this.warnThresholdMs = conf.getLong(WARN_THRESHOLD_KEY, WARN_THRESHOLD_DEFAULT);
++    this.infoThresholdMs = conf.getLong(INFO_THRESHOLD_KEY, INFO_THRESHOLD_DEFAULT);
++  }
++
++  public void start() {
++    Preconditions.checkState(monitorThread == null,
++      "JvmPauseMonitor thread is Already started");
++    monitorThread = new Daemon(new Monitor());
++    monitorThread.start();
++  }
++
++  public void stop() {
++    shouldRun = false;
++    if (isStarted()) {
++      monitorThread.interrupt();
++      try {
++        monitorThread.join();
++      } catch (InterruptedException e) {
++        Thread.currentThread().interrupt();
++      }
++    }
++  }
++
++  public boolean isStarted() {
++    return monitorThread != null;
++  }
++
++  public long getNumGcWarnThreadholdExceeded() {
++    return numGcWarnThresholdExceeded;
++  }
++
++  public long getNumGcInfoThresholdExceeded() {
++    return numGcInfoThresholdExceeded;
++  }
++
++  public long getTotalGcExtraSleepTime() {
++    return totalGcExtraSleepTime;
++  }
++
++  private String formatMessage(long extraSleepTime,
++    Map<String, GcTimes> gcTimesAfterSleep,
++    Map<String, GcTimes> gcTimesBeforeSleep) {
++
++    Set<String> gcBeanNames = Sets.intersection(
++      gcTimesAfterSleep.keySet(),
++      gcTimesBeforeSleep.keySet());
++    List<String> gcDiffs = Lists.newArrayList();
++    for (String name : gcBeanNames) {
++      GcTimes diff = gcTimesAfterSleep.get(name).subtract(
++        gcTimesBeforeSleep.get(name));
++      if (diff.gcCount != 0) {
++        gcDiffs.add("GC pool '" + name + "' had collection(s): " +
++          diff.toString());
++      }
++    }
++
++    String ret = "Detected pause in JVM or host machine (eg GC): " +
++      "pause of approximately " + extraSleepTime + "ms\n";
++    if (gcDiffs.isEmpty()) {
++      ret += "No GCs detected";
++    } else {
++      ret += Joiner.on("\n").join(gcDiffs);
++    }
++    return ret;
++  }
++
++  private Map<String, GcTimes> getGcTimes() {
++    Map<String, GcTimes> map = Maps.newHashMap();
++    List<GarbageCollectorMXBean> gcBeans =
++      ManagementFactory.getGarbageCollectorMXBeans();
++    for (GarbageCollectorMXBean gcBean : gcBeans) {
++      map.put(gcBean.getName(), new GcTimes(gcBean));
++    }
++    return map;
++  }
++
++  private static class GcTimes {
++    private GcTimes(GarbageCollectorMXBean gcBean) {
++      gcCount = gcBean.getCollectionCount();
++      gcTimeMillis = gcBean.getCollectionTime();
++    }
++
++    private GcTimes(long count, long time) {
++      this.gcCount = count;
++      this.gcTimeMillis = time;
++    }
++
++    private GcTimes subtract(GcTimes other) {
++      return new GcTimes(this.gcCount - other.gcCount,
++        this.gcTimeMillis - other.gcTimeMillis);
++    }
++
++    @Override
++    public String toString() {
++      return "count=" + gcCount + " time=" + gcTimeMillis + "ms";
++    }
++
++    private long gcCount;
++    private long gcTimeMillis;
++  }
++
++  private class Monitor implements Runnable {
++    @Override
++    public void run() {
++      Stopwatch sw = new Stopwatch();
++      Map<String, GcTimes> gcTimesBeforeSleep = getGcTimes();
++      while (shouldRun) {
++        sw.reset().start();
++        try {
++          Thread.sleep(SLEEP_INTERVAL_MS);
++        } catch (InterruptedException ie) {
++          return;
++        }
++        long extraSleepTime = sw.elapsedMillis() - SLEEP_INTERVAL_MS;
++        Map<String, GcTimes> gcTimesAfterSleep = getGcTimes();
++
++        if (extraSleepTime > warnThresholdMs) {
++          ++numGcWarnThresholdExceeded;
++          LOG.warn(formatMessage(
++            extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
++          incrementMetricsCounter("jvm.pause.warn-threshold", 1);
++        } else if (extraSleepTime > infoThresholdMs) {
++          ++numGcInfoThresholdExceeded;
++          LOG.info(formatMessage(
++            extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
++          incrementMetricsCounter("jvm.pause.info-threshold", 1);
++        }
++        incrementMetricsCounter("jvm.pause.extraSleepTime", extraSleepTime);
++        totalGcExtraSleepTime += extraSleepTime;
++        gcTimesBeforeSleep = gcTimesAfterSleep;
++      }
++    }
++
++    private void incrementMetricsCounter(String name, long count) {
++      try {
++        MetricsFactory.getMetricsInstance().incrementCounter(name, count);
++      } catch (Exception e) {
++        LOG.warn("Error Reporting JvmPauseMonitor to Metrics system", e);
++      }
++    }
++  }
++
++  /**
++   * Simple 'main' to facilitate manual testing of the pause monitor.
++   *
++   * This main function just leaks memory into a list. Running this class
++   * with a 1GB heap will very quickly go into "GC hell" and result in
++   * log messages about the GC pauses.
++   */
++  public static void main(String []args) throws Exception {
++    new JvmPauseMonitor(new Configuration()).start();
++    List<String> list = Lists.newArrayList();
++    int i = 0;
++    while (true) {
++      list.add(String.valueOf(i++));
++    }
++  }
++}
+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
+new file mode 100644
+index 0000000..14f7afb
+--- /dev/null
++++ b/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java
+@@ -0,0 +1,262 @@
++/**
++ * 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.common.metrics;
++
++import org.apache.hadoop.hive.common.metrics.common.Metrics;
++import org.apache.hadoop.hive.conf.HiveConf;
++
++import java.io.IOException;
++import java.lang.management.ManagementFactory;
++import java.util.HashMap;
++
++import javax.management.MBeanServer;
++import javax.management.MalformedObjectNameException;
++import javax.management.ObjectName;
++
++/**
++ * This class may eventually get superseded by org.apache.hadoop.hive.common.metrics2.Metrics.
++ *
++ * Metrics Subsystem  - allows exposure of a number of named parameters/counters
++ *                      via jmx, intended to be used as a static subsystem
++ *
++ *                      Has a couple of primary ways it can be used:
++ *                      (i) Using the set and get methods to set and get named parameters
++ *                      (ii) Using the incrementCounter method to increment and set named
++ *                      parameters in one go, rather than having to make a get and then a set.
++ *                      (iii) Using the startScope and endScope methods to start and end
++ *                      named "scopes" that record the number of times they've been
++ *                      instantiated and amount of time(in milliseconds) spent inside
++ *                      the scopes.
++ */
++public class LegacyMetrics implements Metrics {
++
++  private LegacyMetrics() {
++    // block
++  }
++
++  /**
++   * MetricsScope : A class that encapsulates an idea of a metered scope.
++   * Instantiating a named scope and then closing it exposes two counters:
++   *   (i) a "number of calls" counter ( &lt;name&gt;.n ), and
++   *  (ii) a "number of msecs spent between scope open and close" counter. ( &lt;name&gt;.t)
++   */
++  public static class MetricsScope {
++
++    final LegacyMetrics metrics;
++
++    final String name;
++    final String numCounter;
++    final String timeCounter;
++    final String avgTimeCounter;
++
++    private boolean isOpen = false;
++    private Long startTime = null;
++
++    /**
++     * Instantiates a named scope - intended to only be called by Metrics, so locally scoped.
++     * @param name - name of the variable
++     * @throws IOException
++     */
++    private MetricsScope(String name, LegacyMetrics metrics) throws IOException {
++      this.metrics = metrics;
++      this.name = name;
++      this.numCounter = name + ".n";
++      this.timeCounter = name + ".t";
++      this.avgTimeCounter = name + ".avg_t";
++      open();
++    }
++
++    public Long getNumCounter() throws IOException {
++      return (Long) metrics.get(numCounter);
++    }
++
++    public Long getTimeCounter() throws IOException {
++      return (Long) metrics.get(timeCounter);
++    }
++
++    /**
++     * Opens scope, and makes note of the time started, increments run counter
++     * @throws IOException
++     *
++     */
++    public void open() throws IOException {
++      if (!isOpen) {
++        isOpen = true;
++        startTime = System.currentTimeMillis();
++      } else {
++        throw new IOException("Scope named " + name + " is not closed, cannot be opened.");
++      }
++    }
++
++    /**
++     * Closes scope, and records the time taken
++     * @throws IOException
++     */
++    public void close() throws IOException {
++      if (isOpen) {
++        Long endTime = System.currentTimeMillis();
++        synchronized(metrics) {
++          Long num = metrics.incrementCounter(numCounter);
++          Long time = metrics.incrementCounter(timeCounter, endTime - startTime);
++          if (num != null && time != null) {
++            metrics.set(avgTimeCounter, Double.valueOf(time.doubleValue() / num.doubleValue()));
++          }
++        }
++      } else {
++        throw new IOException("Scope named " + name + " is not open, cannot be closed.");
++      }
++      isOpen = false;
++    }
++
++
++    /**
++     * Closes scope if open, and reopens it
++     * @throws IOException
++     */
++    public void reopen() throws IOException {
++      if(isOpen) {
++        close();
++      }
++      open();
++    }
++
++  }
++
++  private static final MetricsMBean metrics = new MetricsMBeanImpl();
++
++  private static final ObjectName oname;
++  static {
++    try {
++      oname = new ObjectName(
++          "org.apache.hadoop.hive.common.metrics:type=MetricsMBean");
++    } catch (MalformedObjectNameException mone) {
++      throw new RuntimeException(mone);
++    }
++  }
++
++
++  private static final ThreadLocal<HashMap<String, MetricsScope>> threadLocalScopes
++    = new ThreadLocal<HashMap<String,MetricsScope>>() {
++    @Override
++    protected HashMap<String,MetricsScope> initialValue() {
++      return new HashMap<String,MetricsScope>();
++    }
++  };
++
++  private boolean initialized = false;
++
++  public void init(HiveConf conf) throws Exception {
++    if (!initialized) {
++      MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
++      mbs.registerMBean(metrics, oname);
++      initialized = true;
++    }
++  }
++
++  public boolean isInitialized() {
++    return initialized;
++  }
++
++  public Long incrementCounter(String name) throws IOException{
++    if (!initialized) {
++      return null;
++    }
++    return incrementCounter(name,Long.valueOf(1));
++  }
++
++  public Long incrementCounter(String name, long increment) throws IOException{
++    if (!initialized) {
++      return null;
++    }
++    Long value;
++    synchronized(metrics) {
++      if (!metrics.hasKey(name)) {
++        value = Long.valueOf(increment);
++        set(name, value);
++      } else {
++        value = ((Long)get(name)) + increment;
++        set(name, value);
++      }
++    }
++    return value;
++  }
++
++  public void set(String name, Object value) throws IOException{
++    if (!initialized) {
++      return;
++    }
++    metrics.put(name,value);
++  }
++
++  public Object get(String name) throws IOException{
++    if (!initialized) {
++      return null;
++    }
++    return metrics.get(name);
++  }
++
++  public void startScope(String name) throws IOException{
++    if (!initialized) {
++      return;
++    }
++    if (threadLocalScopes.get().containsKey(name)) {
++      threadLocalScopes.get().get(name).open();
++    } else {
++      threadLocalScopes.get().put(name, new MetricsScope(name, this));
++    }
++  }
++
++  public MetricsScope getScope(String name) throws IOException {
++    if (!initialized) {
++      return null;
++    }
++    if (threadLocalScopes.get().containsKey(name)) {
++      return threadLocalScopes.get().get(name);
++    } else {
++      throw new IOException("No metrics scope named " + name);
++    }
++  }
++
++  public void endScope(String name) throws IOException{
++    if (!initialized) {
++      return;
++    }
++    if (threadLocalScopes.get().containsKey(name)) {
++      threadLocalScopes.get().get(name).close();
++    }
++  }
++
++  /**
++   * Resets the static context state to initial.
++   * Used primarily for testing purposes.
++   *
++   * Note that threadLocalScopes ThreadLocal is *not* cleared in this call.
++   */
++  public void deInit() throws Exception {
++    synchronized (metrics) {
++      if (initialized) {
++        MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
++        if (mbs.isRegistered(oname)) {
++          mbs.unregisterMBean(oname);
++        }
++        metrics.clear();
++        initialized = false;
++      }
++    }
++  }
++}
+diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/Metrics.java b/common/src/java/org/apache/hadoop/hive/common/metrics/Metrics.java
+deleted file mode 100644
+index 01c9d1d..0000000
+--- a/common/src/java/org/apache/hadoop/hive/common/metrics/Metrics.java
++++ /dev/null
+@@ -1,253 +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.common.metrics;
+-
+-import java.io.IOException;
+-import java.lang.management.ManagementFactory;
+-import java.util.HashMap;
+-
+-import javax.management.MBeanServer;
+-import javax.management.MalformedObjectNameException;
+-import javax.management.ObjectName;
+-
+-/**
+- * Metrics Subsystem  - allows exposure of a number of named parameters/counters
+- *                      via jmx, intended to be used as a static subsystem
+- *
+- *                      Has a couple of primary ways it can be used:
+- *                      (i) Using the set and get methods to set and get named parameters
+- *                      (ii) Using the incrementCounter method to increment and set named
+- *                      parameters in one go, rather than having to make a get and then a set.
+- *                      (iii) Using the startScope and endScope methods to start and end
+- *                      named "scopes" that record the number of times they've been
+- *                      instantiated and amount of time(in milliseconds) spent inside
+- *                      the scopes.
+- */
+-public class Metrics {
+-
+-  private Metrics() {
+-    // block
+-  }
+-  
+-  /**
+-   * MetricsScope : A class that encapsulates an idea of a metered scope.
+-   * Instantiating a named scope and then closing it exposes two counters:
+-   *   (i) a "number of calls" counter ( &lt;name&gt;.n ), and
+-   *  (ii) a "number of msecs spent between scope open and close" counter. ( &lt;name&gt;.t)
+-   */
+-  public static class MetricsScope {
+-
+-    final String name;
+-    final String numCounter;
+-    final String timeCounter;
+-    final String avgTimeCounter;
+-    
+-    private boolean isOpen = false;
+-    private Long startTime = null;
+-
+-    /**
+-     * Instantiates a named scope - intended to only be called by Metrics, so locally scoped.
+-     * @param name - name of the variable
+-     * @throws IOException
+-     */
+-    private MetricsScope(String name) throws IOException {
+-      this.name = name;
+-      this.numCounter = name + ".n";
+-      this.timeCounter = name + ".t";
+-      this.avgTimeCounter = name + ".avg_t";
+-      open();
+-    }
+-
+-    public Long getNumCounter() throws IOException {
+-      return (Long)Metrics.get(numCounter);
+-    }
+-
+-    public Long getTimeCounter() throws IOException {
+-      return (Long)Metrics.get(timeCounter);
+-    }
+-
+-    /**
+-     * Opens scope, and makes note of the time started, increments run counter
+-     * @throws IOException
+-     *
+-     */
+-    public void open() throws IOException {
+-      if (!isOpen) {
+-        isOpen = true;
+-        startTime = System.currentTimeMillis();
+-      } else {
+-        throw new IOException("Scope named " + name + " is not closed, cannot be opened.");
+-      }
+-    }
+-
+-    /**
+-     * Closes scope, and records the time taken
+-     * @throws IOException
+-     */
+-    public void close() throws IOException {
+-      if (isOpen) {
+-        Long endTime = System.currentTimeMillis();
+-        synchronized(metrics) {
+-          Long num = Metrics.incrementCounter(numCounter);
+-          Long time = Metrics.incrementCounter(timeCounter, endTime - startTime);
+-          if (num != null && time != null) {
+-            Metrics.set(avgTimeCounter, Double.valueOf(time.doubleValue() / num.doubleValue()));
+-          }
+-        }
+-      } else {
+-        throw new IOException("Scope named " + name + " is not open, cannot be closed.");
+-      }
+-      isOpen = false;
+-    }
+-
+-
+-    /**
+-     * Closes scope if open, and reopens it
+-     * @throws IOException
+-     */
+-    public void reopen() throws IOException {
+-      if(isOpen) {
+-        close();
+-      }
+-      open();
+-    }
+-
+-  }
+-
+-  private static final MetricsMBean metrics = new MetricsMBeanImpl();
+-
+-  private static final ObjectName oname;
+-  static {
+-    try {
+-      oname = new ObjectName(
+-          "org.apache.hadoop.hive.common.metrics:type=MetricsMBean");      
+-    } catch (MalformedObjectNameException mone) {
+-      throw new RuntimeException(mone);
+-    }
+-  }
+-  
+-  
+-  private static final ThreadLocal<HashMap<String, MetricsScope>> threadLocalScopes
+-    = new ThreadLocal<HashMap<String,MetricsScope>>() {
+-    @Override
+-    protected HashMap<String,MetricsScope> initialValue() {
+-      return new HashMap<String,MetricsScope>();
+-    }
+-  };
+-
+-  private static boolean initialized = false;
+-
+-  public static void init() throws Exception {
+-    synchronized (metrics) {
+-      if (!initialized) {
+-        MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+-        mbs.registerMBean(metrics, oname);
+-        initialized = true;
+-      }
+-    }
+-  }
+-
+-  public static Long incrementCounter(String name) throws IOException{
+-    if (!initialized) {
+-      return null;
+-    }
+-    return incrementCounter(name,Long.valueOf(1));
+-  }
+-
+-  public static Long incrementCounter(String name, long increment) throws IOException{
+-    if (!initialized) {
+-      return null;
+-    }
+-    Long value;
+-    synchronized(metrics) {
+-      if (!metrics.hasKey(name)) {
+-        value = Long.valueOf(increment);
+-        set(name, value);
+-      } else {
+-        value = ((Long)get(name)) + increment;
+-        set(name, value);
+-      }
+-    }
+-    return value;
+-  }
+-
+-  public static void set(String name, Object value) throws IOException{
+-    if (!initialized) {
+-      return;
+-    }
+-    metrics.put(name,value);
+-  }
+-
+-  public static Object get(String name) throws IOException{
+-    if (!initialized) {
+-      return null;
+-    }
+-    return metrics.get(name);
+-  }
+-
+-  public static MetricsScope startScope(String name) throws IOException{
+-    if (!initialized) {
+-      return null;
+-    }
+-    if (threadLocalScopes.get().containsKey(name)) {
+-      threadLocalScopes.get().get(name).open();
+-    } else {
+-      threadLocalScopes.get().put(name, new MetricsScope(name));
+-    }
+-    return threadLocalScopes.get().get(name);
+-  }
+-
+-  public static MetricsScope getScope(String name) throws IOException {
+-    if (!initialized) {
+-      return null;
+-    }
+-    if (threadLocalScopes.get().containsKey(name)) {
+-      return threadLocalScopes.get().get(name);
+-    } else {
+-      throw new IOException("No metrics scope named " + name);
+-    }
+-  }
+-
+-  public static void endScope(String name) throws IOException{
+-    if (!initialized) {
+-      return;
+-    }
+-    if (threadLocalScopes.get().containsKey(name)) {
+-      threadLocalScopes.get().get(name).close();
+-    }
+-  }
+-
+-  /**
+-   * Resets the static context state to initial.
+-   * Used primarily for testing purposes.
+-   * 
+-   * Note that threadLocalScopes ThreadLocal is *not* cleared in this call.
+-   */
+-  static void uninit() throws Exception {
+-    synchronized (metrics) {
+-      if (initialized) {
+-        MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+-        if (mbs.isRegistered(oname)) {
+-          mbs.unregisterMBean(oname);
+-        }
+-        metrics.clear();
+-        initialized = false;
+-      }
+-    }
+-  }
+-}
+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
+new file mode 100644
+index 0000000..13a5336
+--- /dev/null
++++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java
+@@ -0,0 +1,68 @@
++/**
++ * 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.common.metrics.common;
++
++import java.io.IOException;
++
++import org.apache.hadoop.hive.conf.HiveConf;
++
++import java.io.IOException;
++
++/**
++ * Generic Metics interface.
++ */
++public interface Metrics {
++
++  /**
++   * Initialize Metrics system with given Hive configuration.
++   * @param conf
++   */
++  public void init(HiveConf conf) throws Exception;
++
++  /**
++   * Deinitializes the Metrics system.
++   */
++  public void deInit() throws Exception;
++
++  /**
++   * @param name
++   * @throws IOException
++   */
++  public void startScope(String name) throws IOException;
++
++  public void endScope(String name) throws IOException;
++
++  //Counter-related methods
++
++  /**
++   * Increments a counter of the given name by 1.
++   * @param name
++   * @return
++   * @throws IOException
++   */
++  public Long incrementCounter(String name) throws IOException;
++
++  /**
++   * 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;
++}
+diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsFactory.java b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsFactory.java
+new file mode 100644
+index 0000000..12a309d
+--- /dev/null
++++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsFactory.java
+@@ -0,0 +1,48 @@
++/**
++ * 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.common.metrics.common;
++
++import org.apache.hadoop.hive.conf.HiveConf;
++import org.apache.hadoop.util.ReflectionUtils;
++
++/**
++ * Class that manages a static Metric instance for this process.
++ */
++public class MetricsFactory {
++
++  private static Metrics metrics;
++  private static Object initLock = new Object();
++
++  public synchronized static void init(HiveConf conf) throws Exception {
++    if (metrics == null) {
++      metrics = (Metrics) ReflectionUtils.newInstance(conf.getClassByName(
++        conf.getVar(HiveConf.ConfVars.HIVE_METRICS_CLASS)), conf);
++    }
++    metrics.init(conf);
++  }
++
++  public synchronized static Metrics getMetricsInstance() {
++    return metrics;
++  }
++
++  public synchronized static void deInit() throws Exception {
++    if (metrics != null) {
++      metrics.deInit();
++    }
++  }
++}
+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
+new file mode 100644
+index 0000000..e59da99
+--- /dev/null
++++ b/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java
+@@ -0,0 +1,366 @@
++/**
++ * 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.common.metrics.metrics2;
++
++import com.codahale.metrics.ConsoleReporter;
++import com.codahale.metrics.Counter;
++import com.codahale.metrics.ExponentiallyDecayingReservoir;
++import com.codahale.metrics.JmxReporter;
++import com.codahale.metrics.Metric;
++import com.codahale.metrics.MetricRegistry;
++import com.codahale.metrics.MetricSet;
++import com.codahale.metrics.Timer;
++import com.codahale.metrics.json.MetricsModule;
++import com.codahale.metrics.jvm.BufferPoolMetricSet;
++import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
++import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
++import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
++import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
++import com.fasterxml.jackson.databind.ObjectMapper;
++import com.google.common.annotations.VisibleForTesting;
++import com.google.common.base.Splitter;
++import com.google.common.cache.CacheBuilder;
++import com.google.common.cache.CacheLoader;
++import com.google.common.cache.LoadingCache;
++import com.google.common.collect.Lists;
++import org.apache.commons.logging.Log;
++import org.apache.commons.logging.LogFactory;
++import org.apache.hadoop.fs.FileSystem;
++import org.apache.hadoop.fs.Path;
++import org.apache.hadoop.fs.permission.FsPermission;
++import org.apache.hadoop.hive.conf.HiveConf;
++
++import java.io.BufferedReader;
++import java.io.BufferedWriter;
++import java.io.Closeable;
++import java.io.IOException;
++import java.io.OutputStreamWriter;
++import java.lang.management.ManagementFactory;
++import java.util.HashMap;
++import java.util.HashSet;
++import java.util.List;
++import java.util.Map;
++import java.util.Set;
++import java.util.TimerTask;
++import java.util.concurrent.ExecutionException;
++import java.util.concurrent.TimeUnit;
++import java.util.concurrent.locks.Lock;
++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 Log LOGGER = LogFactory.getLog(CodahaleMetrics.class);
++
++  public final MetricRegistry metricRegistry = new MetricRegistry();
++  private final Lock timersLock = new ReentrantLock();
++  private final Lock countersLock = new ReentrantLock();
++
++  private LoadingCache<String, Timer> timers;
++  private LoadingCache<String, Counter> counters;
++
++  private boolean initialized = false;
++  private HiveConf conf;
++  private final Set<Closeable> reporters = new HashSet<Closeable>();
++
++  private final ThreadLocal<HashMap<String, MetricsScope>> threadLocalScopes
++    = new ThreadLocal<HashMap<String,MetricsScope>>() {
++    @Override
++    protected HashMap<String,MetricsScope> initialValue() {
++      return new HashMap<String,MetricsScope>();
++    }
++  };
++
++  public static class MetricsScope {
++
++    final String name;
++    final Timer timer;
++    Timer.Context timerContext;
++    CodahaleMetrics metrics;
++
++    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 MetricsScope(String name, CodahaleMetrics metrics) throws IOException {
++      this.name = name;
++      this.metrics = metrics;
++      this.timer = metrics.getTimer(name);
++      open();
++    }
++
++    /**
++     * Opens scope, and makes note of the time started, increments run counter
++     * @throws IOException
++     *
++     */
++    public void open() throws IOException {
++      if (!isOpen) {
++        isOpen = true;
++        this.timerContext = timer.time();
++      } else {
++        throw new IOException("Scope named " + name + " is not closed, cannot be opened.");
++      }
++    }
++
++    /**
++     * Closes scope, and records the time taken
++     * @throws IOException
++     */
++    public void close() throws IOException {
++      if (isOpen) {
++        timerContext.close();
++
++      } else {
++        throw new IOException("Scope named " + name + " is not open, cannot be closed.");
++      }
++      isOpen = false;
++    }
++  }
++
++  public synchronized void init(HiveConf conf) throws Exception {
++    if (initialized) {
++      return;
++    }
++
++    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 {
++          Timer timer = new Timer(new ExponentiallyDecayingReservoir());
++          metricRegistry.register(key, timer);
++          return timer;
++        }
++      }
++    );
++    counters = CacheBuilder.newBuilder().build(
++      new CacheLoader<String, Counter>() {
++        @Override
++        public Counter load(String key) throws Exception {
++          Counter counter = new Counter();
++          metricRegistry.register(key, counter);
++          return counter;
++        }
++      }
++    );
++
++    //register JVM metrics
++    registerAll("gc", new GarbageCollectorMetricSet());
++    registerAll("buffers", new BufferPoolMetricSet(ManagementFactory.getPlatformMBeanServer()));
++    registerAll("memory", new MemoryUsageGaugeSet());
++    registerAll("threads", new ThreadStatesGaugeSet());
++    registerAll("classLoading", new ClassLoadingGaugeSet());
++
++    //Metrics reporter
++    Set<MetricsReporting> finalReporterList = new HashSet<MetricsReporting>();
++    List<String> metricsReporterNames = Lists.newArrayList(
++      Splitter.on(",").trimResults().omitEmptyStrings().split(conf.getVar(HiveConf.ConfVars.HIVE_METRICS_REPORTER)));
++
++    if(metricsReporterNames != null) {
++      for (String metricsReportingName : metricsReporterNames) {
++        try {
++          MetricsReporting reporter = MetricsReporting.valueOf(metricsReportingName.trim().toUpperCase());
++          finalReporterList.add(reporter);
++        } catch (IllegalArgumentException e) {
++          LOGGER.warn("Metrics reporter skipped due to invalid configured reporter: " + metricsReportingName);
++        }
++      }
++    }
++    initReporting(finalReporterList);
++    initialized = true;
++  }
++
++
++  public synchronized void deInit() throws Exception {
++    if (initialized) {
++      if (reporters != null) {
++        for (Closeable reporter : reporters) {
++          reporter.close();
++        }
++      }
++      for (Map.Entry<String, Metric> metric : metricRegistry.getMetrics().entrySet()) {
++        metricRegistry.remove(metric.getKey());
++      }
++      timers.invalidateAll();
++      counters.invalidateAll();
++      initialized = false;
++    }
++  }
++
++  public void startScope(String name) throws IOException {
++    synchronized (this) {
++      if (!initialized) {
++        return;
++      }
++    }
++    name = API_PREFIX + name;
++    if (threadLocalScopes.get().containsKey(name)) {
++      threadLocalScopes.get().get(name).open();
++    } else {
++      threadLocalScopes.get().put(name, new MetricsScope(name, this));
++    }
++  }
++
++  public void endScope(String name) throws IOException{
++    synchronized (this) {
++      if (!initialized) {
++        return;
++      }
++    }
++    name = API_PREFIX + name;
++    if (threadLocalScopes.get().containsKey(name)) {
++      threadLocalScopes.get().get(name).close();
++    }
++  }
++
++  public Long incrementCounter(String name) throws IOException {
++    return incrementCounter(name, 1);
++  }
++
++  public Long incrementCounter(String name, long increment) throws IOException {
++    String key = name;
++    try {
++      countersLock.lock();
++      counters.get(key).inc(increment);
++      return counters.get(key).getCount();
++    } catch(ExecutionException ee) {
++      throw new RuntimeException(ee);
++    } finally {
++      countersLock.unlock();
++    }
++  }
++
++  // This method is necessary to synchronize lazy-creation to the timers.
++  private Timer getTimer(String name) throws IOException {
++    String key = name;
++    try {
++      timersLock.lock();
++      Timer timer = timers.get(key);
++      return timer;
++    } catch (ExecutionException e) {
++      throw new IOException(e);
++    } finally {
++      timersLock.unlock();
++    }
++  }
++
++  private void registerAll(String prefix, MetricSet metricSet) {
++    for (Map.Entry<String, Metric> entry : metricSet.getMetrics().entrySet()) {
++      if (entry.getValue() instanceof MetricSet) {
++        registerAll(prefix + "." + entry.getKey(), (MetricSet) entry.getValue());
++      } else {
++        metricRegistry.register(prefix + "." + entry.getKey(), entry.getValue());
++      }
++    }
++  }
++
++  @VisibleForTesting
++  public MetricRegistry getMetricRegistry() {
++    return metricRegistry;
++  }
++
++  /**
++   * Should be only called once to initialize the reporters
++   */
++  private void initReporting(Set<MetricsReporting> reportingSet) throws Exception {
++    for (MetricsReporting reporting : reportingSet) {
++      switch(reporting) {
++        case CONSOLE:
++          final ConsoleReporter consoleReporter = ConsoleReporter.forRegistry(metricRegistry)
++            .convertRatesTo(TimeUnit.SECONDS)
++            .convertDurationsTo(TimeUnit.MILLISECONDS)
++            .build();
++          consoleReporter.start(1, TimeUnit.SECONDS);
++          reporters.add(consoleReporter);
++          break;
++        case JMX:
++          final JmxReporter jmxReporter = JmxReporter.forRegistry(metricRegistry)
++            .convertRatesTo(TimeUnit.SECONDS)
++            .convertDurationsTo(TimeUnit.MILLISECONDS)
++            .build();
++          jmxReporter.start();
++          reporters.add(jmxReporter);
++          break;
++        case JSON_FILE:
++          final JsonFileReporter jsonFileReporter = new JsonFileReporter();
++          jsonFileReporter.start();
++          reporters.add(jsonFileReporter);
++          break;
++      }
++    }
++  }
++
++  class JsonFileReporter implements Closeable {
++    private ObjectMapper jsonMapper = null;
++    private java.util.Timer timer = null;
++
++    public void start() {
++      this.jsonMapper = new ObjectMapper().registerModule(new MetricsModule(TimeUnit.MILLISECONDS, TimeUnit.MILLISECONDS, false));
++      this.timer = new java.util.Timer(true);
++
++      long time = conf.getTimeVar(HiveConf.ConfVars.HIVE_METRICS_JSON_FILE_INTERVAL, TimeUnit.MILLISECONDS);
++      final String pathString = conf.getVar(HiveConf.ConfVars.HIVE_METRICS_JSON_FILE_LOCATION);
++
++      timer.schedule(new TimerTask() {
++        @Override
++        public void run() {
++          BufferedWriter bw = null;
++          try {
++            String json = jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(metricRegistry);
++            Path tmpPath = new Path(pathString + ".tmp");
++            FileSystem fs = FileSystem.get(conf);
++            fs.delete(tmpPath, true);
++            bw = new BufferedWriter(new OutputStreamWriter(fs.create(tmpPath, true)));
++            bw.write(json);
++            bw.close();
++
++            Path path = new Path(pathString);
++            fs.rename(tmpPath, path);
++            fs.setPermission(path, FsPermission.createImmutable((short) 0644));
++          } catch (Exception e) {
++            LOGGER.warn("Error writing JSON Metrics to file", e);
++          } finally {
++            try {
++              if (bw != null) {
++                bw.close();
++              }
++            } catch (IOException e) {
++              //Ignore.
++            }
++          }
++
++
++        }
++      }, 0, time);
++    }
++
++    public void close() {
++      if (timer != null) {
++        this.timer.cancel();
++      }
++    }
++  }
++}
+diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/MetricsReporting.java b/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/MetricsReporting.java
+new file mode 100644
+index 0000000..643246f
+--- /dev/null
++++ b/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/MetricsReporting.java
+@@ -0,0 +1,27 @@
++/**
++ * 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.common.metrics.metrics2;
++
++/**
++ * Reporting options for org.apache.hadoop.hive.common.metrics.metrics2.Metrics.
++ */
++public enum MetricsReporting {
++  JMX,
++  CONSOLE,
++  JSON_FILE
++}
+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 49b8f97..55a79a9 100644
+--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
++++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+@@ -645,6 +645,7 @@ public void setSparkConfigUpdated(boolean isSparkConfigUpdated) {
+         "Maximum cache full % after which the cache cleaner thread kicks in."),
+     METASTORE_AGGREGATE_STATS_CACHE_CLEAN_UNTIL("hive.metastore.aggregate.stats.cache.clean.until", (float) 0.8,
+         "The cleaner thread cleans until cache reaches this % full size."),
++    METASTORE_METRICS("hive.metastore.metrics.enabled", false, "Enable metrics on the metastore."),
+ 
+     // Parameters for exporting metadata on table drop (requires the use of the)
+     // org.apache.hadoop.hive.ql.parse.MetaDataExportListener preevent listener
+@@ -1688,6 +1689,7 @@ public void setSparkConfigUpdated(boolean isSparkConfigUpdated) {
+         "  EXECUTION: Log completion of tasks\n" +
+         "  PERFORMANCE: Execution + Performance logs \n" +
+         "  VERBOSE: All logs" ),
++    HIVE_SERVER2_METRICS_ENABLED("hive.server2.metrics.enabled", false, "Enable metrics on the HiveServer2."),
+     // logging configuration
+     HIVE_LOG4J_FILE("hive.log4j.file", "",
+         "Hive log4j configuration file.\n" +
+@@ -1715,7 +1717,21 @@ public void setSparkConfigUpdated(boolean isSparkConfigUpdated) {
+     HIVE_AUTOGEN_COLUMNALIAS_PREFIX_INCLUDEFUNCNAME(
+         "hive.autogen.columnalias.prefix.includefuncname", false,
+         "Whether to include function name in the column alias auto generated by Hive."),
+-
++    HIVE_METRICS_CLASS("hive.service.metrics.class",
++        "org.apache.hadoop.hive.common.metrics.metrics2.CodahaleMetrics",
++        new StringSet(
++            "org.apache.hadoop.hive.common.metrics.metrics2.CodahaleMetrics",
++            "org.apache.hadoop.hive.common.metrics.LegacyMetrics"),
++        "Hive metrics subsystem implementation class."),
++    HIVE_METRICS_REPORTER("hive.service.metrics.reporter", "JSON_FILE, JMX",
++        "Reporter type for metric class org.apache.hadoop.hive.common.metrics.metrics2.CodahaleMetrics, comma separated list of JMX, CONSOLE, JSON_FILE"),
++    HIVE_METRICS_JSON_FILE_LOCATION("hive.service.metrics.file.location", "file:///tmp/my-logging.properties",
++        "For metric class org.apache.hadoop.hive.common.metrics.metrics2.CodahaleMetrics JSON_FILE reporter, the location of JSON metrics file.  " +
++        "This file will get overwritten at every interval."),
++    HIVE_METRICS_JSON_FILE_INTERVAL("hive.service.metrics.file.frequency", "5s",
++        new TimeValidator(TimeUnit.MILLISECONDS),
++        "For metric class org.apache.hadoop.hive.common.metrics.metrics2.CodahaleMetrics JSON_FILE reporter, " +
++        "the frequency of updating JSON metrics file."),
+     HIVE_PERF_LOGGER("hive.exec.perf.logger", "org.apache.hadoop.hive.ql.log.PerfLogger",
+         "The class responsible for logging client side performance metrics. \n" +
+         "Must be a subclass of org.apache.hadoop.hive.ql.log.PerfLogger"),
+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
+new file mode 100644
+index 0000000..c14c7ee
+--- /dev/null
++++ b/common/src/test/org/apache/hadoop/hive/common/metrics/TestLegacyMetrics.java
+@@ -0,0 +1,295 @@
++/**
++ * 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.common.metrics;
++
++import java.io.IOException;
++import java.lang.management.ManagementFactory;
++import java.util.concurrent.Callable;
++import java.util.concurrent.ExecutorService;
++import java.util.concurrent.Executors;
++import java.util.concurrent.TimeUnit;
++
++import javax.management.Attribute;
++import javax.management.MBeanAttributeInfo;
++import javax.management.MBeanInfo;
++import javax.management.MBeanOperationInfo;
++import javax.management.MBeanServer;
++import javax.management.ObjectName;
++
++import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
++import org.apache.hadoop.hive.common.metrics.LegacyMetrics.MetricsScope;
++import org.apache.hadoop.hive.conf.HiveConf;
++import org.junit.After;
++import org.junit.Before;
++import org.junit.Test;
++import static org.junit.Assert.*;
++
++public class TestLegacyMetrics {
++
++  private static final String scopeName = "foo";
++  private static final long periodMs = 50L;
++  private static LegacyMetrics metrics;
++
++  @Before
++  public void before() throws Exception {
++    MetricsFactory.deInit();
++    HiveConf conf = new HiveConf();
++    conf.setVar(HiveConf.ConfVars.HIVE_METRICS_CLASS, LegacyMetrics.class.getCanonicalName());
++    MetricsFactory.init(conf);
++    metrics = (LegacyMetrics) MetricsFactory.getMetricsInstance();
++  }
++
++  @After
++  public void after() throws Exception {
++    MetricsFactory.deInit();
++  }
++
++  @Test
++  public void testMetricsMBean() throws Exception {
++    MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
++    final ObjectName oname = new ObjectName(
++        "org.apache.hadoop.hive.common.metrics:type=MetricsMBean");
++    MBeanInfo mBeanInfo = mbs.getMBeanInfo(oname);
++    // check implementation class:
++    assertEquals(MetricsMBeanImpl.class.getName(), mBeanInfo.getClassName());
++
++    // check reset operation:
++    MBeanOperationInfo[] oops = mBeanInfo.getOperations();
++    boolean resetFound = false;
++    for (MBeanOperationInfo op : oops) {
++      if ("reset".equals(op.getName())) {
++        resetFound = true;
++        break;
++      }
++    }
++    assertTrue(resetFound);
++
++    // add metric with a non-null value:
++    Attribute attr = new Attribute("fooMetric", Long.valueOf(-77));
++    mbs.setAttribute(oname, attr);
++
++    mBeanInfo = mbs.getMBeanInfo(oname);
++    MBeanAttributeInfo[] attrinuteInfos = mBeanInfo.getAttributes();
++    assertEquals(1, attrinuteInfos.length);
++    boolean attrFound = false;
++    for (MBeanAttributeInfo info : attrinuteInfos) {
++      if ("fooMetric".equals(info.getName())) {
++        assertEquals("java.lang.Long", info.getType());
++        assertTrue(info.isReadable());
++        assertTrue(info.isWritable());
++        assertFalse(info.isIs());
++
++        attrFound = true;
++        break;
++      }
++    }
++    assertTrue(attrFound);
++
++    // check metric value:
++    Object v = mbs.getAttribute(oname, "fooMetric");
++    assertEquals(Long.valueOf(-77), v);
++
++    // reset the bean:
++    Object result = mbs.invoke(oname, "reset", new Object[0], new String[0]);
++    assertNull(result);
++
++    // the metric value must be zeroed:
++    v = mbs.getAttribute(oname, "fooMetric");
++    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.startScope(scopeName);
++    final MetricsScope fooScope = metrics.getScope(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;
++      }
++    });
++
++    assertSame(fooScope, metrics.getScope(scopeName));
++    Thread.sleep(periodMs+ 1);
++    // 1st close:
++    // closing of open scope should be ok:
++    metrics.endScope(scopeName);
++    expectIOE(new Callable<Void>() {
++      @Override
++      public Void call() throws Exception {
++        metrics.endScope(scopeName); // closing of closed scope not allowed
++        return null;
++      }
++    });
++
++    assertEquals(Long.valueOf(1), fooScope.getNumCounter());
++    final long t1 = fooScope.getTimeCounter().longValue();
++    assertTrue(t1 > periodMs);
++
++    assertSame(fooScope, metrics.getScope(scopeName));
++
++   // opening allowed after closing:
++    metrics.startScope(scopeName);
++    // opening of already open scope not allowed:
++    expectIOE(new Callable<Void>() {
++      @Override
++      public Void call() throws Exception {
++        metrics.startScope(scopeName);
++        return null;
++      }
++    });
++
++    assertEquals(Long.valueOf(1), fooScope.getNumCounter());
++    assertEquals(t1, fooScope.getTimeCounter().longValue());
++
++    assertSame(fooScope, metrics.getScope(scopeName));
++    Thread.sleep(periodMs + 1);
++    // Reopening (close + open) allowed in opened state:
++    fooScope.reopen();
++
++    assertEquals(Long.valueOf(2), fooScope.getNumCounter());
++    assertTrue(fooScope.getTimeCounter().longValue() > 2 * periodMs);
++
++    Thread.sleep(periodMs + 1);
++    // 3rd close:
++    fooScope.close();
++
++    assertEquals(Long.valueOf(3), fooScope.getNumCounter());
++    assertTrue(fooScope.getTimeCounter().longValue() > 3 * periodMs);
++    Double avgT = (Double) metrics.get("foo.avg_t");
++    assertTrue(avgT.doubleValue() > periodMs);
++  }
++
++  @Test
++  public void testScopeConcurrency() throws Exception {
++    metrics.startScope(scopeName);
++    MetricsScope fooScope = metrics.getScope(scopeName);
++    final int threads = 10;
++    ExecutorService executorService = Executors.newFixedThreadPool(threads);
++    for (int i=0; i<threads; i++) {
++      final int n = i;
++      executorService.submit(new Callable<Void>() {
++        @Override
++        public Void call() throws Exception {
++          testScopeImpl(n);
++          return null;
++        }
++      });
++    }
++    executorService.shutdown();
++    assertTrue(executorService.awaitTermination(periodMs * 3 * threads, TimeUnit.MILLISECONDS));
++
++    fooScope = metrics.getScope(scopeName);
++    assertEquals(Long.valueOf(3 * threads), fooScope.getNumCounter());
++    assertTrue(fooScope.getTimeCounter().longValue() > 3 * periodMs * threads);
++    Double avgT = (Double) metrics.get("foo.avg_t");
++    assertTrue(avgT.doubleValue() > periodMs);
++    metrics.endScope(scopeName);
++  }
++
++  void testScopeImpl(int n) throws Exception {
++    metrics.startScope(scopeName);
++    final MetricsScope fooScope = metrics.getScope(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.getScope(scopeName));
++    Thread.sleep(periodMs+ 1);
++    // 1st close:
++    metrics.endScope(scopeName); // closing of open scope should be ok.
++
++    assertTrue(fooScope.getNumCounter().longValue() >= 1);
++    final long t1 = fooScope.getTimeCounter().longValue();
++    assertTrue(t1 > periodMs);
++
++    expectIOE(new Callable<Void>() {
++      @Override
++      public Void call() throws Exception {
++        metrics.endScope(scopeName); // closing of closed scope not allowed
++        return null;
++      }
++    });
++
++    assertSame(fooScope, metrics.getScope(scopeName));
++
++   // opening allowed after closing:
++    metrics.startScope(scopeName);
++
++    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.startScope(scopeName);
++        return null;
++      }
++    });
++
++    assertSame(fooScope, metrics.getScope(scopeName));
++    Thread.sleep(periodMs + 1);
++    // Reopening (close + open) allowed in opened state:
++    fooScope.reopen();
++
++    assertTrue(fooScope.getNumCounter().longValue() >= 2);
++    assertTrue(fooScope.getTimeCounter().longValue() > 2 * periodMs);
++
++    Thread.sleep(periodMs + 1);
++    // 3rd close:
++    fooScope.close();
++
++    assertTrue(fooScope.getNumCounter().longValue() >= 3);
++    assertTrue(fooScope.getTimeCounter().longValue() > 3 * periodMs);
++    Double avgT = (Double) metrics.get("foo.avg_t");
++    assertTrue(avgT.doubleValue() > periodMs);
++  }
++}
+diff --git a/common/src/test/org/apache/hadoop/hive/common/metrics/TestMetrics.java b/common/src/test/org/apache/hadoop/hive/common/metrics/TestMetrics.java
+deleted file mode 100644
+index e85d3f8..0000000
+--- a/common/src/test/org/apache/hadoop/hive/common/metrics/TestMetrics.java
++++ /dev/null
+@@ -1,286 +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.common.metrics;
+-
+-import java.io.IOException;
+-import java.lang.management.ManagementFactory;
+-import java.util.concurrent.Callable;
+-import java.util.concurrent.ExecutorService;
+-import java.util.concurrent.Executors;
+-import java.util.concurrent.TimeUnit;
+-
+-import javax.management.Attribute;
+-import javax.management.MBeanAttributeInfo;
+-import javax.management.MBeanInfo;
+-import javax.management.MBeanOperationInfo;
+-import javax.management.MBeanServer;
+-import javax.management.ObjectName;
+-
+-import org.apache.hadoop.hive.common.metrics.Metrics.MetricsScope;
+-import org.junit.After;
+-import org.junit.Before;
+-import org.junit.Test;
+-import static org.junit.Assert.*;
+-
+-public class TestMetrics {
+-
+-  private static final String scopeName = "foo";
+-  private static final long periodMs = 50L;
+-
+-  @Before
+-  public void before() throws Exception {
+-    Metrics.uninit();
+-    Metrics.init();
+-  }
+-  
+-  @After
+-  public void after() throws Exception {
+-    Metrics.uninit();
+-  }
+-  
+-  @Test
+-  public void testMetricsMBean() throws Exception {
+-    MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+-    final ObjectName oname = new ObjectName(
+-        "org.apache.hadoop.hive.common.metrics:type=MetricsMBean");
+-    MBeanInfo mBeanInfo = mbs.getMBeanInfo(oname);
+-    // check implementation class:
+-    assertEquals(MetricsMBeanImpl.class.getName(), mBeanInfo.getClassName());
+-
+-    // check reset operation:
+-    MBeanOperationInfo[] oops = mBeanInfo.getOperations();
+-    boolean resetFound = false;
+-    for (MBeanOperationInfo op : oops) {
+-      if ("reset".equals(op.getName())) {
+-        resetFound = true;
+-        break;
+-      }
+-    }
+-    assertTrue(resetFound);
+-
+-    // add metric with a non-null value:
+-    Attribute attr = new Attribute("fooMetric", Long.valueOf(-77));
+-    mbs.setAttribute(oname, attr);
+-
+-    mBeanInfo = mbs.getMBeanInfo(oname);
+-    MBeanAttributeInfo[] attrinuteInfos = mBeanInfo.getAttributes();
+-    assertEquals(1, attrinuteInfos.length);
+-    boolean attrFound = false;
+-    for (MBeanAttributeInfo info : attrinuteInfos) {
+-      if ("fooMetric".equals(info.getName())) {
+-        assertEquals("java.lang.Long", info.getType());
+-        assertTrue(info.isReadable());
+-        assertTrue(info.isWritable());
+-        assertFalse(info.isIs());
+-
+-        attrFound = true;
+-        break;
+-      }
+-    }
+-    assertTrue(attrFound);
+-
+-    // check metric value:
+-    Object v = mbs.getAttribute(oname, "fooMetric");
+-    assertEquals(Long.valueOf(-77), v);
+-
+-    // reset the bean:
+-    Object result = mbs.invoke(oname, "reset", new Object[0], new String[0]);
+-    assertNull(result);
+-
+-    // the metric value must be zeroed:
+-    v = mbs.getAttribute(oname, "fooMetric");
+-    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 {
+-    final MetricsScope fooScope = Metrics.startScope(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;
+-      }
+-    });
+-    
+-    assertSame(fooScope, Metrics.getScope(scopeName));
+-    Thread.sleep(periodMs+1);
+-    // 1st close:
+-    // closing of open scope should be ok:
+-    Metrics.endScope(scopeName); 
+-    expectIOE(new Callable<Void>() {
+-      @Override
+-      public Void call() throws Exception {
+-        Metrics.endScope(scopeName); // closing of closed scope not allowed
+-        return null;
+-      }
+-    });
+-    
+-    assertEquals(Long.valueOf(1), fooScope.getNumCounter());
+-    final long t1 = fooScope.getTimeCounter().longValue(); 
+-    assertTrue(t1 > periodMs);
+-    
+-    assertSame(fooScope, Metrics.getScope(scopeName));
+-    
+-   // opening allowed after closing:
+-    Metrics.startScope(scopeName);
+-    // opening of already open scope not allowed:
+-    expectIOE(new Callable<Void>() {
+-      @Override
+-      public Void call() throws Exception {
+-        Metrics.startScope(scopeName); 
+-        return null;
+-      }
+-    });
+-    
+-    assertEquals(Long.valueOf(1), fooScope.getNumCounter());
+-    assertEquals(t1, fooScope.getTimeCounter().longValue());
+-    
+-    assertSame(fooScope, Metrics.getScope(scopeName));
+-    Thread.sleep(periodMs + 1);
+-    // Reopening (close + open) allowed in opened state: 
+-    fooScope.reopen();
+-
+-    assertEquals(Long.valueOf(2), fooScope.getNumCounter());
+-    assertTrue(fooScope.getTimeCounter().longValue() > 2 * periodMs);
+-    
+-    Thread.sleep(periodMs + 1);
+-    // 3rd close:
+-    fooScope.close();
+-    
+-    assertEquals(Long.valueOf(3), fooScope.getNumCounter());
+-    assertTrue(fooScope.getTimeCounter().longValue() > 3 * periodMs);
+-    Double avgT = (Double)Metrics.get("foo.avg_t");
+-    assertTrue(avgT.doubleValue() > periodMs);
+-  }
+-  
+-  @Test
+-  public void testScopeConcurrency() throws Exception {
+-    MetricsScope fooScope = Metrics.startScope(scopeName);
+-    final int threads = 10;
+-    ExecutorService executorService = Executors.newFixedThreadPool(threads);
+-    for (int i=0; i<threads; i++) {
+-      final int n = i;
+-      executorService.submit(new Callable<Void>() {
+-        @Override
+-        public Void call() throws Exception {
+-          testScopeImpl(n);
+-          return null; 
+-        }
+-      });
+-    }
+-    executorService.shutdown();
+-    assertTrue(executorService.awaitTermination(periodMs * 3 * threads, TimeUnit.MILLISECONDS));
+-
+-    fooScope = Metrics.getScope(scopeName);
+-    assertEquals(Long.valueOf(3 * threads), fooScope.getNumCounter());
+-    assertTrue(fooScope.getTimeCounter().longValue() > 3 * periodMs * threads);
+-    Double avgT = (Double)Metrics.get("foo.avg_t");
+-    assertTrue(avgT.doubleValue() > periodMs);
+-    Metrics.endScope(scopeName);
+-  }
+-  
+-  void testScopeImpl(int n) throws Exception {
+-    final MetricsScope fooScope = Metrics.startScope(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.getScope(scopeName));
+-    Thread.sleep(periodMs+1);
+-    // 1st close:
+-    Metrics.endScope(scopeName); // closing of open scope should be ok.
+-    
+-    assertTrue(fooScope.getNumCounter().longValue() >= 1);
+-    final long t1 = fooScope.getTimeCounter().longValue(); 
+-    assertTrue(t1 > periodMs);
+-    
+-    expectIOE(new Callable<Void>() {
+-      @Override
+-      public Void call() throws Exception {
+-        Metrics.endScope(scopeName); // closing of closed scope not allowed
+-        return null;
+-      }
+-    });
+-    
+-    assertSame(fooScope, Metrics.getScope(scopeName));
+-    
+-   // opening allowed after closing:
+-    Metrics.startScope(scopeName);
+-    
+-    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.startScope(scopeName); 
+-        return null;
+-      }
+-    });
+-    
+-    assertSame(fooScope, Metrics.getScope(scopeName));
+-    Thread.sleep(periodMs + 1);
+-    // Reopening (close + open) allowed in opened state: 
+-    fooScope.reopen();
+-
+-    assertTrue(fooScope.getNumCounter().longValue() >= 2);
+-    assertTrue(fooScope.getTimeCounter().longValue() > 2 * periodMs);
+-    
+-    Thread.sleep(periodMs + 1);
+-    // 3rd close:
+-    fooScope.close();
+-    
+-    assertTrue(fooScope.getNumCounter().longValue() >= 3);
+-    assertTrue(fooScope.getTimeCounter().longValue() > 3 * periodMs);
+-    Double avgT = (Double)Metrics.get("foo.avg_t");
+-    assertTrue(avgT.doubleValue() > periodMs);
+-  }
+-}
+diff --git a/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java b/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java
+new file mode 100644
+index 0000000..8749349
+--- /dev/null
++++ b/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java
+@@ -0,0 +1,138 @@
++/**
++ * 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.common.metrics.metrics2;
++
++import com.codahale.metrics.Counter;
++import com.codahale.metrics.MetricRegistry;
++import com.codahale.metrics.Timer;
++import com.fasterxml.jackson.databind.JsonNode;
++import com.fasterxml.jackson.databind.ObjectMapper;
++import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
++import org.apache.hadoop.hive.conf.HiveConf;
++import org.apache.hadoop.hive.shims.ShimLoader;
++import org.junit.After;
++import org.junit.Assert;
++import org.junit.Before;
++import org.junit.Test;
++
++import java.io.File;
++import java.nio.file.Files;
++import java.nio.file.Paths;
++import java.util.concurrent.Callable;
++import java.util.concurrent.ExecutorService;
++import java.util.concurrent.Executors;
++import java.util.concurrent.TimeUnit;
++
++import static org.junit.Assert.assertTrue;
++
++/**
++ * Unit test for new Metrics subsystem.
++ */
++public class TestCodahaleMetrics {
++
++  private static File workDir = new File(System.getProperty("test.tmp.dir"));
++  private static File jsonReportFile;
++  public static MetricRegistry metricRegistry;
++
++  @Before
++  public void before() throws Exception {
++    HiveConf conf = new HiveConf();
++
++    jsonReportFile = new File(workDir, "json_reporting");
++    jsonReportFile.delete();
++    String defaultFsName = ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPFS");
++    conf.set(defaultFsName, "local");
++    conf.setVar(HiveConf.ConfVars.HIVE_METRICS_CLASS, CodahaleMetrics.class.getCanonicalName());
++    conf.setVar(HiveConf.ConfVars.HIVE_METRICS_REPORTER, MetricsReporting.JSON_FILE.name() + "," + MetricsReporting.JMX.name());
++    conf.setVar(HiveConf.ConfVars.HIVE_METRICS_JSON_FILE_LOCATION, jsonReportFile.toString());
++    conf.setVar(HiveConf.ConfVars.HIVE_METRICS_JSON_FILE_INTERVAL, "100ms");
++
++    MetricsFactory.init(conf);
++    metricRegistry = ((CodahaleMetrics) MetricsFactory.getMetricsInstance()).getMetricRegistry();
++  }
++
++  @After
++  public void after() throws Exception {
++    MetricsFactory.deInit();
++  }
++
++  @Test
++  public void testScope() throws Exception {
++    int runs = 5;
++    for (int i = 0; i < runs; i++) {
++      MetricsFactory.getMetricsInstance().startScope("method1");
++      MetricsFactory.getMetricsInstance().endScope("method1");
++    }
++
++    Timer timer = metricRegistry.getTimers().get("api_method1");
++    Assert.assertEquals(5, timer.getCount());
++    Assert.assertTrue(timer.getMeanRate() > 0);
++  }
++
++
++  @Test
++  public void testCount() throws Exception {
++    int runs = 5;
++    for (int i = 0; i < runs; i++) {
++      MetricsFactory.getMetricsInstance().incrementCounter("count1");
++    }
++    Counter counter = metricRegistry.getCounters().get("count1");
++    Assert.assertEquals(5L, counter.getCount());
++  }
++
++  @Test
++  public void testConcurrency() throws Exception {
++    int threads = 4;
++    ExecutorService executorService = Executors.newFixedThreadPool(threads);
++    for (int i=0; i< threads; i++) {
++      final int n = i;
++      executorService.submit(new Callable<Void>() {
++        @Override
++        public Void call() throws Exception {
++          MetricsFactory.getMetricsInstance().startScope("method2");
++          MetricsFactory.getMetricsInstance().endScope("method2");
++          return null;
++        }
++      });
++    }
++    executorService.shutdown();
++    assertTrue(executorService.awaitTermination(10000, TimeUnit.MILLISECONDS));
++    Timer timer = metricRegistry.getTimers().get("api_method2");
++    Assert.assertEquals(4, timer.getCount());
++    Assert.assertTrue(timer.getMeanRate() > 0);
++  }
++
++  @Test
++  public void testFileReporting() throws Exception {
++    int runs = 5;
++    for (int i = 0; i < runs; i++) {
++      MetricsFactory.getMetricsInstance().incrementCounter("count2");
++      Thread.sleep(100);
++    }
++
++    Thread.sleep(2000);
++    byte[] jsonData = Files.readAllBytes(Paths.get(jsonReportFile.getAbsolutePath()));
++    ObjectMapper objectMapper = new ObjectMapper();
++
++    JsonNode rootNode = objectMapper.readTree(jsonData);
++    JsonNode countersNode = rootNode.path("counters");
++    JsonNode methodCounterNode = countersNode.path("count2");
++    JsonNode countNode = methodCounterNode.path("count");
++    Assert.assertEquals(countNode.asInt(), 5);
++  }
++}
+diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java
+new file mode 100644
+index 0000000..25f34d1
+--- /dev/null
++++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.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.hadoop.hive.metastore;
++
++import com.fasterxml.jackson.databind.JsonNode;
++import com.fasterxml.jackson.databind.ObjectMapper;
++import junit.framework.TestCase;
++import org.apache.hadoop.hive.cli.CliSessionState;
++import org.apache.hadoop.hive.common.metrics.metrics2.MetricsReporting;
++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.After;
++import org.junit.AfterClass;
++import org.junit.Assert;
++import org.junit.Before;
++import org.junit.BeforeClass;
++import org.junit.Test;
++
++import java.io.File;
++import java.io.IOException;
++import java.nio.file.Files;
++import java.nio.file.Paths;
++
++/**
++ * Tests Hive Metastore Metrics.
++ */
++public class TestMetaStoreMetrics {
++
++  private static File workDir = new File(System.getProperty("test.tmp.dir"));
++  private static File jsonReportFile;
++
++  private static HiveConf hiveConf;
++  private static Driver driver;
++
++
++  @Before
++  public void before() throws Exception {
++
++    int port = MetaStoreUtils.findFreePort();
++
++    jsonReportFile = new File(workDir, "json_reporting");
++    jsonReportFile.delete();
++
++    hiveConf = new HiveConf(TestMetaStoreMetrics.class);
++    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
++    hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
++    hiveConf.setBoolVar(HiveConf.ConfVars.METASTORE_METRICS, true);
++    hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
++    hiveConf.setVar(HiveConf.ConfVars.HIVE_METRICS_REPORTER, MetricsReporting.JSON_FILE.name() + "," + MetricsReporting.JMX.name());
++    hiveConf.setVar(HiveConf.ConfVars.HIVE_METRICS_JSON_FILE_LOCATION, jsonReportFile.toString());
++    hiveConf.setVar(HiveConf.ConfVars.HIVE_METRICS_JSON_FILE_INTERVAL, "100ms");
++
++    MetaStoreUtils.startMetaStore(port, ShimLoader.getHadoopThriftAuthBridge(), hiveConf);
++
++    SessionState.start(new CliSessionState(hiveConf));
++    driver = new Driver(hiveConf);
++  }
++
++  @Test
++  public void testMetricsFile() throws Exception {
++    driver.run("show databases");
++
++    //give timer thread a chance to print the metrics
++    Thread.sleep(2000);
++
++    //As the file is being written, try a few times.
++    //This can be replaced by CodahaleMetrics's JsonServlet reporter once it is exposed.
++    byte[] jsonData = Files.readAllBytes(Paths.get(jsonReportFile.getAbsolutePath()));
++    ObjectMapper objectMapper = new ObjectMapper();
++
++    JsonNode rootNode = objectMapper.readTree(jsonData);
++    JsonNode countersNode = rootNode.path("timers");
++    JsonNode methodCounterNode = countersNode.path("api_get_all_databases");
++    JsonNode countNode = methodCounterNode.path("count");
++    Assert.assertTrue(countNode.asInt() > 0);
++  }
++}
+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 d81c856..1688920 100644
+--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
++++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+@@ -18,39 +18,14 @@
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+-import static org.apache.commons.lang.StringUtils.join;
+-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_COMMENT;
+-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.validateName;
+-
+-import java.io.IOException;
+-import java.text.DateFormat;
+-import java.text.SimpleDateFormat;
+-import java.util.AbstractMap;
+-import java.util.ArrayList;
+-import java.util.Arrays;
+-import java.util.Collections;
+-import java.util.Formatter;
+-import java.util.HashMap;
+-import java.util.HashSet;
+-import java.util.Iterator;
+-import java.util.LinkedHashMap;
+-import java.util.LinkedList;
+-import java.util.List;
+-import java.util.Map;
+-import java.util.Map.Entry;
+-import java.util.Properties;
+-import java.util.Set;
+-import java.util.Timer;
+-import java.util.concurrent.TimeUnit;
+-import java.util.concurrent.atomic.AtomicBoolean;
+-import java.util.concurrent.locks.Condition;
+-import java.util.concurrent.locks.Lock;
+-import java.util.concurrent.locks.ReentrantLock;
+-import java.util.regex.Pattern;
+-
+-import javax.jdo.JDOException;
+-
++import com.facebook.fb303.FacebookBase;
++import com.facebook.fb303.fb_status;
++import com.google.common.annotations.VisibleForTesting;
++import com.google.common.base.Splitter;
++import com.google.common.collect.ImmutableList;
++import com.google.common.collect.ImmutableListMultimap;
++import com.google.common.collect.Lists;
++import com.google.common.collect.Multimaps;
+ import org.apache.commons.cli.OptionBuilder;
+ import org.apache.commons.logging.Log;
+ import org.apache.commons.logging.LogFactory;
+@@ -58,12 +33,13 @@
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.hive.common.FileUtils;
++import org.apache.hadoop.hive.common.JvmPauseMonitor;
+ import org.apache.hadoop.hive.common.LogUtils;
+ import org.apache.hadoop.hive.common.LogUtils.LogInitializationException;
+ import org.apache.hadoop.hive.common.classification.InterfaceAudience;
+ import org.apache.hadoop.hive.common.classification.InterfaceStability;
+ import org.apache.hadoop.hive.common.cli.CommonCliOptions;
+-import org.apache.hadoop.hive.common.metrics.Metrics;
++import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
+ import org.apache.hadoop.hive.conf.HiveConf;
+ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+ import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
+@@ -221,14 +197,35 @@
+ import org.apache.thrift.transport.TTransport;
+ import org.apache.thrift.transport.TTransportFactory;
+ 
+-import com.facebook.fb303.FacebookBase;
+-import com.facebook.fb303.fb_status;
+-import com.google.common.annotations.VisibleForTesting;
+-import com.google.common.base.Splitter;
+-import com.google.common.collect.ImmutableList;
+-import com.google.common.collect.ImmutableListMultimap;
+-import com.google.common.collect.Lists;
+-import com.google.common.collect.Multimaps;
++import javax.jdo.JDOException;
++import java.io.IOException;
++import java.text.DateFormat;
++import java.text.SimpleDateFormat;
++import java.util.AbstractMap;
++import java.util.ArrayList;
++import java.util.Arrays;
++import java.util.Collections;
++import java.util.Formatter;
++import java.util.HashMap;
++import java.util.HashSet;
++import java.util.Iterator;
++import java.util.LinkedHashMap;
++import java.util.LinkedList;
++import java.util.List;
++import java.util.Map;
++import java.util.Map.Entry;
++import java.util.Properties;
++import java.util.Set;
++import java.util.Timer;
++import java.util.concurrent.TimeUnit;
++import java.util.concurrent.atomic.AtomicBoolean;
++import java.util.concurrent.locks.Condition;
++import java.util.concurrent.locks.Lock;
++import java.util.concurrent.locks.ReentrantLock;
++import java.util.regex.Pattern;
++
++import static org.apache.commons.lang.StringUtils.join;
++import static org.apache.hadoop.hive.metastore.MetaStoreUtils.*;
+ 
+ /**
+  * TODO:pc remove application logic to a separate interface.
+@@ -464,9 +461,10 @@ public void init() throws MetaException {
+         }
+       }
+ 
+-      if (hiveConf.getBoolean("hive.metastore.metrics.enabled", false)) {
++      //Start Metrics for Embedded mode
++      if (hiveConf.getBoolVar(ConfVars.METASTORE_METRICS)) {
+         try {
+-          Metrics.init();
++          MetricsFactory.init(hiveConf);
+         } catch (Exception e) {
+           // log exception, but ignore inability to start
+           LOG.error("error in Metrics init: " + e.getClass().getName() + " "
+@@ -750,11 +748,13 @@ private String startFunction(String function, String extraLogInfo) {
+       incrementCounter(function);
+       logInfo((getIpAddress() == null ? "" : "source:" + getIpAddress() + " ") +
+           function + extraLogInfo);
+-      try {
+-        Metrics.startScope(function);
+-      } catch (IOException e) {
+-        LOG.debug("Exception when starting metrics scope"
++      if (hiveConf.getBoolVar(ConfVars.METASTORE_METRICS)) {
++        try {
++          MetricsFactory.getMetricsInstance().startScope(function);
++        } catch (IOException e) {
++          LOG.debug("Exception when starting metrics scope"
+             + e.getClass().getName() + " " + e.getMessage(), e);
++        }
+       }
+       return function;
+     }
+@@ -792,10 +792,12 @@ private void endFunction(String function, boolean successful, Exception e,
+     }
+ 
+     private void endFunction(String function, MetaStoreEndFunctionContext context) {
+-      try {
+-        Metrics.endScope(function);
+-      } catch (IOException e) {
+-        LOG.debug("Exception when closing metrics scope" + e);
++      if (hiveConf.getBoolVar(ConfVars.METASTORE_METRICS)) {
++        try {
++          MetricsFactory.getMetricsInstance().endScope(function);
++        } catch (IOException e) {
++          LOG.debug("Exception when closing metrics scope" + e);
++        }
+       }
+ 
+       for (MetaStoreEndFunctionListener listener : endFunctionListeners) {
+@@ -819,6 +821,14 @@ public void shutdown() {
+           threadLocalMS.remove();
+         }
+       }
++      if (hiveConf.getBoolVar(ConfVars.METASTORE_METRICS)) {
++        try {
++          MetricsFactory.deInit();
++        } catch (Exception e) {
++          LOG.error("error in Metrics deinit: " + e.getClass().getName() + " "
++            + e.getMessage(), e);
++        }
++      }
+       logInfo("Metastore shutdown complete.");
+     }
+ 
+@@ -5901,6 +5911,16 @@ public void run() {
+         }
+       });
+ 
++      //Start Metrics for Standalone (Remote) Mode
++      if (conf.getBoolVar(ConfVars.METASTORE_METRICS)) {
++        try {
++          MetricsFactory.init(conf);
++        } catch (Exception e) {
++          // log exception, but ignore inability to start
++          LOG.error("error in Metrics init: " + e.getClass().getName() + " "
++            + e.getMessage(), e);
++        }
++      }
+ 
+       Lock startLock = new ReentrantLock();
+       Condition startCondition = startLock.newCondition();
+@@ -6091,7 +6111,13 @@ public void run() {
+         // Wrap the start of the threads in a catch Throwable loop so that any failures
+         // don't doom the rest of the metastore.
+         startLock.lock();
+-        ShimLoader.getHadoopShims().startPauseMonitor(conf);
++        try {
++          JvmPauseMonitor pauseMonitor = new JvmPauseMonitor(conf);
++          pauseMonitor.start();
++        } catch (Throwable t) {
++          LOG.warn("Could not initiate the JvmPauseMonitor thread." + " GCs and Pauses may not be " +
++            "warned upon.", t);
++        }
+ 
+         try {
+           // Per the javadocs on Condition, do not depend on the condition alone as a start gate
+diff --git a/pom.xml b/pom.xml
+index b21d894..35133f2 100644
+--- a/pom.xml
++++ b/pom.xml
+@@ -116,6 +116,7 @@
+     <commons-pool.version>1.5.4</commons-pool.version>
+     <commons-dbcp.version>1.4</commons-dbcp.version>
+     <derby.version>10.11.1.1</derby.version>
++    <dropwizard.version>3.1.0</dropwizard.version>
+     <guava.version>14.0.1</guava.version>
+     <groovy.version>2.1.6</groovy.version>
+     <hadoop-20S.version>1.2.1</hadoop-20S.version>
+@@ -128,6 +129,8 @@
+     <httpcomponents.core.version>4.4</httpcomponents.core.version>
+     <ivy.version>2.4.0</ivy.version>
+     <jackson.version>1.9.2</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>
+     <javaewah.version>0.3.2</javaewah.version>
+     <javolution.version>5.5.1</javolution.version>
+     <jdo-api.version>3.0.1</jdo-api.version>
+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 58e8e49..7820ed5 100644
+--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
++++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
+@@ -42,14 +42,15 @@
+ import org.apache.curator.framework.api.CuratorEventType;
+ import org.apache.curator.framework.recipes.nodes.PersistentEphemeralNode;
+ import org.apache.curator.retry.ExponentialBackoffRetry;
++import org.apache.hadoop.hive.common.JvmPauseMonitor;
+ import org.apache.hadoop.hive.common.LogUtils;
+ import org.apache.hadoop.hive.common.LogUtils.LogInitializationException;
++import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
+ import org.apache.hadoop.hive.conf.HiveConf;
+ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+ import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl;
+ import org.apache.hadoop.hive.ql.exec.tez.TezSessionPoolManager;
+ import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper;
+-import org.apache.hadoop.hive.shims.ShimLoader;
+ import org.apache.hadoop.hive.shims.Util

<TRUNCATED>

[20/23] hive git commit: HIVE-11866 : Add framework to enable testing using LDAPServer using LDAP protocol (Naveen Gangam via Szehon)

Posted by se...@apache.org.
HIVE-11866 : Add framework to enable testing using LDAPServer using LDAP protocol (Naveen Gangam via Szehon)


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

Branch: refs/heads/llap
Commit: 8964c1ebc7f14f03c2c5773a785ed50d318798fe
Parents: a989f69
Author: Szehon Ho <sz...@cloudera.com>
Authored: Mon Oct 5 10:15:20 2015 -0700
Committer: Szehon Ho <sz...@cloudera.com>
Committed: Mon Oct 5 10:15:20 2015 -0700

----------------------------------------------------------------------
 pom.xml                                         |   1 +
 service/pom.xml                                 |   6 +
 .../auth/TestLdapAtnProviderWithLdapServer.java | 215 +++++++++++++++++++
 .../org/apache/hive/service/auth/ldapdata.ldif  |  59 +++++
 4 files changed, 281 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8964c1eb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b11a405..2ef2a09 100644
--- a/pom.xml
+++ b/pom.xml
@@ -166,6 +166,7 @@
     <scala.version>2.10.4</scala.version>
     <tempus-fugit.version>1.1</tempus-fugit.version>
     <snappy.version>0.2</snappy.version>
+    <unboundid.version>2.3.1</unboundid.version>
     <wadl-resourcedoc-doclet.version>1.4</wadl-resourcedoc-doclet.version>
     <velocity.version>1.5</velocity.version>
     <xerces.version>2.9.1</xerces.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/8964c1eb/service/pom.xml
----------------------------------------------------------------------
diff --git a/service/pom.xml b/service/pom.xml
index 07eeb9a..d9bf8d1 100644
--- a/service/pom.xml
+++ b/service/pom.xml
@@ -111,6 +111,12 @@
       <version>${junit.version}</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.unboundid</groupId>
+      <artifactId>unboundid-ldapsdk</artifactId>
+      <version>${unboundid.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/hive/blob/8964c1eb/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithLdapServer.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithLdapServer.java b/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithLdapServer.java
new file mode 100644
index 0000000..8f015b0
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithLdapServer.java
@@ -0,0 +1,215 @@
+/**
+ * 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;
+
+import com.unboundid.ldap.listener.InMemoryDirectoryServer;
+import com.unboundid.ldap.listener.InMemoryDirectoryServerConfig;
+import com.unboundid.ldap.listener.InMemoryListenerConfig;
+import com.unboundid.ldap.sdk.DN;
+import com.unboundid.ldap.sdk.LDAPConnection;
+import com.unboundid.ldif.LDIFReader;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.InputStream;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.util.Set;
+
+import javax.security.sasl.AuthenticationException;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests that use an in-memory LDAP Server (unboundID) to test HS2's
+ * LDAP Authentication Provider. The ldap server uses a sample ldif
+ * file to load ldap data into its directory.
+ * Any of Hive's LDAP Configuration properties are set on the HiveConf
+ * prior to the initialization of LdapAuthenticationProviderImpl.
+ * Each test uses a different set of properties to alter the Atn
+ * provider behavior.
+ */
+public class TestLdapAtnProviderWithLdapServer {
+  private static String ldapUrl;
+  private static InMemoryDirectoryServer server;
+  private static InMemoryDirectoryServerConfig config;
+  private static HiveConf hiveConf;
+  private static byte[] hiveConfBackup;
+  private static LdapAuthenticationProviderImpl ldapProvider;
+  private static final int serverPort = 33300;
+
+  @BeforeClass
+  public static void init() throws Exception {
+    DN dn = new DN("dc=example, dc=com");
+    config = new InMemoryDirectoryServerConfig(dn);
+    config.setSchema(null);
+    config.addAdditionalBindCredentials("cn=user1,ou=People,dc=example,dc=com","user1");
+    config.addAdditionalBindCredentials("cn=user2,ou=People,dc=example,dc=com","user2");
+
+    // listener config only necessary if you want to make sure that the
+    // server listens on port 33300, otherwise a free random port will
+    // be picked at runtime - which might be even better for tests btw.
+    config.setListenerConfigs(
+            new InMemoryListenerConfig("myListener", null, serverPort, null, null, null));
+
+    server = new InMemoryDirectoryServer(config);
+
+    server.startListening();
+
+    File ldifFile = new File(Thread.currentThread().getContextClassLoader()
+                       .getResource("org/apache/hive/service/auth/ldapdata.ldif").getFile());
+    LDIFReader ldifReader = new LDIFReader(ldifFile);
+    // import your test data from ldif files
+    server.importFromLDIF(true, ldifReader);
+
+    LDAPConnection conn = server.getConnection();
+    int port = server.getListenPort();
+    ldapUrl = new String("ldap://localhost:" + port);
+
+    hiveConf = new HiveConf();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    hiveConf.writeXml(baos);
+    baos.close();
+    hiveConfBackup = baos.toByteArray();
+    hiveConf.set("hive.root.logger", "TRACE,console");
+    hiveConf.set("hive.server2.authentication.ldap.url", ldapUrl);
+    hiveConf.set("hive.server2.authentication.ldap.baseDN", "dc=example,dc=com");
+    hiveConf.set("hive.server2.authentication.ldap.userDNPattern", "cn=%s,ou=People,dc=example,dc=com");
+    FileOutputStream fos = new FileOutputStream(new File(hiveConf.getHiveSiteLocation().toURI()));
+    hiveConf.writeXml(fos);
+    fos.close();
+
+    ldapProvider = new LdapAuthenticationProviderImpl();
+  }
+
+  private static void initLdapAtn(Hashtable<String, String> hiveProperties)
+        throws Exception {
+    Set<String> keys = hiveProperties.keySet();
+    Iterator<String> iter = keys.iterator();
+    hiveConf = new HiveConf();
+
+    try {
+      boolean deleted = new File(hiveConf.getHiveSiteLocation().toURI()).delete();
+    } catch (Exception e) {}
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    hiveConf.writeXml(baos);
+    baos.close();
+
+    hiveConf.set("hive.root.logger", "TRACE,console");
+    hiveConf.set("hive.server2.authentication.ldap.url", ldapUrl);
+    hiveConf.set("hive.server2.authentication.ldap.userDNPattern", "cn=%s,ou=People,dc=example,dc=com");
+    hiveConf.set("hive.server2.authentication.ldap.groupDNPattern", "cn=%s,ou=Groups,dc=example,dc=com");
+
+    String key;
+    String value;
+    while (iter.hasNext()) {
+      key = iter.next();
+      value = hiveProperties.get(key);
+      hiveConf.set(key, value);
+    }
+
+    FileOutputStream fos = new FileOutputStream(new File(hiveConf.getHiveSiteLocation().toURI()));
+    hiveConf.writeXml(fos);
+    fos.close();
+
+    ldapProvider = new LdapAuthenticationProviderImpl();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    server.shutDown(true);
+  }
+
+  @Test
+  public void testRoot() throws Exception {
+    Hashtable<String, String> ldapProperties = new Hashtable<String, String>();
+    initLdapAtn(ldapProperties);
+    String user;
+
+    user = "cn=user1,ou=People,dc=example,dc=com";
+    try {
+      ldapProvider.Authenticate(user, "user1");
+      assertTrue(true);
+
+      user = "cn=user2,ou=People,dc=example,dc=com";
+      ldapProvider.Authenticate(user, "user2");
+      assertTrue(true);
+    } catch (AuthenticationException e) {
+      e.printStackTrace();
+      Assert.fail("Authentication failed for user:" + user);
+    }
+  }
+
+  @Test
+  public void testUserBindPositive() throws Exception {
+    Hashtable<String, String> ldapProperties = new Hashtable<String, String>();
+    ldapProperties.put("hive.server2.authentication.ldap.userFilter", "user1,user2");
+    initLdapAtn(ldapProperties);
+    String user;
+
+    user = "cn=user1,ou=People,dc=example,dc=com";
+    try {
+      ldapProvider.Authenticate(user, "user1");
+      assertTrue("testUserBindPositive: Authentication succeeded for user1 as expected", true);
+    } catch (AuthenticationException e) {
+      Assert.fail("testUserBindPositive: Authentication failed for user:" + user +
+                    " with password user1, expected to succeed");
+    }
+
+    user = "cn=user2,ou=People,dc=example,dc=com";
+    try {
+      ldapProvider.Authenticate(user, "user2");
+      assertTrue("testUserBindPositive: Authentication succeeded for user2 as expected", true);
+    } catch (AuthenticationException e) {
+      Assert.fail("testUserBindPositive: Authentication failed for user:" + user +
+                    " with password user2, expected to succeed");
+    }
+  }
+
+  @Test
+  public void testUserBindNegative() throws Exception {
+    Hashtable<String, String> ldapProperties = new Hashtable<String, String>();
+    initLdapAtn(ldapProperties);
+
+    try {
+      ldapProvider.Authenticate("cn=user1,ou=People,dc=example,dc=com", "user2");
+      Assert.fail("testUserBindNegative: Authentication succeeded for user1 with password " +
+                   "user2, expected to fail");
+    } catch (AuthenticationException e) {
+      assertTrue("testUserBindNegative: Authentication failed for user1 as expected", true);
+    }
+
+    try {
+      ldapProvider.Authenticate("cn=user2,ou=People,dc=example,dc=com", "user");
+      Assert.fail("testUserBindNegative: Authentication failed for user2 with password user, " +
+                    "expected to fail");
+    } catch (AuthenticationException e) {
+      assertTrue("testUserBindNegative: Authentication failed for user2 as expected", true);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/8964c1eb/service/src/test/resources/org/apache/hive/service/auth/ldapdata.ldif
----------------------------------------------------------------------
diff --git a/service/src/test/resources/org/apache/hive/service/auth/ldapdata.ldif b/service/src/test/resources/org/apache/hive/service/auth/ldapdata.ldif
new file mode 100644
index 0000000..686fb3f
--- /dev/null
+++ b/service/src/test/resources/org/apache/hive/service/auth/ldapdata.ldif
@@ -0,0 +1,59 @@
+dn: dc=example,dc=com
+distinguishedName: dc=example,dc=com
+objectClass: top
+objectClass: domain
+dc: example
+
+dn: ou=People,dc=example,dc=com
+distinguishedName: ou=People,dc=example,dc=com
+objectClass: top
+objectClass: organizationalUnit
+ou: People
+description: Contains entries which describe persons (seamen)
+
+dn: ou=Groups,dc=example,dc=com
+distinguishedName: ou=Groups,dc=example,dc=com
+objectClass: top
+objectClass: organizationalUnit
+ou: Groups
+description: Contains entries which describe groups (crews, for instance)
+
+dn: cn=group1,ou=Groups,dc=example,dc=com
+distinguishedName: cn=group1,ou=Groups,dc=example,dc=com
+objectClass: group
+objectClass: top
+givenName: Group1
+cn: Test Group1
+sn: group1
+
+dn: cn=group2,ou=Groups,dc=example,dc=com
+distinguishedName: cn=group2,ou=Groups,dc=example,dc=com
+objectClass: group
+objectClass: top
+givenName: Group2
+cn: Test Group2
+sn: group1
+
+dn: cn=user1,ou=People,dc=example,dc=com
+distinguishedName: cn=user1,ou=People,dc=example,dc=com
+objectClass: inetOrgPerson
+objectClass: person
+objectClass: top
+givenName: Test1
+cn: Test User1
+sn: user1
+uid: user1
+userPassword: user1
+memberOf: cn=group1,ou=Groups,dc=example,dc=com
+
+dn: cn=user2,ou=People,dc=example,dc=com
+distinguishedName: cn=user2,ou=People,dc=example,dc=com
+objectClass: inetOrgPerson
+objectClass: person
+objectClass: top
+givenName: Test2
+cn: Test User2
+sn: user2
+uid: user2
+userPassword: user2
+memberOf: cn=group2,ou=Groups,dc=example,dc=com


[08/23] hive git commit: HIVE-11913 : Verify existence of tests for new changes in HiveQA (Szehon, reviewed by Sergio Pena)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/test/resources/HIVE-11271.4.patch
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/resources/HIVE-11271.4.patch b/testutils/ptest2/src/test/resources/HIVE-11271.4.patch
new file mode 100644
index 0000000..4a07c37
--- /dev/null
+++ b/testutils/ptest2/src/test/resources/HIVE-11271.4.patch
@@ -0,0 +1,606 @@
+diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
+index c076d4e112a7edab2106f11fe6224247887313cf..8bcb464de540eda7c14a8c6783bb19a09071af7b 100644
+--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
++++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
+@@ -25,7 +25,9 @@
+ 
+ import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+ import org.apache.hadoop.hive.ql.exec.CommonJoinOperator;
++import org.apache.hadoop.hive.ql.exec.FilterOperator;
+ import org.apache.hadoop.hive.ql.exec.Operator;
++import org.apache.hadoop.hive.ql.exec.OperatorFactory;
+ import org.apache.hadoop.hive.ql.exec.RowSchema;
+ import org.apache.hadoop.hive.ql.exec.SelectOperator;
+ import org.apache.hadoop.hive.ql.exec.UnionOperator;
+@@ -33,6 +35,7 @@
+ import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+ import org.apache.hadoop.hive.ql.parse.ParseContext;
+ import org.apache.hadoop.hive.ql.parse.SemanticException;
++import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+ import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+ import org.apache.hadoop.hive.ql.plan.SelectDesc;
+@@ -241,4 +244,65 @@ public ParseContext getParseContext() {
+     }
+     return columns;
+   }
++
++  /**
++   * If the input filter operator has direct child(ren) which are union operator,
++   * and the filter's column is not the same as union's
++   * create select operator between them. The select operator has same number of columns as
++   * pruned child operator.
++   *
++   * @param curOp
++   *          The filter operator which need to handle children.
++   * @throws SemanticException
++   */
++  public void handleFilterUnionChildren(Operator<? extends OperatorDesc> curOp)
++      throws SemanticException {
++    if (curOp.getChildOperators() == null || !(curOp instanceof FilterOperator)) {
++      return;
++    }
++    List<String> parentPrunList = prunedColLists.get(curOp);
++    if(parentPrunList == null || parentPrunList.size() == 0) {
++      return;
++    }
++    FilterOperator filOp = (FilterOperator)curOp;
++    List<String> prunList = null;
++    List<Integer>[] childToParentIndex = null;
++
++    for (Operator<? extends OperatorDesc> child : curOp.getChildOperators()) {
++      if (child instanceof UnionOperator) {
++        prunList = prunedColLists.get(child);
++        if (prunList == null || prunList.size() == 0 || parentPrunList.size() == prunList.size()) {
++          continue;
++        }
++
++        ArrayList<ExprNodeDesc> exprs = new ArrayList<ExprNodeDesc>();
++        ArrayList<String> outputColNames = new ArrayList<String>();
++        Map<String, ExprNodeDesc> colExprMap = new HashMap<String, ExprNodeDesc>();
++        ArrayList<ColumnInfo> outputRS = new ArrayList<ColumnInfo>();
++        for (ColumnInfo colInfo : child.getSchema().getSignature()) {
++          if (!prunList.contains(colInfo.getInternalName())) {
++            continue;
++          }
++          ExprNodeDesc colDesc = new ExprNodeColumnDesc(colInfo.getType(),
++              colInfo.getInternalName(), colInfo.getTabAlias(), colInfo.getIsVirtualCol());
++          exprs.add(colDesc);
++          outputColNames.add(colInfo.getInternalName());
++          ColumnInfo newCol = new ColumnInfo(colInfo.getInternalName(), colInfo.getType(),
++                  colInfo.getTabAlias(), colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol());
++          newCol.setAlias(colInfo.getAlias());
++          outputRS.add(newCol);
++          colExprMap.put(colInfo.getInternalName(), colDesc);
++        }
++        SelectDesc select = new SelectDesc(exprs, outputColNames, false);
++        curOp.removeChild(child);
++        SelectOperator sel = (SelectOperator) OperatorFactory.getAndMakeChild(
++            select, new RowSchema(outputRS), curOp);
++        OperatorFactory.makeChild(sel, child);
++        sel.setColumnExprMap(colExprMap);
++
++      }
++
++    }
++  }
++
+ }
+diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
+index ac4236c53adf6fa36ad43b2e9029d335f12efde2..2dc15f9f0ae96bdc7f33f3d97ad41c88117734d0 100644
+--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
++++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
+@@ -108,7 +108,7 @@ public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
+           filterOpPrunedColListsOrderPreserved);
+ 
+       pruneOperator(cppCtx, op, cppCtx.getPrunedColLists().get(op));
+-
++      cppCtx.handleFilterUnionChildren(op);
+       return null;
+     }
+   }
+diff --git a/ql/src/test/queries/clientpositive/unionall_unbalancedppd.q b/ql/src/test/queries/clientpositive/unionall_unbalancedppd.q
+new file mode 100644
+index 0000000000000000000000000000000000000000..0825c2d94d0f9815c7ff88549c77662e53adf928
+--- /dev/null
++++ b/ql/src/test/queries/clientpositive/unionall_unbalancedppd.q
+@@ -0,0 +1,120 @@
++set hive.optimize.ppd=true;
++drop table if exists union_all_bug_test_1;
++drop table if exists union_all_bug_test_2;
++create table if not exists union_all_bug_test_1
++(
++f1 int,
++f2 int
++);
++
++create table if not exists union_all_bug_test_2
++(
++f1 int
++);
++
++explain SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1);
++
++SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1);
++
++insert into table union_all_bug_test_1 values (1,1);
++insert into table union_all_bug_test_2 values (1);
++insert into table union_all_bug_test_1 values (0,0);
++insert into table union_all_bug_test_2 values (0);
++
++SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1);
++
++SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 0);
++
++SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1 or filter = 0);
++
++SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (f1 = 1);
+diff --git a/ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out b/ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out
+new file mode 100644
+index 0000000000000000000000000000000000000000..46828e9db772f20b47c3ae6aac1239bbcbabd752
+--- /dev/null
++++ b/ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out
+@@ -0,0 +1,373 @@
++PREHOOK: query: drop table if exists union_all_bug_test_1
++PREHOOK: type: DROPTABLE
++POSTHOOK: query: drop table if exists union_all_bug_test_1
++POSTHOOK: type: DROPTABLE
++PREHOOK: query: drop table if exists union_all_bug_test_2
++PREHOOK: type: DROPTABLE
++POSTHOOK: query: drop table if exists union_all_bug_test_2
++POSTHOOK: type: DROPTABLE
++PREHOOK: query: create table if not exists union_all_bug_test_1
++(
++f1 int,
++f2 int
++)
++PREHOOK: type: CREATETABLE
++PREHOOK: Output: database:default
++PREHOOK: Output: default@union_all_bug_test_1
++POSTHOOK: query: create table if not exists union_all_bug_test_1
++(
++f1 int,
++f2 int
++)
++POSTHOOK: type: CREATETABLE
++POSTHOOK: Output: database:default
++POSTHOOK: Output: default@union_all_bug_test_1
++PREHOOK: query: create table if not exists union_all_bug_test_2
++(
++f1 int
++)
++PREHOOK: type: CREATETABLE
++PREHOOK: Output: database:default
++PREHOOK: Output: default@union_all_bug_test_2
++POSTHOOK: query: create table if not exists union_all_bug_test_2
++(
++f1 int
++)
++POSTHOOK: type: CREATETABLE
++POSTHOOK: Output: database:default
++POSTHOOK: Output: default@union_all_bug_test_2
++PREHOOK: query: explain SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1)
++PREHOOK: type: QUERY
++POSTHOOK: query: explain SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1)
++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: union_all_bug_test_1
++            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
++            Filter Operator
++              predicate: (if(true, f1, f2) = 1) (type: boolean)
++              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
++              Select Operator
++                expressions: f1 (type: int)
++                outputColumnNames: _col0
++                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
++                Union
++                  Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
++                  File Output Operator
++                    compressed: false
++                    Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
++                    table:
++                        input format: org.apache.hadoop.mapred.TextInputFormat
++                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
++                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
++          TableScan
++            alias: union_all_bug_test_2
++            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
++            Filter Operator
++              predicate: false (type: boolean)
++              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
++              Select Operator
++                expressions: f1 (type: int)
++                outputColumnNames: _col0
++                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
++                Union
++                  Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
++                  File Output Operator
++                    compressed: false
++                    Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
++                    table:
++                        input format: org.apache.hadoop.mapred.TextInputFormat
++                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
++                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
++
++  Stage: Stage-0
++    Fetch Operator
++      limit: -1
++      Processor Tree:
++        ListSink
++
++PREHOOK: query: SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1)
++PREHOOK: type: QUERY
++PREHOOK: Input: default@union_all_bug_test_1
++PREHOOK: Input: default@union_all_bug_test_2
++#### A masked pattern was here ####
++POSTHOOK: query: SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1)
++POSTHOOK: type: QUERY
++POSTHOOK: Input: default@union_all_bug_test_1
++POSTHOOK: Input: default@union_all_bug_test_2
++#### A masked pattern was here ####
++PREHOOK: query: insert into table union_all_bug_test_1 values (1,1)
++PREHOOK: type: QUERY
++PREHOOK: Input: default@values__tmp__table__1
++PREHOOK: Output: default@union_all_bug_test_1
++POSTHOOK: query: insert into table union_all_bug_test_1 values (1,1)
++POSTHOOK: type: QUERY
++POSTHOOK: Input: default@values__tmp__table__1
++POSTHOOK: Output: default@union_all_bug_test_1
++POSTHOOK: Lineage: union_all_bug_test_1.f1 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
++POSTHOOK: Lineage: union_all_bug_test_1.f2 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
++PREHOOK: query: insert into table union_all_bug_test_2 values (1)
++PREHOOK: type: QUERY
++PREHOOK: Input: default@values__tmp__table__2
++PREHOOK: Output: default@union_all_bug_test_2
++POSTHOOK: query: insert into table union_all_bug_test_2 values (1)
++POSTHOOK: type: QUERY
++POSTHOOK: Input: default@values__tmp__table__2
++POSTHOOK: Output: default@union_all_bug_test_2
++POSTHOOK: Lineage: union_all_bug_test_2.f1 EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
++PREHOOK: query: insert into table union_all_bug_test_1 values (0,0)
++PREHOOK: type: QUERY
++PREHOOK: Input: default@values__tmp__table__3
++PREHOOK: Output: default@union_all_bug_test_1
++POSTHOOK: query: insert into table union_all_bug_test_1 values (0,0)
++POSTHOOK: type: QUERY
++POSTHOOK: Input: default@values__tmp__table__3
++POSTHOOK: Output: default@union_all_bug_test_1
++POSTHOOK: Lineage: union_all_bug_test_1.f1 EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
++POSTHOOK: Lineage: union_all_bug_test_1.f2 EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
++PREHOOK: query: insert into table union_all_bug_test_2 values (0)
++PREHOOK: type: QUERY
++PREHOOK: Input: default@values__tmp__table__4
++PREHOOK: Output: default@union_all_bug_test_2
++POSTHOOK: query: insert into table union_all_bug_test_2 values (0)
++POSTHOOK: type: QUERY
++POSTHOOK: Input: default@values__tmp__table__4
++POSTHOOK: Output: default@union_all_bug_test_2
++POSTHOOK: Lineage: union_all_bug_test_2.f1 EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
++PREHOOK: query: SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1)
++PREHOOK: type: QUERY
++PREHOOK: Input: default@union_all_bug_test_1
++PREHOOK: Input: default@union_all_bug_test_2
++#### A masked pattern was here ####
++POSTHOOK: query: SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1)
++POSTHOOK: type: QUERY
++POSTHOOK: Input: default@union_all_bug_test_1
++POSTHOOK: Input: default@union_all_bug_test_2
++#### A masked pattern was here ####
++1
++PREHOOK: query: SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 0)
++PREHOOK: type: QUERY
++PREHOOK: Input: default@union_all_bug_test_1
++PREHOOK: Input: default@union_all_bug_test_2
++#### A masked pattern was here ####
++POSTHOOK: query: SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 0)
++POSTHOOK: type: QUERY
++POSTHOOK: Input: default@union_all_bug_test_1
++POSTHOOK: Input: default@union_all_bug_test_2
++#### A masked pattern was here ####
++0
++1
++0
++PREHOOK: query: SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1 or filter = 0)
++PREHOOK: type: QUERY
++PREHOOK: Input: default@union_all_bug_test_1
++PREHOOK: Input: default@union_all_bug_test_2
++#### A masked pattern was here ####
++POSTHOOK: query: SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (filter = 1 or filter = 0)
++POSTHOOK: type: QUERY
++POSTHOOK: Input: default@union_all_bug_test_1
++POSTHOOK: Input: default@union_all_bug_test_2
++#### A masked pattern was here ####
++1
++0
++1
++0
++PREHOOK: query: SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (f1 = 1)
++PREHOOK: type: QUERY
++PREHOOK: Input: default@union_all_bug_test_1
++PREHOOK: Input: default@union_all_bug_test_2
++#### A masked pattern was here ####
++POSTHOOK: query: SELECT f1
++FROM (
++
++SELECT
++f1
++, if('helloworld' like '%hello%' ,f1,f2) as filter
++FROM union_all_bug_test_1
++
++union all
++
++select
++f1
++, 0 as filter
++from union_all_bug_test_2
++) A
++WHERE (f1 = 1)
++POSTHOOK: type: QUERY
++POSTHOOK: Input: default@union_all_bug_test_1
++POSTHOOK: Input: default@union_all_bug_test_2
++#### A masked pattern was here ####
++1
++1

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/test/resources/HIVE-9377.1.patch
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/resources/HIVE-9377.1.patch b/testutils/ptest2/src/test/resources/HIVE-9377.1.patch
new file mode 100644
index 0000000..9d2d5b6
--- /dev/null
+++ b/testutils/ptest2/src/test/resources/HIVE-9377.1.patch
@@ -0,0 +1,25 @@
+Index: ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInFile.java
+IDEA additional info:
+Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
+<+>UTF-8
+===================================================================
+--- ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInFile.java	(date 1421263954000)
++++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInFile.java	(revision )
+@@ -140,6 +140,17 @@
+   }
+ 
+   @Override
++  public void copyToNewInstance(Object newInstance) throws UDFArgumentException {
++    super.copyToNewInstance(newInstance); // Asserts the class invariant. (Same types.)
++    GenericUDFInFile that = (GenericUDFInFile)newInstance;
++    if (that != this) {
++      that.set = (this.set == null ? null : (HashSet<String>)this.set.clone());
++      that.strObjectInspector = this.strObjectInspector;
++      that.fileObjectInspector = this.fileObjectInspector;
++    }
++  }
++
++  @Override
+   public String getDisplayString(String[] children) {
+     assert (children.length == 2);
+     return "in_file(" + children[0] + ", " + children[1] + ")";

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/test/resources/remove-test.patch
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/resources/remove-test.patch b/testutils/ptest2/src/test/resources/remove-test.patch
new file mode 100644
index 0000000..3eac9d7
--- /dev/null
+++ b/testutils/ptest2/src/test/resources/remove-test.patch
@@ -0,0 +1,33 @@
+diff --git a/ql/src/test/queries/clientpositive/join0.q b/ql/src/test/queries/clientpositive/join0.q
+deleted file mode 100644
+index 6ef6843..0000000
+--- a/ql/src/test/queries/clientpositive/join0.q
++++ /dev/null
+@@ -1,27 +0,0 @@
+-set hive.explain.user=false;
+--- JAVA_VERSION_SPECIFIC_OUTPUT
+--- SORT_QUERY_RESULTS
+-
+-EXPLAIN
+-SELECT src1.key as k1, src1.value as v1, 
+-       src2.key as k2, src2.value as v2 FROM 
+-  (SELECT * FROM src WHERE src.key < 10) src1 
+-    JOIN 
+-  (SELECT * FROM src WHERE src.key < 10) src2
+-  SORT BY k1, v1, k2, v2;
+-
+-EXPLAIN FORMATTED
+-SELECT src1.key as k1, src1.value as v1, 
+-       src2.key as k2, src2.value as v2 FROM 
+-  (SELECT * FROM src WHERE src.key < 10) src1 
+-    JOIN 
+-  (SELECT * FROM src WHERE src.key < 10) src2
+-  SORT BY k1, v1, k2, v2;
+-
+-SELECT src1.key as k1, src1.value as v1, 
+-       src2.key as k2, src2.value as v2 FROM 
+-  (SELECT * FROM src WHERE src.key < 10) src1 
+-    JOIN 
+-  (SELECT * FROM src WHERE src.key < 10) src2
+-  SORT BY k1, v1, k2, v2;
+-


[14/23] hive git commit: HIVE-11699: Support special characters in quoted table names (Pengcheng Xiong, reviewed by John Pullokkaran)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/test/results/clientpositive/special_character_in_tabnames_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/special_character_in_tabnames_1.q.out b/ql/src/test/results/clientpositive/special_character_in_tabnames_1.q.out
new file mode 100644
index 0000000..bd0088a
--- /dev/null
+++ b/ql/src/test/results/clientpositive/special_character_in_tabnames_1.q.out
@@ -0,0 +1,19550 @@
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+create table `c/b/o_t1`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@c/b/o_t1
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+create table `c/b/o_t1`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@c/b/o_t1
+PREHOOK: query: create table `//cbo_t2`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@//cbo_t2
+POSTHOOK: query: create table `//cbo_t2`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@//cbo_t2
+PREHOOK: query: create table `cbo_/t3////`(key string, value string, c_int int, c_float float, c_boolean boolean)  row format delimited fields terminated by ',' STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@cbo_/t3////
+POSTHOOK: query: create table `cbo_/t3////`(key string, value string, c_int int, c_float float, c_boolean boolean)  row format delimited fields terminated by ',' STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@cbo_/t3////
+PREHOOK: query: load data local inpath '../../data/files/cbo_t1.txt' into table `c/b/o_t1` partition (dt='2014')
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@c/b/o_t1
+POSTHOOK: query: load data local inpath '../../data/files/cbo_t1.txt' into table `c/b/o_t1` partition (dt='2014')
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@c/b/o_t1
+POSTHOOK: Output: default@c/b/o_t1@dt=2014
+PREHOOK: query: load data local inpath '../../data/files/cbo_t2.txt' into table `//cbo_t2` partition (dt='2014')
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@//cbo_t2
+POSTHOOK: query: load data local inpath '../../data/files/cbo_t2.txt' into table `//cbo_t2` partition (dt='2014')
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@//cbo_t2
+POSTHOOK: Output: default@//cbo_t2@dt=2014
+PREHOOK: query: load data local inpath '../../data/files/cbo_t3.txt' into table `cbo_/t3////`
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@cbo_/t3////
+POSTHOOK: query: load data local inpath '../../data/files/cbo_t3.txt' into table `cbo_/t3////`
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@cbo_/t3////
+PREHOOK: query: CREATE TABLE `p/a/r/t`(
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@p/a/r/t
+POSTHOOK: query: CREATE TABLE `p/a/r/t`(
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@p/a/r/t
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table `p/a/r/t`
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@p/a/r/t
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table `p/a/r/t`
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@p/a/r/t
+PREHOOK: query: CREATE TABLE `line/item` (L_ORDERKEY      INT,
+                                L_PARTKEY       INT,
+                                L_SUPPKEY       INT,
+                                L_LINENUMBER    INT,
+                                L_QUANTITY      DOUBLE,
+                                L_EXTENDEDPRICE DOUBLE,
+                                L_DISCOUNT      DOUBLE,
+                                L_TAX           DOUBLE,
+                                L_RETURNFLAG    STRING,
+                                L_LINESTATUS    STRING,
+                                l_shipdate      STRING,
+                                L_COMMITDATE    STRING,
+                                L_RECEIPTDATE   STRING,
+                                L_SHIPINSTRUCT  STRING,
+                                L_SHIPMODE      STRING,
+                                L_COMMENT       STRING)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@line/item
+POSTHOOK: query: CREATE TABLE `line/item` (L_ORDERKEY      INT,
+                                L_PARTKEY       INT,
+                                L_SUPPKEY       INT,
+                                L_LINENUMBER    INT,
+                                L_QUANTITY      DOUBLE,
+                                L_EXTENDEDPRICE DOUBLE,
+                                L_DISCOUNT      DOUBLE,
+                                L_TAX           DOUBLE,
+                                L_RETURNFLAG    STRING,
+                                L_LINESTATUS    STRING,
+                                l_shipdate      STRING,
+                                L_COMMITDATE    STRING,
+                                L_RECEIPTDATE   STRING,
+                                L_SHIPINSTRUCT  STRING,
+                                L_SHIPMODE      STRING,
+                                L_COMMENT       STRING)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@line/item
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE `line/item`
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@line/item
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE `line/item`
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@line/item
+PREHOOK: query: create table `src/_/cbo` as select * from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src/_/cbo
+POSTHOOK: query: create table `src/_/cbo` as select * from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src/_/cbo
+PREHOOK: query: analyze table `c/b/o_t1` partition (dt) compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Output: default@c/b/o_t1
+PREHOOK: Output: default@c/b/o_t1@dt=2014
+POSTHOOK: query: analyze table `c/b/o_t1` partition (dt) compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Output: default@c/b/o_t1
+POSTHOOK: Output: default@c/b/o_t1@dt=2014
+PREHOOK: query: analyze table `c/b/o_t1` compute statistics for columns key, value, c_int, c_float, c_boolean
+PREHOOK: type: QUERY
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `c/b/o_t1` compute statistics for columns key, value, c_int, c_float, c_boolean
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+PREHOOK: query: analyze table `//cbo_t2` partition (dt) compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Output: default@//cbo_t2
+PREHOOK: Output: default@//cbo_t2@dt=2014
+POSTHOOK: query: analyze table `//cbo_t2` partition (dt) compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Output: default@//cbo_t2
+POSTHOOK: Output: default@//cbo_t2@dt=2014
+PREHOOK: query: analyze table `//cbo_t2` compute statistics for columns key, value, c_int, c_float, c_boolean
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `//cbo_t2` compute statistics for columns key, value, c_int, c_float, c_boolean
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+#### A masked pattern was here ####
+PREHOOK: query: analyze table `cbo_/t3////` compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_/t3////
+PREHOOK: Output: default@cbo_/t3////
+POSTHOOK: query: analyze table `cbo_/t3////` compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_/t3////
+POSTHOOK: Output: default@cbo_/t3////
+PREHOOK: query: analyze table `cbo_/t3////` compute statistics for columns key, value, c_int, c_float, c_boolean
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `cbo_/t3////` compute statistics for columns key, value, c_int, c_float, c_boolean
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+PREHOOK: query: analyze table `src/_/cbo` compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src/_/cbo
+PREHOOK: Output: default@src/_/cbo
+POSTHOOK: query: analyze table `src/_/cbo` compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src/_/cbo
+POSTHOOK: Output: default@src/_/cbo
+PREHOOK: query: analyze table `src/_/cbo` compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src/_/cbo
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `src/_/cbo` compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src/_/cbo
+#### A masked pattern was here ####
+PREHOOK: query: analyze table `p/a/r/t` compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: default@p/a/r/t
+PREHOOK: Output: default@p/a/r/t
+POSTHOOK: query: analyze table `p/a/r/t` compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@p/a/r/t
+POSTHOOK: Output: default@p/a/r/t
+PREHOOK: query: analyze table `p/a/r/t` compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@p/a/r/t
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `p/a/r/t` compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@p/a/r/t
+#### A masked pattern was here ####
+PREHOOK: query: analyze table `line/item` compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: default@line/item
+PREHOOK: Output: default@line/item
+POSTHOOK: query: analyze table `line/item` compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@line/item
+POSTHOOK: Output: default@line/item
+PREHOOK: query: analyze table `line/item` compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@line/item
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `line/item` compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@line/item
+#### A masked pattern was here ####
+PREHOOK: query: select key, (c_int+1)+2 as x, sum(c_int) from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select key, (c_int+1)+2 as x, sum(c_int) from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+ 1	4	2
+ 1 	4	2
+1	4	12
+1 	4	2
+NULL	NULL	NULL
+PREHOOK: query: select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key) R group by y, x
+PREHOOK: type: QUERY
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key) R group by y, x
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+5.0	12	1
+5.0	2	3
+NULL	NULL	1
+PREHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0) group by c_float, `c/b/o_t1`.c_int, key order by a) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key order by q/10 desc, r asc) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c order by `cbo_/t3////`.c_int+c desc, c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0) group by c_float, `c/b/o_t1`.c_int, key order by a) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key order by q/10 desc, r asc) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c order by `cbo_/t3////`.c_int+c desc, c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+1	12	6
+1	2	6
+PREHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) `c/b/o_t1` left outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key  having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `//cbo_t2` on `c/b/o_t1`.a=p left outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c  having `cbo_/t3////`.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0  order by `cbo_/t3////`.c_int % c asc, `cbo_/t3////`.c_int desc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) `c/b/o_t1` left outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key  having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `//cbo_t2` on `c/b/o_t1`.a=p left outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c  having `cbo_/t3////`.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0  order by `cbo_/t3////`.c_int % c asc, `cbo_/t3////`.c_int desc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+1	12	6
+1	2	6
+PREHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) `c/b/o_t1` right outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `//cbo_t2` on `c/b/o_t1`.a=p right outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 2) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) `c/b/o_t1` right outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `//cbo_t2` on `c/b/o_t1`.a=p right outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 2) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+1	12	6
+1	2	6
+PREHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) `c/b/o_t1` full outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by p+q desc, r asc) `//cbo_t2` on `c/b/o_t1`.a=p full outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c having `cbo_/t3////`.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by `cbo_/t3////`.c_int
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) `c/b/o_t1` full outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by p+q desc, r asc) `//cbo_t2` on `c/b/o_t1`.a=p full outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c having `cbo_/t3////`.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by `cbo_/t3////`.c_int
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+1	12	6
+1	2	6
+PREHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+1	12	6
+1	2	6
+PREHOOK: query: -- 21. Test groupby is empty and there is no other cols in aggr
+
+select unionsrc.key FROM (select 'tst1' as key, count(1) as value from src) unionsrc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: -- 21. Test groupby is empty and there is no other cols in aggr
+
+select unionsrc.key FROM (select 'tst1' as key, count(1) as value from src) unionsrc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+tst1
+PREHOOK: query: select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src) unionsrc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src) unionsrc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+tst1	500
+PREHOOK: query: select unionsrc.key FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc order by unionsrc.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select unionsrc.key FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc order by unionsrc.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+avg
+max
+min
+PREHOOK: query: select unionsrc.key, unionsrc.value FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc order by unionsrc.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select unionsrc.key, unionsrc.value FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc order by unionsrc.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+avg	1.5
+max	3.0
+min	1.0
+PREHOOK: query: select unionsrc.key, count(1) FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+    UNION  ALL
+
+        select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc group by unionsrc.key order by unionsrc.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select unionsrc.key, count(1) FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+    UNION  ALL
+
+        select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc group by unionsrc.key order by unionsrc.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+avg	1
+max	1
+min	1
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+-- 4. Test Select + Join + TS
+
+select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` join             `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+-- 4. Test Select + Join + TS
+
+select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` join             `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+PREHOOK: query: select `c/b/o_t1`.key from `c/b/o_t1` join `cbo_/t3////`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.key from `c/b/o_t1` join `cbo_/t3////`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+PREHOOK: query: select `c/b/o_t1`.key from `c/b/o_t1` join `cbo_/t3////` where `c/b/o_t1`.key=`cbo_/t3////`.key and `c/b/o_t1`.key >= 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.key from `c/b/o_t1` join `cbo_/t3////` where `c/b/o_t1`.key=`cbo_/t3////`.key and `c/b/o_t1`.key >= 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+ 1
+ 1
+ 1
+ 1
+ 1 
+ 1 
+ 1 
+ 1 
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1 
+1 
+1 
+1 
+PREHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` left outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` left outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+NULL	NULL
+NULL	NULL
+PREHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` right outer join `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` right outer join `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+NULL	2
+NULL	2
+NULL	2
+NULL	2
+NULL	2
+NULL	NULL
+NULL	NULL
+PREHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` full outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` full outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+NULL	2
+NULL	2
+NULL	2
+NULL	2
+NULL	2
+NULL	NULL
+NULL	NULL
+NULL	NULL
+NULL	NULL
+PREHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+PREHOOK: query: select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+PREHOOK: query: select a, `c/b/o_t1`.b, key, `//cbo_t2`.c_int, `cbo_/t3////`.p from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2`  on `c/b/o_t1`.a=key join (select key as p, c_int as q, `cbo_/t3////`.c_float as r from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.p
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select a, `c/b/o_t1`.b, key, `//cbo_t2`.c_int, `cbo_/t3////`.p from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2`  on `c/b/o_t1`.a=key join (select key as p, c_int as q, `cbo_/t3////`.c_float as r from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.p
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+PREHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.c_int, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.a=`//cbo_t2`.key join `cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.c_int, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.a=`//cbo_t2`.key join `cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+PREHOOK: query: select `cbo_/t3////`.c_int, b, `//cbo_t2`.c_int, `c/b/o_t1`.c from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.a=`//cbo_t2`.key join `cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, b, `//cbo_t2`.c_int, `c/b/o_t1`.c from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.a=`//cbo_t2`.key join `cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+PREHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` left outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` left outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+PREHOOK: query: select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p left outer join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p left outer join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+PREHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` right outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@//cbo_t2
+PREHOOK: Input: default@//cbo_t2@dt=2014
+PREHOOK: Input: default@c/b/o_t1
+PREHOOK: Input: default@c/b/o_t1@dt=2014
+PREHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` right outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@//cbo_t2
+POSTHOOK: Input: default@//cbo_t2@dt=2014
+POSTHOOK: Input: default@c/b/o_t1
+POSTHOOK: Input: default@c/b/o_t1@dt=2014
+POSTHOOK: Input: default@cbo_/t3////
+#### A masked pattern was here ####
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+

<TRUNCATED>

[10/23] hive git commit: HIVE-11913 : Verify existence of tests for new changes in HiveQA (Szehon, reviewed by Sergio Pena)

Posted by se...@apache.org.
HIVE-11913 : Verify existence of tests for new changes in HiveQA (Szehon, 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/bbb312f3
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bbb312f3
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bbb312f3

Branch: refs/heads/llap
Commit: bbb312f36b81b95ef6b0b003606799aaffe30142
Parents: bc1c434
Author: Szehon Ho <sz...@cloudera.com>
Authored: Fri Oct 2 14:20:31 2015 -0700
Committer: Szehon Ho <sz...@cloudera.com>
Committed: Fri Oct 2 14:21:14 2015 -0700

----------------------------------------------------------------------
 .../hive/ptest/execution/JIRAService.java       |  115 +-
 .../org/apache/hive/ptest/execution/PTest.java  |   11 +-
 .../hive/ptest/execution/TestCheckPhase.java    |   77 +
 .../ptest/execution/TestTestCheckPhase.java     |   91 +
 .../src/test/resources/HIVE-10761.6.patch       | 2539 ++++++++++++++++++
 .../src/test/resources/HIVE-11271.4.patch       |  606 +++++
 .../ptest2/src/test/resources/HIVE-9377.1.patch |   25 +
 .../ptest2/src/test/resources/remove-test.patch |   33 +
 8 files changed, 3447 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java
index c7be572..37127ea 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.net.URL;
 import java.util.*;
 
+import com.google.common.collect.Sets;
 import org.apache.commons.cli.*;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.hive.ptest.api.server.TestLogger;
@@ -94,7 +95,12 @@ class JIRAService {
   }
 
   void postComment(boolean error, int numTestsExecuted, SortedSet<String> failedTests,
-      List<String> messages) {
+    List<String> messages) {
+    postComment(error, numTestsExecuted, failedTests, messages, new HashSet<String>());
+  }
+
+  void postComment(boolean error, int numTestsExecuted, SortedSet<String> failedTests,
+    List<String> messages, Set<String> addedTests) {
     DefaultHttpClient httpClient = new DefaultHttpClient();
     try {
       BuildInfo buildInfo = formatBuildTag(mBuildTag);
@@ -102,9 +108,9 @@ class JIRAService {
       List<String> comments = Lists.newArrayList();
       comments.add("");
       comments.add("");
-      if(!failedTests.isEmpty()) {
+      if (!failedTests.isEmpty()) {
         comments.add("{color:red}Overall{color}: -1 at least one tests failed");
-      } else if(numTestsExecuted == 0) {
+      } else if (numTestsExecuted == 0) {
         comments.add("{color:red}Overall{color}: -1 no tests executed");
       } else if (error) {
         comments.add("{color:red}Overall{color}: -1 build exited with an error");
@@ -112,17 +118,23 @@ class JIRAService {
         comments.add("{color:green}Overall{color}: +1 all checks pass");
       }
       comments.add("");
-      if(!mPatch.isEmpty()) {
+      if (!mPatch.isEmpty()) {
         comments.add("Here are the results of testing the latest attachment:");
         comments.add(mPatch);
       }
       comments.add("");
-      if(numTestsExecuted > 0) {
+      if (addedTests.size() > 0) {
+        comments.add(formatSuccess("+1 due to " + addedTests.size() + " test(s) being added or modified."));
+      } else {
+        comments.add(formatError("-1 due to no test(s) being added or modified."));
+      }
+      comments.add("");
+      if (numTestsExecuted > 0) {
         if (failedTests.isEmpty()) {
-          comments.add(formatSuccess("+1 "+ numTestsExecuted + " tests passed"));
+          comments.add(formatSuccess("+1 " + numTestsExecuted + " tests passed"));
         } else {
           comments.add(formatError("-1 due to " + failedTests.size()
-              + " failed/errored test(s), " + numTestsExecuted + " tests executed"));
+            + " failed/errored test(s), " + numTestsExecuted + " tests executed"));
           comments.add("*Failed tests:*");
           comments.add("{noformat}");
           comments.addAll(failedTests);
@@ -131,12 +143,12 @@ class JIRAService {
         comments.add("");
       }
       comments.add("Test results: " + mJenkinsURL + "/" +
-          buildInfo.getFormattedBuildTag() + "/testReport");
+        buildInfo.getFormattedBuildTag() + "/testReport");
       comments.add("Console output: " + mJenkinsURL + "/" +
-          buildInfo.getFormattedBuildTag() + "/console");
+        buildInfo.getFormattedBuildTag() + "/console");
       comments.add("Test logs: " + mLogsURL + buildTagForLogs);
       comments.add("");
-      if(!messages.isEmpty()) {
+      if (!messages.isEmpty()) {
         comments.add("Messages:");
         comments.add("{noformat}");
         comments.addAll(trimMessages(messages));
@@ -147,16 +159,16 @@ class JIRAService {
       String attachmentId = parseAttachementId(mPatch);
       comments.add("");
       comments.add("ATTACHMENT ID: " + attachmentId +
-          " - " + buildInfo.getBuildName());
+        " - " + buildInfo.getBuildName());
       mLogger.info("Comment: " + Joiner.on("\n").join(comments));
       String body = Joiner.on("\n").join(comments);
       String url = String.format("%s/rest/api/2/issue/%s/comment", mUrl, mName);
       URL apiURL = new URL(mUrl);
       httpClient.getCredentialsProvider()
-      .setCredentials(
+        .setCredentials(
           new AuthScope(apiURL.getHost(), apiURL.getPort(),
-              AuthScope.ANY_REALM),
-              new UsernamePasswordCredentials(mUser, mPassword));
+            AuthScope.ANY_REALM),
+          new UsernamePasswordCredentials(mUser, mPassword));
       BasicHttpContext localcontext = new BasicHttpContext();
       localcontext.setAttribute("preemptive-auth", new BasicScheme());
       httpClient.addRequestInterceptor(new PreemptiveAuth(), 0);
@@ -169,36 +181,42 @@ class JIRAService {
       StatusLine statusLine = httpResponse.getStatusLine();
       if (statusLine.getStatusCode() != 201) {
         throw new RuntimeException(statusLine.getStatusCode() + " "
-            + statusLine.getReasonPhrase());
+          + statusLine.getReasonPhrase());
       }
       mLogger.info("JIRA Response Metadata: " + httpResponse);
     } catch (Exception e) {
       mLogger.error("Encountered error attempting to post comment to " + mName,
-          e);
+        e);
     } finally {
       httpClient.getConnectionManager().shutdown();
     }
   }
+
   static List<String> trimMessages(List<String> messages) {
     int size = messages.size();
-    if(size > MAX_MESSAGES) {
+    if (size > MAX_MESSAGES) {
       messages = messages.subList(size - MAX_MESSAGES, size);
       messages.add(0, TRIMMED_MESSAGE);
     }
     return messages;
   }
+
   @SuppressWarnings("unused")
   private static class Body {
     private String body;
+
     public Body() {
 
     }
+
     public Body(String body) {
       this.body = body;
     }
+
     public String getBody() {
       return body;
     }
+
     public void setBody(String body) {
       this.body = body;
     }
@@ -209,7 +227,7 @@ class JIRAService {
     private String buildName;
     private String formattedBuildTag;
 
-    public BuildInfo (String buildName, String formattedBuildTag) {
+    public BuildInfo(String buildName, String formattedBuildTag) {
       this.buildName = buildName;
       this.formattedBuildTag = formattedBuildTag;
     }
@@ -228,7 +246,7 @@ class JIRAService {
    */
   @VisibleForTesting
   static BuildInfo formatBuildTag(String buildTag) {
-    if(buildTag.contains("-")) {
+    if (buildTag.contains("-")) {
       int lastDashIndex = buildTag.lastIndexOf("-");
       String buildName = buildTag.substring(0, lastDashIndex);
       String buildId = buildTag.substring(lastDashIndex + 1);
@@ -237,6 +255,7 @@ class JIRAService {
     }
     throw new IllegalArgumentException("Build tag '" + buildTag + "' must contain a -");
   }
+
   static String formatBuildTagForLogs(String buildTag) {
     if (buildTag.endsWith("/")) {
       return buildTag;
@@ -244,6 +263,7 @@ class JIRAService {
       return buildTag + "/";
     }
   }
+
   private static String formatError(String msg) {
     return String.format("{color:red}ERROR:{color} %s", msg);
   }
@@ -255,7 +275,7 @@ class JIRAService {
   static class PreemptiveAuth implements HttpRequestInterceptor {
 
     public void process(final HttpRequest request, final HttpContext context)
-        throws HttpException, IOException {
+      throws HttpException, IOException {
       AuthState authState = (AuthState) context.getAttribute(ClientContext.TARGET_AUTH_STATE);
       if (authState.getAuthScheme() == null) {
         AuthScheme authScheme = (AuthScheme) context.getAttribute("preemptive-auth");
@@ -263,34 +283,35 @@ class JIRAService {
         HttpHost targetHost = (HttpHost) context.getAttribute(ExecutionContext.HTTP_TARGET_HOST);
         if (authScheme != null) {
           Credentials creds = credsProvider.getCredentials(new AuthScope(
-              targetHost.getHostName(), targetHost.getPort()));
+            targetHost.getHostName(), targetHost.getPort()));
           if (creds == null) {
             throw new HttpException(
-                "No credentials for preemptive authentication");
+              "No credentials for preemptive authentication");
           }
           authState.update(authScheme, creds);
         }
       }
     }
   }
+
   private static String parseAttachementId(String patch) {
-    if(patch == null) {
+    if (patch == null) {
       return "";
     }
     String result = FilenameUtils.getPathNoEndSeparator(patch.trim());
-    if(result == null) {
+    if (result == null) {
       return "";
     }
     result = FilenameUtils.getName(result.trim());
-    if(result == null) {
+    if (result == null) {
       return "";
     }
     return result.trim();
   }
 
   private static void assertRequired(CommandLine commandLine, String[] requiredOptions) throws IllegalArgumentException {
-    for(String requiredOption : requiredOptions) {
-      if(!commandLine.hasOption(requiredOption)) {
+    for (String requiredOption : requiredOptions) {
+      if (!commandLine.hasOption(requiredOption)) {
         throw new IllegalArgumentException("--" + requiredOption + " is required");
       }
     }
@@ -311,7 +332,7 @@ class JIRAService {
   private static final String FIELD_FAILED_TESTS = "failedTests";
   private static final String FIELD_MESSAGES = "messages";
   private static final String FIELD_JIRA_USER = "jiraUser";
-  private static final String FIELD_JIRA_PASS= "jiraPassword";
+  private static final String FIELD_JIRA_PASS = "jiraPassword";
 
   private static Map<String, Class> supportedJsonFields = new HashMap<String, Class>() {
     {
@@ -387,9 +408,9 @@ class JIRAService {
     }
 
     assertRequired(cmd, new String[]{
-        OPT_USER_LONG,
-        OPT_PASS_LONG,
-        OPT_FILE_LONG
+      OPT_USER_LONG,
+      OPT_PASS_LONG,
+      OPT_FILE_LONG
     });
 
     return cmd;
@@ -400,7 +421,7 @@ class JIRAService {
 
     try {
       cmd = parseCommandLine(args);
-    } catch(ParseException e) {
+    } catch (ParseException e) {
       System.out.println("Error parsing command arguments: " + e.getMessage());
       System.exit(1);
     }
@@ -413,25 +434,25 @@ class JIRAService {
     Map<String, Object> jsonValues = parseJsonFile(cmd.getOptionValue(OPT_FILE_LONG));
 
     Map<String, String> context = Maps.newHashMap();
-    context.put(FIELD_JIRA_URL, (String)jsonValues.get(FIELD_JIRA_URL));
+    context.put(FIELD_JIRA_URL, (String) jsonValues.get(FIELD_JIRA_URL));
     context.put(FIELD_JIRA_USER, cmd.getOptionValue(OPT_USER_LONG));
     context.put(FIELD_JIRA_PASS, cmd.getOptionValue(OPT_PASS_LONG));
-    context.put(FIELD_LOGS_URL, (String)jsonValues.get(FIELD_LOGS_URL));
-    context.put(FIELD_REPO, (String)jsonValues.get(FIELD_REPO));
-    context.put(FIELD_REPO_NAME, (String)jsonValues.get(FIELD_REPO_NAME));
-    context.put(FIELD_REPO_TYPE, (String)jsonValues.get(FIELD_REPO_TYPE));
-    context.put(FIELD_REPO_BRANCH, (String)jsonValues.get(FIELD_REPO_BRANCH));
-    context.put(FIELD_JENKINS_URL, (String)jsonValues.get(FIELD_JENKINS_URL));
+    context.put(FIELD_LOGS_URL, (String) jsonValues.get(FIELD_LOGS_URL));
+    context.put(FIELD_REPO, (String) jsonValues.get(FIELD_REPO));
+    context.put(FIELD_REPO_NAME, (String) jsonValues.get(FIELD_REPO_NAME));
+    context.put(FIELD_REPO_TYPE, (String) jsonValues.get(FIELD_REPO_TYPE));
+    context.put(FIELD_REPO_BRANCH, (String) jsonValues.get(FIELD_REPO_BRANCH));
+    context.put(FIELD_JENKINS_URL, (String) jsonValues.get(FIELD_JENKINS_URL));
 
     TestLogger logger = new TestLogger(System.err, TestLogger.LEVEL.TRACE);
     TestConfiguration configuration = new TestConfiguration(new Context(context), logger);
-    configuration.setJiraName((String)jsonValues.get(FIELD_JIRA_NAME));
-    configuration.setPatch((String)jsonValues.get(FIELD_PATCH_URL));
-
-    JIRAService service = new JIRAService(logger, configuration, (String)jsonValues.get(FIELD_BUILD_TAG));
-    List<String> messages = (List)jsonValues.get(FIELD_MESSAGES);
-    SortedSet<String> failedTests = (SortedSet)jsonValues.get(FIELD_FAILED_TESTS);
-    boolean error = (Integer)jsonValues.get(FIELD_BUILD_STATUS) == 0 ? false : true;
-    service.postComment(error, (Integer)jsonValues.get(FIELD_NUM_TESTS_EXECUTED), failedTests, messages);
+    configuration.setJiraName((String) jsonValues.get(FIELD_JIRA_NAME));
+    configuration.setPatch((String) jsonValues.get(FIELD_PATCH_URL));
+
+    JIRAService service = new JIRAService(logger, configuration, (String) jsonValues.get(FIELD_BUILD_TAG));
+    List<String> messages = (List) jsonValues.get(FIELD_MESSAGES);
+    SortedSet<String> failedTests = (SortedSet) jsonValues.get(FIELD_FAILED_TESTS);
+    boolean error = (Integer) jsonValues.get(FIELD_BUILD_STATUS) == 0 ? false : true;
+    service.postComment(error, (Integer) jsonValues.get(FIELD_NUM_TESTS_EXECUTED), failedTests, messages);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
index 7217ef9..35cc752 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
@@ -23,6 +23,7 @@ import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -72,6 +73,7 @@ public class PTest {
 
   private final TestConfiguration mConfiguration;
   private final ListeningExecutorService mExecutor;
+  private final Set<String> mAddedTests;
   private final Set<String> mExecutedTests;
   private final Set<String> mFailedTests;
   private final List<Phase> mPhases;
@@ -92,6 +94,7 @@ public class PTest {
     mBuildTag = buildTag;
     mExecutedTests = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
     mFailedTests = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
+    mAddedTests = new HashSet<String>();
     mExecutionContext = executionContext;
     mSshCommandExecutor = sshCommandExecutor;
     mRsyncCommandExecutor = rsyncCommandExecutor;
@@ -148,6 +151,7 @@ public class PTest {
     }
     mHostExecutors = new CopyOnWriteArrayList<HostExecutor>(hostExecutors);
     mPhases = Lists.newArrayList();
+    mPhases.add(new TestCheckPhase(mHostExecutors, localCommandFactory, templateDefaults, patchFile, logger, mAddedTests));
     mPhases.add(new PrepPhase(mHostExecutors, localCommandFactory, templateDefaults, scratchDir, patchFile, logger));
     mPhases.add(new ExecutionPhase(mHostExecutors, mExecutionContext, hostExecutorBuilder, localCommandFactory, templateDefaults,
         succeededLogDir, failedLogDir, testParser.parse(), mExecutedTests, mFailedTests, logger));
@@ -213,7 +217,7 @@ public class PTest {
       for(Map.Entry<String, Long> entry : elapsedTimes.entrySet()) {
         mLogger.info(String.format("PERF: Phase %s took %d minutes", entry.getKey(), entry.getValue()));
       }
-      publishJiraComment(error, messages, failedTests);
+      publishJiraComment(error, messages, failedTests, mAddedTests);
       if(error || !mFailedTests.isEmpty()) {
         result = 1;
       }
@@ -221,7 +225,7 @@ public class PTest {
     return result;
   }
 
-  private void publishJiraComment(boolean error, List<String> messages, SortedSet<String> failedTests) {
+  private void publishJiraComment(boolean error, List<String> messages, SortedSet<String> failedTests, Set<String> addedTests) {
     if(mConfiguration.getJiraName().isEmpty()) {
       mLogger.info("Skipping JIRA comment as name is empty.");
       return;
@@ -238,8 +242,9 @@ public class PTest {
       mLogger.info("Skipping JIRA comment as password is empty.");
       return;
     }
+    mLogger.info("Added tests: " + addedTests);
     JIRAService jira = new JIRAService(mLogger, mConfiguration, mBuildTag);
-    jira.postComment(error, mExecutedTests.size(), failedTests, messages);
+    jira.postComment(error, mExecutedTests.size(), failedTests, messages, addedTests);
   }
 
   public static class Builder {

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/TestCheckPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/TestCheckPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/TestCheckPhase.java
new file mode 100644
index 0000000..1107dcd
--- /dev/null
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/TestCheckPhase.java
@@ -0,0 +1,77 @@
+/*
+ * 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.ptest.execution;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class TestCheckPhase extends Phase {
+  private final File mPatchFile;
+  private Set<String> modifiedTestFiles;
+
+  private static final Pattern fileNameFromDiff = Pattern.compile("[/][^\\s]*");
+  private static final Pattern javaTest = Pattern.compile("Test.*java");
+
+  public TestCheckPhase(List<HostExecutor> hostExecutors,
+    LocalCommandFactory localCommandFactory,
+    ImmutableMap<String, String> templateDefaults,
+    File patchFile, Logger logger, Set<String> modifiedTestFiles) {
+    super(hostExecutors, localCommandFactory, templateDefaults, logger);
+    this.mPatchFile = patchFile;
+    this.modifiedTestFiles = modifiedTestFiles;
+  }
+  @Override
+  public void execute() throws Exception {
+    if(mPatchFile != null) {
+      logger.info("Reading patchfile " + mPatchFile.getAbsolutePath());
+      FileReader fr = null;
+      try {
+        fr = new FileReader(mPatchFile);
+        BufferedReader br = new BufferedReader(fr);
+        String line;
+        while ((line = br.readLine()) != null) {
+          if(line.startsWith("+++")) {
+            logger.info("Searching line : " + line);
+            Matcher fileNameMatcher = fileNameFromDiff.matcher(line);
+            if (fileNameMatcher.find()) {
+              String filePath = fileNameMatcher.group(0);
+              String fileName = filePath.substring(filePath.lastIndexOf("/")+1);
+              Matcher javaTestMatcher = javaTest.matcher(fileName);
+              if (javaTestMatcher.find() || fileName.endsWith(".q")) {
+                modifiedTestFiles.add(fileName);
+              }
+            }
+          }
+        }
+      } finally {
+        fr.close();
+      }
+    } else {
+      logger.error("Patch file is null");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestTestCheckPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestTestCheckPhase.java b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestTestCheckPhase.java
new file mode 100644
index 0000000..7183ee3
--- /dev/null
+++ b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestTestCheckPhase.java
@@ -0,0 +1,91 @@
+/*
+ * 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.ptest.execution;
+
+import org.approvaltests.Approvals;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.net.URL;
+import java.util.HashSet;
+import java.util.Set;
+
+public class TestTestCheckPhase extends AbstractTestPhase {
+  private TestCheckPhase phase;
+
+  @Before
+  public void setup() throws Exception {
+    initialize(getClass().getSimpleName());
+    createHostExecutor();
+  }
+  @Test
+  public void testNoTests() throws Exception {
+    URL url = this.getClass().getResource("/HIVE-9377.1.patch");
+    File patchFile = new File(url.getFile());
+    Set<String> addedTests = new HashSet<String>();
+    phase = new TestCheckPhase(hostExecutors, localCommandFactory,
+      templateDefaults, patchFile, logger, addedTests);
+    phase.execute();
+
+    Assert.assertEquals(addedTests.size(), 0);
+  }
+
+
+  @Test
+  public void testJavaTests() throws Exception {
+    URL url = this.getClass().getResource("/HIVE-10761.6.patch");
+    File patchFile = new File(url.getFile());
+    Set<String> addedTests = new HashSet<String>();
+    phase = new TestCheckPhase(hostExecutors, localCommandFactory,
+      templateDefaults, patchFile, logger, addedTests);
+    phase.execute();
+
+    Assert.assertEquals(addedTests.size(), 3);
+    Assert.assertTrue(addedTests.contains("TestCodahaleMetrics.java"));
+    Assert.assertTrue(addedTests.contains("TestMetaStoreMetrics.java"));
+    Assert.assertTrue(addedTests.contains("TestLegacyMetrics.java"));
+  }
+
+  @Test
+  public void testQTests() throws Exception {
+    URL url = this.getClass().getResource("/HIVE-11271.4.patch");
+    File patchFile = new File(url.getFile());
+    Set<String> addedTests = new HashSet<String>();
+    phase = new TestCheckPhase(hostExecutors, localCommandFactory,
+      templateDefaults, patchFile, logger, addedTests);
+    phase.execute();
+
+    Assert.assertEquals(addedTests.size(), 1);
+    Assert.assertTrue(addedTests.contains("unionall_unbalancedppd.q"));
+  }
+
+  @Test
+  public void testRemoveTest() throws Exception {
+    URL url = this.getClass().getResource("/remove-test.patch");
+    File patchFile = new File(url.getFile());
+    Set<String> addedTests = new HashSet<String>();
+    phase = new TestCheckPhase(hostExecutors, localCommandFactory,
+      templateDefaults, patchFile, logger, addedTests);
+    phase.execute();
+
+    Assert.assertEquals(addedTests.size(), 0);
+  }
+}


[18/23] hive git commit: HIVE-12002: correct implementation typo (Alex Moundalexis, reviewed by Sergio Pena)

Posted by se...@apache.org.
HIVE-12002: correct implementation typo (Alex Moundalexis, 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/7c164acc
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7c164acc
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7c164acc

Branch: refs/heads/llap
Commit: 7c164acc39f05c807a8c3753d28ab2fc3f628701
Parents: 54ca6fc
Author: Sergio Pena <se...@cloudera.com>
Authored: Mon Oct 5 09:52:07 2015 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Mon Oct 5 09:52:07 2015 -0500

----------------------------------------------------------------------
 beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java   | 4 ++--
 hcatalog/conf/proto-hive-site.xml                                | 2 +-
 hcatalog/src/packages/templates/conf/hive-site.xml.template      | 2 +-
 .../src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java | 2 +-
 .../hadoop/hive/metastore/AlternateFailurePreListener.java       | 2 +-
 5 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7c164acc/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java b/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
index a6b11ba..7e8cc67 100644
--- a/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
+++ b/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
@@ -165,7 +165,7 @@ public class HiveSchemaHelper {
   }
 
   /***
-   * Base implemenation of NestedScriptParser
+   * Base implementation of NestedScriptParser
    * abstractCommandParser.
    *
    */
@@ -494,4 +494,4 @@ public class HiveSchemaHelper {
       throw new IllegalArgumentException("Unknown dbType " + dbName);
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7c164acc/hcatalog/conf/proto-hive-site.xml
----------------------------------------------------------------------
diff --git a/hcatalog/conf/proto-hive-site.xml b/hcatalog/conf/proto-hive-site.xml
index 7df7f46..5530511 100644
--- a/hcatalog/conf/proto-hive-site.xml
+++ b/hcatalog/conf/proto-hive-site.xml
@@ -91,7 +91,7 @@
 <property>
   <name>hive.semantic.analyzer.factory.impl</name>
   <value>org.apache.hive.hcatalog.cli.HCatSemanticAnalyzerFactory</value>
-  <description>controls which SemanticAnalyzerFactory implemenation class is used by CLI</description>
+  <description>controls which SemanticAnalyzerFactory implementation class is used by CLI</description>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/hive/blob/7c164acc/hcatalog/src/packages/templates/conf/hive-site.xml.template
----------------------------------------------------------------------
diff --git a/hcatalog/src/packages/templates/conf/hive-site.xml.template b/hcatalog/src/packages/templates/conf/hive-site.xml.template
index 5f89f4a..638aa35 100644
--- a/hcatalog/src/packages/templates/conf/hive-site.xml.template
+++ b/hcatalog/src/packages/templates/conf/hive-site.xml.template
@@ -88,7 +88,7 @@
 <property>
   <name>hive.semantic.analyzer.factory.impl</name>
   <value>org.apache.hive.hcatalog.cli.HCatSemanticAnalyzerFactory</value>
-  <description>controls which SemanticAnalyzerFactory implemenation class is used by CLI</description>
+  <description>controls which SemanticAnalyzerFactory implementation class is used by CLI</description>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/hive/blob/7c164acc/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 9d10e21..ccb4c98 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -595,7 +595,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     private RawStore newRawStore() throws MetaException {
-      LOG.info(addPrefix("Opening raw store with implemenation class:"
+      LOG.info(addPrefix("Opening raw store with implementation class:"
           + rawStoreClassName));
       Configuration conf = getConf();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7c164acc/metastore/src/test/org/apache/hadoop/hive/metastore/AlternateFailurePreListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/AlternateFailurePreListener.java b/metastore/src/test/org/apache/hadoop/hive/metastore/AlternateFailurePreListener.java
index e5c983b..22146ba 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/AlternateFailurePreListener.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/AlternateFailurePreListener.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hive.metastore.events.PreEventContext;
  *
  * AlternateFailurePreListener.
  *
- * An implemenation of MetaStorePreEventListener which fails every other time it's invoked,
+ * An implementation of MetaStorePreEventListener which fails every other time it's invoked,
  * starting with the first time.
  *
  * It also records and makes available the number of times it's been invoked.


[16/23] hive git commit: HIVE-11835: Type decimal(1, 1) reads 0.0, 0.00, etc from text file as NULL (Reviewed by Szehon)

Posted by se...@apache.org.
HIVE-11835: Type decimal(1,1) reads 0.0, 0.00, etc from text file as NULL (Reviewed by Szehon)


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

Branch: refs/heads/llap
Commit: c0c19d0733a3a9fe785e120f618414212e4976be
Parents: c23841e
Author: Xuefu Zhang <xz...@Cloudera.com>
Authored: Mon Oct 5 05:42:20 2015 -0700
Committer: Xuefu Zhang <xz...@Cloudera.com>
Committed: Mon Oct 5 05:42:20 2015 -0700

----------------------------------------------------------------------
 .../hive/common/type/TestHiveDecimal.java       |  12 +--
 .../test/resources/testconfiguration.properties |   1 +
 .../test/queries/clientpositive/decimal_1_1.q   |   9 ++
 .../results/clientpositive/decimal_1_1.q.out    | 104 +++++++++++++++++++
 .../clientpositive/spark/decimal_1_1.q.out      | 104 +++++++++++++++++++
 .../hadoop/hive/common/type/HiveDecimal.java    |  10 +-
 6 files changed, 233 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c0c19d07/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java b/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
index ba5ef71..f68842c 100644
--- a/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
+++ b/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
@@ -42,15 +42,15 @@ public class TestHiveDecimal {
     Assert.assertTrue("Decimal scale should not go above maximum", dec.scale() <= HiveDecimal.MAX_SCALE);
 
     decStr = "57847525803324040144343378.09799306448796128931113691624";
-    BigDecimal bd = new BigDecimal(decStr);
-    BigDecimal bd1 = HiveDecimal.enforcePrecisionScale(bd, 20, 5);
+    HiveDecimal bd = HiveDecimal.create(decStr);
+    HiveDecimal bd1 = HiveDecimal.enforcePrecisionScale(bd, 20, 5);
     Assert.assertNull(bd1);
     bd1 = HiveDecimal.enforcePrecisionScale(bd, 35, 5);
     Assert.assertEquals("57847525803324040144343378.09799", bd1.toString());
     bd1 = HiveDecimal.enforcePrecisionScale(bd, 45, 20);
     Assert.assertNull(bd1);
 
-    dec = HiveDecimal.create(bd, false);
+    dec = HiveDecimal.create(new BigDecimal(decStr), false);
     Assert.assertNull(dec);
 
     dec = HiveDecimal.create("-1786135888657847525803324040144343378.09799306448796128931113691624");
@@ -65,10 +65,10 @@ public class TestHiveDecimal {
 
     // Rounding numbers that increase int digits
     Assert.assertEquals("10",
-        HiveDecimal.enforcePrecisionScale(new BigDecimal("9.5"), 2, 0).toString());
-    Assert.assertNull(HiveDecimal.enforcePrecisionScale(new BigDecimal("9.5"), 1, 0));
+        HiveDecimal.enforcePrecisionScale(HiveDecimal.create("9.5"), 2, 0).toString());
+    Assert.assertNull(HiveDecimal.enforcePrecisionScale(HiveDecimal.create("9.5"), 1, 0));
     Assert.assertEquals("9",
-        HiveDecimal.enforcePrecisionScale(new BigDecimal("9.4"), 1, 0).toString());
+        HiveDecimal.enforcePrecisionScale(HiveDecimal.create("9.4"), 1, 0).toString());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/c0c19d07/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 700ea63..9c9f4cc 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -667,6 +667,7 @@ spark.query.files=add_part_multiple.q, \
   custom_input_output_format.q, \
   date_join1.q, \
   date_udf.q, \
+  decimal_1_1.q, \
   decimal_join.q, \
   disable_merge_for_bucketing.q, \
   dynamic_rdd_cache.q, \

http://git-wip-us.apache.org/repos/asf/hive/blob/c0c19d07/ql/src/test/queries/clientpositive/decimal_1_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/decimal_1_1.q b/ql/src/test/queries/clientpositive/decimal_1_1.q
new file mode 100644
index 0000000..83ce521
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/decimal_1_1.q
@@ -0,0 +1,9 @@
+drop table if exists decimal_1_1;
+
+create table decimal_1_1 (d decimal(1,1));
+load data local inpath '../../data/files/decimal_1_1.txt' into table decimal_1_1;
+select * from decimal_1_1;
+
+select d from decimal_1_1 order by d desc;
+
+drop table decimal_1_1;

http://git-wip-us.apache.org/repos/asf/hive/blob/c0c19d07/ql/src/test/results/clientpositive/decimal_1_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/decimal_1_1.q.out b/ql/src/test/results/clientpositive/decimal_1_1.q.out
new file mode 100644
index 0000000..b2704c6
--- /dev/null
+++ b/ql/src/test/results/clientpositive/decimal_1_1.q.out
@@ -0,0 +1,104 @@
+PREHOOK: query: drop table if exists decimal_1_1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists decimal_1_1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table decimal_1_1 (d decimal(1,1))
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@decimal_1_1
+POSTHOOK: query: create table decimal_1_1 (d decimal(1,1))
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@decimal_1_1
+PREHOOK: query: load data local inpath '../../data/files/decimal_1_1.txt' into table decimal_1_1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@decimal_1_1
+POSTHOOK: query: load data local inpath '../../data/files/decimal_1_1.txt' into table decimal_1_1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@decimal_1_1
+PREHOOK: query: select * from decimal_1_1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@decimal_1_1
+#### A masked pattern was here ####
+POSTHOOK: query: select * from decimal_1_1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@decimal_1_1
+#### A masked pattern was here ####
+0
+0
+0
+0.1
+0.2
+0.9
+0.9
+NULL
+0.3
+NULL
+NULL
+0
+0
+NULL
+0
+0
+0
+0
+-0.1
+-0.2
+-0.9
+-0.9
+NULL
+-0.3
+NULL
+NULL
+0
+0
+NULL
+0
+PREHOOK: query: select d from decimal_1_1 order by d desc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@decimal_1_1
+#### A masked pattern was here ####
+POSTHOOK: query: select d from decimal_1_1 order by d desc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@decimal_1_1
+#### A masked pattern was here ####
+0.9
+0.9
+0.3
+0.2
+0.1
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+-0.1
+-0.2
+-0.3
+-0.9
+-0.9
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+PREHOOK: query: drop table decimal_1_1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@decimal_1_1
+PREHOOK: Output: default@decimal_1_1
+POSTHOOK: query: drop table decimal_1_1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@decimal_1_1
+POSTHOOK: Output: default@decimal_1_1

http://git-wip-us.apache.org/repos/asf/hive/blob/c0c19d07/ql/src/test/results/clientpositive/spark/decimal_1_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/decimal_1_1.q.out b/ql/src/test/results/clientpositive/spark/decimal_1_1.q.out
new file mode 100644
index 0000000..b2704c6
--- /dev/null
+++ b/ql/src/test/results/clientpositive/spark/decimal_1_1.q.out
@@ -0,0 +1,104 @@
+PREHOOK: query: drop table if exists decimal_1_1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists decimal_1_1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table decimal_1_1 (d decimal(1,1))
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@decimal_1_1
+POSTHOOK: query: create table decimal_1_1 (d decimal(1,1))
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@decimal_1_1
+PREHOOK: query: load data local inpath '../../data/files/decimal_1_1.txt' into table decimal_1_1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@decimal_1_1
+POSTHOOK: query: load data local inpath '../../data/files/decimal_1_1.txt' into table decimal_1_1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@decimal_1_1
+PREHOOK: query: select * from decimal_1_1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@decimal_1_1
+#### A masked pattern was here ####
+POSTHOOK: query: select * from decimal_1_1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@decimal_1_1
+#### A masked pattern was here ####
+0
+0
+0
+0.1
+0.2
+0.9
+0.9
+NULL
+0.3
+NULL
+NULL
+0
+0
+NULL
+0
+0
+0
+0
+-0.1
+-0.2
+-0.9
+-0.9
+NULL
+-0.3
+NULL
+NULL
+0
+0
+NULL
+0
+PREHOOK: query: select d from decimal_1_1 order by d desc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@decimal_1_1
+#### A masked pattern was here ####
+POSTHOOK: query: select d from decimal_1_1 order by d desc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@decimal_1_1
+#### A masked pattern was here ####
+0.9
+0.9
+0.3
+0.2
+0.1
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+-0.1
+-0.2
+-0.3
+-0.9
+-0.9
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+PREHOOK: query: drop table decimal_1_1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@decimal_1_1
+PREHOOK: Output: default@decimal_1_1
+POSTHOOK: query: drop table decimal_1_1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@decimal_1_1
+POSTHOOK: Output: default@decimal_1_1

http://git-wip-us.apache.org/repos/asf/hive/blob/c0c19d07/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
index 12a3936..4ed17a2 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
@@ -272,11 +272,19 @@ public class HiveDecimal implements Comparable<HiveDecimal> {
     return bd;
   }
 
-  public static BigDecimal enforcePrecisionScale(BigDecimal bd, int maxPrecision, int maxScale) {
+  private static BigDecimal enforcePrecisionScale(BigDecimal bd, int maxPrecision, int maxScale) {
     if (bd == null) {
       return null;
     }
 
+    /**
+     * Specially handling the case that bd=0, and we are converting it to a type where precision=scale,
+     * such as decimal(1, 1).
+     */
+    if (bd.compareTo(BigDecimal.ZERO) == 0 && bd.scale() == 0 && maxPrecision == maxScale) {
+      return bd.setScale(maxScale);
+    }
+
     bd = trim(bd);
 
     if (bd.scale() > maxScale) {


[17/23] hive git commit: HIVE-11835: Type decimal(1, 1) reads 0.0, 0.00, etc from text file as NULL (Reviewed by Szehon) -- Add the forgotten new data file

Posted by se...@apache.org.
HIVE-11835: Type decimal(1,1) reads 0.0, 0.00, etc from text file as NULL (Reviewed by Szehon)
-- Add the forgotten new data file


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

Branch: refs/heads/llap
Commit: 54ca6fc51239e28309dcc7d68647ee2a25f1ca98
Parents: c0c19d0
Author: Xuefu Zhang <xz...@Cloudera.com>
Authored: Mon Oct 5 05:46:20 2015 -0700
Committer: Xuefu Zhang <xz...@Cloudera.com>
Committed: Mon Oct 5 05:46:20 2015 -0700

----------------------------------------------------------------------
 data/files/decimal_1_1.txt | 30 ++++++++++++++++++++++++++++++
 1 file changed, 30 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/54ca6fc5/data/files/decimal_1_1.txt
----------------------------------------------------------------------
diff --git a/data/files/decimal_1_1.txt b/data/files/decimal_1_1.txt
new file mode 100644
index 0000000..ec16804
--- /dev/null
+++ b/data/files/decimal_1_1.txt
@@ -0,0 +1,30 @@
+0.0
+0.0000
+.0
+0.1
+0.15
+0.9
+0.94
+0.99
+0.345
+1.0
+1
+0
+00
+22
+1E-9
+-0.0
+-0.0000
+-.0
+-0.1
+-0.15
+-0.9
+-0.94
+-0.99
+-0.345
+-1.0
+-1
+-0
+-00
+-22
+-1E-9


[15/23] hive git commit: HIVE-11699: Support special characters in quoted table names (Pengcheng Xiong, reviewed by John Pullokkaran)

Posted by se...@apache.org.
HIVE-11699: Support special characters in quoted table names (Pengcheng Xiong, reviewed by John Pullokkaran)


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

Branch: refs/heads/llap
Commit: c23841e553cbd4f32d33842d49f9b9e52803d143
Parents: d545935
Author: Pengcheng Xiong <px...@apache.org>
Authored: Sun Oct 4 12:45:21 2015 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Sun Oct 4 12:45:21 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |     5 +
 .../hadoop/hive/metastore/HiveAlterHandler.java |     2 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |     8 +-
 .../hadoop/hive/metastore/MetaStoreUtils.java   |    40 +-
 .../apache/hadoop/hive/metastore/Warehouse.java |     4 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |     2 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |     6 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |     3 +-
 .../hadoop/hive/ql/lockmgr/HiveLockObject.java  |     6 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |     4 +-
 .../apache/hadoop/hive/ql/metadata/Table.java   |     5 +-
 .../RewriteQueryUsingAggregateIndexCtx.java     |     2 +-
 .../ql/parse/ColumnStatsSemanticAnalyzer.java   |     2 +-
 .../hadoop/hive/ql/metadata/TestHive.java       |     2 +-
 .../special_character_in_tabnames_1.q           |    13 +
 .../special_character_in_tabnames_1.q           |  1075 +
 .../special_character_in_tabnames_2.q           |    40 +
 .../special_character_in_tabnames_1.q.out       |    10 +
 .../special_character_in_tabnames_1.q.out       | 19550 +++++++++++++++++
 .../special_character_in_tabnames_2.q.out       |   304 +
 20 files changed, 21060 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/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 33ef654..7f632bc 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2233,6 +2233,11 @@ public class HiveConf extends Configuration {
     HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS("hive.support.sql11.reserved.keywords", true,
         "This flag should be set to true to enable support for SQL2011 reserved keywords.\n" +
         "The default value is true."),
+    HIVE_SUPPORT_SPECICAL_CHARACTERS_IN_TABLE_NAMES("hive.support.special.characters.tablename", true,
+        "This flag should be set to true to enable support for special characters in table names.\n"
+        + "When it is set to false, only [a-zA-Z_0-9]+ are supported.\n"
+        + "The only supported special character right now is '/'. This flag applies only to quoted table names.\n"
+        + "The default value is true."),
     // role names are case-insensitive
     USERS_IN_ADMIN_ROLE("hive.users.in.admin.role", "", false,
         "Comma separated list of users who are in admin role for bootstrapping.\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
index 0082773..45f3515 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
@@ -82,7 +82,7 @@ public class HiveAlterHandler implements AlterHandler {
       throw new InvalidOperationException("New table is invalid: " + newt);
     }
 
-    if (!MetaStoreUtils.validateName(newt.getTableName())) {
+    if (!MetaStoreUtils.validateName(newt.getTableName(), hiveConf)) {
       throw new InvalidOperationException(newt.getTableName()
           + " is not a valid object name");
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/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 8cd1f52..9d10e21 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -865,7 +865,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     private void create_database_core(RawStore ms, final Database db)
         throws AlreadyExistsException, InvalidObjectException, MetaException {
-      if (!validateName(db.getName())) {
+      if (!validateName(db.getName(), null)) {
         throw new InvalidObjectException(db.getName() + " is not a valid database name");
       }
       if (null == db.getLocationUri()) {
@@ -1217,7 +1217,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     private void create_type_core(final RawStore ms, final Type type)
         throws AlreadyExistsException, MetaException, InvalidObjectException {
-      if (!MetaStoreUtils.validateName(type.getName())) {
+      if (!MetaStoreUtils.validateName(type.getName(), null)) {
         throw new InvalidObjectException("Invalid type name");
       }
 
@@ -1351,7 +1351,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         throws AlreadyExistsException, MetaException,
         InvalidObjectException, NoSuchObjectException {
 
-      if (!MetaStoreUtils.validateName(tbl.getTableName())) {
+      if (!MetaStoreUtils.validateName(tbl.getTableName(), hiveConf)) {
         throw new InvalidObjectException(tbl.getTableName()
             + " is not a valid object name");
       }
@@ -5342,7 +5342,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     private void validateFunctionInfo(Function func) throws InvalidObjectException, MetaException {
-      if (!MetaStoreUtils.validateName(func.getFunctionName())) {
+      if (!MetaStoreUtils.validateName(func.getFunctionName(), null)) {
         throw new InvalidObjectException(func.getFunctionName() + " is not a valid object name");
       }
       String className = func.getClassName();

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
index f88f4dd..12f3f16 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hive.common.HiveStatsUtils;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -93,6 +94,14 @@ public class MetaStoreUtils {
 
   public static final String DATABASE_WAREHOUSE_SUFFIX = ".db";
 
+  // Right now we only support one special character '/'.
+  // More special characters can be added accordingly in the future.
+  // NOTE:
+  // If the following array is updated, please also be sure to update the
+  // configuration parameter documentation
+  // HIVE_SUPPORT_SPECICAL_CHARACTERS_IN_TABLE_NAMES in HiveConf as well.
+  public static final char[] specialCharactersInTableNames = new char[] { '/' };
+
   public static Table createColumnsetSchema(String name, List<String> columns,
       List<String> partCols, Configuration conf) throws MetaException {
 
@@ -523,12 +532,23 @@ public class MetaStoreUtils {
    *
    * @param name
    *          the name to validate
+   * @param conf
+   *          hive configuration
    * @return true or false depending on conformance
    * @exception MetaException
    *              if it doesn't match the pattern.
    */
-  static public boolean validateName(String name) {
-    Pattern tpat = Pattern.compile("[\\w_]+");
+  static public boolean validateName(String name, Configuration conf) {
+    Pattern tpat = null;
+    String allowedCharacters = "\\w_";
+    if (conf != null
+        && HiveConf.getBoolVar(conf,
+            HiveConf.ConfVars.HIVE_SUPPORT_SPECICAL_CHARACTERS_IN_TABLE_NAMES)) {
+      for (Character c : specialCharactersInTableNames) {
+        allowedCharacters += c;
+      }
+    }
+    tpat = Pattern.compile("[" + allowedCharacters + "]+");
     Matcher m = tpat.matcher(name);
     if (m.matches()) {
       return true;
@@ -1716,4 +1736,20 @@ public class MetaStoreUtils {
     return new URLClassLoader(curPath.toArray(new URL[0]), loader);
   }
 
+  public static String encodeTableName(String name) {
+    // The encoding method is simple, e.g., replace
+    // all the special characters with the corresponding number in ASCII.
+    // Note that unicode is not supported in table names. And we have explicit
+    // checks for it.
+    String ret = "";
+    for (char ch : name.toCharArray()) {
+      if (Character.isLetterOrDigit(ch) || ch == '_') {
+        ret += ch;
+      } else {
+        ret += "-" + (int) ch + "-";
+      }
+    }
+    return ret;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java b/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
index bc0f6e3..7aab2c7 100755
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
@@ -162,7 +162,7 @@ public class Warehouse {
 
   public Path getTablePath(String whRootString, String tableName) throws MetaException {
     Path whRoot = getDnsPath(new Path(whRootString));
-    return new Path(whRoot, tableName.toLowerCase());
+    return new Path(whRoot, MetaStoreUtils.encodeTableName(tableName.toLowerCase()));
   }
 
   public Path getDatabasePath(Database db) throws MetaException {
@@ -181,7 +181,7 @@ public class Warehouse {
 
   public Path getTablePath(Database db, String tableName)
       throws MetaException {
-    return getDnsPath(new Path(getDatabasePath(db), tableName.toLowerCase()));
+    return getDnsPath(new Path(getDatabasePath(db), MetaStoreUtils.encodeTableName(tableName.toLowerCase())));
   }
 
   public static String getQualifiedName(Table table) {

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/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 57b8618..218b9c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -928,7 +928,7 @@ public class Driver implements CommandProcessor {
         try {
           locks.add(new HiveLockObj(
                       new HiveLockObject(new DummyPartition(p.getTable(), p.getTable().getDbName()
-                                                            + "/" + p.getTable().getTableName()
+                                                            + "/" + MetaStoreUtils.encodeTableName(p.getTable().getTableName())
                                                             + "/" + partialName,
                                                               partialSpec), lockData), mode));
           partialName += "/";

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 74007e7..a6b318a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -3212,7 +3212,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
     if (allPartitions == null) {
       updateModifiedParameters(tbl.getTTable().getParameters(), conf);
-      tbl.checkValidity();
+      tbl.checkValidity(conf);
     } else {
       for (Partition tmpPart: allPartitions) {
         updateModifiedParameters(tmpPart.getParameters(), conf);
@@ -4108,7 +4108,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         oldview.getTTable().getParameters().putAll(crtView.getTblProps());
       }
       oldview.setPartCols(crtView.getPartCols());
-      oldview.checkValidity();
+      oldview.checkValidity(null);
       try {
         db.alterTable(crtView.getViewName(), oldview);
       } catch (InvalidOperationException e) {
@@ -4280,7 +4280,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       {
         // Default database name path is always ignored, use METASTOREWAREHOUSE and object name
         // instead
-        path = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.METASTOREWAREHOUSE), name.toLowerCase());
+        path = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.METASTOREWAREHOUSE), MetaStoreUtils.encodeTableName(name.toLowerCase()));
       }
     }
     else

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
index be5a593..7acc53f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
@@ -22,6 +22,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidReadTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -322,7 +323,7 @@ class DummyTxnManager extends HiveTxnManagerImpl {
         try {
           locks.add(new HiveLockObj(
                       new HiveLockObject(new DummyPartition(p.getTable(), p.getTable().getDbName()
-                                                            + "/" + p.getTable().getTableName()
+                                                            + "/" + MetaStoreUtils.encodeTableName(p.getTable().getTableName())
                                                             + "/" + partialName,
                                                               partialSpec), lockData), mode));
           partialName += "/";

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
index fadd074..d5ea083 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
@@ -22,6 +22,8 @@ import java.util.Arrays;
 import java.util.Map;
 
 import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.metadata.DummyPartition;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -187,12 +189,12 @@ public class HiveLockObject {
   }
 
   public HiveLockObject(Table tbl, HiveLockObjectData lockData) {
-    this(new String[] {tbl.getDbName(), tbl.getTableName()}, lockData);
+    this(new String[] {tbl.getDbName(), MetaStoreUtils.encodeTableName(tbl.getTableName())}, lockData);
   }
 
   public HiveLockObject(Partition par, HiveLockObjectData lockData) {
     this(new String[] {par.getTable().getDbName(),
-        par.getTable().getTableName(), par.getName()}, lockData);
+        MetaStoreUtils.encodeTableName(par.getTable().getTableName()), par.getName()}, lockData);
   }
 
   public HiveLockObject(DummyPartition par, HiveLockObjectData lockData) {

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/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 3ea8e25..4058606 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
@@ -553,7 +553,7 @@ public class Hive {
       if (newTbl.getParameters() != null) {
         newTbl.getParameters().remove(hive_metastoreConstants.DDL_TIME);
       }
-      newTbl.checkValidity();
+      newTbl.checkValidity(conf);
       getMSC().alter_table(names[0], names[1], newTbl.getTTable(), cascade);
     } catch (MetaException e) {
       throw new HiveException("Unable to alter table. " + e.getMessage(), e);
@@ -756,7 +756,7 @@ public class Hive {
         tbl.setFields(MetaStoreUtils.getFieldsFromDeserializer(tbl.getTableName(),
             tbl.getDeserializer()));
       }
-      tbl.checkValidity();
+      tbl.checkValidity(conf);
       if (tbl.getParameters() != null) {
         tbl.getParameters().remove(hive_metastoreConstants.DDL_TIME);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
index 52ed4a3..3d1ca93 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.metadata;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -181,11 +182,11 @@ public class Table implements Serializable {
     return t;
   }
 
-  public void checkValidity() throws HiveException {
+  public void checkValidity(Configuration conf) throws HiveException {
     // check for validity
     String name = tTable.getTableName();
     if (null == name || name.length() == 0
-        || !MetaStoreUtils.validateName(name)) {
+        || !MetaStoreUtils.validateName(name, conf)) {
       throw new HiveException("[" + name + "]: is not a valid table name");
     }
     if (0 == getCols().size()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java
index 4966d89..624ee7f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java
@@ -265,7 +265,7 @@ public final class RewriteQueryUsingAggregateIndexCtx  implements NodeProcessorC
     if (index == 0) {
       // the query contains the sum aggregation GenericUDAF
       String selReplacementCommand = "select sum(`" + rewriteQueryCtx.getAggregateFunction() + "`)"
-          + " from " + rewriteQueryCtx.getIndexName() + " group by "
+          + " from `" + rewriteQueryCtx.getIndexName() + "` group by "
           + rewriteQueryCtx.getIndexKey() + " ";
       // retrieve the operator tree for the query, and the required GroupByOperator from it
       Operator<?> newOperatorTree = RewriteParseContextGenerator.generateOperatorTree(

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
index 582ff32..533bcdf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
@@ -324,7 +324,7 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     rewrittenQueryBuilder.append(" from ");
     rewrittenQueryBuilder.append(tbl.getDbName());
     rewrittenQueryBuilder.append(".");
-    rewrittenQueryBuilder.append(tbl.getTableName());
+    rewrittenQueryBuilder.append("`" + tbl.getTableName() + "`");
     isRewritten = true;
 
     // If partition level statistics is requested, add predicate and group by as needed to rewritten

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
index 1e2feaa..9fd8516 100755
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
@@ -303,7 +303,7 @@ public class TestHive extends TestCase {
 
       ft = hm.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
       assertNotNull("Unable to fetch table", ft);
-      ft.checkValidity();
+      ft.checkValidity(hiveConf);
       assertEquals("Table names didn't match for table: " + tableName, tbl
           .getTableName(), ft.getTableName());
       assertEquals("Table owners didn't match for table: " + tableName, tbl

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/test/queries/clientnegative/special_character_in_tabnames_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/special_character_in_tabnames_1.q b/ql/src/test/queries/clientnegative/special_character_in_tabnames_1.q
new file mode 100644
index 0000000..7785020
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/special_character_in_tabnames_1.q
@@ -0,0 +1,13 @@
+set hive.support.special.characters.tablename=false;
+
+-- If hive.support.special.characters.tablename=false, we can not use special characters in table names.
+-- The same query would work when it is set to true(default value).
+-- Note that there is a positive test with the same name in clientpositive
+
+
+create table `c/b/o_t1`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE;
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q b/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
new file mode 100644
index 0000000..7540d27
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
@@ -0,0 +1,1075 @@
+set hive.cbo.enable=true;
+set hive.exec.check.crossproducts=false;
+set hive.stats.fetch.column.stats=true;
+set hive.auto.convert.join=false;
+
+-- SORT_QUERY_RESULTS
+
+create table `c/b/o_t1`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE;
+create table `//cbo_t2`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE;
+create table `cbo_/t3////`(key string, value string, c_int int, c_float float, c_boolean boolean)  row format delimited fields terminated by ',' STORED AS TEXTFILE;
+
+load data local inpath '../../data/files/cbo_t1.txt' into table `c/b/o_t1` partition (dt='2014');
+load data local inpath '../../data/files/cbo_t2.txt' into table `//cbo_t2` partition (dt='2014');
+load data local inpath '../../data/files/cbo_t3.txt' into table `cbo_/t3////`;
+
+CREATE TABLE `p/a/r/t`(
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table `p/a/r/t`;
+
+CREATE TABLE `line/item` (L_ORDERKEY      INT,
+                                L_PARTKEY       INT,
+                                L_SUPPKEY       INT,
+                                L_LINENUMBER    INT,
+                                L_QUANTITY      DOUBLE,
+                                L_EXTENDEDPRICE DOUBLE,
+                                L_DISCOUNT      DOUBLE,
+                                L_TAX           DOUBLE,
+                                L_RETURNFLAG    STRING,
+                                L_LINESTATUS    STRING,
+                                l_shipdate      STRING,
+                                L_COMMITDATE    STRING,
+                                L_RECEIPTDATE   STRING,
+                                L_SHIPINSTRUCT  STRING,
+                                L_SHIPMODE      STRING,
+                                L_COMMENT       STRING)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|';
+
+LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE `line/item`;
+
+create table `src/_/cbo` as select * from src;
+
+analyze table `c/b/o_t1` partition (dt) compute statistics;
+
+analyze table `c/b/o_t1` compute statistics for columns key, value, c_int, c_float, c_boolean;
+
+analyze table `//cbo_t2` partition (dt) compute statistics;
+
+analyze table `//cbo_t2` compute statistics for columns key, value, c_int, c_float, c_boolean;
+
+analyze table `cbo_/t3////` compute statistics;
+
+analyze table `cbo_/t3////` compute statistics for columns key, value, c_int, c_float, c_boolean;
+
+analyze table `src/_/cbo` compute statistics;
+
+analyze table `src/_/cbo` compute statistics for columns;
+
+analyze table `p/a/r/t` compute statistics;
+
+analyze table `p/a/r/t` compute statistics for columns;
+
+analyze table `line/item` compute statistics;
+
+analyze table `line/item` compute statistics for columns;
+
+select key, (c_int+1)+2 as x, sum(c_int) from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key;
+
+select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key) R group by y, x;
+
+select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0) group by c_float, `c/b/o_t1`.c_int, key order by a) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key order by q/10 desc, r asc) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c order by `cbo_/t3////`.c_int+c desc, c;
+
+select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) `c/b/o_t1` left outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key  having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `//cbo_t2` on `c/b/o_t1`.a=p left outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c  having `cbo_/t3////`.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0  order by `cbo_/t3////`.c_int % c asc, `cbo_/t3////`.c_int desc;
+
+select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) `c/b/o_t1` right outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `//cbo_t2` on `c/b/o_t1`.a=p right outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 2) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c;
+
+
+
+select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) `c/b/o_t1` full outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by p+q desc, r asc) `//cbo_t2` on `c/b/o_t1`.a=p full outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c having `cbo_/t3////`.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by `cbo_/t3////`.c_int;
+
+
+
+select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c;
+
+
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 21. Test groupby is empty and there is no other cols in aggr
+
+select unionsrc.key FROM (select 'tst1' as key, count(1) as value from src) unionsrc;
+
+
+
+select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src) unionsrc;
+
+
+
+select unionsrc.key FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc order by unionsrc.key;
+
+        
+
+select unionsrc.key, unionsrc.value FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc order by unionsrc.key;
+
+
+
+select unionsrc.key, count(1) FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+    UNION  ALL
+
+        select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc group by unionsrc.key order by unionsrc.key;
+
+
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- SORT_QUERY_RESULTS
+
+-- 4. Test Select + Join + TS
+
+select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` join             `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key;
+
+select `c/b/o_t1`.key from `c/b/o_t1` join `cbo_/t3////`;
+
+select `c/b/o_t1`.key from `c/b/o_t1` join `cbo_/t3////` where `c/b/o_t1`.key=`cbo_/t3////`.key and `c/b/o_t1`.key >= 1;
+
+select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` left outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key;
+
+select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` right outer join `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key;
+
+select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` full outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key;
+
+
+
+select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key;
+
+select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a;
+
+select a, `c/b/o_t1`.b, key, `//cbo_t2`.c_int, `cbo_/t3////`.p from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2`  on `c/b/o_t1`.a=key join (select key as p, c_int as q, `cbo_/t3////`.c_float as r from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.p;
+
+select b, `c/b/o_t1`.c, `//cbo_t2`.c_int, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.a=`//cbo_t2`.key join `cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.key;
+
+select `cbo_/t3////`.c_int, b, `//cbo_t2`.c_int, `c/b/o_t1`.c from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.a=`//cbo_t2`.key join `cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.key;
+
+
+
+select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` left outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key;
+
+select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p left outer join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a;
+
+
+
+select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` right outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key;
+
+select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p right outer join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a;
+
+
+
+select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` full outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key;
+
+select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p full outer join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a;
+
+
+
+-- 5. Test Select + Join + FIL + TS
+
+select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key where (`c/b/o_t1`.c_int + `//cbo_t2`.c_int == 2) and (`c/b/o_t1`.c_int > 0 or `//cbo_t2`.c_float >= 0);
+
+select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` left outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key where (`c/b/o_t1`.c_int + `//cbo_t2`.c_int == 2) and (`c/b/o_t1`.c_int > 0 or `//cbo_t2`.c_float >= 0);
+
+select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` right outer join `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key where (`c/b/o_t1`.c_int + `//cbo_t2`.c_int == 2) and (`c/b/o_t1`.c_int > 0 or `//cbo_t2`.c_float >= 0);
+
+select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` full outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key where (`c/b/o_t1`.c_int + `//cbo_t2`.c_int == 2) and (`c/b/o_t1`.c_int > 0 or `//cbo_t2`.c_float >= 0);
+
+
+
+select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or `//cbo_t2`.q >= 0);
+
+
+
+select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` left outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0);
+
+
+
+select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` right outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0);
+
+
+
+select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` full outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0);
+
+
+
+select * from (select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` full outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+
+
+select * from (select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` left outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p left outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+
+
+select * from (select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` left outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p right outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+
+
+select * from (select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` left outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p full outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+
+
+select * from (select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` right outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p right outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+
+
+select * from (select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` right outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p left outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+
+
+select * from (select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` right outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p full outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+
+
+select * from (select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` full outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p full outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+
+
+select * from (select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` full outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p left outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+
+
+select * from (select q, b, `//cbo_t2`.p, `c/b/o_t1`.c, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` full outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p right outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q == 2) and (b > 0 or c_int >= 0)) R where  (q + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 7. Test Select + TS + Join + Fil + GB + GB Having + Limit
+
+select key, (c_int+1)+2 as x, sum(c_int) from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key order by x limit 1;
+
+select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key) R group by y, x order by x,y limit 1;
+
+select key from(select key from (select key from `c/b/o_t1` limit 5)`//cbo_t2`  limit 5)`cbo_/t3////`  limit 5;
+
+select key, c_int from(select key, c_int from (select key, c_int from `c/b/o_t1` order by c_int limit 5)`c/b/o_t1`  order by c_int limit 5)`//cbo_t2`  order by c_int limit 5;
+
+
+
+select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0) group by c_float, `c/b/o_t1`.c_int, key order by a limit 5) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key order by q/10 desc, r asc limit 5) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c order by `cbo_/t3////`.c_int+c desc, c limit 5;
+
+
+
+select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) `c/b/o_t1` left outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key  having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) `//cbo_t2` on `c/b/o_t1`.a=p left outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `//cbo_t2`.q >= 0) and (b > 0 or c_int >= 0) group by `cbo_/t3////`.c_int, c  having `cbo_/t3////`.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0  order by `cbo_/t3////`.c_int % c asc, `cbo_/t3////`.c_int, c desc li
 mit 5;
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 12. SemiJoin
+
+select `c/b/o_t1`.c_int           from `c/b/o_t1` left semi join   `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key;
+
+select `c/b/o_t1`.c_int           from `c/b/o_t1` left semi join   `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0);
+
+select * from (select c, b, a from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` left semi join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p left semi join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + 1 == 2) and (b > 0 or c >= 0)) R where  (b + 1 = 2) and (R.b > 0 or c >= 0);
+
+select * from (select `cbo_/t3////`.c_int, `c/b/o_t1`.c, b from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 = 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` left semi join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p left outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + `cbo_/t3////`.c_int  == 2) and (b > 0 or c_int >= 0)) R where  (R.c_int + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+select * from (select c_int, b, `c/b/o_t1`.c from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` left semi join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p right outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + 1 == 2) and (b > 0 or c_int >= 0)) R where  (c + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+select * from (select c_int, b, `c/b/o_t1`.c from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`  where (`c/b/o_t1`.c_int + 1 == 2) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)) `c/b/o_t1` left semi join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`  where (`//cbo_t2`.c_int + 1 == 2) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)) `//cbo_t2` on `c/b/o_t1`.a=p full outer join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + 1 == 2) and (b > 0 or c_int >= 0)) R where  (c + 1 = 2) and (R.b > 0 or c_int >= 0);
+
+select a, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc) `c/b/o_t1` left semi join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by q+r/10 desc, p) `//cbo_t2` on `c/b/o_t1`.a=p left semi join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + 1  >= 0) and (b > 0 or a >= 0) group by a, c  having a > 0 and (a >=1 or c >= 1) and (a + c) >= 0 order by c, a;
+
+select a, c, count(*)  from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc limit 5) `c/b/o_t1` left semi join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key having `//cbo_t2`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by q+r/10 desc, p limit 5) `//cbo_t2` on `c/b/o_t1`.a=p left semi join `cbo_/t3////` on `c/b/o_t1`.a=key where (b + 1  >= 0) and (b > 0 or a >= 0) group by a, c  having a > 0 and (a >=1 or c >= 1) and (a + c) >= 0 order by c, a;
+
+
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 1. Test Select + TS
+
+select * from `c/b/o_t1`;
+
+select * from `c/b/o_t1` as `c/b/o_t1`;
+
+select * from `c/b/o_t1` as `//cbo_t2`;
+
+
+
+select `c/b/o_t1`.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from `c/b/o_t1`;
+
+select * from `c/b/o_t1` where (((key=1) and (c_float=10)) and (c_int=20)); 
+
+
+
+-- 2. Test Select + TS + FIL
+
+select * from `c/b/o_t1` where `c/b/o_t1`.c_int >= 0;
+
+select * from `c/b/o_t1` as `c/b/o_t1`  where `c/b/o_t1`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+select * from `c/b/o_t1` as `//cbo_t2` where `//cbo_t2`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+
+
+select `//cbo_t2`.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from `c/b/o_t1` as `//cbo_t2`  where `//cbo_t2`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+
+
+-- 3 Test Select + Select + TS + FIL
+
+select * from (select * from `c/b/o_t1` where `c/b/o_t1`.c_int >= 0) as `c/b/o_t1`;
+
+select * from (select * from `c/b/o_t1` as `c/b/o_t1`  where `c/b/o_t1`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as `c/b/o_t1`;
+
+select * from (select * from `c/b/o_t1` as `//cbo_t2` where `//cbo_t2`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as `c/b/o_t1`;
+
+select * from (select `//cbo_t2`.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from `c/b/o_t1` as `//cbo_t2`  where `//cbo_t2`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as `c/b/o_t1`;
+
+
+
+select * from (select * from `c/b/o_t1` where `c/b/o_t1`.c_int >= 0) as `c/b/o_t1` where `c/b/o_t1`.c_int >= 0;
+
+select * from (select * from `c/b/o_t1` as `c/b/o_t1`  where `c/b/o_t1`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as `c/b/o_t1`  where `c/b/o_t1`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+select * from (select * from `c/b/o_t1` as `//cbo_t2` where `//cbo_t2`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as `//cbo_t2` where `//cbo_t2`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+select * from (select `//cbo_t2`.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from `c/b/o_t1` as `//cbo_t2`  where `//cbo_t2`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as `c/b/o_t1` where `c/b/o_t1`.c_int >= 0 and y+c_int >= 0 or x <= 100;
+
+
+
+select `c/b/o_t1`.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from `c/b/o_t1` where `c/b/o_t1`.c_int >= 0) as `c/b/o_t1` where `c/b/o_t1`.c_int >= 0;
+
+select `//cbo_t2`.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from `c/b/o_t1` where `c/b/o_t1`.c_int >= 0) as `//cbo_t2` where `//cbo_t2`.c_int >= 0;
+
+
+
+
+
+
+
+select * from (select * from `c/b/o_t1` where `c/b/o_t1`.c_int >= 0) as `c/b/o_t1` where `c/b/o_t1`.c_int >= 0;
+
+select * from (select * from `c/b/o_t1` as `c/b/o_t1`  where `c/b/o_t1`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as `c/b/o_t1`  where `c/b/o_t1`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+select * from (select * from `c/b/o_t1` as `//cbo_t2` where `//cbo_t2`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as `//cbo_t2` where `//cbo_t2`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+select * from (select `//cbo_t2`.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from `c/b/o_t1` as `//cbo_t2`  where `//cbo_t2`.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as `c/b/o_t1` where `c/b/o_t1`.c_int >= 0 and y+c_int >= 0 or x <= 100;
+
+
+
+select `c/b/o_t1`.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from `c/b/o_t1` where `c/b/o_t1`.c_int >= 0) as `c/b/o_t1` where `c/b/o_t1`.c_int >= 0;
+
+select `//cbo_t2`.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from `c/b/o_t1` where `c/b/o_t1`.c_int >= 0) as `//cbo_t2` where `//cbo_t2`.c_int >= 0;
+
+
+
+
+
+
+
+-- 13. null expr in select list
+
+select null from `cbo_/t3////`;
+
+
+
+-- 14. unary operator
+
+select key from `c/b/o_t1` where c_int = -6  or c_int = +6;
+
+
+
+-- 15. query referencing only partition columns
+
+select count(`c/b/o_t1`.dt) from `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.dt  = `//cbo_t2`.dt  where `c/b/o_t1`.dt = '2014' ;
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 20. Test get stats with empty partition list
+
+select `c/b/o_t1`.value from `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.key = `//cbo_t2`.key where `c/b/o_t1`.dt = '10' and `c/b/o_t1`.c_boolean = true;
+
+
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 18. SubQueries Not Exists
+
+-- distinct, corr
+
+select * 
+
+from `src/_/cbo` b 
+
+where not exists 
+
+  (select distinct a.key 
+
+  from `src/_/cbo` a 
+
+  where b.value = a.value and a.value > 'val_2'
+
+  )
+
+;
+
+
+
+-- no agg, corr, having
+
+select * 
+
+from `src/_/cbo` b 
+
+group by key, value
+
+having not exists 
+
+  (select a.key 
+
+  from `src/_/cbo` a 
+
+  where b.value = a.value  and a.key = b.key and a.value > 'val_12'
+
+  )
+
+;
+
+
+
+-- 19. SubQueries Exists
+
+-- view test
+
+create view cv1 as 
+
+select * 
+
+from `src/_/cbo` b 
+
+where exists
+
+  (select a.key 
+
+  from `src/_/cbo` a 
+
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+
+;
+
+
+
+select * from cv1
+
+;
+
+
+
+-- sq in from
+
+select * 
+
+from (select * 
+
+      from `src/_/cbo` b 
+
+      where exists 
+
+          (select a.key 
+
+          from `src/_/cbo` a 
+
+          where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+
+     ) a
+
+;
+
+
+
+-- sq in from, having
+
+select *
+
+from (select b.key, count(*) 
+
+  from `src/_/cbo` b 
+
+  group by b.key
+
+  having exists 
+
+    (select a.key 
+
+    from `src/_/cbo` a 
+
+    where a.key = b.key and a.value > 'val_9'
+
+    )
+
+) a
+
+;
+
+
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 17. SubQueries In
+
+-- non agg, non corr
+
+select * 
+
+from `src/_/cbo` 
+
+where `src/_/cbo`.key in (select key from `src/_/cbo` s1 where s1.key > '9') order by key
+
+;
+
+
+
+-- agg, corr
+
+-- add back once rank issue fixed for cbo
+
+
+
+-- distinct, corr
+
+select * 
+
+from `src/_/cbo` b 
+
+where b.key in
+
+        (select distinct a.key 
+
+         from `src/_/cbo` a 
+
+         where b.value = a.value and a.key > '9'
+
+        ) order by b.key
+
+;
+
+
+
+-- non agg, corr, with join in Parent Query
+
+select p.p_partkey, li.l_suppkey 
+
+from (select distinct l_partkey as p_partkey from `line/item`) p join `line/item` li on p.p_partkey = li.l_partkey 
+
+where li.l_linenumber = 1 and
+
+ li.l_orderkey in (select l_orderkey from `line/item` where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber)
+
+ order by p.p_partkey
+
+;
+
+
+
+-- where and having
+
+-- Plan is:
+
+-- Stage 1: b semijoin sq1:`src/_/cbo` (subquery in where)
+
+-- Stage 2: group by Stage 1 o/p
+
+-- Stage 5: group by on sq2:`src/_/cbo` (subquery in having)
+
+-- Stage 6: Stage 2 o/p semijoin Stage 5
+
+select key, value, count(*) 
+
+from `src/_/cbo` b
+
+where b.key in (select key from `src/_/cbo` where `src/_/cbo`.key > '8')
+
+group by key, value
+
+having count(*) in (select count(*) from `src/_/cbo` s1 where s1.key > '9' group by s1.key ) order by key
+
+;
+
+
+
+-- non agg, non corr, windowing
+
+select p_mfgr, p_name, avg(p_size) 
+
+from `p/a/r/t` 
+
+group by p_mfgr, p_name
+
+having p_name in 
+
+  (select first_value(p_name) over(partition by p_mfgr order by p_size) from `p/a/r/t`) order by p_mfgr
+
+;
+
+
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 16. SubQueries Not In
+
+-- non agg, non corr
+
+select * 
+
+from `src/_/cbo` 
+
+where `src/_/cbo`.key not in  
+
+  ( select key  from `src/_/cbo` s1 
+
+    where s1.key > '2'
+
+  ) order by key
+
+;
+
+
+
+-- non agg, corr
+
+select p_mfgr, b.p_name, p_size 
+
+from `p/a/r/t` b 
+
+where b.p_name not in 
+
+  (select p_name 
+
+  from (select p_mfgr, p_name, p_size as r from `p/a/r/t`) a 
+
+  where r < 10 and b.p_mfgr = a.p_mfgr 
+
+  ) order by p_mfgr,p_size
+
+;
+
+
+
+-- agg, non corr
+
+select p_name, p_size 
+
+from 
+
+`p/a/r/t` where `p/a/r/t`.p_size not in 
+
+  (select avg(p_size) 
+
+  from (select p_size from `p/a/r/t`) a 
+
+  where p_size < 10
+
+  ) order by p_name
+
+;
+
+
+
+-- agg, corr
+
+select p_mfgr, p_name, p_size 
+
+from `p/a/r/t` b where b.p_size not in 
+
+  (select min(p_size) 
+
+  from (select p_mfgr, p_size from `p/a/r/t`) a 
+
+  where p_size < 10 and b.p_mfgr = a.p_mfgr
+
+  ) order by  p_name
+
+;
+
+
+
+-- non agg, non corr, Group By in Parent Query
+
+select li.l_partkey, count(*) 
+
+from `line/item` li 
+
+where li.l_linenumber = 1 and 
+
+  li.l_orderkey not in (select l_orderkey from `line/item` where l_shipmode = 'AIR') 
+
+group by li.l_partkey order by li.l_partkey
+
+;
+
+
+
+-- add null check test from sq_notin.q once HIVE-7721 resolved.
+
+
+
+-- non agg, corr, having
+
+select b.p_mfgr, min(p_retailprice) 
+
+from `p/a/r/t` b 
+
+group by b.p_mfgr
+
+having b.p_mfgr not in 
+
+  (select p_mfgr 
+
+  from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from `p/a/r/t` group by p_mfgr) a 
+
+  where min(p_retailprice) = l and r - l > 600
+
+  )
+
+  order by b.p_mfgr
+
+;
+
+
+
+-- agg, non corr, having
+
+select b.p_mfgr, min(p_retailprice) 
+
+from `p/a/r/t` b 
+
+group by b.p_mfgr
+
+having b.p_mfgr not in 
+
+  (select p_mfgr 
+
+  from `p/a/r/t` a
+
+  group by p_mfgr
+
+  having max(p_retailprice) - min(p_retailprice) > 600
+
+  )
+
+  order by b.p_mfgr  
+
+;
+
+
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- SORT_QUERY_RESULTS
+
+
+
+-- 8. Test UDF/UDAF
+
+select count(*), count(c_int), sum(c_int), avg(c_int), max(c_int), min(c_int) from `c/b/o_t1`;
+
+select count(*), count(c_int) as a, sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0  then 1 when 1 then 2 else 3 end, sum(case c_int when 0  then 1 when 1 then 2 else 3 end) from `c/b/o_t1` group by c_int order by a;
+
+select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f from `c/b/o_t1`) `c/b/o_t1`;
+
+select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0  then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0  then 1 when 1 then 2 else 3 end) as h from `c/b/o_t1` group by c_int) `c/b/o_t1` order by a;
+
+select f,a,e,b from (select count(*) as a, count(c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f from `c/b/o_t1`) `c/b/o_t1`;
+
+select f,a,e,b from (select count(*) as a, count(distinct c_int) as b, sum(distinct c_int) as c, avg(distinct c_int) as d, max(distinct c_int) as e, min(distinct c_int) as f from `c/b/o_t1`) `c/b/o_t1`;
+
+select key,count(c_int) as a, avg(c_float) from `c/b/o_t1` group by key order by a;
+
+select count(distinct c_int) as a, avg(c_float) from `c/b/o_t1` group by c_float order by a;
+
+select count(distinct c_int) as a, avg(c_float) from `c/b/o_t1` group by c_int order by a;
+
+select count(distinct c_int) as a, avg(c_float) from `c/b/o_t1` group by c_float, c_int order by a;
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- SORT_QUERY_RESULTS
+
+
+
+-- 11. Union All
+
+select * from (select * from `c/b/o_t1` order by key, c_boolean, value, dt)a union all select * from (select * from `//cbo_t2` order by key, c_boolean, value, dt)b;
+
+select key from (select key, c_int from (select * from `c/b/o_t1` union all select * from `//cbo_t2` where `//cbo_t2`.key >=0)r1 union all select key, c_int from `cbo_/t3////`)r2 where key >=0 order by key;
+
+select r2.key from (select key, c_int from (select key, c_int from `c/b/o_t1` union all select key, c_int from `cbo_/t3////` )r1 union all select key, c_int from `cbo_/t3////`)r2 join   (select key, c_int from (select * from `c/b/o_t1` union all select * from `//cbo_t2` where `//cbo_t2`.key >=0)r1 union all select key, c_int from `cbo_/t3////`)r3 on r2.key=r3.key where r3.key >=0 order by r2.key;
+
+
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 10. Test views
+
+create view v1 as select c_int, value, c_boolean, dt from `c/b/o_t1`;
+
+create view v2 as select c_int, value from `//cbo_t2`;
+
+
+
+select value from v1 where c_boolean=false;
+
+select max(c_int) from v1 group by (c_boolean);
+
+
+
+select count(v1.c_int)  from v1 join `//cbo_t2` on v1.c_int = `//cbo_t2`.c_int;
+
+select count(v1.c_int)  from v1 join v2 on v1.c_int = v2.c_int;
+
+
+
+select count(*) from v1 a join v1 b on a.value = b.value;
+
+
+
+create view v3 as select v1.value val from v1 join `c/b/o_t1` on v1.c_boolean = `c/b/o_t1`.c_boolean;
+
+
+
+select count(val) from v3 where val != '1';
+
+with q1 as ( select key from `c/b/o_t1` where key = '1')
+
+select count(*) from q1;
+
+
+
+with q1 as ( select value from v1 where c_boolean = false)
+
+select count(value) from q1 ;
+
+
+
+create view v4 as
+
+with q1 as ( select key,c_int from `c/b/o_t1`  where key = '1')
+
+select * from q1
+
+;
+
+
+
+with q1 as ( select c_int from q2 where c_boolean = false),
+
+q2 as ( select c_int,c_boolean from v1  where value = '1')
+
+select sum(c_int) from (select c_int from q1) a;
+
+
+
+with q1 as ( select `c/b/o_t1`.c_int c_int from q2 join `c/b/o_t1` where q2.c_int = `c/b/o_t1`.c_int  and `c/b/o_t1`.dt='2014'),
+
+q2 as ( select c_int,c_boolean from v1  where value = '1' or dt = '14')
+
+select count(*) from q1 join q2 join v4 on q1.c_int = q2.c_int and v4.c_int = q2.c_int;
+
+
+
+
+
+drop view v1;
+
+drop view v2;
+
+drop view v3;
+
+drop view v4;
+
+set hive.cbo.enable=true;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 9. Test Windowing Functions
+
+-- SORT_QUERY_RESULTS
+
+
+
+select count(c_int) over() from `c/b/o_t1`;
+
+select count(c_int) over(partition by c_float order by key), sum(c_float) over(partition by c_float order by key), max(c_int) over(partition by c_float order by key), min(c_int) over(partition by c_float order by key), row_number() over(partition by c_float order by key) as rn, rank() over(partition by c_float order by key), dense_rank() over(partition by c_float order by key), round(percent_rank() over(partition by c_float order by key), 2), lead(c_int, 2, c_int) over(partition by c_float order by key), lag(c_float, 2, c_float) over(partition by c_float order by key) from `c/b/o_t1` order by rn;
+
+select * from (select count(c_int) over(partition by c_float order by key), sum(c_float) over(partition by c_float order by key), max(c_int) over(partition by c_float order by key), min(c_int) over(partition by c_float order by key), row_number() over(partition by c_float order by key) as rn, rank() over(partition by c_float order by key), dense_rank() over(partition by c_float order by key), round(percent_rank() over(partition by c_float order by key),2), lead(c_int, 2, c_int) over(partition by c_float   order by key  ), lag(c_float, 2, c_float) over(partition by c_float   order by key) from `c/b/o_t1` order by rn) `c/b/o_t1`;
+
+select x from (select count(c_int) over() as x, sum(c_float) over() from `c/b/o_t1`) `c/b/o_t1`;
+
+select 1+sum(c_int) over() from `c/b/o_t1`;
+
+select sum(c_int)+sum(sum(c_int)) over() from `c/b/o_t1`;
+
+select * from (select max(c_int) over (partition by key order by value Rows UNBOUNDED PRECEDING), min(c_int) over (partition by key order by value rows current row), count(c_int) over(partition by key order by value ROWS 1 PRECEDING), avg(value) over (partition by key order by value Rows between unbounded preceding and unbounded following), sum(value) over (partition by key order by value rows between unbounded preceding and current row), avg(c_float) over (partition by key order by value Rows between 1 preceding and unbounded following), sum(c_float) over (partition by key order by value rows between 1 preceding and current row), max(c_float) over (partition by key order by value rows between 1 preceding and unbounded following), min(c_float) over (partition by key order by value rows between 1 preceding and 1 following) from `c/b/o_t1`) `c/b/o_t1`;
+
+select i, a, h, b, c, d, e, f, g, a as x, a +1 as y from (select max(c_int) over (partition by key order by value range UNBOUNDED PRECEDING) a, min(c_int) over (partition by key order by value range current row) b, count(c_int) over(partition by key order by value range 1 PRECEDING) c, avg(value) over (partition by key order by value range between unbounded preceding and unbounded following) d, sum(value) over (partition by key order by value range between unbounded preceding and current row) e, avg(c_float) over (partition by key order by value range between 1 preceding and unbounded following) f, sum(c_float) over (partition by key order by value range between 1 preceding and current row) g, max(c_float) over (partition by key order by value range between 1 preceding and unbounded following) h, min(c_float) over (partition by key order by value range between 1 preceding and 1 following) i from `c/b/o_t1`) `c/b/o_t1`;
+
+select *, rank() over(partition by key order by value) as rr from src1;
+
+select *, rank() over(partition by key order by value) from src1;
+
+insert into table `src/_/cbo` select * from src;
+
+select * from `src/_/cbo` limit 1;
+
+insert overwrite table `src/_/cbo` select * from src;
+
+select * from `src/_/cbo` limit 1;
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/test/queries/clientpositive/special_character_in_tabnames_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/special_character_in_tabnames_2.q b/ql/src/test/queries/clientpositive/special_character_in_tabnames_2.q
new file mode 100644
index 0000000..34dcc90
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/special_character_in_tabnames_2.q
@@ -0,0 +1,40 @@
+-- try the query without indexing, with manual indexing, and with automatic indexing
+-- SORT_QUERY_RESULTS
+
+DROP TABLE IF EXISTS `s/c`;
+
+CREATE TABLE `s/c` (key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH "../../data/files/kv1.txt" INTO TABLE `s/c`;
+
+ANALYZE TABLE `s/c` COMPUTE STATISTICS;
+
+ANALYZE TABLE `s/c` COMPUTE STATISTICS FOR COLUMNS key,value;
+
+-- without indexing
+SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100;
+
+set hive.stats.dbclass=fs;
+CREATE INDEX src_index ON TABLE `s/c`(key) as 'COMPACT' WITH DEFERRED REBUILD;
+ALTER INDEX src_index ON `s/c` REBUILD;
+
+SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+-- manual indexing
+INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_where" SELECT `_bucketname` ,  `_offsets` FROM `default__s/c_src_index__` WHERE key > 80 AND key < 100;
+SET hive.index.compact.file=${system:test.tmp.dir}/index_where;
+SET hive.optimize.index.filter=false;
+SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat;
+
+EXPLAIN SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100;
+SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100;
+
+SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+SET hive.optimize.index.filter=true;
+SET hive.optimize.index.filter.compact.minsize=0;
+
+-- automatic indexing
+EXPLAIN SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100;
+SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100;
+
+DROP INDEX src_index on `s/c`;

http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/test/results/clientnegative/special_character_in_tabnames_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/special_character_in_tabnames_1.q.out b/ql/src/test/results/clientnegative/special_character_in_tabnames_1.q.out
new file mode 100644
index 0000000..cb8e51c
--- /dev/null
+++ b/ql/src/test/results/clientnegative/special_character_in_tabnames_1.q.out
@@ -0,0 +1,10 @@
+PREHOOK: query: -- If hive.support.special.characters.tablename=false, we can not use special characters in table names.
+-- The same query would work when it is set to true(default value).
+-- Note that there is a positive test with the same name in clientpositive
+
+
+create table `c/b/o_t1`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@c/b/o_t1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. org.apache.hadoop.hive.ql.metadata.HiveException: [c/b/o_t1]: is not a valid table name


[07/23] hive git commit: HIVE-11553 : use basic file metadata cache in ETLSplitStrategy-related paths (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-11553 : use basic file metadata cache in ETLSplitStrategy-related paths (Sergey Shelukhin, 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/bc1c434f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bc1c434f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bc1c434f

Branch: refs/heads/llap
Commit: bc1c434f1ffc70dc2d52f34839c6f54b0b0c8d10
Parents: 77f30d4
Author: Sergey Shelukhin <se...@apache.org>
Authored: Fri Oct 2 11:11:36 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Fri Oct 2 13:11:28 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   9 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |  20 +-
 .../hive/metastore/HiveMetaStoreClient.java     |  94 +++++
 .../hadoop/hive/metastore/IMetaStoreClient.java |  20 ++
 .../hive/metastore/hbase/HBaseReadWrite.java    |   9 +-
 ql/pom.xml                                      |   3 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   2 +-
 .../hive/ql/exec/tez/HiveSplitGenerator.java    |   4 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   | 360 +++++++++++++++----
 .../apache/hadoop/hive/ql/io/orc/OrcSplit.java  |   2 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |   3 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  44 ++-
 12 files changed, 484 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/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 e7ed07e..77ca613 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -153,7 +153,7 @@ public class HiveConf extends Configuration {
       HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL,
       HiveConf.ConfVars.METASTORE_END_FUNCTION_LISTENERS,
       HiveConf.ConfVars.METASTORE_PART_INHERIT_TBL_PROPS,
-      HiveConf.ConfVars.METASTORE_BATCH_RETRIEVE_TABLE_PARTITION_MAX,
+      HiveConf.ConfVars.METASTORE_BATCH_RETRIEVE_OBJECTS_MAX,
       HiveConf.ConfVars.METASTORE_INIT_HOOKS,
       HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS,
       HiveConf.ConfVars.HMSHANDLERATTEMPTS,
@@ -567,9 +567,9 @@ public class HiveConf extends Configuration {
         "Maximum number of objects (tables/partitions) can be retrieved from metastore in one batch. \n" +
         "The higher the number, the less the number of round trips is needed to the Hive metastore server, \n" +
         "but it may also cause higher memory requirement at the client side."),
-    METASTORE_BATCH_RETRIEVE_TABLE_PARTITION_MAX(
+    METASTORE_BATCH_RETRIEVE_OBJECTS_MAX(
         "hive.metastore.batch.retrieve.table.partition.max", 1000,
-        "Maximum number of table partitions that metastore internally retrieves in one batch."),
+        "Maximum number of objects that metastore internally retrieves in one batch."),
 
     METASTORE_INIT_HOOKS("hive.metastore.init.hooks", "",
         "A comma separated list of hooks to be invoked at the beginning of HMSHandler initialization. \n" +
@@ -1077,6 +1077,9 @@ public class HiveConf extends Configuration {
         " (split generation reads and caches file footers). HYBRID chooses between the above strategies" +
         " based on heuristics."),
 
+    HIVE_ORC_MS_FOOTER_CACHE_ENABLED("hive.orc.splits.ms.footer.cache.enabled", false,
+        "Whether to enable using file metadata cache in metastore for ORC file footers."),
+
     HIVE_ORC_INCLUDE_FILE_FOOTER_IN_SPLITS("hive.orc.splits.include.file.footer", false,
         "If turned on splits generated by orc will include metadata about the stripes in the file. This\n" +
         "data is read remotely (from the client or HS2 machine) and sent to all the tasks."),

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/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 815f499..8cd1f52 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -208,6 +208,7 @@ import org.apache.thrift.transport.TTransportFactory;
 import javax.jdo.JDOException;
 
 import java.io.IOException;
+import java.lang.reflect.Field;
 import java.nio.ByteBuffer;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
@@ -5718,12 +5719,29 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       if (metadatas != null) {
         assert metadatas.length == fileIds.size();
         for (int i = 0; i < metadatas.length; ++i) {
-          result.putToMetadata(fileIds.get(i), metadatas[i]);
+          ByteBuffer bb = metadatas[i];
+          if (bb == null) continue;
+          bb = handleReadOnlyBufferForThrift(bb);
+          result.putToMetadata(fileIds.get(i), bb);
         }
       }
+      if (!result.isSetMetadata()) {
+        result.setMetadata(new HashMap<Long, ByteBuffer>()); // Set the required field.
+      }
       return result;
     }
 
+    private ByteBuffer handleReadOnlyBufferForThrift(ByteBuffer bb) {
+      if (!bb.isReadOnly()) return bb;
+      // Thrift cannot write read-only buffers... oh well.
+      // TODO: actually thrift never writes to the buffer, so we could use reflection to
+      //       unset the unnecessary read-only flag if allocation/copy perf becomes a problem.
+      ByteBuffer copy = ByteBuffer.allocate(bb.capacity());
+      copy.put(bb);
+      copy.flip();
+      return copy;
+    }
+
     @Override
     public PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req) throws TException {
       getMS().putFileMetadata(req.getFileIds(), req.getMetadata());

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 6f15fd0..8e32966 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -35,9 +35,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -61,6 +64,7 @@ import org.apache.hadoop.hive.metastore.api.AddPartitionsResult;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.CheckLockRequest;
+import org.apache.hadoop.hive.metastore.api.ClearFileMetadataRequest;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
@@ -76,6 +80,8 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.FireEventRequest;
 import org.apache.hadoop.hive.metastore.api.FireEventResponse;
 import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.GetFileMetadataRequest;
+import org.apache.hadoop.hive.metastore.api.GetFileMetadataResult;
 import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
@@ -116,6 +122,7 @@ import org.apache.hadoop.hive.metastore.api.PartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+import org.apache.hadoop.hive.metastore.api.PutFileMetadataRequest;
 import org.apache.hadoop.hive.metastore.api.RequestPartsSpec;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
@@ -170,6 +177,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
   private String tokenStrForm;
   private final boolean localMetaStore;
   private final MetaStoreFilterHook filterHook;
+  private final int fileMetadataBatchSize;
 
   private Map<String, String> currentMetaVars;
 
@@ -195,6 +203,8 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     }
     this.conf = conf;
     filterHook = loadFilterHooks();
+    fileMetadataBatchSize = HiveConf.getIntVar(
+        conf, HiveConf.ConfVars.METASTORE_BATCH_RETRIEVE_OBJECTS_MAX);
 
     String msUri = conf.getVar(HiveConf.ConfVars.METASTOREURIS);
     localMetaStore = HiveConfUtil.isEmbeddedMetaStore(msUri);
@@ -2108,4 +2118,88 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partNames);
     return client.get_aggr_stats_for(req);
   }
+
+  @Override
+  public Iterable<Entry<Long, ByteBuffer>> getFileMetadata(
+      final List<Long> fileIds) throws TException {
+    return new MetastoreMapIterable<Long, ByteBuffer>() {
+      private int listIndex = 0;
+      @Override
+      protected Map<Long, ByteBuffer> fetchNextBatch() throws TException {
+        if (listIndex == fileIds.size()) return null;
+        int endIndex = Math.min(listIndex + fileMetadataBatchSize, fileIds.size());
+        List<Long> subList = fileIds.subList(listIndex, endIndex);
+        GetFileMetadataRequest req = new GetFileMetadataRequest();
+        req.setFileIds(subList);
+        GetFileMetadataResult resp = client.get_file_metadata(req);
+        listIndex = endIndex;
+        return resp.getMetadata();
+      }
+    };
+  }
+
+  public static abstract class MetastoreMapIterable<K, V>
+    implements Iterable<Entry<K, V>>, Iterator<Entry<K, V>> {
+    private Iterator<Entry<K, V>> currentIter;
+
+    protected abstract Map<K, V> fetchNextBatch() throws TException;
+
+    @Override
+    public Iterator<Entry<K, V>> iterator() {
+      return this;
+    }
+
+    @Override
+    public boolean hasNext() {
+      ensureCurrentBatch();
+      return currentIter != null;
+    }
+
+    private void ensureCurrentBatch() {
+      if (currentIter != null && currentIter.hasNext()) return;
+      currentIter = null;
+      Map<K, V> currentBatch;
+      do {
+        try {
+          currentBatch = fetchNextBatch();
+        } catch (TException ex) {
+          throw new RuntimeException(ex);
+        }
+        if (currentBatch == null) return; // No more data.
+      } while (currentBatch.isEmpty());
+      currentIter = currentBatch.entrySet().iterator();
+    }
+
+    @Override
+    public Entry<K, V> next() {
+      ensureCurrentBatch();
+      if (currentIter == null) throw new NoSuchElementException();
+      return currentIter.next();
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  @Override
+  public void clearFileMetadata(List<Long> fileIds) throws TException {
+    ClearFileMetadataRequest req = new ClearFileMetadataRequest();
+    req.setFileIds(fileIds);
+    client.clear_file_metadata(req);
+  }
+
+  @Override
+  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) throws TException {
+    PutFileMetadataRequest req = new PutFileMetadataRequest();
+    req.setFileIds(fileIds);
+    req.setMetadata(metadata);
+    client.put_file_metadata(req);
+  }
+
+  @Override
+  public boolean isSameConfObj(HiveConf c) {
+    return conf == c;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index e4a6cdb..77820ae 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -79,8 +79,10 @@ import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.thrift.TException;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 /**
  * Wrapper around hive metastore thrift api
@@ -1459,4 +1461,22 @@ public interface IMetaStoreClient {
    * flush statistics objects.  This should be called at the beginning of each query.
    */
   void flushCache();
+
+  /**
+   * Gets file metadata, as cached by metastore, for respective file IDs.
+   * The metadata that is not cached in metastore may be missing.
+   */
+  Iterable<Entry<Long, ByteBuffer>> getFileMetadata(List<Long> fileIds) throws TException;
+
+  /**
+   * Cleares the file metadata cache for respective file IDs.
+   */
+  void clearFileMetadata(List<Long> fileIds) throws TException;
+
+  /**
+   * Adds file metadata for respective file IDs to metadata cache in metastore.
+   */
+  void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) throws TException;
+
+  boolean isSameConfObj(HiveConf c);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
index d38c561..f69b4c7 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
@@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -1959,7 +1960,13 @@ public class HBaseReadWrite {
     Result[] results = htab.get(gets);
     for (int i = 0; i < results.length; ++i) {
       Result r = results[i];
-      resultDest[i] = (r.isEmpty() ? null : r.getValueAsByteBuffer(colFam, colName));
+      if (r.isEmpty()) {
+        resultDest[i] = null;
+      } else {
+        Cell cell = r.getColumnLatestCell(colFam, colName);
+        resultDest[i] = ByteBuffer.wrap(
+            cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index 36b3433..587e2ee 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -704,9 +704,12 @@
                   <include>org.apache.hive:hive-common</include>
                   <include>org.apache.hive:hive-exec</include>
                   <include>org.apache.hive:hive-serde</include>
+                  <include>org.apache.hive:hive-metastore</include>
                   <include>com.esotericsoftware.kryo:kryo</include>
                   <include>org.apache.parquet:parquet-hadoop-bundle</include>
                   <include>org.apache.thrift:libthrift</include>
+                  <include>org.apache.thrift:libfb303</include>
+                  <include>javax.jdo:jdo-api</include>
                   <include>commons-lang:commons-lang</include>
                   <include>org.apache.commons:commons-lang3</include>
                   <include>org.jodd:jodd-core</include>

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 3511e73..74007e7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -3635,7 +3635,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     }
 
     int partitionBatchSize = HiveConf.getIntVar(conf,
-        ConfVars.METASTORE_BATCH_RETRIEVE_TABLE_PARTITION_MAX);
+        ConfVars.METASTORE_BATCH_RETRIEVE_OBJECTS_MAX);
 
     // drop the table
     db.dropTable(dropTbl.getTableName(), dropTbl.getIfPurge());

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
index 87881b6..0a43c15 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
@@ -91,10 +91,10 @@ public class HiveSplitGenerator extends InputInitializer {
     userPayloadProto =
         MRInputHelpers.parseMRInputPayload(initializerContext.getInputUserPayload());
 
-    this.conf =
-        TezUtils.createConfFromByteString(userPayloadProto.getConfigurationBytes());
+    this.conf = TezUtils.createConfFromByteString(userPayloadProto.getConfigurationBytes());
 
     this.jobConf = new JobConf(conf);
+
     // Read all credentials into the credentials instance stored in JobConf.
     ShimLoader.getHadoopShims().getMergedCredentials(jobConf);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index 57bde3e..ef62a23 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@ -19,11 +19,14 @@
 package org.apache.hadoop.hive.ql.io.orc;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.concurrent.Callable;
@@ -34,6 +37,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -57,10 +61,13 @@ import org.apache.hadoop.hive.ql.io.InputFormatChecker;
 import org.apache.hadoop.hive.ql.io.RecordIdentifier;
 import org.apache.hadoop.hive.ql.io.StatsProvidingRecordReader;
 import org.apache.hadoop.hive.ql.io.orc.OrcFile.WriterVersion;
+import org.apache.hadoop.hive.ql.io.orc.ReaderImpl.FileMetaInfo;
 import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.serde2.SerDeStats;
@@ -110,7 +117,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
   InputFormatChecker, VectorizedInputFormatInterface,
     AcidInputFormat<NullWritable, OrcStruct>, CombineHiveInputFormat.AvoidSplitCombination {
 
-  static enum SplitStrategyKind{
+  static enum SplitStrategyKind {
     HYBRID,
     BI,
     ETL
@@ -441,7 +448,12 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
    */
   static class Context {
     private final Configuration conf;
-    private static Cache<Path, FileInfo> footerCache;
+
+    // We store all caches in variables to change the main one based on config.
+    // This is not thread safe between different split generations (and wasn't anyway).
+    private static FooterCache footerCache;
+    private static LocalCache localCache;
+    private static MetastoreCache metaCache;
     private static ExecutorService threadPool = null;
     private final int numBuckets;
     private final long maxSize;
@@ -490,13 +502,26 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
                   .setNameFormat("ORC_GET_SPLITS #%d").build());
         }
 
-        if (footerCache == null && cacheStripeDetails) {
-          footerCache = CacheBuilder.newBuilder()
-              .concurrencyLevel(numThreads)
-              .initialCapacity(cacheStripeDetailsSize)
-              .maximumSize(cacheStripeDetailsSize)
-              .softValues()
-              .build();
+        // TODO: local cache is created once, so the configs for future queries will not be honored.
+        if (cacheStripeDetails) {
+          // Note that there's no FS check here; we implicitly only use metastore cache for
+          // HDFS, because only HDFS would return fileIds for us. If fileId is extended using
+          // size/mod time/etc. for other FSes, we might need to check FSes explicitly because
+          // using such an aggregate fileId cache is not bulletproof and should be disable-able.
+          boolean useMetastoreCache = HiveConf.getBoolVar(
+              conf, HiveConf.ConfVars.HIVE_ORC_MS_FOOTER_CACHE_ENABLED);
+          if (localCache == null) {
+            localCache = new LocalCache(numThreads, cacheStripeDetailsSize);
+          }
+          if (useMetastoreCache) {
+            if (metaCache == null) {
+              metaCache = new MetastoreCache(localCache);
+            }
+            assert conf instanceof HiveConf;
+            metaCache.configure((HiveConf)conf);
+          }
+          // Set footer cache for current split generation. See field comment - not thread safe.
+          footerCache = useMetastoreCache ? metaCache : localCache;
         }
       }
       String value = conf.get(ValidTxnList.VALID_TXNS_KEY,
@@ -591,49 +616,37 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
       this.covered = covered;
     }
 
-    private FileInfo verifyCachedFileInfo(FileStatus file) {
-      FileInfo fileInfo = Context.footerCache.getIfPresent(file.getPath());
-      if (fileInfo != null) {
-        if (isDebugEnabled) {
-          LOG.debug("Info cached for path: " + file.getPath());
-        }
-        if (fileInfo.modificationTime == file.getModificationTime() &&
-            fileInfo.size == file.getLen()) {
-          // Cached copy is valid
-          context.cacheHitCounter.incrementAndGet();
-          return fileInfo;
-        } else {
-          // Invalidate
-          Context.footerCache.invalidate(file.getPath());
-          if (isDebugEnabled) {
-            LOG.debug("Meta-Info for : " + file.getPath() +
-                " changed. CachedModificationTime: "
-                + fileInfo.modificationTime + ", CurrentModificationTime: "
-                + file.getModificationTime()
-                + ", CachedLength: " + fileInfo.size + ", CurrentLength: " +
-                file.getLen());
-          }
-        }
-      } else {
-        if (isDebugEnabled) {
-          LOG.debug("Info not cached for path: " + file.getPath());
-        }
-      }
-      return null;
-    }
-
     @Override
     public List<SplitInfo> getSplits() throws IOException {
-      List<SplitInfo> result = Lists.newArrayList();
-      for (HdfsFileStatusWithId file : files) {
-        FileInfo info = null;
-        if (context.cacheStripeDetails) {
-          info = verifyCachedFileInfo(file.getFileStatus());
+      List<SplitInfo> result = new ArrayList<>(files.size());
+      // TODO: Right now, we do the metastore call here, so there will be a metastore call per
+      //       partition. If we had a sync point after getting file lists, we could make just one
+      //       call; this might be too much sync for many partitions and also cause issues with the
+      //       huge metastore call result that cannot be handled with in-API batching. To have an
+      //       optimal number of metastore calls, we should wait for batch-size number of files (a
+      //       few hundreds) to become available, then call metastore.
+      if (context.cacheStripeDetails) {
+        FileInfo[] infos = Context.footerCache.getAndValidate(files);
+        for (int i = 0; i < files.size(); ++i) {
+          FileInfo info = infos[i];
+          if (info != null) {
+            // Cached copy is valid
+            context.cacheHitCounter.incrementAndGet();
+          }
+          HdfsFileStatusWithId file = files.get(i);
+          // ignore files of 0 length
+          if (file.getFileStatus().getLen() > 0) {
+            result.add(new SplitInfo(
+                context, fs, file, info, isOriginal, deltas, true, dir, covered));
+          }
         }
-        // ignore files of 0 length
-        if (file.getFileStatus().getLen() > 0) {
-          result.add(new SplitInfo(
-              context, fs, file, info, isOriginal, deltas, true, dir, covered));
+      } else {
+        for (HdfsFileStatusWithId file : files) {
+          // ignore files of 0 length
+          if (file.getFileStatus().getLen() > 0) {
+            result.add(new SplitInfo(
+                context, fs, file, null, isOriginal, deltas, true, dir, covered));
+          }
         }
       }
       return result;
@@ -808,7 +821,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
       this.file = this.fileWithId.getFileStatus();
       this.blockSize = this.file.getBlockSize();
       this.fileInfo = splitInfo.fileInfo;
-      locations = SHIMS.getLocationsWithOffset(fs, file);
+      locations = SHIMS.getLocationsWithOffset(fs, file); // TODO: potential DFS call
       this.isOriginal = splitInfo.isOriginal;
       this.deltas = splitInfo.deltas;
       this.hasBase = splitInfo.hasBase;
@@ -990,41 +1003,51 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     }
 
     private void populateAndCacheStripeDetails() throws IOException {
-      Reader orcReader = OrcFile.createReader(file.getPath(),
-          OrcFile.readerOptions(context.conf).filesystem(fs));
+      // Only create OrcReader if we are missing some information.
+      OrcProto.Footer footer;
       if (fileInfo != null) {
         stripes = fileInfo.stripeInfos;
         fileMetaInfo = fileInfo.fileMetaInfo;
         metadata = fileInfo.metadata;
-        types = fileInfo.types;
+        types = fileInfo.footer.getTypesList();
         writerVersion = fileInfo.writerVersion;
+        footer = fileInfo.footer;
         // For multiple runs, in case sendSplitsInFooter changes
         if (fileMetaInfo == null && context.footerInSplits) {
+          Reader orcReader = createOrcReader();
           fileInfo.fileMetaInfo = ((ReaderImpl) orcReader).getFileMetaInfo();
-          fileInfo.metadata = orcReader.getMetadata();
-          fileInfo.types = orcReader.getTypes();
-          fileInfo.writerVersion = orcReader.getWriterVersion();
+          assert fileInfo.metadata != null && fileInfo.footer != null
+              && fileInfo.writerVersion != null;
+          footer = fileInfo.footer;
+          // We assume that if we needed to create a reader, we need to cache it to meta cache.
+          // TODO: This will also needlessly overwrite it in local cache for now.
+          Context.footerCache.put(fileWithId.getFileId(), file, fileInfo.fileMetaInfo, orcReader);
         }
       } else {
+        Reader orcReader = createOrcReader();
         stripes = orcReader.getStripes();
         metadata = orcReader.getMetadata();
         types = orcReader.getTypes();
         writerVersion = orcReader.getWriterVersion();
         fileMetaInfo = context.footerInSplits ?
             ((ReaderImpl) orcReader).getFileMetaInfo() : null;
+        footer = orcReader.getFooter();
         if (context.cacheStripeDetails) {
-          // Populate into cache.
-          Context.footerCache.put(file.getPath(),
-              new FileInfo(file.getModificationTime(), file.getLen(), stripes,
-                  metadata, types, fileMetaInfo, writerVersion));
+          Long fileId = fileWithId.getFileId();
+          Context.footerCache.put(fileId, file, fileMetaInfo, orcReader);
         }
       }
       includedCols = genIncludedColumns(types, context.conf, isOriginal);
-      projColsUncompressedSize = computeProjectionSize(orcReader, includedCols, isOriginal);
+      projColsUncompressedSize = computeProjectionSize(footer, includedCols, isOriginal);
+    }
+
+    private Reader createOrcReader() throws IOException {
+      return OrcFile.createReader(file.getPath(),
+          OrcFile.readerOptions(context.conf).filesystem(fs));
     }
 
-    private long computeProjectionSize(final Reader orcReader, final boolean[] includedCols,
-        final boolean isOriginal) {
+    private long computeProjectionSize(
+        OrcProto.Footer footer, final boolean[] includedCols, final boolean isOriginal) {
       final int rootIdx = getRootColumn(isOriginal);
       List<Integer> internalColIds = Lists.newArrayList();
       if (includedCols != null) {
@@ -1034,7 +1057,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
           }
         }
       }
-      return orcReader.getRawDataSizeFromColIndices(internalColIds);
+      return ReaderImpl.getRawDataSizeFromColIndices(internalColIds, footer);
     }
   }
 
@@ -1045,7 +1068,11 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
 
   static List<OrcSplit> generateSplitsInfo(Configuration conf, int numSplits)
       throws IOException {
-    // use threads to resolve directories into splits
+    // Use threads to resolve directories into splits.
+    if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_ORC_MS_FOOTER_CACHE_ENABLED)) {
+      // Create HiveConf once, since this is expensive.
+      conf = new HiveConf(conf, OrcInputFormat.class);
+    }
     Context context = new Context(conf, numSplits);
     List<OrcSplit> splits = Lists.newArrayList();
     List<Future<AcidDirInfo>> pathFutures = Lists.newArrayList();
@@ -1072,6 +1099,8 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
           LOG.debug(splitStrategy);
         }
 
+        // Hack note - different split strategies return differently typed lists, yay Java.
+        // This works purely by magic, because we know which strategy produces which type.
         if (splitStrategy instanceof ETLSplitStrategy) {
           List<SplitInfo> splitInfos = ((ETLSplitStrategy)splitStrategy).getSplits();
           for (SplitInfo splitInfo : splitInfos) {
@@ -1134,26 +1163,28 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
    *
    */
   private static class FileInfo {
-    long modificationTime;
-    long size;
-    List<StripeInformation> stripeInfos;
-    ReaderImpl.FileMetaInfo fileMetaInfo;
-    Metadata metadata;
-    List<OrcProto.Type> types;
+    private final long modificationTime;
+    private final long size;
+    private final Long fileId;
+    private final List<StripeInformation> stripeInfos;
+    private ReaderImpl.FileMetaInfo fileMetaInfo;
+    private Metadata metadata;
+    private OrcProto.Footer footer;
     private OrcFile.WriterVersion writerVersion;
 
 
     FileInfo(long modificationTime, long size,
              List<StripeInformation> stripeInfos,
-             Metadata metadata, List<OrcProto.Type> types,
+             Metadata metadata, OrcProto.Footer footer,
              ReaderImpl.FileMetaInfo fileMetaInfo,
-             OrcFile.WriterVersion writerVersion) {
+             OrcFile.WriterVersion writerVersion, Long fileId) {
       this.modificationTime = modificationTime;
       this.size = size;
+      this.fileId = fileId;
       this.stripeInfos = stripeInfos;
       this.fileMetaInfo = fileMetaInfo;
       this.metadata = metadata;
-      this.types = types;
+      this.footer = footer;
       this.writerVersion = writerVersion;
     }
   }
@@ -1513,5 +1544,186 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
         bucket, validTxnList, new Reader.Options(), deltaDirectory);
   }
 
+  /**
+   * Represents footer cache.
+   */
+  public interface FooterCache {
+    FileInfo[] getAndValidate(List<HdfsFileStatusWithId> files) throws IOException;
+    void put(Long fileId, FileStatus file, FileMetaInfo fileMetaInfo, Reader orcReader)
+        throws IOException;
+  }
+
+  /** Local footer cache using Guava. Stores convoluted Java objects. */
+  private static class LocalCache implements FooterCache {
+    private Cache<Path, FileInfo> cache;
+
+    public LocalCache(int numThreads, int cacheStripeDetailsSize) {
+      cache = CacheBuilder.newBuilder()
+        .concurrencyLevel(numThreads)
+        .initialCapacity(cacheStripeDetailsSize)
+        .maximumSize(cacheStripeDetailsSize)
+        .softValues()
+        .build();
+    }
+
+    @Override
+    public FileInfo[] getAndValidate(List<HdfsFileStatusWithId> files) {
+      // TODO: should local cache also be by fileId? Preserve the original logic for now.
+      FileInfo[] result = new FileInfo[files.size()];
+      int i = -1;
+      for (HdfsFileStatusWithId fileWithId : files) {
+        ++i;
+        FileStatus file = fileWithId.getFileStatus();
+        Path path = file.getPath();
+        Long fileId = fileWithId.getFileId();
+        FileInfo fileInfo = cache.getIfPresent(path);
+        if (isDebugEnabled) {
+          LOG.debug("Info " + (fileInfo == null ? "not " : "") + "cached for path: " + path);
+        }
+        if (fileInfo == null) continue;
+        if ((fileId != null && fileInfo.fileId != null && fileId == fileInfo.fileId)
+            || (fileInfo.modificationTime == file.getModificationTime() &&
+            fileInfo.size == file.getLen())) {
+          result[i] = fileInfo;
+          continue;
+        }
+        // Invalidate
+        cache.invalidate(path);
+        if (isDebugEnabled) {
+          LOG.debug("Meta-Info for : " + path + " changed. CachedModificationTime: "
+              + fileInfo.modificationTime + ", CurrentModificationTime: "
+              + file.getModificationTime() + ", CachedLength: " + fileInfo.size
+              + ", CurrentLength: " + file.getLen());
+        }
+      }
+      return result;
+    }
+
+    public void put(Path path, FileInfo fileInfo) {
+      cache.put(path, fileInfo);
+    }
+
+    @Override
+    public void put(Long fileId, FileStatus file, FileMetaInfo fileMetaInfo, Reader orcReader)
+        throws IOException {
+      cache.put(file.getPath(), new FileInfo(file.getModificationTime(), file.getLen(),
+          orcReader.getStripes(), orcReader.getMetadata(), orcReader.getFooter(), fileMetaInfo,
+          orcReader.getWriterVersion(), fileId));
+    }
+  }
+
+  /** Metastore-based footer cache storing serialized footers. Also has a local cache. */
+  public static class MetastoreCache implements FooterCache {
+    private final LocalCache localCache;
+    private boolean isWarnLogged = false;
+    private HiveConf conf;
+
+    public MetastoreCache(LocalCache lc) {
+      localCache = lc;
+    }
 
+    @Override
+    public FileInfo[] getAndValidate(List<HdfsFileStatusWithId> files) throws IOException {
+      // First, check the local cache.
+      FileInfo[] result = localCache.getAndValidate(files);
+      assert result.length == files.size();
+      // This is an unfortunate consequence of batching/iterating thru MS results.
+      // TODO: maybe have a direct map call for small lists if this becomes a perf issue.
+      HashMap<Long, Integer> posMap = new HashMap<>(files.size());
+      for (int i = 0; i < result.length; ++i) {
+        if (result[i] != null) continue;
+        HdfsFileStatusWithId file = files.get(i);
+        Long fileId = file.getFileId();
+        if (fileId == null) {
+          if (!isWarnLogged || isDebugEnabled) {
+            LOG.warn("Not using metastore cache because fileId is missing: "
+                + file.getFileStatus().getPath());
+            isWarnLogged = true;
+          }
+          continue;
+        }
+        posMap.put(fileId, i);
+      }
+      Iterator<Entry<Long, ByteBuffer>> iter = null;
+      Hive hive;
+      try {
+        hive = getHive();
+        iter = hive.getFileMetadata(Lists.newArrayList(posMap.keySet()), conf).iterator();
+      } catch (HiveException ex) {
+        throw new IOException(ex);
+      }
+      List<Long> corruptIds = null;
+      while (iter.hasNext()) {
+        Entry<Long, ByteBuffer> e = iter.next();
+        int ix = posMap.get(e.getKey());
+        assert result[ix] == null;
+        HdfsFileStatusWithId file = files.get(ix);
+        assert file.getFileId() == e.getKey();
+        result[ix] = createFileInfoFromMs(file, e.getValue());
+        if (result[ix] == null) {
+          if (corruptIds == null) {
+            corruptIds = new ArrayList<>();
+          }
+          corruptIds.add(file.getFileId());
+        } else {
+          localCache.put(file.getFileStatus().getPath(), result[ix]);
+        }
+      }
+      if (corruptIds != null) {
+        try {
+          hive.clearFileMetadata(corruptIds);
+        } catch (HiveException ex) {
+          LOG.error("Failed to clear corrupt cache data", ex);
+        }
+      }
+      return result;
+    }
+
+    private Hive getHive() throws HiveException {
+      // TODO: we wish we could cache the Hive object, but it's not thread safe, and each
+      //       threadlocal we "cache" would need to be reinitialized for every query. This is
+      //       a huge PITA. Hive object will be cached internally, but the compat check will be
+      //       done every time inside get().
+      return Hive.getWithFastCheck(conf);
+    }
+
+    private static FileInfo createFileInfoFromMs(
+        HdfsFileStatusWithId file, ByteBuffer bb) throws IOException {
+      FileStatus fs = file.getFileStatus();
+      ReaderImpl.FooterInfo fi = null;
+      ByteBuffer original = bb.duplicate();
+      try {
+        fi = ReaderImpl.extractMetaInfoFromFooter(bb, fs.getPath());
+      } catch (Exception ex) {
+        byte[] data = new byte[original.remaining()];
+        System.arraycopy(original.array(), original.arrayOffset() + original.position(),
+            data, 0, data.length);
+        String msg = "Failed to parse the footer stored in cache for file ID "
+            + file.getFileId() + " " + original + " [ " + Hex.encodeHexString(data) + " ]";
+        LOG.error(msg, ex);
+        return null;
+      }
+      return new FileInfo(fs.getModificationTime(), fs.getLen(), fi.getStripes(),
+          fi.getMetadata(), fi.getFooter(), fi.getFileMetaInfo(),
+          fi.getFileMetaInfo().writerVersion, file.getFileId());
+    }
+
+    @Override
+    public void put(Long fileId, FileStatus file, FileMetaInfo fileMetaInfo, Reader orcReader)
+        throws IOException {
+      localCache.put(fileId, file, fileMetaInfo, orcReader);
+      if (fileId != null) {
+        try {
+          getHive().putFileMetadata(Lists.newArrayList(fileId),
+              Lists.newArrayList(((ReaderImpl)orcReader).getSerializedFileFooter()));
+        } catch (HiveException e) {
+          throw new IOException(e);
+        }
+      }
+    }
+
+    public void configure(HiveConf queryConfig) {
+      this.conf = queryConfig;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
index cc03df7..8bcda75 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
@@ -153,7 +153,7 @@ public class OrcSplit extends FileSplit {
     }
   }
 
-  ReaderImpl.FileMetaInfo getFileMetaInfo(){
+  public ReaderImpl.FileMetaInfo getFileMetaInfo() {
     return fileMetaInfo;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
index 3bac48a..a36027e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
@@ -233,8 +233,7 @@ public class ReaderImpl implements Reader {
       throw new FileFormatException("Malformed ORC file " + path +
           ". Invalid postscript length " + psLen);
     }
-    int offset = buffer.arrayOffset() + buffer.position() + buffer.limit() - 1
-        - len;
+    int offset = buffer.arrayOffset() + buffer.position() + buffer.limit() - 1 - len;
     byte[] array = buffer.array();
     // now look for the magic string at the end of the postscript.
     if (!Text.decode(array, offset, len).equals(OrcFile.MAGIC)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/bc1c434f/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 8efbb05..3ea8e25 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
@@ -103,6 +103,7 @@ import org.apache.thrift.TException;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -241,15 +242,32 @@ public class Hive {
    *
    */
   public static Hive get(HiveConf c) throws HiveException {
+    return getInternal(c, false);
+  }
+
+  /**
+   * Same as {@link #get(HiveConf)}, except that it checks only the object identity of existing
+   * MS client, assuming the relevant settings would be unchanged within the same conf object.
+   */
+  public static Hive getWithFastCheck(HiveConf c) throws HiveException {
+    return getInternal(c, true);
+  }
+
+  private static Hive getInternal(HiveConf c, boolean isFastCheck) throws HiveException {
     Hive db = hiveDB.get();
     if (db == null || !db.isCurrentUserOwner() ||
-        (db.metaStoreClient != null && !db.metaStoreClient.isCompatibleWith(c))) {
+        (db.metaStoreClient != null && !isCompatible(db, c, isFastCheck))) {
       return get(c, true);
     }
     db.conf = c;
     return db;
   }
 
+  private static boolean isCompatible(Hive db, HiveConf c, boolean isFastCheck) {
+    return isFastCheck
+        ? db.metaStoreClient.isSameConfObj(c) : db.metaStoreClient.isCompatibleWith(c);
+  }
+
   /**
    * get a connection to metastore. see get(HiveConf) function for comments
    *
@@ -3339,4 +3357,28 @@ private void constructOneLBLocationMap(FileStatus fSta,
     return true;
   }
 
+  public Iterable<Map.Entry<Long, ByteBuffer>> getFileMetadata(
+      List<Long> fileIds, Configuration conf) throws HiveException {
+    try {
+      return getMSC().getFileMetadata(fileIds);
+    } catch (TException e) {
+      throw new HiveException(e);
+    }
+  }
+
+  public void clearFileMetadata(List<Long> fileIds) throws HiveException {
+    try {
+      getMSC().clearFileMetadata(fileIds);
+    } catch (TException e) {
+      throw new HiveException(e);
+    }
+  }
+
+  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) throws HiveException {
+    try {
+      getMSC().putFileMetadata(fileIds, metadata);
+    } catch (TException e) {
+      throw new HiveException(e);
+    }
+  }
 };


[03/23] hive git commit: HIVE-11998 - Improve Compaction process logging (Eugene Koifman, reviewed by Jason Dere)

Posted by se...@apache.org.
HIVE-11998 - Improve Compaction process logging (Eugene Koifman, reviewed by Jason Dere)


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

Branch: refs/heads/llap
Commit: a1bac802a21efef8a2c10d616b2aeb680ffedd9c
Parents: b1eb0c0
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Fri Oct 2 10:12:29 2015 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Fri Oct 2 10:12:29 2015 -0700

----------------------------------------------------------------------
 .../metastore/txn/CompactionTxnHandler.java     | 36 ++++++++++++++------
 .../hive/ql/txn/AcidHouseKeeperService.java     |  5 +--
 .../hadoop/hive/ql/txn/compactor/Cleaner.java   |  3 +-
 .../hive/ql/txn/compactor/CompactorThread.java  |  9 ++---
 .../hadoop/hive/ql/txn/compactor/Initiator.java |  5 ++-
 5 files changed, 38 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a1bac802/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
index 328a65c..44ee5c6 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
@@ -122,8 +122,9 @@ public class CompactionTxnHandler extends TxnHandler {
         stmt = dbConn.createStatement();
         String s = "update COMPACTION_QUEUE set cq_run_as = '" + user + "' where cq_id = " + cq_id;
         LOG.debug("Going to execute update <" + s + ">");
-        if (stmt.executeUpdate(s) != 1) {
-          LOG.error("Unable to update compaction record");
+        int updCnt = stmt.executeUpdate(s);
+        if (updCnt != 1) {
+          LOG.error("Unable to set cq_run_as=" + user + " for compaction record with cq_id=" + cq_id + ".  updCnt=" + updCnt);
           LOG.debug("Going to rollback");
           dbConn.rollback();
         }
@@ -182,8 +183,10 @@ public class CompactionTxnHandler extends TxnHandler {
         s = "update COMPACTION_QUEUE set cq_worker_id = '" + workerId + "', " +
           "cq_start = " + now + ", cq_state = '" + WORKING_STATE + "' where cq_id = " + info.id;
         LOG.debug("Going to execute update <" + s + ">");
-        if (stmt.executeUpdate(s) != 1) {
-          LOG.error("Unable to update compaction record");
+        int updCount = stmt.executeUpdate(s);
+        if (updCount != 1) {
+          LOG.error("Unable to set to cq_state=" + WORKING_STATE + " for compaction record: " +
+            info + ". updCnt=" + updCount);
           LOG.debug("Going to rollback");
           dbConn.rollback();
         }
@@ -221,8 +224,9 @@ public class CompactionTxnHandler extends TxnHandler {
         String s = "update COMPACTION_QUEUE set cq_state = '" + READY_FOR_CLEANING + "', " +
           "cq_worker_id = null where cq_id = " + info.id;
         LOG.debug("Going to execute update <" + s + ">");
-        if (stmt.executeUpdate(s) != 1) {
-          LOG.error("Unable to update compaction record");
+        int updCnt = stmt.executeUpdate(s);
+        if (updCnt != 1) {
+          LOG.error("Unable to set cq_state=" + READY_FOR_CLEANING + " for compaction record: " + info + ". updCnt=" + updCnt);
           LOG.debug("Going to rollback");
           dbConn.rollback();
         }
@@ -298,6 +302,17 @@ public class CompactionTxnHandler extends TxnHandler {
   /**
    * This will remove an entry from the queue after
    * it has been compacted.
+   * 
+   * todo: possibly a problem?  Worker will start with DB in state X (wrt this partition).
+   * while it's working more txns will happen, against partition it's compacting.
+   * then this will delete state up to X and since then.  There may be new delta files created
+   * between compaction starting and cleaning.  These will not be compacted until more
+   * transactions happen.  So this ideally should only delete
+   * up to TXN_ID that was compacted (i.e. HWM in Worker?)  Then this can also run
+   * at READ_COMMITTED
+   * 
+   * Also, by using this method when Worker fails, we prevent future compactions from
+   * running until more data is written to tale or compaction is invoked explicitly
    * @param info info on the compaction entry to remove
    */
   public void markCleaned(CompactionInfo info) throws MetaException {
@@ -309,8 +324,9 @@ public class CompactionTxnHandler extends TxnHandler {
         stmt = dbConn.createStatement();
         String s = "delete from COMPACTION_QUEUE where cq_id = " + info.id;
         LOG.debug("Going to execute update <" + s + ">");
-        if (stmt.executeUpdate(s) != 1) {
-          LOG.error("Unable to delete compaction record");
+        int updCount = stmt.executeUpdate(s);
+        if (updCount != 1) {
+          LOG.error("Unable to delete compaction record: " + info +  ".  Update count=" + updCount);
           LOG.debug("Going to rollback");
           dbConn.rollback();
         }
@@ -348,7 +364,7 @@ public class CompactionTxnHandler extends TxnHandler {
             else buf.append(", ");
             buf.append(id);
           }
-
+          //because 1 txn may include different partitions/tables even in auto commit mode
           buf.append(") and tc_database = '");
           buf.append(info.dbname);
           buf.append("' and tc_table = '");
@@ -415,7 +431,7 @@ public class CompactionTxnHandler extends TxnHandler {
           String bufStr = buf.toString();
           LOG.debug("Going to execute update <" + bufStr + ">");
           int rc = stmt.executeUpdate(bufStr);
-          LOG.debug("Removed " + rc + " records from txns");
+          LOG.info("Removed " + rc + "  empty Aborted transactions: " + txnids + " from TXNS");
           LOG.debug("Going to commit");
           dbConn.commit();
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/a1bac802/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidHouseKeeperService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidHouseKeeperService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidHouseKeeperService.java
index d22ca8d..23a77e6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidHouseKeeperService.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidHouseKeeperService.java
@@ -84,9 +84,10 @@ public class AcidHouseKeeperService implements HouseKeeperService {
     @Override
     public void run() {
       try {
+        long startTime = System.currentTimeMillis();
         txnHandler.performTimeOuts();
-        owner.isAliveCounter.incrementAndGet();
-        LOG.info("timeout reaper ran");
+        int count = owner.isAliveCounter.incrementAndGet();
+        LOG.info("timeout reaper ran for " + (System.currentTimeMillis() - startTime)/1000 + "seconds.  isAliveCounter=" + count);
       }
       catch(Throwable t) {
         LOG.fatal("Serious error in " + Thread.currentThread().getName() + ": " + t.getMessage(), t);

http://git-wip-us.apache.org/repos/asf/hive/blob/a1bac802/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
index 16d2c81..622bf54 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
@@ -212,7 +212,7 @@ public class Cleaner extends CompactorThread {
       if (runJobAsSelf(ci.runAs)) {
         removeFiles(location, txnList);
       } else {
-        LOG.info("Cleaning as user " + ci.runAs);
+        LOG.info("Cleaning as user " + ci.runAs + " for " + ci.getFullPartitionName());
         UserGroupInformation ugi = UserGroupInformation.createProxyUser(ci.runAs,
             UserGroupInformation.getLoginUser());
         ugi.doAs(new PrivilegedExceptionAction<Object>() {
@@ -245,6 +245,7 @@ public class Cleaner extends CompactorThread {
           ", that hardly seems right.");
       return;
     }
+    LOG.info("About to remove " + filesToDelete.size() + " obsolete directories from " + location);
     FileSystem fs = filesToDelete.get(0).getFileSystem(conf);
 
     for (Path dead : filesToDelete) {

http://git-wip-us.apache.org/repos/asf/hive/blob/a1bac802/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
index 38cd95e..c956f58 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
@@ -119,8 +119,8 @@ abstract class CompactorThread extends Thread implements MetaStoreThread {
         throw e;
       }
       if (parts.size() != 1) {
-        LOG.error(ci.getFullPartitionName() + " does not refer to a single partition");
-        throw new MetaException("Too many partitions");
+        LOG.error(ci.getFullPartitionName() + " does not refer to a single partition. " + parts);
+        throw new MetaException("Too many partitions for : " + ci.getFullPartitionName());
       }
       return parts.get(0);
     } else {
@@ -179,8 +179,9 @@ abstract class CompactorThread extends Thread implements MetaStoreThread {
         return wrapper.get(0);
       }
     }
-    LOG.error("Unable to stat file as either current user or table owner, giving up");
-    throw new IOException("Unable to stat file");
+    LOG.error("Unable to stat file " + p + " as either current user(" + UserGroupInformation.getLoginUser() +
+      ") or table owner(" + t.getOwner() + "), giving up");
+    throw new IOException("Unable to stat file: " + p);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/a1bac802/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
index 9bf725d..f265311 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
@@ -99,7 +99,7 @@ public class Initiator extends CompactorThread {
 
               // check if no compaction set for this table
               if (noAutoCompactSet(t)) {
-                LOG.info("Table " + tableName(t) + " marked true so we will not compact it.");
+                LOG.info("Table " + tableName(t) + " marked " + hive_metastoreConstants.TABLE_NO_AUTO_COMPACT + "=true so we will not compact it.");
                 continue;
               }
 
@@ -297,11 +297,10 @@ public class Initiator extends CompactorThread {
   }
 
   private void requestCompaction(CompactionInfo ci, String runAs, CompactionType type) throws MetaException {
-    String s = "Requesting " + type.toString() + " compaction for " + ci.getFullPartitionName();
-    LOG.info(s);
     CompactionRequest rqst = new CompactionRequest(ci.dbname, ci.tableName, type);
     if (ci.partName != null) rqst.setPartitionname(ci.partName);
     rqst.setRunas(runAs);
+    LOG.info("Requesting compaction: " + rqst);
     txnHandler.compact(rqst);
   }
 


[19/23] hive git commit: HIVE-12007 : Hive LDAP Authenticator should allow just Domain without baseDN (for AD) (Naveen Gangam via Szehon)

Posted by se...@apache.org.
HIVE-12007 : Hive LDAP Authenticator should allow just Domain without baseDN (for AD) (Naveen Gangam via Szehon)


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

Branch: refs/heads/llap
Commit: a989f697605276bf3f47419c0237391cda3b9e3c
Parents: 7c164ac
Author: Szehon Ho <sz...@cloudera.com>
Authored: Mon Oct 5 10:11:47 2015 -0700
Committer: Szehon Ho <sz...@cloudera.com>
Committed: Mon Oct 5 10:12:18 2015 -0700

----------------------------------------------------------------------
 .../auth/LdapAuthenticationProviderImpl.java     | 19 ++++++++++++-------
 1 file changed, 12 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a989f697/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 b2c4daf..f2a4a5b 100644
--- a/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
+++ b/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
@@ -77,7 +77,7 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
             LOG.warn("Unexpected format for groupDNPattern..ignoring " + groupTokens[i]);
           }
         }
-      } else {
+      } else if (baseDN != null) {
         groupBases.add("CN=%s," + baseDN);
       }
 
@@ -101,7 +101,7 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
             LOG.warn("Unexpected format for userDNPattern..ignoring " + userTokens[i]);
           }
         }
-      } else {
+      } else if (baseDN != null) {
         userBases.add("CN=%s," + baseDN);
       }
 
@@ -151,22 +151,22 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
       // Create initial context
       ctx = new InitialDirContext(env);
 
-      if (isDN(user)) {
+      if (isDN(user) || hasDomain(user)) {
         userName = extractName(user);
       } else {
         userName = user;
       }
 
-      if (userFilter == null && groupFilter == null && customQuery == null) {
+      if (userFilter == null && groupFilter == null && customQuery == null && userBases.size() > 0) {
         if (isDN(user)) {
-          userDN = findUserDNByDN(ctx, user);
+          userDN = findUserDNByDN(ctx, userName);
         } else {
           if (userDN == null) {
-            userDN = findUserDNByPattern(ctx, user);
+            userDN = findUserDNByPattern(ctx, userName);
           }
 
           if (userDN == null) {
-            userDN = findUserDNByName(ctx, baseDN, user);
+            userDN = findUserDNByName(ctx, baseDN, userName);
           }
         }
 
@@ -564,6 +564,11 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
   }
 
   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(","));
     }


[22/23] hive git commit: HIVE-12036 : LLAP: merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-12036 : LLAP: merge master into branch (Sergey Shelukhin)


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

Branch: refs/heads/llap
Commit: b28aec9a0f5e2507f05f9bb4c235bb6136bb5df2
Parents: c5ccf66 0ca9ff8
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Oct 5 12:42:39 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Oct 5 12:42:39 2015 -0700

----------------------------------------------------------------------
 .../apache/hive/beeline/HiveSchemaHelper.java   |     4 +-
 .../hadoop/hive/common/StatsSetupConst.java     |     8 -
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    20 +-
 .../hive/common/type/TestHiveDecimal.java       |    12 +-
 data/files/decimal_1_1.txt                      |    30 +
 data/files/sample2.json                         |     2 +
 .../hadoop/hive/hbase/HBaseStatsAggregator.java |   128 -
 .../hadoop/hive/hbase/HBaseStatsPublisher.java  |   154 -
 .../hive/hbase/HBaseStatsSetupConstants.java    |    34 -
 .../hadoop/hive/hbase/HBaseStatsUtils.java      |   135 -
 .../src/test/queries/positive/hbase_stats.q     |    30 -
 .../src/test/queries/positive/hbase_stats2.q    |    31 -
 .../positive/hbase_stats_empty_partition.q      |    13 -
 .../src/test/results/positive/hbase_stats.q.out |   311 -
 .../test/results/positive/hbase_stats2.q.out    |   311 -
 .../positive/hbase_stats_empty_partition.q.out  |    63 -
 hcatalog/conf/proto-hive-site.xml               |     2 +-
 .../templates/conf/hive-site.xml.template       |     2 +-
 hcatalog/streaming/pom.xml                      |     7 +
 .../streaming/AbstractRecordWriter.java         |    93 +-
 .../streaming/DelimitedInputWriter.java         |    54 +-
 .../hcatalog/streaming/StrictJsonWriter.java    |    46 +-
 .../hive/hcatalog/streaming/TestStreaming.java  |   698 +-
 .../test/resources/testconfiguration.properties |     1 +
 .../hadoop/hive/metastore/HiveAlterHandler.java |     2 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |    30 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    94 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    20 +
 .../hadoop/hive/metastore/MetaStoreUtils.java   |    40 +-
 .../apache/hadoop/hive/metastore/Warehouse.java |     4 +-
 .../hive/metastore/hbase/HBaseReadWrite.java    |     9 +-
 .../metastore/txn/CompactionTxnHandler.java     |    36 +-
 .../metastore/AlternateFailurePreListener.java  |     2 +-
 pom.xml                                         |     3 +-
 ql/pom.xml                                      |     3 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  |     2 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |     8 +-
 .../hive/ql/exec/tez/HashTableLoader.java       |    25 +-
 .../hive/ql/exec/tez/HiveSplitGenerator.java    |     4 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   366 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |     3 +-
 .../serde/ArrayWritableObjectInspector.java     |     7 +
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |     3 +-
 .../hadoop/hive/ql/lockmgr/HiveLockObject.java  |     6 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    48 +-
 .../apache/hadoop/hive/ql/metadata/Table.java   |     5 +-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |     4 +
 .../optimizer/SortedDynPartitionOptimizer.java  |     7 +-
 .../correlation/CorrelationUtilities.java       |    33 -
 .../RewriteQueryUsingAggregateIndexCtx.java     |     2 +-
 .../ql/parse/ColumnStatsSemanticAnalyzer.java   |     2 +-
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |    12 -
 .../hive/ql/txn/AcidHouseKeeperService.java     |     5 +-
 .../hadoop/hive/ql/txn/compactor/Cleaner.java   |     3 +-
 .../hive/ql/txn/compactor/CompactorMR.java      |    10 +-
 .../hive/ql/txn/compactor/CompactorThread.java  |     9 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |     5 +-
 .../hadoop/hive/ql/metadata/StringAppender.java |     2 +-
 .../hadoop/hive/ql/metadata/TestHive.java       |     2 +-
 .../special_character_in_tabnames_1.q           |    13 +
 ql/src/test/queries/clientpositive/cross_join.q |     8 +
 .../test/queries/clientpositive/decimal_1_1.q   |     9 +
 .../dynpart_sort_opt_vectorization.q            |     2 -
 .../clientpositive/dynpart_sort_optimization.q  |     2 -
 .../clientpositive/dynpart_sort_optimization2.q |     2 -
 .../queries/clientpositive/load_non_hdfs_path.q |     6 +
 .../parquet_mixed_partition_formats2.q          |    31 +
 .../special_character_in_tabnames_1.q           |  1075 +
 .../special_character_in_tabnames_2.q           |    40 +
 .../special_character_in_tabnames_1.q.out       |    10 +
 .../results/clientpositive/cross_join.q.out     |   196 +
 .../results/clientpositive/decimal_1_1.q.out    |   104 +
 .../clientpositive/load_non_hdfs_path.q.out     |    16 +
 .../parquet_mixed_partition_formats2.q.out      |    99 +
 .../clientpositive/spark/cross_join.q.out       |   211 +
 .../clientpositive/spark/decimal_1_1.q.out      |   104 +
 .../special_character_in_tabnames_1.q.out       | 19550 +++++++++++++++++
 .../special_character_in_tabnames_2.q.out       |   304 +
 .../results/clientpositive/tez/auto_join0.q.out |     1 -
 .../tez/auto_sortmerge_join_12.q.out            |     1 -
 .../results/clientpositive/tez/cross_join.q.out |   187 +
 .../tez/cross_product_check_2.q.out             |     6 -
 .../tez/dynamic_partition_pruning.q.out         |     1 -
 .../vectorized_dynamic_partition_pruning.q.out  |     1 -
 .../objectinspector/ObjectInspectorUtils.java   |    29 +
 service/pom.xml                                 |     6 +
 .../auth/LdapAuthenticationProviderImpl.java    |    19 +-
 .../cli/operation/LogDivertAppender.java        |     2 +-
 .../auth/TestLdapAtnProviderWithLdapServer.java |   215 +
 .../org/apache/hive/service/auth/ldapdata.ldif  |    59 +
 .../hadoop/hive/common/type/HiveDecimal.java    |    10 +-
 .../hive/ptest/execution/JIRAService.java       |   115 +-
 .../org/apache/hive/ptest/execution/PTest.java  |    11 +-
 .../hive/ptest/execution/TestCheckPhase.java    |    77 +
 .../ptest/execution/TestTestCheckPhase.java     |    91 +
 .../src/test/resources/HIVE-10761.6.patch       |  2539 +++
 .../src/test/resources/HIVE-11271.4.patch       |   606 +
 .../ptest2/src/test/resources/HIVE-9377.1.patch |    25 +
 .../ptest2/src/test/resources/remove-test.patch |    33 +
 99 files changed, 27175 insertions(+), 1646 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/hive/blob/b28aec9a/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/b28aec9a/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/b28aec9a/ql/pom.xml
----------------------------------------------------------------------
diff --cc ql/pom.xml
index 99c22a3,587e2ee..83b9ebf
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@@ -709,7 -704,7 +709,8 @@@
                    <include>org.apache.hive:hive-common</include>
                    <include>org.apache.hive:hive-exec</include>
                    <include>org.apache.hive:hive-serde</include>
 +                  <include>org.apache.hive:hive-llap-client</include>
+                   <include>org.apache.hive:hive-metastore</include>
                    <include>com.esotericsoftware.kryo:kryo</include>
                    <include>org.apache.parquet:parquet-hadoop-bundle</include>
                    <include>org.apache.thrift:libthrift</include>

http://git-wip-us.apache.org/repos/asf/hive/blob/b28aec9a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/b28aec9a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/b28aec9a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index 8941db1,ef62a23..6ffec30
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@@ -108,10 -114,10 +114,10 @@@ import com.google.common.util.concurren
   * delete events have null for row.
   */
  public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
 -  InputFormatChecker, VectorizedInputFormatInterface,
 +  InputFormatChecker, VectorizedInputFormatInterface, LlapWrappableInputFormatInterface,
      AcidInputFormat<NullWritable, OrcStruct>, CombineHiveInputFormat.AvoidSplitCombination {
  
-   static enum SplitStrategyKind{
+   static enum SplitStrategyKind {
      HYBRID,
      BI,
      ETL
@@@ -811,7 -821,7 +823,8 @@@
        this.file = this.fileWithId.getFileStatus();
        this.blockSize = this.file.getBlockSize();
        this.fileInfo = splitInfo.fileInfo;
-       locations = SHIMS.getLocationsWithOffset(fs, fileWithId.getFileStatus());
 -      locations = SHIMS.getLocationsWithOffset(fs, file); // TODO: potential DFS call
++      // TODO: potential DFS call
++      this.locations = SHIMS.getLocationsWithOffset(fs, fileWithId.getFileStatus());
        this.isOriginal = splitInfo.isOriginal;
        this.deltas = splitInfo.deltas;
        this.hasBase = splitInfo.hasBase;
@@@ -993,41 -1003,51 +1006,51 @@@
      }
  
      private void populateAndCacheStripeDetails() throws IOException {
-       Reader orcReader = OrcFile.createReader(fileWithId.getFileStatus().getPath(),
-           OrcFile.readerOptions(context.conf).filesystem(fs));
+       // Only create OrcReader if we are missing some information.
 -      OrcProto.Footer footer;
++      List<OrcProto.ColumnStatistics> colStatsLocal;
++      List<OrcProto.Type> typesLocal;
        if (fileInfo != null) {
          stripes = fileInfo.stripeInfos;
 +        stripeStats = fileInfo.stripeStats;
          fileMetaInfo = fileInfo.fileMetaInfo;
-         types = fileInfo.types;
 -        metadata = fileInfo.metadata;
 -        types = fileInfo.footer.getTypesList();
++        typesLocal = types = fileInfo.types;
++        colStatsLocal = fileInfo.fileStats;
          writerVersion = fileInfo.writerVersion;
 -        footer = fileInfo.footer;
          // For multiple runs, in case sendSplitsInFooter changes
          if (fileMetaInfo == null && context.footerInSplits) {
+           Reader orcReader = createOrcReader();
            fileInfo.fileMetaInfo = ((ReaderImpl) orcReader).getFileMetaInfo();
-           fileInfo.types = orcReader.getTypes();
-           fileInfo.writerVersion = orcReader.getWriterVersion();
 -          assert fileInfo.metadata != null && fileInfo.footer != null
++          assert fileInfo.stripeStats != null && fileInfo.types != null
+               && fileInfo.writerVersion != null;
 -          footer = fileInfo.footer;
+           // We assume that if we needed to create a reader, we need to cache it to meta cache.
+           // TODO: This will also needlessly overwrite it in local cache for now.
+           Context.footerCache.put(fileWithId.getFileId(), file, fileInfo.fileMetaInfo, orcReader);
          }
        } else {
+         Reader orcReader = createOrcReader();
          stripes = orcReader.getStripes();
 -        metadata = orcReader.getMetadata();
--        types = orcReader.getTypes();
++        typesLocal = types = orcReader.getTypes();
++        colStatsLocal = orcReader.getOrcProtoFileStatistics();
          writerVersion = orcReader.getWriterVersion();
 +        stripeStats = orcReader.getStripeStatistics();
          fileMetaInfo = context.footerInSplits ?
              ((ReaderImpl) orcReader).getFileMetaInfo() : null;
 -        footer = orcReader.getFooter();
          if (context.cacheStripeDetails) {
-           // Populate into cache.
-           Context.footerCache.put(fileWithId.getFileStatus().getPath(),
-               new FileInfo(fileWithId.getFileStatus().getModificationTime(),
-                   fileWithId.getFileStatus().getLen(), stripes,
-                   stripeStats, types, fileMetaInfo, writerVersion));
+           Long fileId = fileWithId.getFileId();
+           Context.footerCache.put(fileId, file, fileMetaInfo, orcReader);
          }
        }
        includedCols = genIncludedColumns(types, context.conf, isOriginal);
-       projColsUncompressedSize = computeProjectionSize(orcReader, includedCols, isOriginal);
 -      projColsUncompressedSize = computeProjectionSize(footer, includedCols, isOriginal);
++      projColsUncompressedSize = computeProjectionSize(typesLocal, colStatsLocal, includedCols, isOriginal);
+     }
+ 
+     private Reader createOrcReader() throws IOException {
+       return OrcFile.createReader(file.getPath(),
+           OrcFile.readerOptions(context.conf).filesystem(fs));
      }
  
-     private long computeProjectionSize(final Reader orcReader, final boolean[] includedCols,
-         final boolean isOriginal) {
 -    private long computeProjectionSize(
 -        OrcProto.Footer footer, final boolean[] includedCols, final boolean isOriginal) {
++    private long computeProjectionSize(List<OrcProto.Type> types,
++        List<OrcProto.ColumnStatistics> stats, boolean[] includedCols, boolean isOriginal) {
        final int rootIdx = getRootColumn(isOriginal);
        List<Integer> internalColIds = Lists.newArrayList();
        if (includedCols != null) {
@@@ -1037,7 -1057,7 +1060,7 @@@
            }
          }
        }
-       return orcReader.getRawDataSizeFromColIndices(internalColIds);
 -      return ReaderImpl.getRawDataSizeFromColIndices(internalColIds, footer);
++      return ReaderImpl.getRawDataSizeFromColIndices(internalColIds, types, stats);
      }
    }
  
@@@ -1048,9 -1068,12 +1071,13 @@@
  
    static List<OrcSplit> generateSplitsInfo(Configuration conf, int numSplits)
        throws IOException {
-     // use threads to resolve directories into splits
+     // Use threads to resolve directories into splits.
+     if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_ORC_MS_FOOTER_CACHE_ENABLED)) {
+       // Create HiveConf once, since this is expensive.
+       conf = new HiveConf(conf, OrcInputFormat.class);
+     }
      Context context = new Context(conf, numSplits);
 +    boolean useFileIds = HiveConf.getBoolVar(conf, ConfVars.HIVE_ORC_INCLUDE_FILE_ID_IN_SPLITS);
      List<OrcSplit> splits = Lists.newArrayList();
      List<Future<AcidDirInfo>> pathFutures = Lists.newArrayList();
      List<Future<List<OrcSplit>>> splitFutures = Lists.newArrayList();
@@@ -1137,26 -1163,28 +1166,29 @@@
     *
     */
    private static class FileInfo {
-     long modificationTime;
-     long size;
-     List<StripeInformation> stripeInfos;
-     FileMetaInfo fileMetaInfo;
-     List<StripeStatistics> stripeStats;
-     List<OrcProto.Type> types;
+     private final long modificationTime;
+     private final long size;
+     private final Long fileId;
+     private final List<StripeInformation> stripeInfos;
 -    private ReaderImpl.FileMetaInfo fileMetaInfo;
 -    private Metadata metadata;
 -    private OrcProto.Footer footer;
++    private FileMetaInfo fileMetaInfo;
++    private List<StripeStatistics> stripeStats;
++    private List<OrcProto.ColumnStatistics> fileStats;
++    private List<OrcProto.Type> types;
      private OrcFile.WriterVersion writerVersion;
  
  
--    FileInfo(long modificationTime, long size,
--             List<StripeInformation> stripeInfos,
 -             Metadata metadata, OrcProto.Footer footer,
 -             ReaderImpl.FileMetaInfo fileMetaInfo,
++    FileInfo(long modificationTime, long size, List<StripeInformation> stripeInfos,
 +             List<StripeStatistics> stripeStats, List<OrcProto.Type> types,
-              FileMetaInfo fileMetaInfo,
-              OrcFile.WriterVersion writerVersion) {
++             List<OrcProto.ColumnStatistics> fileStats, FileMetaInfo fileMetaInfo,
+              OrcFile.WriterVersion writerVersion, Long fileId) {
        this.modificationTime = modificationTime;
        this.size = size;
+       this.fileId = fileId;
        this.stripeInfos = stripeInfos;
        this.fileMetaInfo = fileMetaInfo;
 -      this.metadata = metadata;
 -      this.footer = footer;
 +      this.stripeStats = stripeStats;
 +      this.types = types;
++      this.fileStats = fileStats;
        this.writerVersion = writerVersion;
      }
    }
@@@ -1516,5 -1544,186 +1548,187 @@@
          bucket, validTxnList, new Reader.Options(), deltaDirectory);
    }
  
+   /**
+    * Represents footer cache.
+    */
+   public interface FooterCache {
+     FileInfo[] getAndValidate(List<HdfsFileStatusWithId> files) throws IOException;
+     void put(Long fileId, FileStatus file, FileMetaInfo fileMetaInfo, Reader orcReader)
+         throws IOException;
+   }
+ 
+   /** Local footer cache using Guava. Stores convoluted Java objects. */
+   private static class LocalCache implements FooterCache {
+     private Cache<Path, FileInfo> cache;
+ 
+     public LocalCache(int numThreads, int cacheStripeDetailsSize) {
+       cache = CacheBuilder.newBuilder()
+         .concurrencyLevel(numThreads)
+         .initialCapacity(cacheStripeDetailsSize)
+         .maximumSize(cacheStripeDetailsSize)
+         .softValues()
+         .build();
+     }
  
+     @Override
+     public FileInfo[] getAndValidate(List<HdfsFileStatusWithId> files) {
+       // TODO: should local cache also be by fileId? Preserve the original logic for now.
+       FileInfo[] result = new FileInfo[files.size()];
+       int i = -1;
+       for (HdfsFileStatusWithId fileWithId : files) {
+         ++i;
+         FileStatus file = fileWithId.getFileStatus();
+         Path path = file.getPath();
+         Long fileId = fileWithId.getFileId();
+         FileInfo fileInfo = cache.getIfPresent(path);
+         if (isDebugEnabled) {
+           LOG.debug("Info " + (fileInfo == null ? "not " : "") + "cached for path: " + path);
+         }
+         if (fileInfo == null) continue;
+         if ((fileId != null && fileInfo.fileId != null && fileId == fileInfo.fileId)
+             || (fileInfo.modificationTime == file.getModificationTime() &&
+             fileInfo.size == file.getLen())) {
+           result[i] = fileInfo;
+           continue;
+         }
+         // Invalidate
+         cache.invalidate(path);
+         if (isDebugEnabled) {
+           LOG.debug("Meta-Info for : " + path + " changed. CachedModificationTime: "
+               + fileInfo.modificationTime + ", CurrentModificationTime: "
+               + file.getModificationTime() + ", CachedLength: " + fileInfo.size
+               + ", CurrentLength: " + file.getLen());
+         }
+       }
+       return result;
+     }
+ 
+     public void put(Path path, FileInfo fileInfo) {
+       cache.put(path, fileInfo);
+     }
+ 
+     @Override
+     public void put(Long fileId, FileStatus file, FileMetaInfo fileMetaInfo, Reader orcReader)
+         throws IOException {
+       cache.put(file.getPath(), new FileInfo(file.getModificationTime(), file.getLen(),
 -          orcReader.getStripes(), orcReader.getMetadata(), orcReader.getFooter(), fileMetaInfo,
 -          orcReader.getWriterVersion(), fileId));
++          orcReader.getStripes(), orcReader.getStripeStatistics(), orcReader.getTypes(),
++          orcReader.getOrcProtoFileStatistics(), fileMetaInfo, orcReader.getWriterVersion(),
++          fileId));
+     }
+   }
+ 
+   /** Metastore-based footer cache storing serialized footers. Also has a local cache. */
+   public static class MetastoreCache implements FooterCache {
+     private final LocalCache localCache;
+     private boolean isWarnLogged = false;
+     private HiveConf conf;
+ 
+     public MetastoreCache(LocalCache lc) {
+       localCache = lc;
+     }
+ 
+     @Override
+     public FileInfo[] getAndValidate(List<HdfsFileStatusWithId> files) throws IOException {
+       // First, check the local cache.
+       FileInfo[] result = localCache.getAndValidate(files);
+       assert result.length == files.size();
+       // This is an unfortunate consequence of batching/iterating thru MS results.
+       // TODO: maybe have a direct map call for small lists if this becomes a perf issue.
+       HashMap<Long, Integer> posMap = new HashMap<>(files.size());
+       for (int i = 0; i < result.length; ++i) {
+         if (result[i] != null) continue;
+         HdfsFileStatusWithId file = files.get(i);
+         Long fileId = file.getFileId();
+         if (fileId == null) {
+           if (!isWarnLogged || isDebugEnabled) {
+             LOG.warn("Not using metastore cache because fileId is missing: "
+                 + file.getFileStatus().getPath());
+             isWarnLogged = true;
+           }
+           continue;
+         }
+         posMap.put(fileId, i);
+       }
+       Iterator<Entry<Long, ByteBuffer>> iter = null;
+       Hive hive;
+       try {
+         hive = getHive();
+         iter = hive.getFileMetadata(Lists.newArrayList(posMap.keySet()), conf).iterator();
+       } catch (HiveException ex) {
+         throw new IOException(ex);
+       }
+       List<Long> corruptIds = null;
+       while (iter.hasNext()) {
+         Entry<Long, ByteBuffer> e = iter.next();
+         int ix = posMap.get(e.getKey());
+         assert result[ix] == null;
+         HdfsFileStatusWithId file = files.get(ix);
+         assert file.getFileId() == e.getKey();
+         result[ix] = createFileInfoFromMs(file, e.getValue());
+         if (result[ix] == null) {
+           if (corruptIds == null) {
+             corruptIds = new ArrayList<>();
+           }
+           corruptIds.add(file.getFileId());
+         } else {
+           localCache.put(file.getFileStatus().getPath(), result[ix]);
+         }
+       }
+       if (corruptIds != null) {
+         try {
+           hive.clearFileMetadata(corruptIds);
+         } catch (HiveException ex) {
+           LOG.error("Failed to clear corrupt cache data", ex);
+         }
+       }
+       return result;
+     }
+ 
+     private Hive getHive() throws HiveException {
+       // TODO: we wish we could cache the Hive object, but it's not thread safe, and each
+       //       threadlocal we "cache" would need to be reinitialized for every query. This is
+       //       a huge PITA. Hive object will be cached internally, but the compat check will be
+       //       done every time inside get().
+       return Hive.getWithFastCheck(conf);
+     }
+ 
+     private static FileInfo createFileInfoFromMs(
+         HdfsFileStatusWithId file, ByteBuffer bb) throws IOException {
+       FileStatus fs = file.getFileStatus();
+       ReaderImpl.FooterInfo fi = null;
+       ByteBuffer original = bb.duplicate();
+       try {
+         fi = ReaderImpl.extractMetaInfoFromFooter(bb, fs.getPath());
+       } catch (Exception ex) {
+         byte[] data = new byte[original.remaining()];
+         System.arraycopy(original.array(), original.arrayOffset() + original.position(),
+             data, 0, data.length);
+         String msg = "Failed to parse the footer stored in cache for file ID "
+             + file.getFileId() + " " + original + " [ " + Hex.encodeHexString(data) + " ]";
+         LOG.error(msg, ex);
+         return null;
+       }
 -      return new FileInfo(fs.getModificationTime(), fs.getLen(), fi.getStripes(),
 -          fi.getMetadata(), fi.getFooter(), fi.getFileMetaInfo(),
++      return new FileInfo(fs.getModificationTime(), fs.getLen(), fi.getStripes(), fi.getMetadata(),
++          fi.getFooter().getTypesList(), fi.getFooter().getStatisticsList(), fi.getFileMetaInfo(),
+           fi.getFileMetaInfo().writerVersion, file.getFileId());
+     }
+ 
+     @Override
+     public void put(Long fileId, FileStatus file, FileMetaInfo fileMetaInfo, Reader orcReader)
+         throws IOException {
+       localCache.put(fileId, file, fileMetaInfo, orcReader);
+       if (fileId != null) {
+         try {
+           getHive().putFileMetadata(Lists.newArrayList(fileId),
+               Lists.newArrayList(((ReaderImpl)orcReader).getSerializedFileFooter()));
+         } catch (HiveException e) {
+           throw new IOException(e);
+         }
+       }
+     }
+ 
+     public void configure(HiveConf queryConfig) {
+       this.conf = queryConfig;
+     }
+   }
  }

http://git-wip-us.apache.org/repos/asf/hive/blob/b28aec9a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/b28aec9a/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out
----------------------------------------------------------------------


[04/23] hive git commit: HIVE-11990 : Loading data inpath from a temporary table dir fails on Windows (Hari Subramaniyan, reviewed by Jason Dere)

Posted by se...@apache.org.
HIVE-11990 : Loading data inpath from a temporary table dir fails on Windows (Hari Subramaniyan, reviewed by Jason Dere)


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

Branch: refs/heads/llap
Commit: c3d62ad94be5bf896d3e868b39f472ff6b1ce992
Parents: a1bac80
Author: Hari Subramaniyan <ha...@apache.org>
Authored: Fri Oct 2 11:11:46 2015 -0700
Committer: Hari Subramaniyan <ha...@apache.org>
Committed: Fri Oct 2 11:11:46 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/LoadSemanticAnalyzer.java  | 12 ------------
 .../queries/clientpositive/load_non_hdfs_path.q     |  6 ++++++
 .../results/clientpositive/load_non_hdfs_path.q.out | 16 ++++++++++++++++
 3 files changed, 22 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c3d62ad9/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
index 9d2702f..c488029 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
@@ -159,18 +159,6 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException(ErrorMsg.INVALID_PATH.getMsg(ast), e);
     }
 
-    // only in 'local' mode do we copy stuff from one place to another.
-    // reject different scheme/authority in other cases.
-    if (!isLocal
-        && (!StringUtils.equals(fromURI.getScheme(), toURI.getScheme()) || !StringUtils
-        .equals(fromURI.getAuthority(), toURI.getAuthority()))) {
-      String reason = "Move from: " + fromURI.toString() + " to: "
-          + toURI.toString() + " is not valid. "
-          + "Please check that values for params \"default.fs.name\" and "
-          + "\"hive.metastore.warehouse.dir\" do not conflict.";
-      throw new SemanticException(ErrorMsg.ILLEGAL_PATH.getMsg(ast, reason));
-    }
-
     return srcs;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c3d62ad9/ql/src/test/queries/clientpositive/load_non_hdfs_path.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/load_non_hdfs_path.q b/ql/src/test/queries/clientpositive/load_non_hdfs_path.q
new file mode 100644
index 0000000..824ce69
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/load_non_hdfs_path.q
@@ -0,0 +1,6 @@
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/non_hdfs_path;
+dfs -touchz ${system:test.tmp.dir}/non_hdfs_path/1.txt;
+dfs -chmod 555 ${system:test.tmp.dir}/non_hdfs_path/1.txt;
+
+create table t1(i int);
+load data inpath 'pfile:${system:test.tmp.dir}/non_hdfs_path/' overwrite into table t1;

http://git-wip-us.apache.org/repos/asf/hive/blob/c3d62ad9/ql/src/test/results/clientpositive/load_non_hdfs_path.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/load_non_hdfs_path.q.out b/ql/src/test/results/clientpositive/load_non_hdfs_path.q.out
new file mode 100644
index 0000000..31c40d8
--- /dev/null
+++ b/ql/src/test/results/clientpositive/load_non_hdfs_path.q.out
@@ -0,0 +1,16 @@
+PREHOOK: query: create table t1(i int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1
+POSTHOOK: query: create table t1(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1
+#### A masked pattern was here ####
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@t1
+#### A masked pattern was here ####
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@t1


[21/23] hive git commit: HIVE-11983 - Hive streaming API uses incorrect logic to assign buckets to incoming records (Roshan Naik via Eugene Koifman)

Posted by se...@apache.org.
HIVE-11983 - Hive streaming API uses incorrect logic to assign buckets to incoming records (Roshan Naik via Eugene Koifman)


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

Branch: refs/heads/llap
Commit: 0ca9ff86514f5e76cb0cd99022d8d5d1cf39e626
Parents: 8964c1e
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Mon Oct 5 11:06:58 2015 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Mon Oct 5 11:06:58 2015 -0700

----------------------------------------------------------------------
 hcatalog/streaming/pom.xml                      |   7 +
 .../streaming/AbstractRecordWriter.java         |  93 ++-
 .../streaming/DelimitedInputWriter.java         |  54 +-
 .../hcatalog/streaming/StrictJsonWriter.java    |  46 +-
 .../hive/hcatalog/streaming/TestStreaming.java  | 698 +++++++++++++++----
 .../objectinspector/ObjectInspectorUtils.java   |  29 +
 6 files changed, 773 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0ca9ff86/hcatalog/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/pom.xml b/hcatalog/streaming/pom.xml
index 6d03ce1..ba9f731 100644
--- a/hcatalog/streaming/pom.xml
+++ b/hcatalog/streaming/pom.xml
@@ -104,6 +104,13 @@
       <scope>test</scope>
     </dependency>
 
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-common</artifactId>
+      <scope>test</scope>
+      <version>${hadoop-23.version}</version>
+    </dependency>
+
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hive/blob/0ca9ff86/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java
index c959222..a2cd2f5 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -32,13 +33,20 @@ import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
 import org.apache.hadoop.hive.ql.io.RecordUpdater;
 import org.apache.hadoop.hive.serde2.SerDe;
 import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hive.hcatalog.common.HCatUtil;
 import org.apache.thrift.TException;
 
 import java.io.IOException;
 
-import java.util.Random;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
 
 abstract class AbstractRecordWriter implements RecordWriter {
   static final private Log LOG = LogFactory.getLog(AbstractRecordWriter.class.getName());
@@ -48,14 +56,15 @@ abstract class AbstractRecordWriter implements RecordWriter {
   final Table tbl;
 
   final IMetaStoreClient msClient;
-  RecordUpdater updater = null;
+  protected final List<Integer> bucketIds;
+  ArrayList<RecordUpdater> updaters = null;
+
+  public final int totalBuckets;
 
-  private final int totalBuckets;
-  private Random rand = new Random();
-  private int currentBucketId = 0;
   private final Path partitionPath;
 
   final AcidOutputFormat<?,?> outf;
+  private Object[] bucketFieldData; // Pre-allocated in constructor. Updated on each write.
 
   protected AbstractRecordWriter(HiveEndPoint endPoint, HiveConf conf)
           throws ConnectionError, StreamingException {
@@ -71,8 +80,11 @@ abstract class AbstractRecordWriter implements RecordWriter {
         throw new StreamingException("Cannot stream to table that has not been bucketed : "
                 + endPoint);
       }
+      this.bucketIds = getBucketColIDs(tbl.getSd().getBucketCols(), tbl.getSd().getCols()) ;
+      this.bucketFieldData = new Object[bucketIds.size()];
       String outFormatName = this.tbl.getSd().getOutputFormat();
       outf = (AcidOutputFormat<?,?>) ReflectionUtils.newInstance(JavaUtils.loadClass(outFormatName), conf);
+      bucketFieldData = new Object[bucketIds.size()];
     } catch (MetaException e) {
       throw new ConnectionError(endPoint, e);
     } catch (NoSuchObjectException e) {
@@ -86,17 +98,37 @@ abstract class AbstractRecordWriter implements RecordWriter {
     }
   }
 
-  protected AbstractRecordWriter(HiveEndPoint endPoint)
-          throws ConnectionError, StreamingException {
-    this(endPoint, HiveEndPoint.createHiveConf(AbstractRecordWriter.class, endPoint.metaStoreUri) );
+  // return the column numbers of the bucketed columns
+  private List<Integer> getBucketColIDs(List<String> bucketCols, List<FieldSchema> cols) {
+    ArrayList<Integer> result =  new ArrayList<Integer>(bucketCols.size());
+    HashSet<String> bucketSet = new HashSet<String>(bucketCols);
+    for (int i = 0; i < cols.size(); i++) {
+      if( bucketSet.contains(cols.get(i).getName()) ) {
+        result.add(i);
+      }
+    }
+    return result;
   }
 
   abstract SerDe getSerde() throws SerializationError;
 
+  protected abstract ObjectInspector[] getBucketObjectInspectors();
+  protected abstract StructObjectInspector getRecordObjectInspector();
+  protected abstract StructField[] getBucketStructFields();
+
+  // returns the bucket number to which the record belongs to
+  protected int getBucket(Object row) throws SerializationError {
+    ObjectInspector[] inspectors = getBucketObjectInspectors();
+    Object[] bucketFields = getBucketFields(row);
+    return ObjectInspectorUtils.getBucketNumber(bucketFields, inspectors, totalBuckets);
+  }
+
   @Override
   public void flush() throws StreamingIOFailure {
     try {
-      updater.flush();
+      for (RecordUpdater updater : updaters) {
+        updater.flush();
+      }
     } catch (IOException e) {
       throw new StreamingIOFailure("Unable to flush recordUpdater", e);
     }
@@ -116,9 +148,8 @@ abstract class AbstractRecordWriter implements RecordWriter {
   public void newBatch(Long minTxnId, Long maxTxnID)
           throws StreamingIOFailure, SerializationError {
     try {
-      this.currentBucketId = rand.nextInt(totalBuckets);
       LOG.debug("Creating Record updater");
-      updater = createRecordUpdater(currentBucketId, minTxnId, maxTxnID);
+      updaters = createRecordUpdaters(totalBuckets, minTxnId, maxTxnID);
     } catch (IOException e) {
       LOG.error("Failed creating record updater", e);
       throw new StreamingIOFailure("Unable to get new record Updater", e);
@@ -128,13 +159,49 @@ abstract class AbstractRecordWriter implements RecordWriter {
   @Override
   public void closeBatch() throws StreamingIOFailure {
     try {
-      updater.close(false);
-      updater = null;
+      for (RecordUpdater updater : updaters) {
+        updater.close(false);
+      }
+      updaters.clear();
     } catch (IOException e) {
       throw new StreamingIOFailure("Unable to close recordUpdater", e);
     }
   }
 
+  protected static ObjectInspector[] getObjectInspectorsForBucketedCols(List<Integer> bucketIds
+          , StructObjectInspector recordObjInspector)
+          throws SerializationError {
+    ObjectInspector[] result = new ObjectInspector[bucketIds.size()];
+
+    for (int i = 0; i < bucketIds.size(); i++) {
+      int bucketId = bucketIds.get(i);
+      result[i] =
+              recordObjInspector.getAllStructFieldRefs().get( bucketId ).getFieldObjectInspector();
+    }
+    return result;
+  }
+
+
+  private Object[] getBucketFields(Object row) throws SerializationError {
+    StructObjectInspector recordObjInspector = getRecordObjectInspector();
+    StructField[] bucketStructFields = getBucketStructFields();
+    for (int i = 0; i < bucketIds.size(); i++) {
+      bucketFieldData[i] = recordObjInspector.getStructFieldData(row,  bucketStructFields[i]);
+    }
+    return bucketFieldData;
+  }
+
+
+
+  private ArrayList<RecordUpdater> createRecordUpdaters(int bucketCount, Long minTxnId, Long maxTxnID)
+          throws IOException, SerializationError {
+    ArrayList<RecordUpdater> result = new ArrayList<RecordUpdater>(bucketCount);
+    for (int bucket = 0; bucket < bucketCount; bucket++) {
+      result.add(createRecordUpdater(bucket, minTxnId, maxTxnID) );
+    }
+    return result;
+  }
+
   private RecordUpdater createRecordUpdater(int bucketId, Long minTxnId, Long maxTxnID)
           throws IOException, SerializationError {
     try {

http://git-wip-us.apache.org/repos/asf/hive/blob/0ca9ff86/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java
index 6dc69f0..fd36a38 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java
@@ -26,12 +26,14 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.SerDe;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
-import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters;
+import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazySimpleStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.io.BytesWritable;
 
 import java.io.IOException;
@@ -51,7 +53,11 @@ public class DelimitedInputWriter extends AbstractRecordWriter {
   private char serdeSeparator;
   private int[] fieldToColMapping;
   private final ArrayList<String> tableColumns;
-  private AbstractSerDe serde = null;
+  private LazySimpleSerDe serde = null;
+
+  private final LazySimpleStructObjectInspector recordObjInspector;
+  private final ObjectInspector[] bucketObjInspectors;
+  private final StructField[] bucketStructFields;
 
   static final private Log LOG = LogFactory.getLog(DelimitedInputWriter.class.getName());
 
@@ -120,6 +126,22 @@ public class DelimitedInputWriter extends AbstractRecordWriter {
     this.reorderingNeeded = isReorderingNeeded(delimiter, getTableColumns());
     LOG.debug("Field reordering needed = " + this.reorderingNeeded + ", for endpoint " + endPoint);
     this.serdeSeparator = serdeSeparator;
+    this.serde = createSerde(tbl, conf, serdeSeparator);
+
+    // get ObjInspectors for entire record and bucketed cols
+    try {
+      this.recordObjInspector = (LazySimpleStructObjectInspector) serde.getObjectInspector();
+      this.bucketObjInspectors = getObjectInspectorsForBucketedCols(bucketIds, recordObjInspector);
+    } catch (SerDeException e) {
+      throw new SerializationError("Unable to get ObjectInspector for bucket columns", e);
+    }
+
+    // get StructFields for bucketed cols
+    bucketStructFields = new StructField[bucketIds.size()];
+    List<? extends StructField> allFields = recordObjInspector.getAllStructFieldRefs();
+    for (int i = 0; i < bucketIds.size(); i++) {
+      bucketStructFields[i] = allFields.get(bucketIds.get(i));
+    }
   }
 
   private boolean isReorderingNeeded(String delimiter, ArrayList<String> tableColumns) {
@@ -173,14 +195,14 @@ public class DelimitedInputWriter extends AbstractRecordWriter {
     }
     String[] reorderedFields = new String[getTableColumns().size()];
     String decoded = new String(record);
-    String[] fields = decoded.split(delimiter);
+    String[] fields = decoded.split(delimiter,-1);
     for (int i=0; i<fieldToColMapping.length; ++i) {
       int newIndex = fieldToColMapping[i];
       if(newIndex != -1) {
         reorderedFields[newIndex] = fields[i];
       }
     }
-    return join(reorderedFields,getSerdeSeparator());
+    return join(reorderedFields, getSerdeSeparator());
   }
 
   // handles nulls in items[]
@@ -212,7 +234,8 @@ public class DelimitedInputWriter extends AbstractRecordWriter {
     try {
       byte[] orderedFields = reorderFields(record);
       Object encodedRow = encode(orderedFields);
-      updater.insert(transactionId, encodedRow);
+      int bucket = getBucket(encodedRow);
+      updaters.get(bucket).insert(transactionId, encodedRow);
     } catch (IOException e) {
       throw new StreamingIOFailure("Error writing record in transaction ("
               + transactionId + ")", e);
@@ -221,13 +244,22 @@ public class DelimitedInputWriter extends AbstractRecordWriter {
 
   @Override
   SerDe getSerde() throws SerializationError {
-    if(serde!=null) {
-      return serde;
-    }
-    serde = createSerde(tbl, conf);
     return serde;
   }
 
+  protected LazySimpleStructObjectInspector getRecordObjectInspector() {
+    return recordObjInspector;
+  }
+
+  @Override
+  protected StructField[] getBucketStructFields() {
+    return bucketStructFields;
+  }
+
+  protected ObjectInspector[] getBucketObjectInspectors() {
+    return bucketObjInspectors;
+  }
+
   private Object encode(byte[] record) throws SerializationError {
     try {
       BytesWritable blob = new BytesWritable();
@@ -244,7 +276,7 @@ public class DelimitedInputWriter extends AbstractRecordWriter {
    * @throws SerializationError if serde could not be initialized
    * @param tbl
    */
-  protected LazySimpleSerDe createSerde(Table tbl, HiveConf conf)
+  protected static LazySimpleSerDe createSerde(Table tbl, HiveConf conf, char serdeSeparator)
           throws SerializationError {
     try {
       Properties tableProps = MetaStoreUtils.getTableMetadata(tbl);

http://git-wip-us.apache.org/repos/asf/hive/blob/0ca9ff86/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java
index 6d6beb8..6ab21eb 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java
@@ -24,10 +24,14 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.serde2.SerDe;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.io.Text;
+import org.apache.hive.hcatalog.data.HCatRecordObjectInspector;
 import org.apache.hive.hcatalog.data.JsonSerDe;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Properties;
 
 /**
@@ -37,6 +41,10 @@ import java.util.Properties;
 public class StrictJsonWriter extends AbstractRecordWriter {
   private JsonSerDe serde;
 
+  private final HCatRecordObjectInspector recordObjInspector;
+  private final ObjectInspector[] bucketObjInspectors;
+  private final StructField[] bucketStructFields;
+
   /**
    *
    * @param endPoint the end point to write to
@@ -46,7 +54,7 @@ public class StrictJsonWriter extends AbstractRecordWriter {
    */
   public StrictJsonWriter(HiveEndPoint endPoint)
           throws ConnectionError, SerializationError, StreamingException {
-    super(endPoint, null);
+    this(endPoint, null);
   }
 
   /**
@@ -60,23 +68,49 @@ public class StrictJsonWriter extends AbstractRecordWriter {
   public StrictJsonWriter(HiveEndPoint endPoint, HiveConf conf)
           throws ConnectionError, SerializationError, StreamingException {
     super(endPoint, conf);
+    this.serde = createSerde(tbl, conf);
+    // get ObjInspectors for entire record and bucketed cols
+    try {
+      recordObjInspector = ( HCatRecordObjectInspector ) serde.getObjectInspector();
+      this.bucketObjInspectors = getObjectInspectorsForBucketedCols(bucketIds, recordObjInspector);
+    } catch (SerDeException e) {
+      throw new SerializationError("Unable to get ObjectInspector for bucket columns", e);
+    }
+
+    // get StructFields for bucketed cols
+    bucketStructFields = new StructField[bucketIds.size()];
+    List<? extends StructField> allFields = recordObjInspector.getAllStructFieldRefs();
+    for (int i = 0; i < bucketIds.size(); i++) {
+      bucketStructFields[i] = allFields.get(bucketIds.get(i));
+    }
   }
 
   @Override
   SerDe getSerde() throws SerializationError {
-    if(serde!=null) {
-      return serde;
-    }
-    serde = createSerde(tbl, conf);
     return serde;
   }
 
+  protected HCatRecordObjectInspector getRecordObjectInspector() {
+    return recordObjInspector;
+  }
+
+  @Override
+  protected StructField[] getBucketStructFields() {
+    return bucketStructFields;
+  }
+
+  protected ObjectInspector[] getBucketObjectInspectors() {
+    return bucketObjInspectors;
+  }
+
+
   @Override
   public void write(long transactionId, byte[] record)
           throws StreamingIOFailure, SerializationError {
     try {
       Object encodedRow = encode(record);
-      updater.insert(transactionId, encodedRow);
+      int bucket = getBucket(encodedRow);
+      updaters.get(bucket).insert(transactionId, encodedRow);
     } catch (IOException e) {
       throw new StreamingIOFailure("Error writing record in transaction("
               + transactionId + ")", e);

http://git-wip-us.apache.org/repos/asf/hive/blob/0ca9ff86/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
index c28d4aa..2f6baec 100644
--- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
@@ -18,33 +18,38 @@
 
 package org.apache.hive.hcatalog.streaming;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
-import org.apache.hadoop.hive.metastore.TableType;
-import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.SerDeInfo;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.io.HiveInputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcFile;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
-import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
-import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
 import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
+import org.apache.hadoop.hive.ql.io.orc.Reader;
+import org.apache.hadoop.hive.ql.io.orc.RecordReader;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.txn.AcidHouseKeeperService;
 import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableStringObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableIntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableLongObjectInspector;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
@@ -68,7 +73,6 @@ import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 
@@ -118,6 +122,7 @@ public class TestStreaming {
   private static final String COL2 = "msg";
 
   private final HiveConf conf;
+  private Driver driver;
   private final IMetaStoreClient msClient;
 
   final String metaStoreURI = null;
@@ -127,13 +132,23 @@ public class TestStreaming {
   private final static String tblName = "alerts";
   private final static String[] fieldNames = new String[]{COL1,COL2};
   List<String> partitionVals;
-  private static String partLocation;
+  private static Path partLoc;
+  private static Path partLoc2;
 
   // unpartitioned table
-  private final static String dbName2 = "testing";
+  private final static String dbName2 = "testing2";
   private final static String tblName2 = "alerts";
   private final static String[] fieldNames2 = new String[]{COL1,COL2};
 
+
+  // for bucket join testing
+  private final static String dbName3 = "testing3";
+  private final static String tblName3 = "dimensionTable";
+  private final static String dbName4 = "testing4";
+  private final static String tblName4 = "factTable";
+  List<String> partitionVals2;
+
+
   private final String PART1_CONTINENT = "Asia";
   private final String PART1_COUNTRY = "India";
 
@@ -146,14 +161,21 @@ public class TestStreaming {
     partitionVals.add(PART1_CONTINENT);
     partitionVals.add(PART1_COUNTRY);
 
+    partitionVals2 = new ArrayList<String>(1);
+    partitionVals2.add(PART1_COUNTRY);
+
+
     conf = new HiveConf(this.getClass());
     conf.set("fs.raw.impl", RawFileSystem.class.getName());
+    conf.set("hive.enforce.bucketing", "true");
     TxnDbUtil.setConfValues(conf);
     if (metaStoreURI!=null) {
       conf.setVar(HiveConf.ConfVars.METASTOREURIS, metaStoreURI);
     }
     conf.setBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, true);
     conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true);
+    dbFolder.create();
+
 
     //1) Start from a clean slate (metastore)
     TxnDbUtil.cleanDb();
@@ -165,17 +187,35 @@ public class TestStreaming {
 
   @Before
   public void setup() throws Exception {
+    SessionState.start(new CliSessionState(conf));
+    driver = new Driver(conf);
+    driver.setMaxRows(200002);//make sure Driver returns all results
     // drop and recreate the necessary databases and tables
     dropDB(msClient, dbName);
-    createDbAndTable(msClient, dbName, tblName, partitionVals);
+
+    String[] colNames = new String[] {COL1, COL2};
+    String[] colTypes = new String[] {serdeConstants.INT_TYPE_NAME, serdeConstants.STRING_TYPE_NAME};
+    String[] bucketCols = new String[] {COL1};
+    String loc1 = dbFolder.newFolder(dbName + ".db").toString();
+    String[] partNames = new String[]{"Continent", "Country"};
+    partLoc = createDbAndTable(driver, dbName, tblName, partitionVals, colNames, colTypes, bucketCols, partNames, loc1, 1);
 
     dropDB(msClient, dbName2);
-    createDbAndTable(msClient, dbName2, tblName2, partitionVals);
+    String loc2 = dbFolder.newFolder(dbName2 + ".db").toString();
+    partLoc2 = createDbAndTable(driver, dbName2, tblName2, partitionVals, colNames, colTypes, bucketCols, partNames, loc2, 2);
+
+    String loc3 = dbFolder.newFolder("testing5.db").toString();
+    createStoreSales("testing5", loc3);
+
+    runDDL(driver, "drop table testBucketing3.streamedtable");
+    runDDL(driver, "drop table testBucketing3.finaltable");
+    runDDL(driver, "drop table testBucketing3.nobucket");
   }
 
   @After
   public void cleanup() throws Exception {
     msClient.close();
+    driver.close();
   }
 
   private static List<FieldSchema> getPartitionKeys() {
@@ -186,10 +226,170 @@ public class TestStreaming {
     return fields;
   }
 
-  private void checkDataWritten(long minTxn, long maxTxn, int buckets, int numExpectedFiles,
+  private void createStoreSales(String dbName, String loc) throws Exception {
+    String dbUri = "raw://" + new Path(loc).toUri().toString();
+    String tableLoc = dbUri + Path.SEPARATOR + "store_sales";
+
+    boolean success = runDDL(driver, "create database IF NOT EXISTS " + dbName + " location '" + dbUri + "'");
+    Assert.assertTrue(success);
+    success = runDDL(driver, "use " + dbName);
+    Assert.assertTrue(success);
+
+    success = runDDL(driver, "drop table if exists store_sales");
+    Assert.assertTrue(success);
+    success = runDDL(driver, "create table store_sales\n" +
+      "(\n" +
+      "    ss_sold_date_sk           int,\n" +
+      "    ss_sold_time_sk           int,\n" +
+      "    ss_item_sk                int,\n" +
+      "    ss_customer_sk            int,\n" +
+      "    ss_cdemo_sk               int,\n" +
+      "    ss_hdemo_sk               int,\n" +
+      "    ss_addr_sk                int,\n" +
+      "    ss_store_sk               int,\n" +
+      "    ss_promo_sk               int,\n" +
+      "    ss_ticket_number          int,\n" +
+      "    ss_quantity               int,\n" +
+      "    ss_wholesale_cost         decimal(7,2),\n" +
+      "    ss_list_price             decimal(7,2),\n" +
+      "    ss_sales_price            decimal(7,2),\n" +
+      "    ss_ext_discount_amt       decimal(7,2),\n" +
+      "    ss_ext_sales_price        decimal(7,2),\n" +
+      "    ss_ext_wholesale_cost     decimal(7,2),\n" +
+      "    ss_ext_list_price         decimal(7,2),\n" +
+      "    ss_ext_tax                decimal(7,2),\n" +
+      "    ss_coupon_amt             decimal(7,2),\n" +
+      "    ss_net_paid               decimal(7,2),\n" +
+      "    ss_net_paid_inc_tax       decimal(7,2),\n" +
+      "    ss_net_profit             decimal(7,2)\n" +
+      ")\n" +
+      " partitioned by (dt string)\n" +
+      "clustered by (ss_store_sk, ss_promo_sk)\n" +
+      "INTO 4 BUCKETS stored as orc " + " location '" + tableLoc +  "'" + "  TBLPROPERTIES ('orc.compress'='NONE', 'transactional'='true')");
+    Assert.assertTrue(success);
+
+    success = runDDL(driver, "alter table store_sales add partition(dt='2015')");
+    Assert.assertTrue(success);
+  }
+  /**
+   * make sure it works with table where bucket col is not 1st col
+   * @throws Exception
+   */
+  @Test
+  public void testBucketingWhereBucketColIsNotFirstCol() throws Exception {
+    List<String> partitionVals = new ArrayList<String>();
+    partitionVals.add("2015");
+    HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, "testing5", "store_sales", partitionVals);
+    DelimitedInputWriter writer = new DelimitedInputWriter(new String[] {"ss_sold_date_sk","ss_sold_time_sk", "ss_item_sk",
+      "ss_customer_sk", "ss_cdemo_sk", "ss_hdemo_sk", "ss_addr_sk", "ss_store_sk", "ss_promo_sk", "ss_ticket_number", "ss_quantity",
+      "ss_wholesale_cost", "ss_list_price", "ss_sales_price", "ss_ext_discount_amt", "ss_ext_sales_price", "ss_ext_wholesale_cost",
+      "ss_ext_list_price", "ss_ext_tax", "ss_coupon_amt", "ss_net_paid", "ss_net_paid_inc_tax", "ss_net_profit"},",", endPt);
+    StreamingConnection connection = endPt.newConnection(false, null);//should this really be null?
+
+    TransactionBatch txnBatch =  connection.fetchTransactionBatch(2, writer);
+    txnBatch.beginNextTransaction();
+
+    StringBuilder row = new StringBuilder();
+    for(int i = 0; i < 10; i++) {
+      for(int ints = 0; ints < 11; ints++) {
+        row.append(ints).append(',');
+      }
+      for(int decs = 0; decs < 12; decs++) {
+        row.append(i + 0.1).append(',');
+      }
+      row.setLength(row.length() - 1);
+      txnBatch.write(row.toString().getBytes());
+    }
+    txnBatch.commit();
+    txnBatch.close();
+    connection.close();
+
+    ArrayList<String> res = queryTable(driver, "select row__id.bucketid, * from testing5.store_sales");
+    for (String re : res) {
+      System.out.println(re);
+    }
+  }
+
+
+  // stream data into streaming table with N buckets, then copy the data into another bucketed table
+  // check if bucketing in both was done in the same way
+  @Test
+  public void testStreamBucketingMatchesRegularBucketing() throws Exception {
+    int bucketCount = 100;
+
+    String dbUri = "raw://" + new Path(dbFolder.newFolder().toString()).toUri().toString();
+    String tableLoc  = "'" + dbUri + Path.SEPARATOR + "streamedtable" + "'";
+    String tableLoc2 = "'" + dbUri + Path.SEPARATOR + "finaltable" + "'";
+    String tableLoc3 = "'" + dbUri + Path.SEPARATOR + "nobucket" + "'";
+
+    runDDL(driver, "create database testBucketing3");
+    runDDL(driver, "use testBucketing3");
+    runDDL(driver, "create table streamedtable ( key1 string,key2 int,data string ) clustered by ( key1,key2 ) into "
+            + bucketCount + " buckets  stored as orc  location " + tableLoc + " TBLPROPERTIES ('transactional'='true')") ;
+//  In 'nobucket' table we capture bucketid from streamedtable to workaround a hive bug that prevents joins two identically bucketed tables
+    runDDL(driver, "create table nobucket ( bucketid int, key1 string,key2 int,data string ) location " + tableLoc3) ;
+    runDDL(driver, "create table finaltable ( bucketid int, key1 string,key2 int,data string ) clustered by ( key1,key2 ) into "
+            + bucketCount + " buckets  stored as orc location " + tableLoc2 + " TBLPROPERTIES ('transactional'='true')");
+
+
+    String[] records = new String[] {
+    "PSFAHYLZVC,29,EPNMA",
+    "PPPRKWAYAU,96,VUTEE",
+    "MIAOFERCHI,3,WBDSI",
+    "CEGQAZOWVN,0,WCUZL",
+    "XWAKMNSVQF,28,YJVHU",
+    "XBWTSAJWME,2,KDQFO",
+    "FUVLQTAXAY,5,LDSDG",
+    "QTQMDJMGJH,6,QBOMA",
+    "EFLOTLWJWN,71,GHWPS",
+    "PEQNAOJHCM,82,CAAFI",
+    "MOEKQLGZCP,41,RUACR",
+    "QZXMCOPTID,37,LFLWE",
+    "EYALVWICRD,13,JEZLC",
+    "VYWLZAYTXX,16,DMVZX",
+    "OSALYSQIXR,47,HNZVE",
+    "JGKVHKCEGQ,25,KSCJB",
+    "WQFMMYDHET,12,DTRWA",
+    "AJOVAYZKZQ,15,YBKFO",
+    "YAQONWCUAU,31,QJNHZ",
+    "DJBXUEUOEB,35,IYCBL"
+    };
+
+    HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, "testBucketing3", "streamedtable", null);
+    String[] colNames1 = new String[] { "key1", "key2", "data" };
+    DelimitedInputWriter wr = new DelimitedInputWriter(colNames1,",",  endPt);
+    StreamingConnection connection = endPt.newConnection(false);
+
+    TransactionBatch txnBatch =  connection.fetchTransactionBatch(2, wr);
+    txnBatch.beginNextTransaction();
+
+    for (String record : records) {
+      txnBatch.write(record.toString().getBytes());
+    }
+
+    txnBatch.commit();
+    txnBatch.close();
+    connection.close();
+
+    ArrayList<String> res1 = queryTable(driver, "select row__id.bucketid, * from streamedtable order by key2");
+    for (String re : res1) {
+      System.out.println(re);
+    }
+
+    driver.run("insert into nobucket select row__id.bucketid,* from streamedtable");
+    runDDL(driver, " insert into finaltable select * from nobucket");
+    ArrayList<String> res2 = queryTable(driver, "select row__id.bucketid,* from finaltable where row__id.bucketid<>bucketid");
+    for (String s : res2) {
+      LOG.error(s);
+    }
+    Assert.assertTrue(res2.isEmpty());
+  }
+
+
+  private void checkDataWritten(Path partitionPath, long minTxn, long maxTxn, int buckets, int numExpectedFiles,
                                 String... records) throws Exception {
     ValidTxnList txns = msClient.getValidTxns();
-    AcidUtils.Directory dir = AcidUtils.getAcidState(new Path(partLocation), conf, txns);
+    AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, txns);
     Assert.assertEquals(0, dir.getObsolete().size());
     Assert.assertEquals(0, dir.getOriginalFiles().size());
     List<AcidUtils.ParsedDelta> current = dir.getCurrentDirectories();
@@ -197,7 +397,7 @@ public class TestStreaming {
     for (AcidUtils.ParsedDelta pd : current) System.out.println(pd.getPath().toString());
     Assert.assertEquals(numExpectedFiles, current.size());
 
-    // find the absolute mininum transaction
+    // find the absolute minimum transaction
     long min = Long.MAX_VALUE;
     long max = Long.MIN_VALUE;
     for (AcidUtils.ParsedDelta pd : current) {
@@ -209,11 +409,11 @@ public class TestStreaming {
 
     InputFormat inf = new OrcInputFormat();
     JobConf job = new JobConf();
-    job.set("mapred.input.dir", partLocation.toString());
+    job.set("mapred.input.dir", partitionPath.toString());
     job.set("bucket_count", Integer.toString(buckets));
     job.set(ValidTxnList.VALID_TXNS_KEY, txns.toString());
-    InputSplit[] splits = inf.getSplits(job, 1);
-    Assert.assertEquals(1, splits.length);
+    InputSplit[] splits = inf.getSplits(job, buckets);
+    Assert.assertEquals(buckets, splits.length);
     org.apache.hadoop.mapred.RecordReader<NullWritable, OrcStruct> rr =
             inf.getRecordReader(splits[0], job, Reporter.NULL);
 
@@ -226,9 +426,9 @@ public class TestStreaming {
     Assert.assertEquals(false, rr.next(key, value));
   }
 
-  private void checkNothingWritten() throws Exception {
+  private void checkNothingWritten(Path partitionPath) throws Exception {
     ValidTxnList txns = msClient.getValidTxns();
-    AcidUtils.Directory dir = AcidUtils.getAcidState(new Path(partLocation), conf, txns);
+    AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, txns);
     Assert.assertEquals(0, dir.getObsolete().size());
     Assert.assertEquals(0, dir.getOriginalFiles().size());
     List<AcidUtils.ParsedDelta> current = dir.getCurrentDirectories();
@@ -398,7 +598,7 @@ public class TestStreaming {
     txnBatch.write("1,Hello streaming".getBytes());
     txnBatch.commit();
 
-    checkDataWritten(1, 10, 1, 1, "{1, Hello streaming}");
+    checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}");
 
     Assert.assertEquals(TransactionBatch.TxnState.COMMITTED
             , txnBatch.getCurrentTransactionState());
@@ -410,11 +610,11 @@ public class TestStreaming {
     txnBatch.write("2,Welcome to streaming".getBytes());
 
     // data should not be visible
-    checkDataWritten(1, 10, 1, 1, "{1, Hello streaming}");
+    checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}");
 
     txnBatch.commit();
 
-    checkDataWritten(1, 10, 1, 1, "{1, Hello streaming}",
+    checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}",
         "{2, Welcome to streaming}");
 
     txnBatch.close();
@@ -459,7 +659,7 @@ public class TestStreaming {
     txnBatch.write(rec1.getBytes());
     txnBatch.commit();
 
-    checkDataWritten(1, 10, 1, 1, "{1, Hello streaming}");
+    checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}");
 
     Assert.assertEquals(TransactionBatch.TxnState.COMMITTED
             , txnBatch.getCurrentTransactionState());
@@ -518,7 +718,7 @@ public class TestStreaming {
               , txnBatch.getCurrentTransactionState());
       ++batch;
     }
-    Assert.assertEquals(0,txnBatch.remainingTransactions());
+    Assert.assertEquals(0, txnBatch.remainingTransactions());
     txnBatch.close();
 
     Assert.assertEquals(TransactionBatch.TxnState.INACTIVE
@@ -542,7 +742,7 @@ public class TestStreaming {
     txnBatch.write("2,Welcome to streaming".getBytes());
     txnBatch.abort();
 
-    checkNothingWritten();
+    checkNothingWritten(partLoc);
 
     Assert.assertEquals(TransactionBatch.TxnState.ABORTED
             , txnBatch.getCurrentTransactionState());
@@ -550,7 +750,7 @@ public class TestStreaming {
     txnBatch.close();
     connection.close();
 
-    checkNothingWritten();
+    checkNothingWritten(partLoc);
 
   }
 
@@ -569,7 +769,7 @@ public class TestStreaming {
     txnBatch.write("2,Welcome to streaming".getBytes());
     txnBatch.abort();
 
-    checkNothingWritten();
+    checkNothingWritten(partLoc);
 
     Assert.assertEquals(TransactionBatch.TxnState.ABORTED
             , txnBatch.getCurrentTransactionState());
@@ -579,8 +779,8 @@ public class TestStreaming {
     txnBatch.write("2,Welcome to streaming".getBytes());
     txnBatch.commit();
 
-    checkDataWritten(1, 10, 1, 1, "{1, Hello streaming}",
-        "{2, Welcome to streaming}");
+    checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}",
+            "{2, Welcome to streaming}");
 
     txnBatch.close();
     connection.close();
@@ -598,14 +798,14 @@ public class TestStreaming {
     txnBatch.write("1,Hello streaming".getBytes());
     txnBatch.commit();
 
-    checkDataWritten(1, 10, 1, 1, "{1, Hello streaming}");
+    checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}");
 
     txnBatch.beginNextTransaction();
     txnBatch.write("2,Welcome to streaming".getBytes());
     txnBatch.commit();
 
-    checkDataWritten(1, 10, 1, 1, "{1, Hello streaming}",
-        "{2, Welcome to streaming}");
+    checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}",
+            "{2, Welcome to streaming}");
 
     txnBatch.close();
 
@@ -615,16 +815,16 @@ public class TestStreaming {
     txnBatch.write("3,Hello streaming - once again".getBytes());
     txnBatch.commit();
 
-    checkDataWritten(1, 20, 1, 2, "{1, Hello streaming}",
-        "{2, Welcome to streaming}", "{3, Hello streaming - once again}");
+    checkDataWritten(partLoc, 1, 20, 1, 2, "{1, Hello streaming}",
+            "{2, Welcome to streaming}", "{3, Hello streaming - once again}");
 
     txnBatch.beginNextTransaction();
     txnBatch.write("4,Welcome to streaming - once again".getBytes());
     txnBatch.commit();
 
-    checkDataWritten(1, 20, 1, 2, "{1, Hello streaming}",
-        "{2, Welcome to streaming}", "{3, Hello streaming - once again}",
-        "{4, Welcome to streaming - once again}");
+    checkDataWritten(partLoc, 1, 20, 1, 2, "{1, Hello streaming}",
+            "{2, Welcome to streaming}", "{3, Hello streaming - once again}",
+            "{4, Welcome to streaming - once again}");
 
     Assert.assertEquals(TransactionBatch.TxnState.COMMITTED
             , txnBatch.getCurrentTransactionState());
@@ -655,15 +855,15 @@ public class TestStreaming {
     txnBatch1.write("1,Hello streaming".getBytes());
     txnBatch2.write("3,Hello streaming - once again".getBytes());
 
-    checkNothingWritten();
+    checkNothingWritten(partLoc);
 
     txnBatch2.commit();
 
-    checkDataWritten(11, 20, 1, 1, "{3, Hello streaming - once again}");
+    checkDataWritten(partLoc, 11, 20, 1, 1, "{3, Hello streaming - once again}");
 
     txnBatch1.commit();
 
-    checkDataWritten(1, 20, 1, 2, "{1, Hello streaming}", "{3, Hello streaming - once again}");
+    checkDataWritten(partLoc, 1, 20, 1, 2, "{1, Hello streaming}", "{3, Hello streaming - once again}");
 
     txnBatch1.beginNextTransaction();
     txnBatch1.write("2,Welcome to streaming".getBytes());
@@ -671,17 +871,17 @@ public class TestStreaming {
     txnBatch2.beginNextTransaction();
     txnBatch2.write("4,Welcome to streaming - once again".getBytes());
 
-    checkDataWritten(1, 20, 1, 2, "{1, Hello streaming}", "{3, Hello streaming - once again}");
+    checkDataWritten(partLoc, 1, 20, 1, 2, "{1, Hello streaming}", "{3, Hello streaming - once again}");
 
     txnBatch1.commit();
 
-    checkDataWritten(1, 20, 1, 2, "{1, Hello streaming}",
+    checkDataWritten(partLoc, 1, 20, 1, 2, "{1, Hello streaming}",
         "{2, Welcome to streaming}",
         "{3, Hello streaming - once again}");
 
     txnBatch2.commit();
 
-    checkDataWritten(1, 20, 1, 2, "{1, Hello streaming}",
+    checkDataWritten(partLoc, 1, 20, 1, 2, "{1, Hello streaming}",
         "{2, Welcome to streaming}",
         "{3, Hello streaming - once again}",
         "{4, Welcome to streaming - once again}");
@@ -772,6 +972,164 @@ public class TestStreaming {
     }
   }
 
+
+  private ArrayList<SampleRec> dumpBucket(Path orcFile) throws IOException {
+    org.apache.hadoop.fs.FileSystem fs = org.apache.hadoop.fs.FileSystem.getLocal(new Configuration());
+    Reader reader = OrcFile.createReader(orcFile,
+            OrcFile.readerOptions(conf).filesystem(fs));
+
+    RecordReader rows = reader.rows(null);
+    StructObjectInspector inspector = (StructObjectInspector) reader
+            .getObjectInspector();
+
+    System.out.format("Found Bucket File : %s \n", orcFile.getName());
+    ArrayList<SampleRec> result = new ArrayList<SampleRec>();
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      SampleRec rec = (SampleRec) deserializeDeltaFileRow(row, inspector)[5];
+      result.add(rec);
+    }
+
+    return result;
+  }
+
+  // Assumes stored data schema = [acid fields],string,int,string
+  // return array of 6 fields, where the last field has the actual data
+  private static Object[] deserializeDeltaFileRow(Object row, StructObjectInspector inspector) {
+    List<? extends StructField> fields = inspector.getAllStructFieldRefs();
+
+    WritableIntObjectInspector f0ins = (WritableIntObjectInspector) fields.get(0).getFieldObjectInspector();
+    WritableLongObjectInspector f1ins = (WritableLongObjectInspector) fields.get(1).getFieldObjectInspector();
+    WritableIntObjectInspector f2ins = (WritableIntObjectInspector) fields.get(2).getFieldObjectInspector();
+    WritableLongObjectInspector f3ins = (WritableLongObjectInspector) fields.get(3).getFieldObjectInspector();
+    WritableLongObjectInspector f4ins = (WritableLongObjectInspector)  fields.get(4).getFieldObjectInspector();
+    StructObjectInspector f5ins = (StructObjectInspector) fields.get(5).getFieldObjectInspector();
+
+    int f0 = f0ins.get(inspector.getStructFieldData(row, fields.get(0)));
+    long f1 = f1ins.get(inspector.getStructFieldData(row, fields.get(1)));
+    int f2 = f2ins.get(inspector.getStructFieldData(row, fields.get(2)));
+    long f3 = f3ins.get(inspector.getStructFieldData(row, fields.get(3)));
+    long f4 = f4ins.get(inspector.getStructFieldData(row, fields.get(4)));
+    SampleRec f5 = deserializeInner(inspector.getStructFieldData(row, fields.get(5)), f5ins);
+
+    return new Object[] {f0, f1, f2, f3, f4, f5};
+  }
+
+  // Assumes row schema => string,int,string
+  private static SampleRec deserializeInner(Object row, StructObjectInspector inspector) {
+    List<? extends StructField> fields = inspector.getAllStructFieldRefs();
+
+    WritableStringObjectInspector f0ins = (WritableStringObjectInspector) fields.get(0).getFieldObjectInspector();
+    WritableIntObjectInspector f1ins = (WritableIntObjectInspector) fields.get(1).getFieldObjectInspector();
+    WritableStringObjectInspector f2ins = (WritableStringObjectInspector) fields.get(2).getFieldObjectInspector();
+
+    String f0 = f0ins.getPrimitiveJavaObject(inspector.getStructFieldData(row, fields.get(0)));
+    int f1 = f1ins.get(inspector.getStructFieldData(row, fields.get(1)));
+    String f2 = f2ins.getPrimitiveJavaObject(inspector.getStructFieldData(row, fields.get(2)));
+    return new SampleRec(f0, f1, f2);
+  }
+
+  @Test
+  public void testBucketing() throws Exception {
+    dropDB(msClient, dbName3);
+    dropDB(msClient, dbName4);
+
+    // 1) Create two bucketed tables
+    String dbLocation = dbFolder.newFolder(dbName3).getCanonicalPath() + ".db";
+    dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths
+    String[] colNames = "key1,key2,data".split(",");
+    String[] colTypes = "string,int,string".split(",");
+    String[] bucketNames = "key1,key2".split(",");
+    int bucketCount = 4;
+    createDbAndTable(driver, dbName3, tblName3, null, colNames, colTypes, bucketNames
+            , null, dbLocation, bucketCount);
+
+    String dbLocation2 = dbFolder.newFolder(dbName4).getCanonicalPath() + ".db";
+    dbLocation2 = dbLocation2.replaceAll("\\\\","/"); // for windows paths
+    String[] colNames2 = "key3,key4,data2".split(",");
+    String[] colTypes2 = "string,int,string".split(",");
+    String[] bucketNames2 = "key3,key4".split(",");
+    createDbAndTable(driver, dbName4, tblName4, null, colNames2, colTypes2, bucketNames2
+            , null, dbLocation2, bucketCount);
+
+
+    // 2) Insert data into both tables
+    HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName3, tblName3, null);
+    DelimitedInputWriter writer = new DelimitedInputWriter(colNames,",", endPt);
+    StreamingConnection connection = endPt.newConnection(false);
+
+    TransactionBatch txnBatch =  connection.fetchTransactionBatch(2, writer);
+    txnBatch.beginNextTransaction();
+    txnBatch.write("name0,1,Hello streaming".getBytes());
+    txnBatch.write("name2,2,Welcome to streaming".getBytes());
+    txnBatch.write("name4,2,more Streaming unlimited".getBytes());
+    txnBatch.write("name5,2,even more Streaming unlimited".getBytes());
+    txnBatch.commit();
+
+
+    HiveEndPoint endPt2 = new HiveEndPoint(metaStoreURI, dbName4, tblName4, null);
+    DelimitedInputWriter writer2 = new DelimitedInputWriter(colNames2,",", endPt2);
+    StreamingConnection connection2 = endPt2.newConnection(false);
+    TransactionBatch txnBatch2 =  connection2.fetchTransactionBatch(2, writer2);
+    txnBatch2.beginNextTransaction();
+
+    txnBatch2.write("name5,2,fact3".getBytes());  // bucket 0
+    txnBatch2.write("name8,2,fact3".getBytes());  // bucket 1
+    txnBatch2.write("name0,1,fact1".getBytes());  // bucket 2
+    // no data for bucket 3 -- expect 0 length bucket file
+
+
+    txnBatch2.commit();
+
+    // 3 Check data distribution in  buckets
+
+    HashMap<Integer, ArrayList<SampleRec>> actual1 = dumpAllBuckets(dbLocation, tblName3);
+    HashMap<Integer, ArrayList<SampleRec>> actual2 = dumpAllBuckets(dbLocation2, tblName4);
+    System.err.println("\n  Table 1");
+    System.err.println(actual1);
+    System.err.println("\n  Table 2");
+    System.err.println(actual2);
+
+    // assert bucket listing is as expected
+    Assert.assertEquals("number of buckets does not match expectation", actual1.values().size(), 4);
+    Assert.assertEquals("records in bucket does not match expectation", actual1.get(0).size(), 2);
+    Assert.assertEquals("records in bucket does not match expectation", actual1.get(1).size(), 1);
+    Assert.assertEquals("records in bucket does not match expectation", actual1.get(2).size(), 0);
+    Assert.assertEquals("records in bucket does not match expectation", actual1.get(3).size(), 1);
+
+
+  }
+
+
+    // assumes un partitioned table
+  // returns a map<bucketNum, list<record> >
+  private HashMap<Integer, ArrayList<SampleRec>> dumpAllBuckets(String dbLocation, String tableName)
+          throws IOException {
+    HashMap<Integer, ArrayList<SampleRec>> result = new HashMap<Integer, ArrayList<SampleRec>>();
+
+    for (File deltaDir : new File(dbLocation + "/" + tableName).listFiles()) {
+      if(!deltaDir.getName().startsWith("delta"))
+        continue;
+      File[] bucketFiles = deltaDir.listFiles();
+      for (File bucketFile : bucketFiles) {
+        if(bucketFile.toString().endsWith("length"))
+          continue;
+        Integer bucketNum = getBucketNumber(bucketFile);
+        ArrayList<SampleRec>  recs = dumpBucket(new Path(bucketFile.toString()));
+        result.put(bucketNum, recs);
+      }
+    }
+    return result;
+  }
+
+  //assumes bucket_NNNNN format of file name
+  private Integer getBucketNumber(File bucketFile) {
+    String fname = bucketFile.getName();
+    int start = fname.indexOf('_');
+    String number = fname.substring(start+1, fname.length());
+    return Integer.parseInt(number);
+  }
+
   // delete db and all tables in it
   public static void dropDB(IMetaStoreClient client, String databaseName) {
     try {
@@ -784,90 +1142,182 @@ public class TestStreaming {
 
   }
 
-  public void createDbAndTable(IMetaStoreClient client, String databaseName,
-                               String tableName, List<String> partVals)
+
+
+  ///////// -------- UTILS ------- /////////
+  // returns Path of the partition created (if any) else Path of table
+  public static Path createDbAndTable(Driver driver, String databaseName,
+                                      String tableName, List<String> partVals,
+                                      String[] colNames, String[] colTypes,
+                                      String[] bucketCols,
+                                      String[] partNames, String dbLocation, int bucketCount)
           throws Exception {
-    Database db = new Database();
-    db.setName(databaseName);
-    String dbLocation = "raw://" + dbFolder.newFolder(databaseName + ".db").toURI().getPath();
-    db.setLocationUri(dbLocation);
-    client.createDatabase(db);
-
-    Table tbl = new Table();
-    tbl.setDbName(databaseName);
-    tbl.setTableName(tableName);
-    tbl.setTableType(TableType.MANAGED_TABLE.toString());
-    StorageDescriptor sd = new StorageDescriptor();
-    sd.setCols(getTableColumns());
-    sd.setNumBuckets(1);
-    sd.setLocation(dbLocation + Path.SEPARATOR + tableName);
-    tbl.setPartitionKeys(getPartitionKeys());
-
-    tbl.setSd(sd);
-
-    sd.setBucketCols(new ArrayList<String>(2));
-    sd.setSerdeInfo(new SerDeInfo());
-    sd.getSerdeInfo().setName(tbl.getTableName());
-    sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-    sd.getSerdeInfo().getParameters().put(serdeConstants.SERIALIZATION_FORMAT, "1");
-
-    sd.getSerdeInfo().setSerializationLib(OrcSerde.class.getName());
-    sd.setInputFormat(HiveInputFormat.class.getName());
-    sd.setOutputFormat(OrcOutputFormat.class.getName());
-
-    Map<String, String> tableParams = new HashMap<String, String>();
-    tbl.setParameters(tableParams);
-    client.createTable(tbl);
 
-    try {
-      addPartition(client, tbl, partVals);
-    } catch (AlreadyExistsException e) {
-    }
-    Partition createdPartition = client.getPartition(databaseName, tableName, partVals);
-    partLocation = createdPartition.getSd().getLocation();
-  }
-
-  private static void addPartition(IMetaStoreClient client, Table tbl
-          , List<String> partValues)
-          throws IOException, TException {
-    Partition part = new Partition();
-    part.setDbName(tbl.getDbName());
-    part.setTableName(tblName);
-    StorageDescriptor sd = new StorageDescriptor(tbl.getSd());
-    sd.setLocation(sd.getLocation() + Path.SEPARATOR + makePartPath(tbl.getPartitionKeys()
-            , partValues));
-    part.setSd(sd);
-    part.setValues(partValues);
-    client.add_partition(part);
-  }
-
-  private static String makePartPath(List<FieldSchema> partKeys, List<String> partVals) {
-    if (partKeys.size()!=partVals.size()) {
-      throw new IllegalArgumentException("Partition values:" + partVals
-              + ", does not match the partition Keys in table :" + partKeys );
-    }
-    StringBuilder buff = new StringBuilder(partKeys.size()*20);
-    buff.append(" ( ");
-    int i=0;
-    for (FieldSchema schema : partKeys) {
-      buff.append(schema.getName());
-      buff.append("='");
-      buff.append(partVals.get(i));
-      buff.append("'");
-      if (i!=partKeys.size()-1) {
-        buff.append(Path.SEPARATOR);
+    String dbUri = "raw://" + new Path(dbLocation).toUri().toString();
+    String tableLoc = dbUri + Path.SEPARATOR + tableName;
+
+    runDDL(driver, "create database IF NOT EXISTS " + databaseName + " location '" + dbUri + "'");
+    runDDL(driver, "use " + databaseName);
+    String crtTbl = "create table " + tableName +
+            " ( " +  getTableColumnsStr(colNames,colTypes) + " )" +
+            getPartitionStmtStr(partNames) +
+            " clustered by ( " + join(bucketCols, ",") + " )" +
+            " into " + bucketCount + " buckets " +
+            " stored as orc " +
+            " location '" + tableLoc +  "'";
+    runDDL(driver, crtTbl);
+    if(partNames!=null && partNames.length!=0) {
+      return addPartition(driver, tableName, partVals, partNames);
+    }
+    return new Path(tableLoc);
+  }
+
+  private static Path addPartition(Driver driver, String tableName, List<String> partVals, String[] partNames) throws QueryFailedException, CommandNeedRetryException, IOException {
+    String partSpec = getPartsSpec(partNames, partVals);
+    String addPart = "alter table " + tableName + " add partition ( " + partSpec  + " )";
+    runDDL(driver, addPart);
+    return getPartitionPath(driver, tableName, partSpec);
+  }
+
+  private static Path getPartitionPath(Driver driver, String tableName, String partSpec) throws CommandNeedRetryException, IOException {
+    ArrayList<String> res = queryTable(driver, "describe extended " + tableName + " PARTITION (" + partSpec + ")");
+    String partInfo = res.get(res.size() - 1);
+    int start = partInfo.indexOf("location:") + "location:".length();
+    int end = partInfo.indexOf(",",start);
+    return new Path( partInfo.substring(start,end) );
+  }
+
+  private static String getTableColumnsStr(String[] colNames, String[] colTypes) {
+    StringBuffer sb = new StringBuffer();
+    for (int i=0; i < colNames.length; ++i) {
+      sb.append(colNames[i] + " " + colTypes[i]);
+      if (i<colNames.length-1) {
+        sb.append(",");
       }
-      ++i;
     }
-    buff.append(" )");
-    return buff.toString();
+    return sb.toString();
   }
 
+  // converts partNames into "partName1 string, partName2 string"
+  private static String getTablePartsStr(String[] partNames) {
+    if (partNames==null || partNames.length==0) {
+      return "";
+    }
+    StringBuffer sb = new StringBuffer();
+    for (int i=0; i < partNames.length; ++i) {
+      sb.append(partNames[i] + " string");
+      if (i < partNames.length-1) {
+        sb.append(",");
+      }
+    }
+    return sb.toString();
+  }
 
-  private static List<FieldSchema> getTableColumns() {
-    List<FieldSchema> fields = new ArrayList<FieldSchema>();
-    fields.add(new FieldSchema(COL1, serdeConstants.INT_TYPE_NAME, ""));
-    fields.add(new FieldSchema(COL2, serdeConstants.STRING_TYPE_NAME, ""));
-    return fields;
+  // converts partNames,partVals into "partName1=val1, partName2=val2"
+  private static String getPartsSpec(String[] partNames, List<String> partVals) {
+    StringBuffer sb = new StringBuffer();
+    for (int i=0; i < partVals.size(); ++i) {
+      sb.append(partNames[i] + " = '" + partVals.get(i) + "'");
+      if(i < partVals.size()-1) {
+        sb.append(",");
+      }
+    }
+    return sb.toString();
+  }
+
+  private static String join(String[] values, String delimiter) {
+    if(values==null)
+      return null;
+    StringBuffer strbuf = new StringBuffer();
+
+    boolean first = true;
+
+    for (Object value : values)  {
+      if (!first) { strbuf.append(delimiter); } else { first = false; }
+      strbuf.append(value.toString());
+    }
+
+    return strbuf.toString();
+  }
+  private static String getPartitionStmtStr(String[] partNames) {
+    if ( partNames == null || partNames.length == 0) {
+      return "";
+    }
+    return " partitioned by (" + getTablePartsStr(partNames) + " )";
+  }
+
+  private static boolean runDDL(Driver driver, String sql) throws QueryFailedException {
+    LOG.debug(sql);
+    System.out.println(sql);
+    int retryCount = 1; // # of times to retry if first attempt fails
+    for (int attempt=0; attempt <= retryCount; ++attempt) {
+      try {
+        //LOG.debug("Running Hive Query: "+ sql);
+        CommandProcessorResponse cpr = driver.run(sql);
+        if(cpr.getResponseCode() == 0) {
+          return true;
+        }
+        LOG.error("Statement: " + sql + " failed: " + cpr);
+      } catch (CommandNeedRetryException e) {
+        if (attempt == retryCount) {
+          throw new QueryFailedException(sql, e);
+        }
+        continue;
+      }
+    } // for
+    return false;
+  }
+
+
+  public static ArrayList<String> queryTable(Driver driver, String query)
+          throws CommandNeedRetryException, IOException {
+    driver.run(query);
+    ArrayList<String> res = new ArrayList<String>();
+    driver.getResults(res);
+    if(res.isEmpty())
+      System.err.println(driver.getErrorMsg());
+    return res;
+  }
+
+  private static class SampleRec {
+    public String field1;
+    public int field2;
+    public String field3;
+
+    public SampleRec(String field1, int field2, String field3) {
+      this.field1 = field1;
+      this.field2 = field2;
+      this.field3 = field3;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      SampleRec that = (SampleRec) o;
+
+      if (field2 != that.field2) return false;
+      if (field1 != null ? !field1.equals(that.field1) : that.field1 != null) return false;
+      return !(field3 != null ? !field3.equals(that.field3) : that.field3 != null);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = field1 != null ? field1.hashCode() : 0;
+      result = 31 * result + field2;
+      result = 31 * result + (field3 != null ? field3.hashCode() : 0);
+      return result;
+    }
+
+    @Override
+    public String toString() {
+      return " { " +
+              "'" + field1 + '\'' +
+              "," + field2 +
+              ",'" + field3 + '\'' +
+              " }";
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/0ca9ff86/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
index 00a6384..54ae48e 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
@@ -494,6 +494,35 @@ public final class ObjectInspectorUtils {
     }
   }
 
+  /**
+   * Computes the bucket number to which the bucketFields belong to
+   * @param bucketFields  the bucketed fields of the row
+   * @param bucketFieldInspectors  the ObjectInpsectors for each of the bucketed fields
+   * @param totalBuckets the number of buckets in the table
+   * @return the bucket number
+   */
+  public static int getBucketNumber(Object[] bucketFields, ObjectInspector[] bucketFieldInspectors, int totalBuckets) {
+    int hashCode = getBucketHashCode(bucketFields, bucketFieldInspectors);
+    int bucketID = (hashCode & Integer.MAX_VALUE) % totalBuckets;
+    return bucketID;
+  }
+
+  /**
+   * Computes the hash code for the given bucketed fields
+   * @param bucketFields
+   * @param bucketFieldInspectors
+   * @return
+   */
+  private static int getBucketHashCode(Object[] bucketFields, ObjectInspector[] bucketFieldInspectors) {
+    int hashCode = 0;
+    for (int i = 0; i < bucketFields.length; i++) {
+      int fieldHash = ObjectInspectorUtils.hashCode(bucketFields[i], bucketFieldInspectors[i]);
+      hashCode = 31 * hashCode + fieldHash;
+    }
+    return hashCode;
+  }
+
+
   public static int hashCode(Object o, ObjectInspector objIns) {
     if (o == null) {
       return 0;


[13/23] hive git commit: HIVE-11699: Support special characters in quoted table names (Pengcheng Xiong, reviewed by John Pullokkaran)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c23841e5/ql/src/test/results/clientpositive/special_character_in_tabnames_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/special_character_in_tabnames_2.q.out b/ql/src/test/results/clientpositive/special_character_in_tabnames_2.q.out
new file mode 100644
index 0000000..51d31e0
--- /dev/null
+++ b/ql/src/test/results/clientpositive/special_character_in_tabnames_2.q.out
@@ -0,0 +1,304 @@
+PREHOOK: query: -- try the query without indexing, with manual indexing, and with automatic indexing
+-- SORT_QUERY_RESULTS
+
+DROP TABLE IF EXISTS `s/c`
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: -- try the query without indexing, with manual indexing, and with automatic indexing
+-- SORT_QUERY_RESULTS
+
+DROP TABLE IF EXISTS `s/c`
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE `s/c` (key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@s/c
+POSTHOOK: query: CREATE TABLE `s/c` (key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@s/c
+PREHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/kv1.txt" INTO TABLE `s/c`
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@s/c
+POSTHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/kv1.txt" INTO TABLE `s/c`
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@s/c
+PREHOOK: query: ANALYZE TABLE `s/c` COMPUTE STATISTICS
+PREHOOK: type: QUERY
+PREHOOK: Input: default@s/c
+PREHOOK: Output: default@s/c
+POSTHOOK: query: ANALYZE TABLE `s/c` COMPUTE STATISTICS
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@s/c
+POSTHOOK: Output: default@s/c
+PREHOOK: query: ANALYZE TABLE `s/c` COMPUTE STATISTICS FOR COLUMNS key,value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@s/c
+#### A masked pattern was here ####
+POSTHOOK: query: ANALYZE TABLE `s/c` COMPUTE STATISTICS FOR COLUMNS key,value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@s/c
+#### A masked pattern was here ####
+PREHOOK: query: -- without indexing
+SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@s/c
+#### A masked pattern was here ####
+POSTHOOK: query: -- without indexing
+SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@s/c
+#### A masked pattern was here ####
+82	val_82
+83	val_83
+83	val_83
+84	val_84
+84	val_84
+85	val_85
+86	val_86
+87	val_87
+90	val_90
+90	val_90
+90	val_90
+92	val_92
+95	val_95
+95	val_95
+96	val_96
+97	val_97
+97	val_97
+98	val_98
+98	val_98
+PREHOOK: query: CREATE INDEX src_index ON TABLE `s/c`(key) as 'COMPACT' WITH DEFERRED REBUILD
+PREHOOK: type: CREATEINDEX
+PREHOOK: Input: default@s/c
+POSTHOOK: query: CREATE INDEX src_index ON TABLE `s/c`(key) as 'COMPACT' WITH DEFERRED REBUILD
+POSTHOOK: type: CREATEINDEX
+POSTHOOK: Input: default@s/c
+POSTHOOK: Output: default@default__s/c_src_index__
+PREHOOK: query: ALTER INDEX src_index ON `s/c` REBUILD
+PREHOOK: type: ALTERINDEX_REBUILD
+PREHOOK: Input: default@s/c
+PREHOOK: Output: default@default__s/c_src_index__
+POSTHOOK: query: ALTER INDEX src_index ON `s/c` REBUILD
+POSTHOOK: type: ALTERINDEX_REBUILD
+POSTHOOK: Input: default@s/c
+POSTHOOK: Output: default@default__s/c_src_index__
+POSTHOOK: Lineage: default__s/c_src_index__._bucketname SIMPLE [(s/c)s/c.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ]
+POSTHOOK: Lineage: default__s/c_src_index__._offsets EXPRESSION [(s/c)s/c.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ]
+POSTHOOK: Lineage: default__s/c_src_index__.key SIMPLE [(s/c)s/c.FieldSchema(name:key, type:string, comment:default), ]
+PREHOOK: query: -- manual indexing
+#### A masked pattern was here ####
+PREHOOK: type: QUERY
+PREHOOK: Input: default@default__s/c_src_index__
+#### A masked pattern was here ####
+POSTHOOK: query: -- manual indexing
+#### A masked pattern was here ####
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@default__s/c_src_index__
+#### A masked pattern was here ####
+PREHOOK: query: EXPLAIN SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100
+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: s/c
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) (type: boolean)
+              Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@s/c
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@s/c
+#### A masked pattern was here ####
+82	val_82
+83	val_83
+83	val_83
+84	val_84
+84	val_84
+85	val_85
+86	val_86
+87	val_87
+90	val_90
+90	val_90
+90	val_90
+92	val_92
+95	val_95
+95	val_95
+96	val_96
+97	val_97
+97	val_97
+98	val_98
+98	val_98
+PREHOOK: query: -- automatic indexing
+EXPLAIN SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100
+PREHOOK: type: QUERY
+POSTHOOK: query: -- automatic indexing
+EXPLAIN SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-3 is a root stage
+  Stage-8 depends on stages: Stage-3 , consists of Stage-5, Stage-4, Stage-6
+  Stage-5
+  Stage-2 depends on stages: Stage-5, Stage-4, Stage-7
+  Stage-1 depends on stages: Stage-2
+  Stage-4
+  Stage-6
+  Stage-7 depends on stages: Stage-6
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: default__s/c_src_index__
+            filterExpr: ((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) (type: boolean)
+            Filter Operator
+              predicate: ((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) (type: boolean)
+              Select Operator
+                expressions: _bucketname (type: string), _offsets (type: array<bigint>)
+                outputColumnNames: _col0, _col1
+                File Output Operator
+                  compressed: false
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-8
+    Conditional Operator
+
+  Stage: Stage-5
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-2
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: s/c
+            filterExpr: ((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) (type: boolean)
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((UDFToDouble(key) > 80.0) and (UDFToDouble(key) < 100.0)) (type: boolean)
+              Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-6
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-7
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@default__s/c_src_index__
+PREHOOK: Input: default@s/c
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT key, value FROM `s/c` WHERE key > 80 AND key < 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@default__s/c_src_index__
+POSTHOOK: Input: default@s/c
+#### A masked pattern was here ####
+82	val_82
+83	val_83
+83	val_83
+84	val_84
+84	val_84
+85	val_85
+86	val_86
+87	val_87
+90	val_90
+90	val_90
+90	val_90
+92	val_92
+95	val_95
+95	val_95
+96	val_96
+97	val_97
+97	val_97
+98	val_98
+98	val_98
+PREHOOK: query: DROP INDEX src_index on `s/c`
+PREHOOK: type: DROPINDEX
+PREHOOK: Input: default@s/c
+POSTHOOK: query: DROP INDEX src_index on `s/c`
+POSTHOOK: type: DROPINDEX
+POSTHOOK: Input: default@s/c


[11/23] hive git commit: HIVE-12005 : Remove hbase based stats collection mechanism (Ashutosh Chauhan via Prasanth J)

Posted by se...@apache.org.
HIVE-12005 : Remove hbase based stats collection mechanism (Ashutosh Chauhan via Prasanth J)

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/a014cffe
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a014cffe
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a014cffe

Branch: refs/heads/llap
Commit: a014cffed990f8f499d797453f50f436b94fd280
Parents: bbb312f
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Wed Sep 30 18:15:07 2015 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Sat Oct 3 11:36:23 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/common/StatsSetupConst.java     |   8 -
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   4 +-
 .../hadoop/hive/hbase/HBaseStatsAggregator.java | 128 --------
 .../hadoop/hive/hbase/HBaseStatsPublisher.java  | 154 ---------
 .../hive/hbase/HBaseStatsSetupConstants.java    |  34 --
 .../hadoop/hive/hbase/HBaseStatsUtils.java      | 135 --------
 .../src/test/queries/positive/hbase_stats.q     |  30 --
 .../src/test/queries/positive/hbase_stats2.q    |  31 --
 .../positive/hbase_stats_empty_partition.q      |  13 -
 .../src/test/results/positive/hbase_stats.q.out | 311 -------------------
 .../test/results/positive/hbase_stats2.q.out    | 311 -------------------
 .../positive/hbase_stats_empty_partition.q.out  |  63 ----
 12 files changed, 2 insertions(+), 1220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java b/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
index c70cdfb..b16b231 100644
--- a/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
+++ b/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
@@ -31,14 +31,6 @@ import java.util.Map;
 public class StatsSetupConst {
 
   public enum StatDB {
-    hbase {
-      @Override
-      public String getPublisher(Configuration conf) {
-        return "org.apache.hadoop.hive.hbase.HBaseStatsPublisher"; }
-      @Override
-      public String getAggregator(Configuration conf) {
-        return "org.apache.hadoop.hive.hbase.HBaseStatsAggregator"; }
-    },
     jdbc {
       @Override
       public String getPublisher(Configuration conf) {

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/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 77ca613..33ef654 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1341,11 +1341,11 @@ public class HiveConf extends Configuration {
     // Statistics
     HIVESTATSAUTOGATHER("hive.stats.autogather", true,
         "A flag to gather statistics automatically during the INSERT OVERWRITE command."),
-    HIVESTATSDBCLASS("hive.stats.dbclass", "fs", new PatternSet("jdbc(:.*)", "hbase", "counter", "custom", "fs"),
+    HIVESTATSDBCLASS("hive.stats.dbclass", "fs", new PatternSet("jdbc(:.*)", "counter", "custom", "fs"),
         "The storage that stores temporary Hive statistics. In filesystem based statistics collection ('fs'), \n" +
         "each task writes statistics it has collected in a file on the filesystem, which will be aggregated \n" +
         "after the job has finished. Supported values are fs (filesystem), jdbc:database (where database \n" +
-        "can be derby, mysql, etc.), hbase, counter, and custom as defined in StatsSetupConst.java."), // StatsSetupConst.StatDB
+        "can be derby, mysql, etc.), counter, and custom as defined in StatsSetupConst.java."), // StatsSetupConst.StatDB
     HIVESTATSJDBCDRIVER("hive.stats.jdbcdriver",
         "org.apache.derby.jdbc.EmbeddedDriver",
         "The JDBC driver for the database that stores temporary Hive statistics."),

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsAggregator.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsAggregator.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsAggregator.java
deleted file mode 100644
index 1b96232..0000000
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsAggregator.java
+++ /dev/null
@@ -1,128 +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.hbase;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.filter.PrefixFilter;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.stats.StatsAggregator;
-
-
-/**
- * A class that implements the StatsAggregator interface through HBase.
- */
-public class HBaseStatsAggregator implements StatsAggregator {
-
-  private HTable htable;
-  private final Log LOG = LogFactory.getLog(this.getClass().getName());
-
-  /**
-   * Does the necessary HBase initializations.
-   */
-  public boolean connect(Configuration hiveconf, Task sourceTask) {
-
-    try {
-      htable = new HTable(HBaseConfiguration.create(hiveconf),
-        HBaseStatsSetupConstants.PART_STAT_TABLE_NAME);
-
-      return true;
-    } catch (IOException e) {
-      LOG.error("Error during HBase connection. ", e);
-      return false;
-    }
-  }
-
-  /**
-   * Aggregates temporary stats from HBase;
-   */
-  public String aggregateStats(String rowID, String key) {
-
-    byte[] family, column;
-    if (!HBaseStatsUtils.isValidStatistic(key)) {
-      LOG.warn("Warning. Invalid statistic: " + key + ", supported stats: " +
-          HBaseStatsUtils.getSupportedStatistics());
-      return null;
-    }
-
-    family = HBaseStatsUtils.getFamilyName();
-    column = HBaseStatsUtils.getColumnName(key);
-
-    try {
-
-      long retValue = 0;
-      Scan scan = new Scan();
-      scan.addColumn(family, column);
-      // Filter the row by its ID
-      // The complete key is "tableName/PartSpecs/jobID/taskID"
-      // This is a prefix filter, the prefix is "tableName/PartSpecs/JobID", i.e. the taskID is
-      // ignored. In SQL, this is equivalent to
-      // "Select * FROM tableName where ID LIKE 'tableName/PartSpecs/JobID%';"
-      PrefixFilter filter = new PrefixFilter(Bytes.toBytes(rowID));
-      scan.setFilter(filter);
-      ResultScanner scanner = htable.getScanner(scan);
-
-      for (Result result : scanner) {
-        retValue += Long.parseLong(Bytes.toString(result.getValue(family, column)));
-      }
-      return Long.toString(retValue);
-    } catch (IOException e) {
-      LOG.error("Error during publishing aggregation. ", e);
-      return null;
-    }
-  }
-
-  public boolean closeConnection() {
-    return true;
-  }
-
-  public boolean cleanUp(String rowID) {
-    try {
-      Scan scan = new Scan();
-      // Filter the row by its ID
-      // The complete key is "tableName/PartSpecs/jobID/taskID"
-      // This is a prefix filter, the prefix is "JobID"
-      // In SQL, this is equivalent to "Select * FROM tableName where ID LIKE 'JobID%';"
-      PrefixFilter filter = new PrefixFilter(Bytes.toBytes(rowID));
-      scan.setFilter(filter);
-      ResultScanner scanner = htable.getScanner(scan);
-      ArrayList<Delete> toDelete = new ArrayList<Delete>();
-      for (Result result : scanner) {
-        Delete delete = new Delete(result.getRow());
-        toDelete.add(delete);
-      }
-      htable.delete(toDelete);
-      return true;
-    } catch (IOException e) {
-      LOG.error("Error during publishing aggregation. ", e);
-      return false;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsPublisher.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsPublisher.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsPublisher.java
deleted file mode 100644
index 8266b33..0000000
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsPublisher.java
+++ /dev/null
@@ -1,154 +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.hbase;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hive.ql.stats.StatsPublisher;
-
-/**
- * A class that implements the StatsPublisher interface through HBase.
- */
-public class HBaseStatsPublisher implements StatsPublisher {
-
-  private HTable htable;
-  private final Log LOG = LogFactory.getLog(this.getClass().getName());
-
-  /**
-   * Does the necessary HBase initializations.
-   */
-  public boolean connect(Configuration hiveconf) {
-
-    try {
-      htable = new HTable(HBaseConfiguration.create(hiveconf),
-        HBaseStatsSetupConstants.PART_STAT_TABLE_NAME);
-      // for performance reason, defer update until the closeConnection
-      htable.setAutoFlush(false);
-    } catch (IOException e) {
-      LOG.error("Error during HBase connection. " + e);
-      return false;
-    }
-
-    return true;
-  }
-
-  /**
-   * Writes temporary statistics into HBase;
-   */
-  public boolean publishStat(String rowID, Map<String, String> stats) {
-
-    // Write in HBase
-
-    if (stats.isEmpty()) {
-      // If there are no stats to publish, nothing to do.
-      return true;
-    }
-
-    if (!HBaseStatsUtils.isValidStatisticSet(stats.keySet())) {
-      LOG.warn("Warning. Invalid statistic: " + stats.keySet().toString()
-          + ", supported stats: "
-          + HBaseStatsUtils.getSupportedStatistics());
-      return false;
-    }
-
-    try {
-
-      // check the basic stat (e.g., row_count)
-
-      Get get = new Get(Bytes.toBytes(rowID));
-      Result result = htable.get(get);
-
-      byte[] family = HBaseStatsUtils.getFamilyName();
-      byte[] column = HBaseStatsUtils.getColumnName(HBaseStatsUtils.getBasicStat());
-
-      long val = Long.parseLong(HBaseStatsUtils.getStatFromMap(HBaseStatsUtils.getBasicStat(),
-          stats));
-      long oldVal = 0;
-
-      if (!result.isEmpty()) {
-        oldVal = Long.parseLong(Bytes.toString(result.getValue(family, column)));
-      }
-
-      if (oldVal >= val) {
-        return true; // we do not need to publish anything
-      }
-
-      // we need to update
-      Put row = new Put(Bytes.toBytes(rowID));
-      for (String statType : HBaseStatsUtils.getSupportedStatistics()) {
-        column = HBaseStatsUtils.getColumnName(statType);
-        row.add(family, column, Bytes.toBytes(HBaseStatsUtils.getStatFromMap(statType, stats)));
-      }
-
-      htable.put(row);
-      return true;
-
-    } catch (IOException e) {
-      LOG.error("Error during publishing statistics. " + e);
-      return false;
-    }
-  }
-
-  public boolean closeConnection() {
-    // batch update
-    try {
-      htable.flushCommits();
-      return true;
-    } catch (IOException e) {
-      LOG.error("Cannot commit changes in stats publishing.", e);
-      return false;
-    }
-  }
-
-
-  /**
-   * Does the necessary HBase initializations.
-   */
-  public boolean init(Configuration hiveconf) {
-    try {
-      HBaseAdmin hbase = new HBaseAdmin(HBaseConfiguration.create(hiveconf));
-
-      // Creating table if not exists
-      if (!hbase.tableExists(HBaseStatsSetupConstants.PART_STAT_TABLE_NAME)) {
-        HTableDescriptor table = new HTableDescriptor(HBaseStatsSetupConstants.PART_STAT_TABLE_NAME);
-        HColumnDescriptor family = new HColumnDescriptor(HBaseStatsUtils.getFamilyName());
-        table.addFamily(family);
-        hbase.createTable(table);
-      }
-    } catch (IOException e) {
-      LOG.error("Error during HBase initialization. " + e);
-      return false;
-    }
-
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsSetupConstants.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsSetupConstants.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsSetupConstants.java
deleted file mode 100644
index af97edf..0000000
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsSetupConstants.java
+++ /dev/null
@@ -1,34 +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.hbase;
-
-public final class HBaseStatsSetupConstants {
-
-  public static final String PART_STAT_TABLE_NAME = "PARTITION_STAT_TBL";
-
-  public static final String PART_STAT_COLUMN_FAMILY = "PARTITION_STAT_FAMILY";
-
-  //supported stats
-
-  public static final String PART_STAT_ROW_COUNT_COLUMN_NAME = "ROW_COUNT";
-
-  public static final String PART_STAT_RAW_DATA_SIZE_COLUMN_NAME = "RAW_DATA_SIZE";
-
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsUtils.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsUtils.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsUtils.java
deleted file mode 100644
index 6e8fac5..0000000
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsUtils.java
+++ /dev/null
@@ -1,135 +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.hbase;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hive.common.StatsSetupConst;
-
-
-
-public class HBaseStatsUtils {
-
-  private static final List<String> supportedStats = new ArrayList<String>();
-  private static final Map<String, String> columnNameMapping = new HashMap<String, String>();
-
-  static {
-    // supported statistics
-    supportedStats.add(StatsSetupConst.ROW_COUNT);
-    supportedStats.add(StatsSetupConst.RAW_DATA_SIZE);
-
-    // row count statistics
-    columnNameMapping.put(StatsSetupConst.ROW_COUNT,
-        HBaseStatsSetupConstants.PART_STAT_ROW_COUNT_COLUMN_NAME);
-
-    // raw data size
-    columnNameMapping.put(StatsSetupConst.RAW_DATA_SIZE,
-        HBaseStatsSetupConstants.PART_STAT_RAW_DATA_SIZE_COLUMN_NAME);
-
-  }
-
-  /**
-   * Returns the set of supported statistics
-   */
-  public static List<String> getSupportedStatistics() {
-    return supportedStats;
-  }
-
-  /**
-   * Retrieves the value for a particular stat from the published map.
-   *
-   * @param statType
-   *          - statistic type to be retrieved from the map
-   * @param stats
-   *          - stats map
-   * @return value for the given statistic as string, "0" if the statistic is not present
-   */
-  public static String getStatFromMap(String statType, Map<String, String> stats) {
-    String value = stats.get(statType);
-    if (value == null) {
-      return "0";
-    }
-    return value;
-  }
-
-  /**
-   * Check if the set to be published is within the supported statistics.
-   * It must also contain at least the basic statistics (used for comparison).
-   *
-   * @param stats
-   *          - stats to be published
-   * @return true if is a valid statistic set, false otherwise
-   */
-
-  public static boolean isValidStatisticSet(Collection<String> stats) {
-    if(!stats.contains(getBasicStat())) {
-      return false;
-    }
-    for (String stat : stats) {
-      if (!supportedStats.contains(stat)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Check if a particular statistic type is supported
-   *
-   * @param statType
-   *          - statistic to be published
-   * @return true if statType is supported, false otherwise
-   */
-  public static boolean isValidStatistic(String statType) {
-    return supportedStats.contains(statType);
-  }
-
-  /**
-   * Returns the HBase column where the statistics for the given type are stored.
-   *
-   * @param statType
-   *          - supported statistic.
-   * @return column name for the given statistic.
-   */
-  public static byte[] getColumnName(String statType) {
-    return Bytes.toBytes(columnNameMapping.get(statType));
-  }
-
-  /**
-   * Returns the family name for stored statistics.
-   */
-  public static byte[] getFamilyName() {
-    return Bytes.toBytes(HBaseStatsSetupConstants.PART_STAT_COLUMN_FAMILY);
-  }
-
-  /**
-   * Returns the basic type of the supported statistics.
-   * It is used to determine which statistics are fresher.
-   */
-
-  public static String getBasicStat() {
-    return supportedStats.get(0);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/hbase-handler/src/test/queries/positive/hbase_stats.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_stats.q b/hbase-handler/src/test/queries/positive/hbase_stats.q
deleted file mode 100644
index 3350dde..0000000
--- a/hbase-handler/src/test/queries/positive/hbase_stats.q
+++ /dev/null
@@ -1,30 +0,0 @@
-set datanucleus.cache.collections=false;
-set hive.stats.autogather=true;
-set hive.stats.atomic=false;
-
-set hive.stats.dbclass=hbase;
-
-create table stats_src like src;
-insert overwrite table stats_src select * from src;
-analyze table stats_src compute statistics;
-desc formatted stats_src;
-
-create table stats_part like srcpart;
-
-insert overwrite table stats_part partition (ds='2010-04-08', hr = '11') select key, value from src;
-insert overwrite table stats_part partition (ds='2010-04-08', hr = '12') select key, value from src;
-
-analyze table stats_part partition(ds='2010-04-08', hr='11') compute statistics;
-analyze table stats_part partition(ds='2010-04-08', hr='12') compute statistics;
-
-insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
-
-desc formatted stats_part;
-desc formatted stats_part partition (ds='2010-04-08', hr = '11');
-desc formatted stats_part partition (ds='2010-04-08', hr = '12');
-
-analyze table stats_part partition(ds, hr) compute statistics;
-desc formatted stats_part;
-
-drop table stats_src;
-drop table stats_part;

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/hbase-handler/src/test/queries/positive/hbase_stats2.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_stats2.q b/hbase-handler/src/test/queries/positive/hbase_stats2.q
deleted file mode 100644
index f6c71c3..0000000
--- a/hbase-handler/src/test/queries/positive/hbase_stats2.q
+++ /dev/null
@@ -1,31 +0,0 @@
-set datanucleus.cache.collections=false;
-set hive.stats.autogather=true;
-set hive.stats.atomic=false;
-set hive.stats.collect.rawdatasize=false;
-
-set hive.stats.dbclass=hbase;
-
-create table stats_src like src;
-insert overwrite table stats_src select * from src;
-analyze table stats_src compute statistics;
-desc formatted stats_src;
-
-create table stats_part like srcpart;
-
-insert overwrite table stats_part partition (ds='2010-04-08', hr = '11') select key, value from src;
-insert overwrite table stats_part partition (ds='2010-04-08', hr = '12') select key, value from src;
-
-analyze table stats_part partition(ds='2010-04-08', hr='11') compute statistics;
-analyze table stats_part partition(ds='2010-04-08', hr='12') compute statistics;
-
-insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
-
-desc formatted stats_part;
-desc formatted stats_part partition (ds='2010-04-08', hr = '11');
-desc formatted stats_part partition (ds='2010-04-08', hr = '12');
-
-analyze table stats_part partition(ds, hr) compute statistics;
-desc formatted stats_part;
-
-drop table stats_src;
-drop table stats_part;

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/hbase-handler/src/test/queries/positive/hbase_stats_empty_partition.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_stats_empty_partition.q b/hbase-handler/src/test/queries/positive/hbase_stats_empty_partition.q
deleted file mode 100644
index 24f4616..0000000
--- a/hbase-handler/src/test/queries/positive/hbase_stats_empty_partition.q
+++ /dev/null
@@ -1,13 +0,0 @@
--- This test verifies that writing an empty partition succeeds when
--- hive.stats.reliable is set to true.
-
-create table tmptable(key string, value string) partitioned by (part string);
-
-set hive.stats.autogather=true;
-set hive.stats.reliable=true;
-
-set hive.stats.dbclass=hbase;
-
-insert overwrite table tmptable partition (part = '1') select * from src where key = 'no_such_value';
-
-describe formatted tmptable partition (part = '1');

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/hbase-handler/src/test/results/positive/hbase_stats.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_stats.q.out b/hbase-handler/src/test/results/positive/hbase_stats.q.out
deleted file mode 100644
index f34720d..0000000
--- a/hbase-handler/src/test/results/positive/hbase_stats.q.out
+++ /dev/null
@@ -1,311 +0,0 @@
-PREHOOK: query: create table stats_src like src
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@stats_src
-POSTHOOK: query: create table stats_src like src
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@stats_src
-PREHOOK: query: insert overwrite table stats_src select * from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@stats_src
-POSTHOOK: query: insert overwrite table stats_src select * from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@stats_src
-POSTHOOK: Lineage: stats_src.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: stats_src.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: analyze table stats_src compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@stats_src
-PREHOOK: Output: default@stats_src
-POSTHOOK: query: analyze table stats_src compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@stats_src
-POSTHOOK: Output: default@stats_src
-PREHOOK: query: desc formatted stats_src
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@stats_src
-POSTHOOK: query: desc formatted stats_src
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@stats_src
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	default             
-value               	string              	default             
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: create table stats_part like srcpart
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@stats_part
-POSTHOOK: query: create table stats_part like srcpart
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@stats_part
-PREHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '11') select key, value from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '11') select key, value from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=11).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=11).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '12') select key, value from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '12') select key, value from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=12).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=12).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: analyze table stats_part partition(ds='2010-04-08', hr='11') compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@stats_part
-PREHOOK: Input: default@stats_part@ds=2010-04-08/hr=11
-PREHOOK: Output: default@stats_part
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: query: analyze table stats_part partition(ds='2010-04-08', hr='11') compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@stats_part
-POSTHOOK: Input: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: Output: default@stats_part
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-PREHOOK: query: analyze table stats_part partition(ds='2010-04-08', hr='12') compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@stats_part
-PREHOOK: Input: default@stats_part@ds=2010-04-08/hr=12
-PREHOOK: Output: default@stats_part
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: query: analyze table stats_part partition(ds='2010-04-08', hr='12') compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@stats_part
-POSTHOOK: Input: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: Output: default@stats_part
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-PREHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
-POSTHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: desc formatted stats_part
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@stats_part
-POSTHOOK: query: desc formatted stats_part
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@stats_part
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	default             
-value               	string              	default             
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	                    
-hr                  	string              	                    
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '11')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@stats_part
-POSTHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '11')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@stats_part
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	default             
-value               	string              	default             
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	                    
-hr                  	string              	                    
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2010-04-08, 11]    	 
-Database:           	default             	 
-Table:              	stats_part          	 
-#### A masked pattern was here ####
-Partition Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '12')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@stats_part
-POSTHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '12')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@stats_part
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	default             
-value               	string              	default             
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	                    
-hr                  	string              	                    
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2010-04-08, 12]    	 
-Database:           	default             	 
-Table:              	stats_part          	 
-#### A masked pattern was here ####
-Partition Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: analyze table stats_part partition(ds, hr) compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@stats_part
-PREHOOK: Input: default@stats_part@ds=2010-04-08/hr=11
-PREHOOK: Input: default@stats_part@ds=2010-04-08/hr=12
-PREHOOK: Input: default@stats_part@ds=2010-04-08/hr=13
-PREHOOK: Output: default@stats_part
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
-POSTHOOK: query: analyze table stats_part partition(ds, hr) compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@stats_part
-POSTHOOK: Input: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: Input: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: Input: default@stats_part@ds=2010-04-08/hr=13
-POSTHOOK: Output: default@stats_part
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
-PREHOOK: query: desc formatted stats_part
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@stats_part
-POSTHOOK: query: desc formatted stats_part
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@stats_part
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	default             
-value               	string              	default             
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	                    
-hr                  	string              	                    
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: drop table stats_src
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@stats_src
-PREHOOK: Output: default@stats_src
-POSTHOOK: query: drop table stats_src
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@stats_src
-POSTHOOK: Output: default@stats_src
-PREHOOK: query: drop table stats_part
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@stats_part
-PREHOOK: Output: default@stats_part
-POSTHOOK: query: drop table stats_part
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@stats_part
-POSTHOOK: Output: default@stats_part

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/hbase-handler/src/test/results/positive/hbase_stats2.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_stats2.q.out b/hbase-handler/src/test/results/positive/hbase_stats2.q.out
deleted file mode 100644
index aad2e3a..0000000
--- a/hbase-handler/src/test/results/positive/hbase_stats2.q.out
+++ /dev/null
@@ -1,311 +0,0 @@
-PREHOOK: query: create table stats_src like src
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@stats_src
-POSTHOOK: query: create table stats_src like src
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@stats_src
-PREHOOK: query: insert overwrite table stats_src select * from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@stats_src
-POSTHOOK: query: insert overwrite table stats_src select * from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@stats_src
-POSTHOOK: Lineage: stats_src.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: stats_src.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: analyze table stats_src compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@stats_src
-PREHOOK: Output: default@stats_src
-POSTHOOK: query: analyze table stats_src compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@stats_src
-POSTHOOK: Output: default@stats_src
-PREHOOK: query: desc formatted stats_src
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@stats_src
-POSTHOOK: query: desc formatted stats_src
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@stats_src
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	default             
-value               	string              	default             
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	0                   
-	totalSize           	5812                
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: create table stats_part like srcpart
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@stats_part
-POSTHOOK: query: create table stats_part like srcpart
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@stats_part
-PREHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '11') select key, value from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '11') select key, value from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=11).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=11).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '12') select key, value from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '12') select key, value from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=12).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=12).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: analyze table stats_part partition(ds='2010-04-08', hr='11') compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@stats_part
-PREHOOK: Input: default@stats_part@ds=2010-04-08/hr=11
-PREHOOK: Output: default@stats_part
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: query: analyze table stats_part partition(ds='2010-04-08', hr='11') compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@stats_part
-POSTHOOK: Input: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: Output: default@stats_part
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-PREHOOK: query: analyze table stats_part partition(ds='2010-04-08', hr='12') compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@stats_part
-PREHOOK: Input: default@stats_part@ds=2010-04-08/hr=12
-PREHOOK: Output: default@stats_part
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: query: analyze table stats_part partition(ds='2010-04-08', hr='12') compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@stats_part
-POSTHOOK: Input: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: Output: default@stats_part
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-PREHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
-POSTHOOK: query: insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: desc formatted stats_part
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@stats_part
-POSTHOOK: query: desc formatted stats_part
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@stats_part
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	default             
-value               	string              	default             
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	                    
-hr                  	string              	                    
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '11')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@stats_part
-POSTHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '11')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@stats_part
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	default             
-value               	string              	default             
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	                    
-hr                  	string              	                    
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2010-04-08, 11]    	 
-Database:           	default             	 
-Table:              	stats_part          	 
-#### A masked pattern was here ####
-Partition Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	0                   
-	totalSize           	5812                
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '12')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@stats_part
-POSTHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '12')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@stats_part
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	default             
-value               	string              	default             
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	                    
-hr                  	string              	                    
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2010-04-08, 12]    	 
-Database:           	default             	 
-Table:              	stats_part          	 
-#### A masked pattern was here ####
-Partition Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	0                   
-	totalSize           	5812                
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: analyze table stats_part partition(ds, hr) compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@stats_part
-PREHOOK: Input: default@stats_part@ds=2010-04-08/hr=11
-PREHOOK: Input: default@stats_part@ds=2010-04-08/hr=12
-PREHOOK: Input: default@stats_part@ds=2010-04-08/hr=13
-PREHOOK: Output: default@stats_part
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
-POSTHOOK: query: analyze table stats_part partition(ds, hr) compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@stats_part
-POSTHOOK: Input: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: Input: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: Input: default@stats_part@ds=2010-04-08/hr=13
-POSTHOOK: Output: default@stats_part
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=11
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=12
-POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
-PREHOOK: query: desc formatted stats_part
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@stats_part
-POSTHOOK: query: desc formatted stats_part
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@stats_part
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	default             
-value               	string              	default             
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	                    
-hr                  	string              	                    
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: drop table stats_src
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@stats_src
-PREHOOK: Output: default@stats_src
-POSTHOOK: query: drop table stats_src
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@stats_src
-POSTHOOK: Output: default@stats_src
-PREHOOK: query: drop table stats_part
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@stats_part
-PREHOOK: Output: default@stats_part
-POSTHOOK: query: drop table stats_part
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@stats_part
-POSTHOOK: Output: default@stats_part

http://git-wip-us.apache.org/repos/asf/hive/blob/a014cffe/hbase-handler/src/test/results/positive/hbase_stats_empty_partition.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_stats_empty_partition.q.out b/hbase-handler/src/test/results/positive/hbase_stats_empty_partition.q.out
deleted file mode 100644
index c13817e..0000000
--- a/hbase-handler/src/test/results/positive/hbase_stats_empty_partition.q.out
+++ /dev/null
@@ -1,63 +0,0 @@
-PREHOOK: query: -- This test verifies that writing an empty partition succeeds when
--- hive.stats.reliable is set to true.
-
-create table tmptable(key string, value string) partitioned by (part string)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@tmptable
-POSTHOOK: query: -- This test verifies that writing an empty partition succeeds when
--- hive.stats.reliable is set to true.
-
-create table tmptable(key string, value string) partitioned by (part string)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@tmptable
-PREHOOK: query: insert overwrite table tmptable partition (part = '1') select * from src where key = 'no_such_value'
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@tmptable@part=1
-POSTHOOK: query: insert overwrite table tmptable partition (part = '1') select * from src where key = 'no_such_value'
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@tmptable@part=1
-POSTHOOK: Lineage: tmptable PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: tmptable PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: describe formatted tmptable partition (part = '1')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@tmptable
-POSTHOOK: query: describe formatted tmptable partition (part = '1')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@tmptable
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	                    
-value               	string              	                    
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-part                	string              	                    
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[1]                 	 
-Database:           	default             	 
-Table:              	tmptable            	 
-#### A masked pattern was here ####
-Partition Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	0                   
-	rawDataSize         	0                   
-	totalSize           	0                   
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   


[12/23] hive git commit: HIVE-12016-Update log4j2 version to 2.4 (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-12016-Update log4j2 version to 2.4 (Prasanth Jayachandran 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/d545935a
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d545935a
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d545935a

Branch: refs/heads/llap
Commit: d545935a56136607dae8b1b69fcf6c3cc664c11b
Parents: a014cff
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Sun Oct 4 00:07:18 2015 -0500
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Sun Oct 4 00:07:18 2015 -0500

----------------------------------------------------------------------
 pom.xml                                                            | 2 +-
 ql/src/test/org/apache/hadoop/hive/ql/metadata/StringAppender.java | 2 +-
 .../org/apache/hive/service/cli/operation/LogDivertAppender.java   | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d545935a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 12a832f..b11a405 100644
--- a/pom.xml
+++ b/pom.xml
@@ -148,7 +148,7 @@
     <kryo.version>2.22</kryo.version>
     <libfb303.version>0.9.2</libfb303.version>
     <libthrift.version>0.9.2</libthrift.version>
-    <log4j2.version>2.3</log4j2.version>
+    <log4j2.version>2.4</log4j2.version>
     <opencsv.version>2.3</opencsv.version>
     <mockito-all.version>1.9.5</mockito-all.version>
     <mina.version>2.0.0-M5</mina.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/d545935a/ql/src/test/org/apache/hadoop/hive/ql/metadata/StringAppender.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/StringAppender.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/StringAppender.java
index 17b64d6..275b66b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/StringAppender.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/StringAppender.java
@@ -113,7 +113,7 @@ public class StringAppender
 
     protected StringOutputStreamManager(ByteArrayOutputStream os, String streamName,
         Layout<?> layout) {
-      super(os, streamName, layout);
+      super(os, streamName, layout, true);
       stream = os;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d545935a/service/src/java/org/apache/hive/service/cli/operation/LogDivertAppender.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/LogDivertAppender.java b/service/src/java/org/apache/hive/service/cli/operation/LogDivertAppender.java
index fb3921f..c1bc547 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/LogDivertAppender.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/LogDivertAppender.java
@@ -231,7 +231,7 @@ public class LogDivertAppender
 
     protected StringOutputStreamManager(ByteArrayOutputStream os, String streamName,
         Layout<?> layout) {
-      super(os, streamName, layout);
+      super(os, streamName, layout, true);
       stream = os;
     }
 


[06/23] hive git commit: HIVE-11714: Turn off hybrid grace hash join for cross product join (Wei Zheng via Gunther Hagleitner)

Posted by se...@apache.org.
HIVE-11714: Turn off hybrid grace hash join for cross product join (Wei Zheng via Gunther Hagleitner)


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

Branch: refs/heads/llap
Commit: 77f30d4f98a7d0b7729747136ed0e0926dd56e6b
Parents: 0d36e82
Author: Gunther Hagleitner <gu...@apache.org>
Authored: Fri Oct 2 12:51:40 2015 -0700
Committer: Gunther Hagleitner <gu...@apache.org>
Committed: Fri Oct 2 12:55:47 2015 -0700

----------------------------------------------------------------------
 .../hive/ql/exec/tez/HashTableLoader.java       |  25 ++-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |   4 +
 ql/src/test/queries/clientpositive/cross_join.q |   8 +
 .../results/clientpositive/cross_join.q.out     | 196 +++++++++++++++++
 .../clientpositive/spark/cross_join.q.out       | 211 +++++++++++++++++++
 .../results/clientpositive/tez/auto_join0.q.out |   1 -
 .../tez/auto_sortmerge_join_12.q.out            |   1 -
 .../results/clientpositive/tez/cross_join.q.out | 187 ++++++++++++++++
 .../tez/cross_product_check_2.q.out             |   6 -
 .../tez/dynamic_partition_pruning.q.out         |   1 -
 .../vectorized_dynamic_partition_pruning.q.out  |   1 -
 11 files changed, 626 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java
index f7d165a..8a3647c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
@@ -38,6 +39,7 @@ import org.apache.hadoop.hive.ql.exec.persistence.MapJoinObjectSerDeContext;
 import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainer;
 import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainerSerDe;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
@@ -77,6 +79,12 @@ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTable
     Map<Integer, String> parentToInput = desc.getParentToInput();
     Map<Integer, Long> parentKeyCounts = desc.getParentKeyCounts();
 
+    boolean isCrossProduct = false;
+    List<ExprNodeDesc> joinExprs = desc.getKeys().values().iterator().next();
+    if (joinExprs.size() == 0) {
+      isCrossProduct = true;
+    }
+
     boolean useOptimizedTables = HiveConf.getBoolVar(
         hconf, HiveConf.ConfVars.HIVEMAPJOINUSEOPTIMIZEDTABLE);
     boolean useHybridGraceHashJoin = desc.isHybridHashJoin();
@@ -184,11 +192,18 @@ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTable
           }
         }
 
-        MapJoinTableContainer tableContainer = useOptimizedTables
-            ? (useHybridGraceHashJoin ? new HybridHashTableContainer(hconf, keyCount,
-                                            memory, desc.getParentDataSizes().get(pos), nwayConf)
-                                      : new MapJoinBytesTableContainer(hconf, valCtx, keyCount, 0))
-            : new HashMapWrapper(hconf, keyCount);
+        MapJoinTableContainer tableContainer;
+        if (useOptimizedTables) {
+          if (!useHybridGraceHashJoin || isCrossProduct) {
+            tableContainer = new MapJoinBytesTableContainer(hconf, valCtx, keyCount, 0);
+          } else {
+            tableContainer = new HybridHashTableContainer(hconf, keyCount, memory,
+                desc.getParentDataSizes().get(pos), nwayConf);
+          }
+        } else {
+          tableContainer = new HashMapWrapper(hconf, keyCount);
+        }
+
         LOG.info("Using tableContainer " + tableContainer.getClass().getSimpleName());
 
         while (kvReader.next()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index c357329..024849e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -634,6 +634,10 @@ public class ConvertJoinMapJoin implements NodeProcessor {
             joinOp.getConf().getMapAliases(), bigTablePosition, true, removeReduceSink);
     mapJoinOp.getConf().setHybridHashJoin(HiveConf.getBoolVar(context.conf,
         HiveConf.ConfVars.HIVEUSEHYBRIDGRACEHASHJOIN));
+    List<ExprNodeDesc> joinExprs = mapJoinOp.getConf().getKeys().values().iterator().next();
+    if (joinExprs.size() == 0) {  // In case of cross join, we disable hybrid grace hash join
+      mapJoinOp.getConf().setHybridHashJoin(false);
+    }
 
     Operator<? extends OperatorDesc> parentBigTableOp =
         mapJoinOp.getParentOperators().get(bigTablePosition);

http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/test/queries/clientpositive/cross_join.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cross_join.q b/ql/src/test/queries/clientpositive/cross_join.q
index 8eb949e..d30df50 100644
--- a/ql/src/test/queries/clientpositive/cross_join.q
+++ b/ql/src/test/queries/clientpositive/cross_join.q
@@ -5,3 +5,11 @@ explain select src.key from src join src src2;
 explain select src.key from src cross join src src2;
 -- appending condition is allowed
 explain select src.key from src cross join src src2 on src.key=src2.key;
+
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.noconditionaltask.size=10000000;
+set hive.mapjoin.hybridgrace.hashtable=true;
+
+explain select src.key from src join src src2;
+explain select src.key from src cross join src src2;
+explain select src.key from src cross join src src2 on src.key=src2.key;

http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/test/results/clientpositive/cross_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cross_join.q.out b/ql/src/test/results/clientpositive/cross_join.q.out
index a6dcd28..c543007 100644
--- a/ql/src/test/results/clientpositive/cross_join.q.out
+++ b/ql/src/test/results/clientpositive/cross_join.q.out
@@ -179,3 +179,199 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+Warning: Map Join MAPJOIN[11][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
+PREHOOK: query: explain select src.key from src join src src2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key from src join src src2
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-4 is a root stage
+  Stage-3 depends on stages: Stage-4
+  Stage-0 depends on stages: Stage-3
+
+STAGE PLANS:
+  Stage: Stage-4
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        $hdt$_0:src 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        $hdt$_0:src 
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              HashTable Sink Operator
+                keys:
+                  0 
+                  1 
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0
+                Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Local Work:
+        Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Map Join MAPJOIN[11][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
+PREHOOK: query: explain select src.key from src cross join src src2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key from src cross join src src2
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-4 is a root stage
+  Stage-3 depends on stages: Stage-4
+  Stage-0 depends on stages: Stage-3
+
+STAGE PLANS:
+  Stage: Stage-4
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        $hdt$_0:src 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        $hdt$_0:src 
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              HashTable Sink Operator
+                keys:
+                  0 
+                  1 
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0
+                Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Local Work:
+        Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select src.key from src cross join src src2 on src.key=src2.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key from src cross join src src2 on src.key=src2.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-4 is a root stage
+  Stage-3 depends on stages: Stage-4
+  Stage-0 depends on stages: Stage-3
+
+STAGE PLANS:
+  Stage: Stage-4
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        $hdt$_0:src 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        $hdt$_0:src 
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Local Work:
+        Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+

http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/test/results/clientpositive/spark/cross_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/cross_join.q.out b/ql/src/test/results/clientpositive/spark/cross_join.q.out
index 5322d39..af49a79 100644
--- a/ql/src/test/results/clientpositive/spark/cross_join.q.out
+++ b/ql/src/test/results/clientpositive/spark/cross_join.q.out
@@ -203,3 +203,214 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+Warning: Map Join MAPJOIN[11][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: explain select src.key from src join src src2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key from src join src src2
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 
+                        1 
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: key (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 
+                        1 
+                      outputColumnNames: _col0
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Map Join MAPJOIN[11][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: explain select src.key from src cross join src src2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key from src cross join src src2
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 
+                        1 
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: key (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 
+                        1 
+                      outputColumnNames: _col0
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select src.key from src cross join src src2 on src.key=src2.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key from src cross join src src2 on src.key=src2.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Spark HashTable Sink Operator
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0
+                        input vertices:
+                          1 Map 2
+                        Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                          table:
+                              input format: org.apache.hadoop.mapred.TextInputFormat
+                              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+

http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/test/results/clientpositive/tez/auto_join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_join0.q.out b/ql/src/test/results/clientpositive/tez/auto_join0.q.out
index beaac17..1564b1c 100644
--- a/ql/src/test/results/clientpositive/tez/auto_join0.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_join0.q.out
@@ -87,7 +87,6 @@ STAGE PLANS:
                   input vertices:
                     1 Reducer 6
                   Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
-                  HybridGraceHashJoin: true
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                     sort order: ++++

http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_12.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_12.q.out
index e90af15..b83aa27 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_12.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_12.q.out
@@ -402,7 +402,6 @@ STAGE PLANS:
                           1 Map 5
                         Position of Big Table: 0
                         Statistics: Num rows: 69 Data size: 7032 Basic stats: COMPLETE Column stats: NONE
-                        HybridGraceHashJoin: true
                         Group By Operator
                           aggregations: count()
                           mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/test/results/clientpositive/tez/cross_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/cross_join.q.out b/ql/src/test/results/clientpositive/tez/cross_join.q.out
index 6d10c1e..431aa63 100644
--- a/ql/src/test/results/clientpositive/tez/cross_join.q.out
+++ b/ql/src/test/results/clientpositive/tez/cross_join.q.out
@@ -203,3 +203,190 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+Warning: Map Join MAPJOIN[11][bigTable=?] in task 'Map 1' is a cross product
+PREHOOK: query: explain select src.key from src join src src2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key from src join src src2
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: key (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 
+                        1 
+                      outputColumnNames: _col0
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Map Join MAPJOIN[11][bigTable=?] in task 'Map 1' is a cross product
+PREHOOK: query: explain select src.key from src cross join src src2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key from src cross join src src2
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: key (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 
+                        1 
+                      outputColumnNames: _col0
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select src.key from src cross join src src2 on src.key=src2.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key from src cross join src src2 on src.key=src2.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0
+                        input vertices:
+                          1 Map 2
+                        Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                        HybridGraceHashJoin: true
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                          table:
+                              input format: org.apache.hadoop.mapred.TextInputFormat
+                              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+

http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/test/results/clientpositive/tez/cross_product_check_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/cross_product_check_2.q.out b/ql/src/test/results/clientpositive/tez/cross_product_check_2.q.out
index 14dd820..f30caa1 100644
--- a/ql/src/test/results/clientpositive/tez/cross_product_check_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/cross_product_check_2.q.out
@@ -63,7 +63,6 @@ STAGE PLANS:
                       input vertices:
                         1 Map 2
                       Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                      HybridGraceHashJoin: true
                       File Output Operator
                         compressed: false
                         Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
@@ -160,7 +159,6 @@ STAGE PLANS:
                     input vertices:
                       0 Map 1
                     Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                    HybridGraceHashJoin: true
                     Select Operator
                       expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
@@ -260,7 +258,6 @@ STAGE PLANS:
                     input vertices:
                       1 Reducer 2
                     Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                    HybridGraceHashJoin: true
                     Select Operator
                       expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string)
                       outputColumnNames: _col0, _col1, _col2
@@ -324,7 +321,6 @@ STAGE PLANS:
                     input vertices:
                       1 Map 3
                     Statistics: Num rows: 11 Data size: 105 Basic stats: COMPLETE Column stats: NONE
-                    HybridGraceHashJoin: true
                     Group By Operator
                       keys: _col0 (type: string)
                       mode: hash
@@ -358,7 +354,6 @@ STAGE PLANS:
                     input vertices:
                       1 Reducer 2
                     Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-                    HybridGraceHashJoin: true
                     Select Operator
                       expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string)
                       outputColumnNames: _col0, _col1, _col2
@@ -488,7 +483,6 @@ STAGE PLANS:
                   input vertices:
                     1 Reducer 4
                   Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
-                  HybridGraceHashJoin: true
                   File Output Operator
                     compressed: false
                     Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
index e11f3e5..bbc2e16 100644
--- a/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
@@ -4162,7 +4162,6 @@ STAGE PLANS:
                       input vertices:
                         1 Reducer 4
                       Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE
-                      HybridGraceHashJoin: true
                       Group By Operator
                         aggregations: count()
                         mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/77f30d4f/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out
index f0ddc5b..c779368 100644
--- a/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out
@@ -4215,7 +4215,6 @@ STAGE PLANS:
                       input vertices:
                         1 Reducer 4
                       Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE
-                      HybridGraceHashJoin: true
                       Group By Operator
                         aggregations: count()
                         mode: hash


[02/23] hive git commit: HIVE-11997 - Add ability to send Compaction Jobs to specific queue (Eugene Koifman, reviewed by Jason Dere)

Posted by se...@apache.org.
HIVE-11997 - Add ability to send Compaction Jobs to specific queue (Eugene Koifman, reviewed by Jason Dere)


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

Branch: refs/heads/llap
Commit: b1eb0c0f1c3fc0f503bc675281c8be8356d1f081
Parents: ff9822e
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Fri Oct 2 10:11:00 2015 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Fri Oct 2 10:11:00 2015 -0700

----------------------------------------------------------------------
 common/src/java/org/apache/hadoop/hive/conf/HiveConf.java |  2 ++
 .../apache/hadoop/hive/ql/txn/compactor/CompactorMR.java  | 10 +++++++++-
 2 files changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b1eb0c0f/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 dffdb5c..e7ed07e 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1571,6 +1571,8 @@ public class HiveConf extends Configuration {
 
     HIVE_COMPACTOR_CLEANER_RUN_INTERVAL("hive.compactor.cleaner.run.interval", "5000ms",
         new TimeValidator(TimeUnit.MILLISECONDS), "Time between runs of the cleaner thread"),
+    COMPACTOR_JOB_QUEUE("hive.compactor.job.queue", "", "Used to specify name of Hadoop queue to which\n" +
+      "Compaction jobs will be submitted.  Set to empty string to let Hadoop choose the queue."),
     HIVE_TIMEDOUT_TXN_REAPER_START("hive.timedout.txn.reaper.start", "100s",
       new TimeValidator(TimeUnit.MILLISECONDS), "Time delay of 1st reaper run after metastore start"),
     HIVE_TIMEDOUT_TXN_REAPER_INTERVAL("hive.timedout.txn.reaper.interval", "180s",

http://git-wip-us.apache.org/repos/asf/hive/blob/b1eb0c0f/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
index 02fa725..3ee9346 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
@@ -117,6 +117,11 @@ public class CompactorMR {
     job.setInputFormat(CompactorInputFormat.class);
     job.setOutputFormat(NullOutputFormat.class);
     job.setOutputCommitter(CompactorOutputCommitter.class);
+    
+    String queueName = conf.getVar(HiveConf.ConfVars.COMPACTOR_JOB_QUEUE);
+    if(queueName != null && queueName.length() > 0) {
+      job.setQueueName(queueName);
+    }
 
     job.set(FINAL_LOCATION, sd.getLocation());
     job.set(TMP_LOCATION, sd.getLocation() + "/" + TMPDIR + "_" + UUID.randomUUID().toString());
@@ -189,7 +194,10 @@ public class CompactorMR {
     LOG.debug("Setting maximume transaction to " + maxTxn);
 
     RunningJob rj = JobClient.runJob(job);
-    LOG.info("Submitted " + (isMajor ? CompactionType.MAJOR : CompactionType.MINOR) + " compaction job '" + jobName + "' with jobID=" + rj.getID());
+    LOG.info("Submitted " + (isMajor ? CompactionType.MAJOR : CompactionType.MINOR) + " compaction job '" +
+      jobName + "' with jobID=" + rj.getID() + " to " + job.getQueueName() + " queue.  " +
+      "(current delta dirs count=" + dir.getCurrentDirectories().size() +
+      ", obsolete delta dirs count=" + dir.getObsolete());
     rj.waitForCompletion();
     su.gatherStats();
   }


[23/23] hive git commit: HIVE-12034 : HIVE-4243 broke things for llap branch (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-12034 : HIVE-4243 broke things for llap branch (Sergey Shelukhin)


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

Branch: refs/heads/llap
Commit: edd3079f771d4b70914d79221bc0df8141e1957a
Parents: b28aec9
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Oct 5 12:43:23 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Oct 5 12:43:23 2015 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/edd3079f/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
index 7d9353a..68d3255 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
@@ -107,6 +107,7 @@ public final class OrcFile {
     ORIGINAL(0),
       HIVE_8732(1), // corrupted stripe/file maximum column statistics
       HIVE_4243(2), // use real column names from Hive tables
+// Don't use any magic numbers here except for the below:
       FUTURE(Integer.MAX_VALUE); // a version from a future writer
 
     private final int id;
@@ -125,7 +126,7 @@ public final class OrcFile {
       int max = Integer.MIN_VALUE;
       for (WriterVersion v : WriterVersion.values()) {
         if (v.id < 0) throw new AssertionError();
-        if (v.id > max) {
+        if (v.id > max && FUTURE.id != v.id) {
           max = v.id;
         }
       }
@@ -136,6 +137,7 @@ public final class OrcFile {
     }
 
     public static WriterVersion from(int val) {
+      if (val == FUTURE.id) return FUTURE; // Special handling for the magic value.
       return values[val];
     }
   }


[05/23] hive git commit: HIVE-11980 : Follow up on HIVE-11696, exception is thrown from CTAS from the table with table-level serde is Parquet while partition-level serde is JSON (Aihua Xu via Szehon)

Posted by se...@apache.org.
HIVE-11980 : Follow up on HIVE-11696, exception is thrown from CTAS from the table with table-level serde is Parquet while partition-level serde is JSON (Aihua Xu via Szehon)


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

Branch: refs/heads/llap
Commit: 0d36e82479a47dac7e55875364503881fdbc069e
Parents: c3d62ad
Author: Szehon Ho <sz...@cloudera.com>
Authored: Fri Oct 2 12:54:08 2015 -0700
Committer: Szehon Ho <sz...@cloudera.com>
Committed: Fri Oct 2 12:54:51 2015 -0700

----------------------------------------------------------------------
 data/files/sample2.json                         |  2 +
 .../serde/ArrayWritableObjectInspector.java     |  7 ++
 .../parquet_mixed_partition_formats2.q          | 31 ++++++
 .../parquet_mixed_partition_formats2.q.out      | 99 ++++++++++++++++++++
 4 files changed, 139 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0d36e824/data/files/sample2.json
----------------------------------------------------------------------
diff --git a/data/files/sample2.json b/data/files/sample2.json
new file mode 100644
index 0000000..4e1802f
--- /dev/null
+++ b/data/files/sample2.json
@@ -0,0 +1,2 @@
+{"id": 1, "reports": [2,3], "address": {"country": 1, "state": 1}}
+{"id": 2, "reports": [], "address": {"country": 1, "state": 2}}

http://git-wip-us.apache.org/repos/asf/hive/blob/0d36e824/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java
index 6091882..ae545b8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java
@@ -169,6 +169,13 @@ public class ArrayWritableObjectInspector extends SettableStructObjectInspector
       return new ArrayList<Object>(Arrays.asList(arrWritable));
     }
 
+    //since setStructFieldData and create return a list, getStructFieldData should be able to
+    //handle list data. This is required when table serde is ParquetHiveSerDe and partition serde
+    //is something else.
+    if (data instanceof List) {
+      return ((List) data);
+    }
+
     throw new UnsupportedOperationException("Cannot inspect " + data.getClass().getCanonicalName());
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0d36e824/ql/src/test/queries/clientpositive/parquet_mixed_partition_formats2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_mixed_partition_formats2.q b/ql/src/test/queries/clientpositive/parquet_mixed_partition_formats2.q
new file mode 100644
index 0000000..e0b21d1
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/parquet_mixed_partition_formats2.q
@@ -0,0 +1,31 @@
+add jar ${system:maven.local.repository}/org/apache/hive/hcatalog/hive-hcatalog-core/${system:hive.version}/hive-hcatalog-core-${system:hive.version}.jar;
+
+CREATE TABLE parquet_table_json_partition (
+id bigint COMMENT 'from deserializer',
+address struct<country:bigint,state:bigint> COMMENT 'from deserializer',
+reports array<bigint> COMMENT 'from deserializer')
+PARTITIONED BY (
+ts string)
+ROW FORMAT SERDE
+'org.apache.hive.hcatalog.data.JsonSerDe'
+STORED AS INPUTFORMAT
+'org.apache.hadoop.mapred.TextInputFormat'
+OUTPUTFORMAT
+'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat';
+
+LOAD DATA LOCAL INPATH '../../data/files/sample2.json' INTO TABLE parquet_table_json_partition PARTITION(ts='20150101');
+
+SELECT * FROM parquet_table_json_partition LIMIT 100;
+
+ALTER TABLE parquet_table_json_partition
+  SET FILEFORMAT INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+                 OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+                 SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe';
+
+SELECT * FROM parquet_table_json_partition LIMIT 100;
+
+CREATE TABLE new_table AS SELECT * FROM parquet_table_json_partition LIMIT 100;
+
+SELECT * FROM new_table;
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/0d36e824/ql/src/test/results/clientpositive/parquet_mixed_partition_formats2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_mixed_partition_formats2.q.out b/ql/src/test/results/clientpositive/parquet_mixed_partition_formats2.q.out
new file mode 100644
index 0000000..c4d7197
--- /dev/null
+++ b/ql/src/test/results/clientpositive/parquet_mixed_partition_formats2.q.out
@@ -0,0 +1,99 @@
+PREHOOK: query: CREATE TABLE parquet_table_json_partition (
+id bigint COMMENT 'from deserializer',
+address struct<country:bigint,state:bigint> COMMENT 'from deserializer',
+reports array<bigint> COMMENT 'from deserializer')
+PARTITIONED BY (
+ts string)
+ROW FORMAT SERDE
+'org.apache.hive.hcatalog.data.JsonSerDe'
+STORED AS INPUTFORMAT
+'org.apache.hadoop.mapred.TextInputFormat'
+OUTPUTFORMAT
+'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_table_json_partition
+POSTHOOK: query: CREATE TABLE parquet_table_json_partition (
+id bigint COMMENT 'from deserializer',
+address struct<country:bigint,state:bigint> COMMENT 'from deserializer',
+reports array<bigint> COMMENT 'from deserializer')
+PARTITIONED BY (
+ts string)
+ROW FORMAT SERDE
+'org.apache.hive.hcatalog.data.JsonSerDe'
+STORED AS INPUTFORMAT
+'org.apache.hadoop.mapred.TextInputFormat'
+OUTPUTFORMAT
+'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_table_json_partition
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/sample2.json' INTO TABLE parquet_table_json_partition PARTITION(ts='20150101')
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@parquet_table_json_partition
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/sample2.json' INTO TABLE parquet_table_json_partition PARTITION(ts='20150101')
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@parquet_table_json_partition
+POSTHOOK: Output: default@parquet_table_json_partition@ts=20150101
+PREHOOK: query: SELECT * FROM parquet_table_json_partition LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_table_json_partition
+PREHOOK: Input: default@parquet_table_json_partition@ts=20150101
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM parquet_table_json_partition LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_table_json_partition
+POSTHOOK: Input: default@parquet_table_json_partition@ts=20150101
+#### A masked pattern was here ####
+1	{"country":1,"state":1}	[2,3]	20150101
+2	{"country":1,"state":2}	[]	20150101
+PREHOOK: query: ALTER TABLE parquet_table_json_partition
+  SET FILEFORMAT INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+                 OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+                 SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+PREHOOK: type: ALTERTABLE_FILEFORMAT
+PREHOOK: Input: default@parquet_table_json_partition
+PREHOOK: Output: default@parquet_table_json_partition
+POSTHOOK: query: ALTER TABLE parquet_table_json_partition
+  SET FILEFORMAT INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+                 OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+                 SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+POSTHOOK: type: ALTERTABLE_FILEFORMAT
+POSTHOOK: Input: default@parquet_table_json_partition
+POSTHOOK: Output: default@parquet_table_json_partition
+PREHOOK: query: SELECT * FROM parquet_table_json_partition LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_table_json_partition
+PREHOOK: Input: default@parquet_table_json_partition@ts=20150101
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM parquet_table_json_partition LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_table_json_partition
+POSTHOOK: Input: default@parquet_table_json_partition@ts=20150101
+#### A masked pattern was here ####
+1	{"country":1,"state":1}	[2,3]	20150101
+2	{"country":1,"state":2}	[]	20150101
+PREHOOK: query: CREATE TABLE new_table AS SELECT * FROM parquet_table_json_partition LIMIT 100
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@parquet_table_json_partition
+PREHOOK: Input: default@parquet_table_json_partition@ts=20150101
+PREHOOK: Output: database:default
+PREHOOK: Output: default@new_table
+POSTHOOK: query: CREATE TABLE new_table AS SELECT * FROM parquet_table_json_partition LIMIT 100
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@parquet_table_json_partition
+POSTHOOK: Input: default@parquet_table_json_partition@ts=20150101
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@new_table
+PREHOOK: query: SELECT * FROM new_table
+PREHOOK: type: QUERY
+PREHOOK: Input: default@new_table
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM new_table
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@new_table
+#### A masked pattern was here ####
+2	{"country":1,"state":2}	[]	20150101
+1	{"country":1,"state":1}	[2,3]	20150101